From 9d30a1f996ab6900332c3721bc5f50e510e89d48 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Tue, 8 Aug 2023 15:56:49 +0200 Subject: [PATCH 01/54] initial commit for intraprocedural analysis --- .../intraprocedural/AbstractFlowAnalysis.java | 102 ++++ .../BitSetBasedPriorityQueue.java | 311 ++++++++++ .../intraprocedural/FlowAnalysis.java | 529 ++++++++++++++++++ .../intraprocedural/ForwardFlowAnalysis.java | 22 + 4 files changed, 964 insertions(+) create mode 100644 sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java create mode 100644 sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java create mode 100644 sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java create mode 100644 sootup.analysis/src/main/java/sootup/analysis/intraprocedural/ForwardFlowAnalysis.java diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java new file mode 100644 index 00000000000..ab52929d165 --- /dev/null +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java @@ -0,0 +1,102 @@ +package sootup.analysis.intraprocedural; + +/*- + * #%L + * Soot - a J*va Optimization Framework + * %% + * Copyright (C) 1997 - 1999 Raja Vallee-Rai + * %% + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Lesser General Public License as + * published by the Free Software Foundation, either version 2.1 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Lesser Public License for more details. + * + * You should have received a copy of the GNU General Lesser Public + * License along with this program. If not, see + * . + * #L% + */ + +import java.util.IdentityHashMap; +import java.util.Map; +import javax.annotation.Nonnull; +import sootup.core.graph.BasicBlock; +import sootup.core.graph.StmtGraph; +import sootup.core.jimple.common.stmt.Stmt; + +/** + * An abstract class providing a meta-framework for carrying out dataflow analysis. This class + * provides common methods and fields required by the BranchedFlowAnalysis and FlowAnalysis abstract + * classes. + * + * @param abstraction type for the Facts + */ +public abstract class AbstractFlowAnalysis { + + /** The graph being analysed. */ + protected final StmtGraph> graph; + + /** Maps graph nodes to IN sets. */ + protected final Map stmtToBeforeFlow; + + /** Constructs a flow analysis on the given StmtGraph. */ + public AbstractFlowAnalysis(StmtGraph> graph) { + this.graph = graph; + this.stmtToBeforeFlow = new IdentityHashMap<>(graph.getNodes().size() * 2 + 1); + } + + /** Returns the flow object corresponding to the initial values for each graph node. */ + @Nonnull + protected abstract F newInitialFlow(); + + /** Determines whether entryInitialFlow() is applied to trap handlers. */ + protected boolean treatTrapHandlersAsEntries() { + return false; + } + + /** Returns true if this analysis is forwards. */ + protected abstract boolean isForward(); + + /** + * Compute the merge of the in1 and in2 sets, putting the result into + * out. The behavior of this function depends on the implementation ( it may be + * necessary to check whether in1 and in2 are equal or aliased ). Used + * by the doAnalysis method. + */ + protected abstract void merge(@Nonnull F in1, @Nonnull F in2, @Nonnull F out); + + /** + * Merges in1 and in2 into out, just before node succNode. By default, this method just calls + * merge(A,A,A), ignoring the node. + */ + protected void merge(@Nonnull Stmt succNode, @Nonnull F in1, @Nonnull F in2, @Nonnull F out) { + merge(in1, in2, out); + } + + /** Creates a copy of the source flow object in dest. */ + protected abstract void copy(@Nonnull F source, @Nonnull F dest); + + /** + * Carries out the actual flow analysis. Typically called from a concrete FlowAnalysis's + * constructor. + */ + protected abstract void execute(); + + /** Accessor function returning value of IN set for s. */ + @Nonnull + public F getFlowBefore(@Nonnull Stmt s) { + return stmtToBeforeFlow.get(s); + } + + /** Merges in into inout, just before node succNode. */ + protected void mergeInto(@Nonnull Stmt succNode, @Nonnull F inout, @Nonnull F in) { + F tmp = newInitialFlow(); + merge(succNode, inout, in, tmp); + copy(tmp, inout); + } +} diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java new file mode 100644 index 00000000000..98adb9172fd --- /dev/null +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java @@ -0,0 +1,311 @@ +package sootup.analysis.intraprocedural; + +/*- + * #%L + * Soot - a J*va Optimization Framework + * %% + * Copyright (C) 1997 - 2018 Raja Vallée-Rai and others + * %% + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Lesser General Public License as + * published by the Free Software Foundation, either version 2.1 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Lesser Public License for more details. + * + * You should have received a copy of the GNU General Lesser Public + * License along with this program. If not, see + * . + * #L% + */ + +import java.util.*; +import javax.annotation.Nonnull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A fixed size priority queue based on bitsets. The elements of the priority queue are ordered + * according to a given universe. This priority queue does not permit {@code null} elements. + * Inserting of elements that are not part of the universe is also permitted (doing so will result + * in a {@code NoSuchElementException}). + * + * @author Steven Lambeth + * @param the type of elements held in the universe + */ +public abstract class BitSetBasedPriorityQueue extends AbstractQueue { + private static final Logger logger = LoggerFactory.getLogger(BitSetBasedPriorityQueue.class); + + private final List universe; + private final Map ordinalMap; + final int N; + int min = Integer.MAX_VALUE; + + BitSetBasedPriorityQueue(List universe, Map ordinalMap) { + assert ordinalMap.size() == universe.size(); + this.universe = universe; + this.ordinalMap = ordinalMap; + this.N = universe.size(); + } + + abstract class Itr implements Iterator { + long expected = getExpected(); + int next = min; + int now = Integer.MAX_VALUE; + + abstract long getExpected(); + + @Override + public boolean hasNext() { + return next < N; + } + + @Override + public E next() { + if (expected != getExpected()) { + throw new ConcurrentModificationException(); + } + if (next >= N) { + throw new NoSuchElementException(); + } + + now = next; + next = nextSetBit(next + 1); + return universe.get(now); + } + + @Override + public void remove() { + if (now >= N) { + throw new IllegalStateException(); + } + if (expected != getExpected()) { + throw new ConcurrentModificationException(); + } + + BitSetBasedPriorityQueue.this.remove(now); + expected = getExpected(); + now = Integer.MAX_VALUE; + } + } + + int getOrdinal(@Nonnull Object o) { + Integer i = ordinalMap.get(o); + if (i == null) { + throw new NoSuchElementException(); + } + return i; + } + + /** Adds all elements of the universe to this queue. */ + abstract void addAll(); + + /** + * Returns the index of the first bit that is set to true that occurs on or after the + * specified starting index. If no such bit exists then a value bigger that {@code N} is returned. + * + * @param fromIndex the index to start checking from (inclusive). + * @return the index of the next set bit. + */ + abstract int nextSetBit(int fromIndex); + + abstract boolean remove(int ordinal); + + abstract boolean add(int ordinal); + + abstract boolean contains(int ordinal); + + /** {@inheritDoc} */ + @Override + public final E peek() { + return isEmpty() ? null : universe.get(min); + } + + /** {@inheritDoc} */ + @Override + public final E poll() { + if (isEmpty()) { + return null; + } + E e = universe.get(min); + remove(min); + return e; + } + + /** + * {@inheritDoc} + * + * @throws NoSuchElementException if e not part of the universe + * @throws NullPointerException if e is {@code null} + */ + @Override + public final boolean add(E e) { + return offer(e); + } + + /** + * {@inheritDoc} + * + * @throws NoSuchElementException if e not part of the universe + * @throws NullPointerException if e is {@code null} + */ + @Override + public final boolean offer(E e) { + return add(getOrdinal(e)); + } + + /** {@inheritDoc} */ + @Override + public final boolean remove(Object o) { + if (o == null || isEmpty()) { + return false; + } + try { + if (o.equals(peek())) { + remove(min); + return true; + } else { + return remove(getOrdinal(o)); + } + } catch (NoSuchElementException e) { + logger.debug(e.getMessage()); + return false; + } + } + + /** {@inheritDoc} */ + @Override + public final boolean contains(Object o) { + if (o == null) { + return false; + } + try { + if (o.equals(peek())) { + return true; + } else { + return contains(getOrdinal(o)); + } + } catch (NoSuchElementException e) { + logger.debug(e.getMessage()); + return false; + } + } + + /** {@inheritDoc} */ + @Override + public boolean isEmpty() { + return min >= N; + } + + /** + * Creates a new full priority queue + * + * @param + * @param universe + * @return + */ + public static BitSetBasedPriorityQueue of(List universe) { + BitSetBasedPriorityQueue q = noneOf(universe); + q.addAll(); + return q; + } + + /** + * Creates a new empty priority queue + * + * @param + * @param universe + * @return + */ + public static BitSetBasedPriorityQueue noneOf(List universe) { + Map ordinalMap = new HashMap<>(2 * universe.size() / 3); + int i = 0; + for (E e : universe) { + if (e == null) { + throw new NullPointerException("null is not allowed"); + } + if (ordinalMap.put(e, i++) != null) { + throw new IllegalArgumentException("duplicate key found"); + } + } + return newPriorityQueue(universe, ordinalMap); + } + + private static BitSetBasedPriorityQueue newPriorityQueue( + List universe, Map ordinalMap) { + return new LargeBitSetBasedPriorityQueue(universe, ordinalMap); + } + + static class LargeBitSetBasedPriorityQueue extends BitSetBasedPriorityQueue { + + private final BitSet queue; + private long modCount = 0; + + LargeBitSetBasedPriorityQueue(List universe, Map ordinalMap) { + super(universe, ordinalMap); + queue = new BitSet(N); + } + + @Override + boolean add(int ordinal) { + if (contains(ordinal)) { + return false; + } + queue.set(ordinal); + min = Math.min(min, ordinal); + modCount++; + return true; + } + + @Override + void addAll() { + queue.set(0, N); + min = 0; + modCount++; + } + + @Override + int nextSetBit(int fromIndex) { + int i = queue.nextSetBit(fromIndex); + return (i < 0) ? Integer.MAX_VALUE : i; + } + + @Override + boolean remove(int ordinal) { + if (!contains(ordinal)) { + return false; + } + queue.clear(ordinal); + + if (min == ordinal) { + min = nextSetBit(min + 1); + } + + modCount++; + return true; + } + + @Override + boolean contains(int ordinal) { + return queue.get(ordinal); + } + + @Override + public Iterator iterator() { + return new Itr() { + @Override + long getExpected() { + return modCount; + } + }; + } + + @Override + public int size() { + return queue.cardinality(); + } + } +} diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java new file mode 100644 index 00000000000..b4331dfaf8d --- /dev/null +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java @@ -0,0 +1,529 @@ +package sootup.analysis.intraprocedural; + +/*- + * #%L + * Soot - a J*va Optimization Framework + * %% + * Copyright (C) 1997 - 1999 Raja Vallee-Rai + * %% + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Lesser General Public License as + * published by the Free Software Foundation, either version 2.1 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Lesser Public License for more details. + * + * You should have received a copy of the GNU General Lesser Public + * License along with this program. If not, see + * . + * #L% + */ + +import java.util.*; +import javax.annotation.Nonnull; +import sootup.core.graph.BasicBlock; +import sootup.core.graph.StmtGraph; +import sootup.core.jimple.common.stmt.JGotoStmt; +import sootup.core.jimple.common.stmt.Stmt; + +/** + * An abstract class providing a framework for carrying out dataflow analysis. Subclassing either + * BackwardFlowAnalysis or ForwardFlowAnalysis and providing implementations for the abstract + * methods will allow Soot to compute the corresponding flow analysis. + */ +public abstract class FlowAnalysis extends AbstractFlowAnalysis { + + public enum Flow { + IN { + @Override + F getFlow(Entry e) { + return e.inFlow; + } + }, + OUT { + @Override + F getFlow(Entry e) { + return e.outFlow; + } + }; + + abstract F getFlow(Entry e); + } + + static class Entry { + final Stmt data; + int number; + + /** This Entry is part of a real scc. */ + boolean isRealStronglyConnected; + + Entry[] in; + Entry[] out; + F inFlow; + F outFlow; + + @SuppressWarnings("unchecked") + Entry(Stmt u, Entry pred) { + in = new Entry[] {pred}; + data = u; + number = Integer.MIN_VALUE; + isRealStronglyConnected = false; + } + + @Override + public String toString() { + return data == null ? "" : data.toString(); + } + } + + static class Orderer { + /** + * Creates a new {@code Entry} graph based on a {@code DirectedGraph}. This includes pseudo + * topological order, local access for predecessors and successors, a graph entry-point, a + * {@code Numberable} interface and a real strongly connected component marker. + * + * @param g + * @param direction + * @param entryFlow + * @return + */ + static List> newUniverse( + @Nonnull StmtGraph> g, + @Nonnull AnalysisDirection direction, + @Nonnull F entryFlow) { + final int size = g.getNodes().size(); + final int n = size; + + Deque> s = new ArrayDeque<>(n); + List> universe = new ArrayList<>(n); + Map> visited = new HashMap<>(((n + 1) * 4) / 3); + + // out of universe node + Entry superEntry = new Entry(null, null); + + List entries; + List actualEntries = direction.getEntries(g); + + if (!actualEntries.isEmpty()) { + // normal cases: there is at least + // one return statement for a backward analysis + // or one entry statement for a forward analysis + entries = actualEntries; + } else { + // cases without any entry statement + + if (AnalysisDirection.FORWARD == direction) { + // case of a forward flow analysis on + // a method without any entry point + throw new RuntimeException("error: no entry point for method in forward analysis"); + } else { + // case of backward analysis on + // a method which potentially has + // an infinite loop and no return statement + entries = new ArrayList<>(); + + // a single head is expected + final Collection entrypoints = g.getEntrypoints(); + assert entrypoints.size() == 1; + Stmt head = entrypoints.iterator().next(); + + // collect all 'goto' statements to catch the 'goto' from the infinite loop + Set visitedNodes = new HashSet<>(); + List workList = new ArrayList<>(); + workList.add(head); + for (Stmt currentStmt; !workList.isEmpty(); ) { + currentStmt = workList.remove(0); + visitedNodes.add(currentStmt); + + // only add 'goto' statements + if (currentStmt instanceof JGotoStmt) { + entries.add(currentStmt); + } + + for (Stmt successor : g.successors(currentStmt)) { + if (visitedNodes.contains(successor)) { + continue; + } + workList.add(successor); + } + } + + // + if (entries.isEmpty()) { + throw new RuntimeException("error: backward analysis on an empty entry set."); + } + } + } + + visitEntry(visited, superEntry, entries); + superEntry.inFlow = entryFlow; + superEntry.outFlow = entryFlow; + + @SuppressWarnings("unchecked") + Entry[] sv = new Entry[size]; + int[] si = new int[size]; + int index = 0; + + int i = 0; + Entry v = superEntry; + + while (true) { + if (i < v.out.length) { + Entry w = v.out[i++]; + + // an unvisited child node + if (w.number == Integer.MIN_VALUE) { + w.number = s.size(); + s.add(w); + + visitEntry(visited, w, direction.getOut(g, w.data)); + + // save old + si[index] = i; + sv[index] = v; + index++; + + i = 0; + v = w; + } + } else { + if (index == 0) { + assert universe.size() <= size; + Collections.reverse(universe); + return universe; + } + + universe.add(v); + sccPop(s, v); + + // restore old + index--; + v = sv[index]; + i = si[index]; + } + } + } + + @Nonnull + private static Entry[] visitEntry( + Map> visited, Entry v, List out) { + final int n = out.size(); + @SuppressWarnings("unchecked") + Entry[] a = new Entry[n]; + + assert (out instanceof RandomAccess); + + for (int i = 0; i < n; i++) { + a[i] = getEntryOf(visited, out.get(i), v); + } + + return v.out = a; + } + + @Nonnull + private static Entry getEntryOf( + @Nonnull Map> visited, @Nonnull Stmt stmt, @Nonnull Entry v) { + // either we reach a new node or a merge node, the latter one is rare + // so put and restore should be better that a lookup + + // add and restore if required + Entry newEntry = new Entry<>(stmt, v); + Entry oldEntry = visited.putIfAbsent(stmt, newEntry); + + // no restore required + if (oldEntry == null) { + return newEntry; + } + + // adding self ref (real strongly connected with itself) + if (oldEntry == v) { + oldEntry.isRealStronglyConnected = true; + } + + // merge nodes are rare, so this is ok + int l = oldEntry.in.length; + oldEntry.in = Arrays.copyOf(oldEntry.in, l + 1); + oldEntry.in[l] = v; + return oldEntry; + } + + private static void sccPop(@Nonnull Deque> s, @Nonnull Entry v) { + int min = v.number; + for (Entry e : v.out) { + assert e.number > Integer.MIN_VALUE; + min = Math.min(min, e.number); + } + + // not our SCC + if (min != v.number) { + v.number = min; + return; + } + + // we only want real SCCs (size > 1) + Entry w = s.removeLast(); + w.number = Integer.MAX_VALUE; + if (w == v) { + return; + } + + w.isRealStronglyConnected = true; + for (; ; ) { + w = s.removeLast(); + assert w.number >= v.number; + w.isRealStronglyConnected = true; + w.number = Integer.MAX_VALUE; + if (w == v) { + assert w.in.length >= 2; + return; + } + } + } + } + + enum AnalysisDirection { + BACKWARD { + @Override + @Nonnull + List getEntries(StmtGraph> g) { + return g.getTails(); + } + + @Override + @Nonnull + List getOut(StmtGraph> g, Stmt s) { + return g.predecessors(s); + } + }, + FORWARD { + @Override + @Nonnull + List getEntries(StmtGraph> g) { + return (List) g.getEntrypoints(); + } + + @Override + @Nonnull + List getOut(StmtGraph> g, Stmt s) { + return g.successors(s); + } + }; + + @Nonnull + abstract List getEntries(StmtGraph> g); + + @Nonnull + abstract List getOut(StmtGraph> g, Stmt s); + } + + /** Maps graph nodes to OUT sets. */ + @Nonnull protected final Map stmtToAfterFlow; + + /** Filtered: Maps graph nodes to OUT sets. */ + @Nonnull protected Map filterStmtToAfterFlow; + + /** Constructs a flow analysis on the given DirectedGraph. */ + public FlowAnalysis(@Nonnull StmtGraph> graph) { + super(graph); + this.stmtToAfterFlow = new IdentityHashMap<>(graph.getNodes().size() * 2 + 1); + this.filterStmtToAfterFlow = Collections.emptyMap(); + } + + /** + * Given the merge of the out sets, compute the in set for s + * (or in to out, depending on direction). + * + *

This function often causes confusion, because the same interface is used for both forward + * and backward flow analyses. The first parameter is always the argument to the flow function + * (i.e. it is the "in" set in a forward analysis and the "out" set in a backward analysis), and + * the third parameter is always the result of the flow function (i.e. it is the "out" set in a + * forward analysis and the "in" set in a backward analysis). + * + * @param in the input flow + * @param d the current node + * @param out the returned flow + */ + protected abstract void flowThrough(@Nonnull A in, Stmt d, @Nonnull A out); + + /** Accessor function returning value of OUT set for s. */ + public A getFlowAfter(@Nonnull Stmt s) { + A a = stmtToAfterFlow.get(s); + return a == null ? newInitialFlow() : a; + } + + @Nonnull + @Override + public A getFlowBefore(@Nonnull Stmt s) { + A a = stmtToBeforeFlow.get(s); + return a == null ? newInitialFlow() : a; + } + + private void initFlow( + @Nonnull Iterable> universe, @Nonnull Map in, @Nonnull Map out) { + + // If a node has only a single in-flow, the in-flow is always equal + // to the out-flow if its predecessor, so we use the same object. + // this saves memory and requires less object creation and copy calls. + + // Furthermore a node can be marked as omissible, this allows us to use + // the same "flow-set" for out-flow and in-flow. A merge node with within + // a real scc cannot be omitted, as it could cause endless loops within + // the fixpoint-iteration! + + for (Entry n : universe) { + boolean omit = true; + if (n.in.length > 1) { + n.inFlow = newInitialFlow(); + + // no merge points in loops + omit = !n.isRealStronglyConnected; + } else { + assert n.in.length == 1 : "missing superhead"; + n.inFlow = getFlow(n.in[0], n); + assert n.inFlow != null : "topological order is broken"; + } + + if (omit && omissible(n.data)) { + // We could recalculate the graph itself but thats more expensive than + // just falling through such nodes. + n.outFlow = n.inFlow; + } else { + n.outFlow = newInitialFlow(); + } + + // for legacy api (ms: already a soot comment) + in.put(n.data, n.inFlow); + out.put(n.data, n.outFlow); + } + } + + /** + * If a flow node can be omitted return true, otherwise false. There is + * no guarantee a node will be omitted. A omissible node does not influence the result of an + * analysis. + * + *

If you are unsure, don't overwrite this method + * + * @param stmt the node to check + * @return false + */ + protected boolean omissible(@Nonnull Stmt stmt) { + return false; + } + + /** + * You can specify which flow set you would like to use of node {@code from} + * + * @param from + * @param mergeNode + * @return Flow.OUT + */ + protected Flow getFlow(@Nonnull Stmt from, @Nonnull Stmt mergeNode) { + return Flow.OUT; + } + + private A getFlow(@Nonnull Entry o, @Nonnull Entry e) { + return (o.inFlow == o.outFlow) ? o.outFlow : getFlow(o.data, e.data).getFlow(o); + } + + private void meetFlows(@Nonnull Entry entry) { + assert entry.in.length >= 1; + + if (entry.in.length > 1) { + boolean copy = true; + for (Entry o : entry.in) { + A flow = getFlow(o, entry); + if (copy) { + copy = false; + copy(flow, entry.inFlow); + } else { + mergeInto(entry.data, entry.inFlow, flow); + } + } + } + } + + final int execute(@Nonnull Map inFlow, @Nonnull Map outFlow) { + + final boolean isForward = isForward(); + final List> universe = + Orderer.newUniverse( + graph, + isForward ? AnalysisDirection.FORWARD : AnalysisDirection.BACKWARD, + newInitialFlow()); + initFlow(universe, inFlow, outFlow); + + Queue> q = BitSetBasedPriorityQueue.of(universe); + + // Perform fixed point flow analysis + for (int numComputations = 0; ; numComputations++) { + Entry e = q.poll(); + if (e == null) { + return numComputations; + } + + meetFlows(e); + + // Compute beforeFlow and store it. + // ifh.handleFlowIn(this, e.data); + boolean hasChanged = flowThrough(e); + // ifh.handleFlowOut(this, e.data); + + // Update queue appropriately + if (hasChanged) { + q.addAll(Arrays.asList(e.out)); + } + } + } + + private boolean flowThrough(Entry d) { + // omitted, just fall through + if (d.inFlow == d.outFlow) { + assert !d.isRealStronglyConnected || d.in.length == 1; + return true; + } + + if (d.isRealStronglyConnected) { + // A flow node that is influenced by at least one back-reference. + // It's essential to check if "flowThrough" changes the result. + // This requires the calculation of "equals", which itself + // can be really expensive - depending on the used flow-model. + // Depending on the "merge"+"flowThrough" costs, it can be cheaper + // to fall through. Only nodes with real back-references always + // need to be checked for changes + A out = newInitialFlow(); + flowThrough(d.inFlow, d.data, out); + if (out.equals(d.outFlow)) { + return false; + } + // copy back the result, as it has changed (former: copyFreshToExisting) + copy(out, d.outFlow); + return true; + } + + // no back-references, just calculate "flowThrough" + flowThrough(d.inFlow, d.data, d.outFlow); + return true; + } + + /* + * Copies a *fresh* copy of in to dest. The input is not referenced somewhere else. This allows + * subclasses for a smarter and faster copying. + * + * @param in + * @param dest + * + protected void copyFreshToExisting(A in, A dest) { + if (in instanceof FlowSet && dest instanceof FlowSet) { + FlowSet fin = (FlowSet) in; + FlowSet fdest = (FlowSet) dest; + fin.copyFreshToExisting(fdest); + } else { + copy(in, dest); + } + } + */ +} diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/ForwardFlowAnalysis.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/ForwardFlowAnalysis.java new file mode 100644 index 00000000000..8ad8d7b45ba --- /dev/null +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/ForwardFlowAnalysis.java @@ -0,0 +1,22 @@ +package sootup.analysis.intraprocedural; + +import sootup.core.graph.BasicBlock; +import sootup.core.graph.StmtGraph; + +public abstract class ForwardFlowAnalysis extends FlowAnalysis { + + /** Construct the analysis from StmtGraph. */ + public > ForwardFlowAnalysis(StmtGraph graph) { + super(graph); + } + + @Override + protected boolean isForward() { + return true; + } + + @Override + protected void execute() { + int i = execute(stmtToBeforeFlow, stmtToAfterFlow); + } +} From 7d7dda0c3b2cb34d065f23b372e3059300294696 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 1 Sep 2023 21:22:26 +0200 Subject: [PATCH 02/54] progress on priority queue backed by indexed universe --- .../intraprocedural/AbstractFlowAnalysis.java | 1 + .../intraprocedural/FlowAnalysis.java | 2 +- ....java => UniverseSortedPriorityQueue.java} | 49 ++++++++++--------- 3 files changed, 27 insertions(+), 25 deletions(-) rename sootup.analysis/src/main/java/sootup/analysis/intraprocedural/{BitSetBasedPriorityQueue.java => UniverseSortedPriorityQueue.java} (81%) diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java index ab52929d165..245c32e99ae 100644 --- a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/AbstractFlowAnalysis.java @@ -55,6 +55,7 @@ public AbstractFlowAnalysis(StmtGraph> graph) { protected abstract F newInitialFlow(); /** Determines whether entryInitialFlow() is applied to trap handlers. */ + // FIXME: [ms] implement as an option protected boolean treatTrapHandlersAsEntries() { return false; } diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java index b4331dfaf8d..20a443d4379 100644 --- a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/FlowAnalysis.java @@ -456,7 +456,7 @@ final int execute(@Nonnull Map inFlow, @Nonnull Map outFlow) { newInitialFlow()); initFlow(universe, inFlow, outFlow); - Queue> q = BitSetBasedPriorityQueue.of(universe); + Queue> q = UniverseSortedPriorityQueue.of(universe); // Perform fixed point flow analysis for (int numComputations = 0; ; numComputations++) { diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java similarity index 81% rename from sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java rename to sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java index 98adb9172fd..191150f8fba 100644 --- a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/BitSetBasedPriorityQueue.java +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java @@ -36,19 +36,19 @@ * @author Steven Lambeth * @param the type of elements held in the universe */ -public abstract class BitSetBasedPriorityQueue extends AbstractQueue { - private static final Logger logger = LoggerFactory.getLogger(BitSetBasedPriorityQueue.class); +public abstract class UniverseSortedPriorityQueue extends AbstractQueue { + private static final Logger logger = LoggerFactory.getLogger(UniverseSortedPriorityQueue.class); - private final List universe; - private final Map ordinalMap; - final int N; + protected final List universe; + protected final Map ordinalMap; int min = Integer.MAX_VALUE; - BitSetBasedPriorityQueue(List universe, Map ordinalMap) { + UniverseSortedPriorityQueue(List universe, Map ordinalMap) { + // TODO: [ms] we should index the ordinalMap ourselves? Or for intended reuse just wrap it + // together with the universe? and use an IdentityHashMap.. assert ordinalMap.size() == universe.size(); this.universe = universe; this.ordinalMap = ordinalMap; - this.N = universe.size(); } abstract class Itr implements Iterator { @@ -60,7 +60,7 @@ abstract class Itr implements Iterator { @Override public boolean hasNext() { - return next < N; + return next < universe.size(); } @Override @@ -68,7 +68,7 @@ public E next() { if (expected != getExpected()) { throw new ConcurrentModificationException(); } - if (next >= N) { + if (next >= universe.size()) { throw new NoSuchElementException(); } @@ -79,14 +79,14 @@ public E next() { @Override public void remove() { - if (now >= N) { + if (now >= universe.size()) { throw new IllegalStateException(); } if (expected != getExpected()) { throw new ConcurrentModificationException(); } - BitSetBasedPriorityQueue.this.remove(now); + UniverseSortedPriorityQueue.this.remove(now); expected = getExpected(); now = Integer.MAX_VALUE; } @@ -164,7 +164,7 @@ public final boolean remove(Object o) { return false; } try { - if (o.equals(peek())) { + if (o == peek()) { remove(min); return true; } else { @@ -183,7 +183,7 @@ public final boolean contains(Object o) { return false; } try { - if (o.equals(peek())) { + if (o == peek()) { return true; } else { return contains(getOrdinal(o)); @@ -197,7 +197,7 @@ public final boolean contains(Object o) { /** {@inheritDoc} */ @Override public boolean isEmpty() { - return min >= N; + return min >= universe.size(); } /** @@ -207,8 +207,8 @@ public boolean isEmpty() { * @param universe * @return */ - public static BitSetBasedPriorityQueue of(List universe) { - BitSetBasedPriorityQueue q = noneOf(universe); + public static UniverseSortedPriorityQueue of(List universe) { + UniverseSortedPriorityQueue q = noneOf(universe); q.addAll(); return q; } @@ -220,12 +220,12 @@ public static BitSetBasedPriorityQueue of(List universe) { * @param universe * @return */ - public static BitSetBasedPriorityQueue noneOf(List universe) { + public static UniverseSortedPriorityQueue noneOf(List universe) { Map ordinalMap = new HashMap<>(2 * universe.size() / 3); int i = 0; for (E e : universe) { if (e == null) { - throw new NullPointerException("null is not allowed"); + throw new IllegalArgumentException("null is not allowed"); } if (ordinalMap.put(e, i++) != null) { throw new IllegalArgumentException("duplicate key found"); @@ -234,19 +234,19 @@ public static BitSetBasedPriorityQueue noneOf(List universe) return newPriorityQueue(universe, ordinalMap); } - private static BitSetBasedPriorityQueue newPriorityQueue( + private static UniverseSortedPriorityQueue newPriorityQueue( List universe, Map ordinalMap) { - return new LargeBitSetBasedPriorityQueue(universe, ordinalMap); + return new LargeUniverseSortedPriorityQueue(universe, ordinalMap); } - static class LargeBitSetBasedPriorityQueue extends BitSetBasedPriorityQueue { + static class LargeUniverseSortedPriorityQueue extends UniverseSortedPriorityQueue { private final BitSet queue; private long modCount = 0; - LargeBitSetBasedPriorityQueue(List universe, Map ordinalMap) { + LargeUniverseSortedPriorityQueue(List universe, Map ordinalMap) { super(universe, ordinalMap); - queue = new BitSet(N); + queue = new BitSet(universe.size()); } @Override @@ -262,7 +262,7 @@ boolean add(int ordinal) { @Override void addAll() { - queue.set(0, N); + queue.set(0, universe.size()); min = 0; modCount++; } @@ -293,6 +293,7 @@ boolean contains(int ordinal) { return queue.get(ordinal); } + @Nonnull @Override public Iterator iterator() { return new Itr() { From 992975ad245eee025d09008e1af1dfc28b54eccc Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 1 Sep 2023 21:31:33 +0200 Subject: [PATCH 03/54] refactor copy() as a ctor --- .../sootup/analysis/intraprocedural/Fact.java | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 sootup.analysis/src/main/java/sootup/analysis/intraprocedural/Fact.java diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/Fact.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/Fact.java new file mode 100644 index 00000000000..67cbcd420e9 --- /dev/null +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/Fact.java @@ -0,0 +1,77 @@ +package sootup.analysis.intraprocedural; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import javax.annotation.Nonnull; +import sootup.core.jimple.basic.Local; + +/** simple dataflow fact for interprocedural dataflow analysis adaptable with a state enum * */ +public class Fact { + + /** The aliases that point to the same object. */ + @Nonnull private final Set aliases; + + /** The state of the object. */ + @Nonnull private S state; + + public Fact(@Nonnull S initialState) { + this(new HashSet<>(), initialState); + } + + public Fact(@Nonnull Fact originFact) { + this(new HashSet<>(originFact.aliases), originFact.state); + } + + protected Fact(@Nonnull Set aliases, @Nonnull S initialState) { + this.aliases = aliases; + this.state = initialState; + } + + public void updateState(@Nonnull S state) { + this.state = state; + } + + public void addAlias(@Nonnull Local alias) { + this.aliases.add(alias); + } + + public boolean containsAlias(@Nonnull Local value) { + return aliases.contains(value); + } + + @Nonnull + public S getState() { + return state; + } + + @Override + public String toString() { + return "(" + aliases + ", " + state + ")"; + } + + @Override + public int hashCode() { + return Objects.hash(aliases, state); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + Fact other = (Fact) obj; + if (!aliases.equals(other.aliases)) { + return false; + } + + return state == other.state; + } +} From 4fdc12363e6a4a8fa7ead924b73182e911773c72 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 11:37:34 +0200 Subject: [PATCH 04/54] refactor getDefsOfLocal; JFieldref as leftOp of an assignment is not a Local.. --- .../java/sootup/core/graph/StmtGraph.java | 3 +- .../java/sootup/core/jimple/basic/Local.java | 13 +- .../bytecode/interceptors/Aggregator.java | 220 +++++++++--------- .../ConstantPropagatorAndFolder.java | 3 +- .../bytecode/interceptors/CopyPropagator.java | 10 +- .../MinimalBytecodeTestSuiteBase.java | 27 +++ 6 files changed, 158 insertions(+), 118 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java index 81cbd28b479..5a1e6237817 100644 --- a/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java @@ -245,7 +245,8 @@ public void validateStmtConnectionsInGraph() { /** * Look for a path in graph, from def to use. This path has to lie inside an extended basic block - * (and this property implies uniqueness.). The path returned includes from and to. + * (and this property implies uniqueness.). The path returned includes from and to. FIXME: ms: + * explain better * * @param from start point for the path. * @param to end point for the path. diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java index 27874dc900d..0ca645c7e8b 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java @@ -123,12 +123,13 @@ public Position getPosition() { return position; } - public List> getDefsOfLocal(List defs) { - List> localDefs = new ArrayList<>(); + /** returns the returned List can contain: Locals, JFieldRefs, JArrayRefs */ + public List> getDefsOfLocal(List defs) { + List> localDefs = new ArrayList<>(); for (Stmt stmt : defs) { if (stmt instanceof AbstractDefinitionStmt - && ((AbstractDefinitionStmt) stmt).getLeftOp().equals(this)) { - localDefs.add((AbstractDefinitionStmt) stmt); + && ((AbstractDefinitionStmt) stmt).getLeftOp().equals(this)) { + localDefs.add((AbstractDefinitionStmt) stmt); } } return localDefs; @@ -156,7 +157,9 @@ public List getDefsForLocalUse(StmtGraph graph, Stmt stmt) { if (s instanceof AbstractDefinitionStmt && s.getDefs().get(0).equivTo(this)) { defStmts.add(s); } else { - queue.addAll(graph.predecessors(s)); + if (graph.containsNode(s)) { + queue.addAll(graph.predecessors(s)); + } } } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index ab41bdeba61..4fae9d4722c 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -66,125 +66,131 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi List stmts = builder.getStmts(); for (Stmt stmt : Lists.newArrayList(stmts)) { - if (stmt instanceof JAssignStmt) { - final JAssignStmt assignStmt = (JAssignStmt) stmt; - Value lhs = assignStmt.getLeftOp(); - if (lhs instanceof Local) { - Local lhsLocal = (Local) lhs; - if (onlyStackVars && !lhsLocal.getName().startsWith("$stack")) { - continue; - } - for (Value val : assignStmt.getUses()) { - if (val instanceof Local) { - List> defs = ((Local) val).getDefsOfLocal(stmts); - if (defs.size() == 1) { - Stmt relevantDef = defs.get(0); - List path = graph.getExtendedBasicBlockPathBetween(relevantDef, stmt); + if (!(stmt instanceof JAssignStmt)) { + continue; + } + final JAssignStmt assignStmt = (JAssignStmt) stmt; + Value lhs = assignStmt.getLeftOp(); + if (!(lhs instanceof Local)) { + continue; + } + Local lhsLocal = (Local) lhs; + if (onlyStackVars && !lhsLocal.getName().startsWith("$stack")) { + continue; + } + for (Value val : assignStmt.getUses()) { + if (!(val instanceof Local)) { + continue; + } + List> defs = ((Local) val).getDefsOfLocal(stmts); + if (defs.size() != 1) { + continue; + } - boolean cantAggr = false; - boolean propagatingInvokeExpr = false; - boolean propagatingFieldRef = false; - boolean propagatingArrayRef = false; - List fieldRefList = new ArrayList<>(); + Stmt relevantDef = defs.get(0); + List path = graph.getExtendedBasicBlockPathBetween(relevantDef, stmt); - List localsUsed = new ArrayList<>(); - for (Stmt pathStmt : path) { - List allDefs = pathStmt.getDefs(); - for (Value def : allDefs) { - if (def instanceof Local) { - localsUsed.add(def); - } else if (def instanceof AbstractInstanceInvokeExpr) { - propagatingInvokeExpr = true; - } else if (def instanceof JArrayRef) { - propagatingArrayRef = true; - } else if (def instanceof JFieldRef) { - propagatingFieldRef = true; - fieldRefList.add((JFieldRef) def); - } - } - } - for (Stmt pathStmt : path) { - if (pathStmt != stmt && pathStmt != relevantDef) { - for (Value stmtDef : pathStmt.getDefs()) { - if (localsUsed.contains(stmtDef)) { - cantAggr = true; - break; - } - if (propagatingInvokeExpr || propagatingFieldRef || propagatingArrayRef) { - if (stmtDef instanceof JFieldRef) { - if (propagatingInvokeExpr) { - cantAggr = true; - break; - } else if (propagatingFieldRef) { - // Can't aggregate a field access if passing a definition of a field - // with the same name, because they might be aliased - for (JFieldRef fieldRef : fieldRefList) { - if (fieldRef.equals((JFieldRef) stmtDef)) { - cantAggr = true; - break; - } - } - } - } else if (stmtDef instanceof JArrayRef) { - if (propagatingInvokeExpr || propagatingArrayRef) { - // Cannot aggregate an invoke expr past an array write and cannot - // aggregate an array read past a write - cantAggr = true; - break; - } - } - } - } - } - // Check for intervening side effects due to method calls - if (propagatingInvokeExpr || propagatingFieldRef || propagatingArrayRef) { - for (final Value value : stmt.getUses()) { - if (pathStmt == stmt && value == lhs) { - break; - } - if (value instanceof AbstractInstanceInvokeExpr - || (propagatingInvokeExpr - && (value instanceof JFieldRef || value instanceof JArrayRef))) { + boolean cantAggr = false; + boolean propagatingInvokeExpr = false; + boolean propagatingFieldRef = false; + boolean propagatingArrayRef = false; + List fieldRefList = new ArrayList<>(); + + List localsUsed = new ArrayList<>(); + for (Stmt pathStmt : path) { + List allDefs = pathStmt.getDefs(); + for (Value def : allDefs) { + if (def instanceof Local) { + localsUsed.add(def); + } else if (def instanceof AbstractInstanceInvokeExpr) { + propagatingInvokeExpr = true; + } else if (def instanceof JArrayRef) { + propagatingArrayRef = true; + } else if (def instanceof JFieldRef) { + propagatingFieldRef = true; + fieldRefList.add((JFieldRef) def); + } + } + } + + for (Stmt pathStmt : path) { + if (pathStmt != stmt && pathStmt != relevantDef) { + for (Value stmtDef : pathStmt.getDefs()) { + if (localsUsed.contains(stmtDef)) { + cantAggr = true; + break; + } + if (propagatingInvokeExpr || propagatingFieldRef || propagatingArrayRef) { + if (stmtDef instanceof JFieldRef) { + if (propagatingInvokeExpr) { + cantAggr = true; + break; + } else if (propagatingFieldRef) { + // Can't aggregate a field access if passing a definition of a field + // with the same name, because they might be aliased + for (JFieldRef fieldRef : fieldRefList) { + if (fieldRef.equals((JFieldRef) stmtDef)) { cantAggr = true; break; } } } - } - - if (cantAggr) { - continue; - } - - Value aggregatee = ((JAssignStmt) relevantDef).getRightOp(); - JAssignStmt newStmt = null; - if (assignStmt.getRightOp() instanceof AbstractBinopExpr) { - AbstractBinopExpr rightOp = (AbstractBinopExpr) assignStmt.getRightOp(); - if (rightOp.getOp1() == val) { - AbstractBinopExpr newBinopExpr = rightOp.withOp1((Immediate) aggregatee); - newStmt = - new JAssignStmt<>( - assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); - } else if (rightOp.getOp2() == val) { - AbstractBinopExpr newBinopExpr = rightOp.withOp2((Immediate) aggregatee); - newStmt = - new JAssignStmt<>( - assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); - } - } else { - newStmt = assignStmt.withRValue(aggregatee); - } - if (newStmt != null) { - builder.replaceStmt(stmt, newStmt); - if (graph.getStartingStmt() == relevantDef) { - Stmt newStartingStmt = builder.getStmtGraph().successors(relevantDef).get(0); - builder.setStartingStmt(newStartingStmt); + } else if (stmtDef instanceof JArrayRef) { + if (propagatingInvokeExpr || propagatingArrayRef) { + // Cannot aggregate an invoke expr past an array write and cannot + // aggregate an array read past a write + cantAggr = true; + break; } - builder.removeStmt(relevantDef); } } } } + // Check for intervening side effects due to method calls + if (propagatingInvokeExpr || propagatingFieldRef || propagatingArrayRef) { + for (final Value value : stmt.getUses()) { + if (pathStmt == stmt && value == lhs) { + break; + } + if (value instanceof AbstractInstanceInvokeExpr + || (propagatingInvokeExpr + && (value instanceof JFieldRef || value instanceof JArrayRef))) { + cantAggr = true; + break; + } + } + } + } + + if (cantAggr) { + continue; + } + + Value aggregatee = ((JAssignStmt) relevantDef).getRightOp(); + JAssignStmt newStmt = null; + if (assignStmt.getRightOp() instanceof AbstractBinopExpr) { + AbstractBinopExpr rightOp = (AbstractBinopExpr) assignStmt.getRightOp(); + if (rightOp.getOp1() == val) { + AbstractBinopExpr newBinopExpr = rightOp.withOp1((Immediate) aggregatee); + newStmt = + new JAssignStmt<>( + assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); + } else if (rightOp.getOp2() == val) { + AbstractBinopExpr newBinopExpr = rightOp.withOp2((Immediate) aggregatee); + newStmt = + new JAssignStmt<>( + assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); + } + } else { + newStmt = assignStmt.withRValue(aggregatee); + } + if (newStmt != null) { + builder.replaceStmt(stmt, newStmt); + if (graph.getStartingStmt() == relevantDef) { + Stmt newStartingStmt = builder.getStmtGraph().successors(relevantDef).get(0); + builder.setStartingStmt(newStartingStmt); + } + builder.removeStmt(relevantDef); } } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java index 71a9cb341f3..4ab42d828c0 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java @@ -69,8 +69,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } else if (stmt instanceof JReturnStmt) { for (Value value : stmt.getUses()) { if (value instanceof Local) { - List> defsOfUse = - ((Local) value).getDefsOfLocal(defs); + List> defsOfUse = ((Local) value).getDefsOfLocal(defs); if (defsOfUse.size() == 1) { AbstractDefinitionStmt definitionStmt = defsOfUse.get(0); Value rhs = definitionStmt.getRightOp(); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java index d7df4845b0c..468b1b96827 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java @@ -40,7 +40,11 @@ import sootup.core.types.ReferenceType; import sootup.core.views.View; -/** @author Zun Wang */ +/** + * FIXME: *WHAT DOES IT DO* + * + * @author Zun Wang + */ public class CopyPropagator implements BodyInterceptor { @Override @@ -51,7 +55,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (use instanceof Local) { List defsOfUse = ((Local) use).getDefsForLocalUse(stmtGraph, stmt); - if (isPropagable(defsOfUse)) { + if (isPropatabable(defsOfUse)) { AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) defsOfUse.get(0); Value rhs = defStmt.getRightOp(); // if rhs is a constant, then replace use, if it is possible @@ -86,7 +90,7 @@ private void replaceUse( } } - private boolean isPropagable(List defsOfUse) { + private boolean isPropatabable(List defsOfUse) { // If local is defined just one time, then the propagation of this local available. boolean isPropagateable = false; if (defsOfUse.size() == 1) { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java index a0420d4a11f..8855065666a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java @@ -4,20 +4,27 @@ import categories.Java8Test; import java.io.File; +import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; +import javax.annotation.Nonnull; import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.rules.TestWatcher; import org.junit.runner.Description; +import sootup.core.inputlocation.AnalysisInputLocation; +import sootup.core.inputlocation.ClassLoadingOptions; import sootup.core.model.Body; import sootup.core.model.SootMethod; import sootup.core.signatures.MethodSignature; +import sootup.core.transform.BodyInterceptor; import sootup.core.types.ClassType; import sootup.core.util.Utils; import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; +import sootup.java.bytecode.interceptors.*; import sootup.java.core.JavaIdentifierFactory; import sootup.java.core.JavaProject; import sootup.java.core.JavaSootClass; @@ -61,6 +68,26 @@ protected void starting(Description description) { + File.separator)) .build(); javaView = project.createView(); + + Function, ClassLoadingOptions> bana = + (clazz) -> + new ClassLoadingOptions() { + @Nonnull + @Override + public List getBodyInterceptors() { + final ArrayList interceptors = new ArrayList<>(); + // interceptors.add(new LocalSplitter()); + interceptors.add(new Aggregator()); + interceptors.add(new CopyPropagator()); + // interceptors.add(new UnusedLocalEliminator()); + interceptors.add(new DeadAssignmentEliminator()); + interceptors.add(new UnreachableCodeEliminator()); + + return interceptors; + } + }; + + javaView.configBodyInterceptors(bana); } } From 377cd317b376a6638d87425fff5394c4dc7c4f26 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 11:59:55 +0200 Subject: [PATCH 05/54] improve comment --- .../intraprocedural/UniverseSortedPriorityQueue.java | 7 ++++++- .../sootup/java/bytecode/interceptors/Aggregator.java | 11 ++++++++--- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java index 191150f8fba..93cc4bd1155 100644 --- a/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java +++ b/sootup.analysis/src/main/java/sootup/analysis/intraprocedural/UniverseSortedPriorityQueue.java @@ -28,7 +28,12 @@ import org.slf4j.LoggerFactory; /** - * A fixed size priority queue based on bitsets. The elements of the priority queue are ordered + * TLDR: this class implements sorting a priorityquueue based on the order of items in a List (i.e. + * the Universe") it seems equivalent to using a PriorityQueue with a custom Comparator sorting the + * items ordered using the same Universe. as a rough guess this class is faster as the sorting-call + * seems to be called just once vs multiple times TODO: check that. + * + *

A fixed size priority queue based on bitsets. The elements of the priority queue are ordered * according to a given universe. This priority queue does not permit {@code null} elements. * Inserting of elements that are not part of the universe is also permitted (doing so will result * in a {@code NoSuchElementException}). diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index 4fae9d4722c..91f3e7ad0c2 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -20,6 +20,7 @@ * . * #L% */ + import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.List; @@ -32,7 +33,7 @@ import sootup.core.jimple.common.expr.AbstractInstanceInvokeExpr; import sootup.core.jimple.common.ref.JArrayRef; import sootup.core.jimple.common.ref.JFieldRef; -import sootup.core.jimple.common.stmt.*; +import sootup.core.jimple.common.stmt.AbstractDefinitionStmt; import sootup.core.jimple.common.stmt.JAssignStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; @@ -86,10 +87,14 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (defs.size() != 1) { continue; } - Stmt relevantDef = defs.get(0); + if (!graph.containsNode(relevantDef)) { + continue; + } List path = graph.getExtendedBasicBlockPathBetween(relevantDef, stmt); - + if (path == null) { + continue; + } boolean cantAggr = false; boolean propagatingInvokeExpr = false; boolean propagatingFieldRef = false; From 543bde6bf3077c0af38b5cbd9b276a701ceecd1c Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 12:12:56 +0200 Subject: [PATCH 06/54] set default BodyInterceptors --- .../inputlocation/ClassLoadingOptionKey.java | 25 ----------------- .../BytecodeBodyInterceptors.java | 28 +++++++++---------- .../MinimalBytecodeTestSuiteBase.java | 26 ----------------- 3 files changed, 13 insertions(+), 66 deletions(-) delete mode 100644 sootup.core/src/main/java/sootup/core/inputlocation/ClassLoadingOptionKey.java diff --git a/sootup.core/src/main/java/sootup/core/inputlocation/ClassLoadingOptionKey.java b/sootup.core/src/main/java/sootup/core/inputlocation/ClassLoadingOptionKey.java deleted file mode 100644 index 6345bd7d61a..00000000000 --- a/sootup.core/src/main/java/sootup/core/inputlocation/ClassLoadingOptionKey.java +++ /dev/null @@ -1,25 +0,0 @@ -/*- - * #%L - * Soot - a J*va Optimization Framework - * %% - * Copyright (C) 2020 Markus Schmidt and others - * %% - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Lesser General Public License as - * published by the Free Software Foundation, either version 2.1 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Lesser Public License for more details. - * - * You should have received a copy of the GNU General Lesser Public - * License along with this program. If not, see - * . - * #L% - */ -package sootup.core.inputlocation; - -/** An Extensible way to support configurations for a View */ -public class ClassLoadingOptionKey {} diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java index 6de964013b7..2eab5cc898f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java @@ -29,21 +29,19 @@ /** Built-in sets of {@link BodyInterceptor}s for the bytecode frontend */ public enum BytecodeBodyInterceptors { Default( - /* - new CastAndReturnInliner(), - new UnreachableCodeEliminator(), - // // new LocalSplitter(), - // new Aggregator(), - new TypeAssigner(), - // new LocalNameStandardizer(), - new CopyPropagator(), - new DeadAssignmentEliminator(), - new ConditionalBranchFolder(), - new EmptySwitchEliminator(), - new NopEliminator(), - new UnusedLocalEliminator() - */ - ); + new CastAndReturnInliner(), + new UnreachableCodeEliminator(), + new LocalSplitter(), + new Aggregator(), + new TypeAssigner(), + // ms: is already called from typeassigner? new LocalNameStandardizer(), + new CopyPropagator(), + new DeadAssignmentEliminator(), + new ConditionalBranchFolder(), + new EmptySwitchEliminator(), + new NopEliminator(), + new UnusedLocalEliminator(), + new UnreachableCodeEliminator()); @Nonnull private final List bodyInterceptors; diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java index 8855065666a..0d7200f16c9 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/MinimalBytecodeTestSuiteBase.java @@ -4,23 +4,17 @@ import categories.Java8Test; import java.io.File; -import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.annotation.Nonnull; import org.junit.ClassRule; import org.junit.experimental.categories.Category; import org.junit.rules.TestWatcher; import org.junit.runner.Description; -import sootup.core.inputlocation.AnalysisInputLocation; -import sootup.core.inputlocation.ClassLoadingOptions; import sootup.core.model.Body; import sootup.core.model.SootMethod; import sootup.core.signatures.MethodSignature; -import sootup.core.transform.BodyInterceptor; import sootup.core.types.ClassType; import sootup.core.util.Utils; import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; @@ -68,26 +62,6 @@ protected void starting(Description description) { + File.separator)) .build(); javaView = project.createView(); - - Function, ClassLoadingOptions> bana = - (clazz) -> - new ClassLoadingOptions() { - @Nonnull - @Override - public List getBodyInterceptors() { - final ArrayList interceptors = new ArrayList<>(); - // interceptors.add(new LocalSplitter()); - interceptors.add(new Aggregator()); - interceptors.add(new CopyPropagator()); - // interceptors.add(new UnusedLocalEliminator()); - interceptors.add(new DeadAssignmentEliminator()); - interceptors.add(new UnreachableCodeEliminator()); - - return interceptors; - } - }; - - javaView.configBodyInterceptors(bana); } } From 42803b3daf9b4e7b8a3a73cc2293d9469ee2975e Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 12:33:12 +0200 Subject: [PATCH 07/54] add comment --- .../src/main/java/sootup/core/model/SourceType.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/model/SourceType.java b/sootup.core/src/main/java/sootup/core/model/SourceType.java index 1aba39958c4..ebf48269c1a 100644 --- a/sootup.core/src/main/java/sootup/core/model/SourceType.java +++ b/sootup.core/src/main/java/sootup/core/model/SourceType.java @@ -23,7 +23,10 @@ */ public enum SourceType { - Phantom, - Application, - Library + Phantom, // in soot: code that we have no access to and only can assume the class layout.. TODO: + // which seems to makes no sense anymore as we now reference indirectly to SootClasses + // and dont need a "catch all" Phantom SootClass? + Application, // code that we want to analyze e.g. in call graph generation its traversed + Library // code that is given but should not be analyzed e.g. in call graph generation its not + // traversed } From ebf120faae9584f03473770e42dc543dce1d4dfe Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 12:37:36 +0200 Subject: [PATCH 08/54] remove obsolete BodyGenerator reference which could lead to confusion, as it is used with another instance directly in the ClassSources, too --- sootup.core/src/main/java/sootup/core/model/Body.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 09886d27961..68ebd82e544 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -33,7 +33,6 @@ import sootup.core.jimple.common.ref.*; import sootup.core.jimple.common.stmt.*; import sootup.core.signatures.MethodSignature; -import sootup.core.types.Type; import sootup.core.util.Copyable; import sootup.core.util.EscapedWriter; import sootup.core.util.ImmutableUtils; @@ -365,7 +364,6 @@ public static BodyBuilder builder(@Nonnull Body body, Set modifi */ public static class BodyBuilder { @Nonnull private Set locals = new LinkedHashSet<>(); - @Nonnull private final LocalGenerator localGen = new LocalGenerator(locals); @Nonnull private Set modifiers = Collections.emptySet(); @Nullable private Position position = null; @@ -419,12 +417,6 @@ public BodyBuilder setLocals(@Nonnull Set locals) { return this; } - @Nonnull - public BodyBuilder addLocal(@Nonnull String name, Type type) { - locals.add(localGen.generateLocal(type)); - return this; - } - @Nonnull public BodyBuilder addLocal(@Nonnull Local local) { locals.add(local); From e8ab49733726532242c5cf94f52f362db095bc13 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 12:38:46 +0200 Subject: [PATCH 09/54] style --- sootup.core/src/main/java/sootup/core/model/SourceType.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/model/SourceType.java b/sootup.core/src/main/java/sootup/core/model/SourceType.java index ebf48269c1a..95afd02f847 100644 --- a/sootup.core/src/main/java/sootup/core/model/SourceType.java +++ b/sootup.core/src/main/java/sootup/core/model/SourceType.java @@ -24,9 +24,9 @@ public enum SourceType { Phantom, // in soot: code that we have no access to and only can assume the class layout.. TODO: - // which seems to makes no sense anymore as we now reference indirectly to SootClasses - // and dont need a "catch all" Phantom SootClass? + // which seems to makes no sense anymore as we now reference indirectly to SootClasses + // and dont need a "catch all" Phantom SootClass? Application, // code that we want to analyze e.g. in call graph generation its traversed Library // code that is given but should not be analyzed e.g. in call graph generation its not - // traversed + // traversed } From a280859d06761c792e3a615fdf88ff1daac45ee9 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 13:46:59 +0200 Subject: [PATCH 10/54] make Aggregator work as in the description --- .../bytecode/interceptors/Aggregator.java | 21 ++++++++++++------- .../bytecode/interceptors/CopyPropagator.java | 7 ++++++- .../interceptors/UnusedLocalEliminator.java | 4 +++- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index 91f3e7ad0c2..651e6628d21 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -40,25 +40,30 @@ import sootup.core.transform.BodyInterceptor; import sootup.core.views.View; +/* + * The Jimple Local Aggregator removes some unnecessary copies by combining local variables. + * Essentially, it finds definitions which have only a single use and, if it is safe to do so, + * removes the original definition after replacing the use with the definition's right-hand side. + * At this stage in JimpleBody construction, local aggregation serves largely to remove the copies to and + * from stack variables which simulate load and store instructions in the original bytecode. + * */ public class Aggregator implements BodyInterceptor { - // TODO: [ms] the onlyStackVars flag kind of enable/disables ***everything*** that does - // something in this Interceptor.. check with old soot again (see usage in big if) - boolean onlyStackVars; + // if this is true, only aggregate variables starting with "$" which are the ones which are *not* + // referring to a field of a class + protected boolean dontAggregateFieldLocals; public Aggregator() { this(false); } - public Aggregator(boolean onlyStackVars) { - this.onlyStackVars = onlyStackVars; + public Aggregator(boolean dontAggregateFieldLocals) { + this.dontAggregateFieldLocals = dontAggregateFieldLocals; } /** * Traverse the statements in the given body, looking for aggregation possibilities; that is, * given a def d and a use u, d has no other uses, u has no other defs, collapse d and u. - * - *

option: only-stack-locals; if this is true, only aggregate variables starting with $ */ @Override public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View view) { @@ -76,7 +81,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi continue; } Local lhsLocal = (Local) lhs; - if (onlyStackVars && !lhsLocal.getName().startsWith("$stack")) { + if (dontAggregateFieldLocals && !lhsLocal.getName().startsWith("$")) { continue; } for (Value val : assignStmt.getUses()) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java index 468b1b96827..e84fcfcd4df 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java @@ -41,7 +41,12 @@ import sootup.core.views.View; /** - * FIXME: *WHAT DOES IT DO* + * The CopyPropagator performs cascaded copy propagation. If the propagator encounters situations of + * the form: A: a = ...; ... B: x = a; ... C: ... = ... x; where a and x are each defined only once + * (at A and B, respectively), then it can propagate immediately without checking between B and C + * for redefinitions of a. In this case the propagator is global. Otherwise, if a has multiple + * definitions then the propagator checks for redefinitions and propagates copies only within + * extended basic blocks. * * @author Zun Wang */ diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/UnusedLocalEliminator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/UnusedLocalEliminator.java index fad2df1df43..078c8820de5 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/UnusedLocalEliminator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/UnusedLocalEliminator.java @@ -32,7 +32,9 @@ import sootup.core.views.View; /** - * A BodyTransformer that removes all unused local variables from a given Body. + * The UnusedLocalEliminator removes any unused locals from the method. TODO: ms: it basically + * collects all locals and assigns the collection of Locals to the Body... i.e. its use replaces the + * need of a validator and the manual assignment of the Locals to the body. * * @author Marcus Nachtigall */ From 38b9734319742b5a8f06c8009b72a61a9b2fe766 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 14:20:54 +0200 Subject: [PATCH 11/54] replace generics in JIdentityStmt, JAssignStmt with covariant return types and moving fields to subclasses to remove necessity of internal casts --- .../callgraph/AbstractCallGraphAlgorithm.java | 2 +- .../callgraph/RapidTypeAnalysisAlgorithm.java | 2 +- .../core/graph/MutableBlockStmtGraph.java | 21 ++++---- .../main/java/sootup/core/jimple/Jimple.java | 9 ++-- .../java/sootup/core/jimple/basic/Local.java | 8 +-- .../common/stmt/AbstractDefinitionStmt.java | 23 +++----- .../core/jimple/common/stmt/JAssignStmt.java | 41 ++++++++++----- .../jimple/common/stmt/JIdentityStmt.java | 37 +++++++++---- .../jimple/visitor/AbstractStmtVisitor.java | 4 +- .../jimple/visitor/ReplaceUseStmtVisitor.java | 4 +- .../core/jimple/visitor/StmtVisitor.java | 4 +- .../src/main/java/sootup/core/model/Body.java | 23 ++++---- .../bytecode/frontend/AsmMethodSource.java | 52 ++++++++----------- .../java/bytecode/frontend/StackFrame.java | 16 +++--- .../bytecode/interceptors/Aggregator.java | 14 +++-- .../interceptors/CastAndReturnInliner.java | 2 +- .../ConstantPropagatorAndFolder.java | 4 +- .../bytecode/interceptors/CopyPropagator.java | 2 +- .../DeadAssignmentEliminator.java | 8 +-- .../bytecode/interceptors/LocalPacker.java | 6 +-- .../bytecode/interceptors/LocalSplitter.java | 6 +-- .../StaticSingleAssignmentFormer.java | 10 ++-- .../typeresolving/CastCounter.java | 11 ++-- .../typeresolving/TypeChecker.java | 6 +-- .../typeresolving/TypeResolver.java | 6 +-- .../java/core/model/SootMethodTest.java | 2 +- 26 files changed, 166 insertions(+), 157 deletions(-) diff --git a/sootup.callgraph/src/main/java/sootup/callgraph/AbstractCallGraphAlgorithm.java b/sootup.callgraph/src/main/java/sootup/callgraph/AbstractCallGraphAlgorithm.java index eb899f1e0af..eeef50551a2 100644 --- a/sootup.callgraph/src/main/java/sootup/callgraph/AbstractCallGraphAlgorithm.java +++ b/sootup.callgraph/src/main/java/sootup/callgraph/AbstractCallGraphAlgorithm.java @@ -246,7 +246,7 @@ protected Stream resolveAllStaticInitializerCallsFromSourceMeth // constructor calls if (stmt instanceof JAssignStmt) { - Value rightOp = ((JAssignStmt) stmt).getRightOp(); + Value rightOp = ((JAssignStmt) stmt).getRightOp(); instantiateVisitor.init(); rightOp.accept(instantiateVisitor); ClassType classType = instantiateVisitor.getResult(); diff --git a/sootup.callgraph/src/main/java/sootup/callgraph/RapidTypeAnalysisAlgorithm.java b/sootup.callgraph/src/main/java/sootup/callgraph/RapidTypeAnalysisAlgorithm.java index 69fee270cab..d92abca3183 100644 --- a/sootup.callgraph/src/main/java/sootup/callgraph/RapidTypeAnalysisAlgorithm.java +++ b/sootup.callgraph/src/main/java/sootup/callgraph/RapidTypeAnalysisAlgorithm.java @@ -108,7 +108,7 @@ protected void collectInstantiatedClassesInMethod(SootMethod method) { Set instantiated = method.getBody().getStmts().stream() .filter(stmt -> stmt instanceof JAssignStmt) - .map(stmt -> ((JAssignStmt) stmt).getRightOp()) + .map(stmt -> ((JAssignStmt) stmt).getRightOp()) .filter(value -> value instanceof JNewExpr) .map(value -> ((JNewExpr) value).getType()) .collect(Collectors.toSet()); diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 285170c3033..7a264665d07 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -152,14 +152,14 @@ public void initializeWith( final boolean isRemoved = currentTrapMap.remove(exceptionType, trap); final PriorityQueue overridenTrapHandlers = overlappingTraps.get(exceptionType); if (overridenTrapHandlers != null) { - if (!isRemoved && overridenTrapHandlers.size() > 0) { + if (!isRemoved && !overridenTrapHandlers.isEmpty()) { // check if theres an overlapping trap that has a less specific TrapRange which is // ending before it gets the active exception information again // not logical as a compiler output... but possible. overridenTrapHandlers.remove(trap); } - if (overridenTrapHandlers.size() > 0) { + if (!overridenTrapHandlers.isEmpty()) { currentTrapMap.put(exceptionType, overridenTrapHandlers.poll()); } } @@ -178,7 +178,7 @@ public void initializeWith( overlappingTraps.computeIfAbsent( trap.getExceptionType(), k -> - new PriorityQueue( + new PriorityQueue<>( (trapA, trapB) -> { if (trapA.getEndStmt() == trapB.getEndStmt()) { final Integer startIdxA = trapstmtToIdx.get(trapA.getBeginStmt()); @@ -422,7 +422,7 @@ private MutableBasicBlock addBlockInternal( final Iterator iterator = stmts.iterator(); final Stmt node = iterator.next(); MutableBasicBlock block = getOrCreateBlock(node); - if (block.getHead() != node || block.getSuccessors().size() > 0) { + if (block.getHead() != node || !block.getSuccessors().isEmpty()) { throw new IllegalArgumentException( "The first Stmt in the List is already in the StmtGraph and and is not the head of a Block where currently no successor are set, yet."); } else if (block.getStmtCount() > 1) { @@ -662,10 +662,7 @@ protected boolean isMergeable( return false; } // check if the same traps are applied to both blocks - if (!firstBlock.getExceptionalSuccessors().equals(followingBlock.getExceptionalSuccessors())) { - return false; - } - return true; + return firstBlock.getExceptionalSuccessors().equals(followingBlock.getExceptionalSuccessors()); } /** trys to merge the second block into the first one if possible */ @@ -998,7 +995,7 @@ protected void removeBlockBorderEdgesInternal( // TODO: reuse tryMerge*Block? // add BlockB to BlockA if blockA has no branchingstmt as tail && same traps - if (blockOfFrom.getStmts().size() > 0 && from == blockOfFrom.getTail()) { + if (!blockOfFrom.getStmts().isEmpty() && from == blockOfFrom.getTail()) { if (blockOfFrom.getPredecessors().size() == 1) { MutableBasicBlock singlePreviousBlock = blockOfFrom.getPredecessors().get(0); if (!singlePreviousBlock.getTail().branches() && singlePreviousBlock != blockOfFrom) { @@ -1018,7 +1015,7 @@ protected void removeBlockBorderEdgesInternal( // remove outgoing connections from blockA if from stmt is the tail if (!from.branches()) { - if (blockOfFrom.getStmts().size() > 0 && blockOfFrom.getSuccessors().size() == 1) { + if (!blockOfFrom.getStmts().isEmpty() && blockOfFrom.getSuccessors().size() == 1) { // merge previous block if possible i.e. no branchingstmt as tail && same traps && no // other predesccorblocks MutableBasicBlock singleSuccessorBlock = blockOfFrom.getSuccessors().get(0); @@ -1156,7 +1153,7 @@ public List exceptionalPredecessors(@Nonnull MutableBasicBlock block) { Stmt head = block.getHead(); if (!(head instanceof JIdentityStmt - && ((JIdentityStmt) head).getRightOp() instanceof JCaughtExceptionRef)) { + && ((JIdentityStmt) head).getRightOp() instanceof JCaughtExceptionRef)) { // only an exception handler stmt can have exceptional predecessors return Collections.emptyList(); } @@ -1174,7 +1171,7 @@ public List exceptionalPredecessorBlocks(@Nonnull MutableBasi Stmt head = block.getHead(); if (!(head instanceof JIdentityStmt - && ((JIdentityStmt) head).getRightOp() instanceof JCaughtExceptionRef)) { + && ((JIdentityStmt) head).getRightOp() instanceof JCaughtExceptionRef)) { // only an exception handler stmt can have exceptional predecessors return Collections.emptyList(); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java index 35eda4a4ccb..11f3d69f4ce 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java +++ b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java @@ -519,15 +519,14 @@ public static JIfStmt newIfStmt(AbstractConditionExpr condition, StmtPositionInf } /** Constructs a IdentityStmt(Local, IdentityRef) grammar chunk. */ - public static JIdentityStmt newIdentityStmt( + public static JIdentityStmt newIdentityStmt( Local local, L identityRef, StmtPositionInfo posInfo) { - return new JIdentityStmt<>(local, identityRef, posInfo); + return new JIdentityStmt(local, identityRef, posInfo); } /** Constructs a AssignStmt(Variable, RValue) grammar chunk. */ - public static JAssignStmt newAssignStmt( - L variable, R rvalue, StmtPositionInfo posInfo) { - return new JAssignStmt<>(variable, rvalue, posInfo); + public static JAssignStmt newAssignStmt(Value variable, Value rvalue, StmtPositionInfo posInfo) { + return new JAssignStmt(variable, rvalue, posInfo); } /** Constructs a InvokeStmt(InvokeExpr) grammar chunk. */ diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java index 0ca645c7e8b..f8dfb861b3f 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java @@ -124,12 +124,12 @@ public Position getPosition() { } /** returns the returned List can contain: Locals, JFieldRefs, JArrayRefs */ - public List> getDefsOfLocal(List defs) { - List> localDefs = new ArrayList<>(); + public List getDefsOfLocal(List defs) { + List localDefs = new ArrayList<>(); for (Stmt stmt : defs) { if (stmt instanceof AbstractDefinitionStmt - && ((AbstractDefinitionStmt) stmt).getLeftOp().equals(this)) { - localDefs.add((AbstractDefinitionStmt) stmt); + && ((AbstractDefinitionStmt) stmt).getLeftOp().equals(this)) { + localDefs.add((AbstractDefinitionStmt) stmt); } } return localDefs; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index 86b6a1de640..e109cafb518 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -30,27 +30,17 @@ import sootup.core.jimple.basic.Value; import sootup.core.types.Type; -public abstract class AbstractDefinitionStmt extends Stmt { +public abstract class AbstractDefinitionStmt extends Stmt { - @Nonnull private final L leftOp; - @Nonnull private final R rightOp; - - AbstractDefinitionStmt( - @Nonnull L leftOp, @Nonnull R rightOp, @Nonnull StmtPositionInfo positionInfo) { + AbstractDefinitionStmt(@Nonnull StmtPositionInfo positionInfo) { super(positionInfo); - this.leftOp = leftOp; - this.rightOp = rightOp; } @Nonnull - public final L getLeftOp() { - return leftOp; - } + public abstract Value getLeftOp(); @Nonnull - public R getRightOp() { - return rightOp; - } + public abstract Value getRightOp(); @Nonnull public Type getType() { @@ -61,14 +51,15 @@ public Type getType() { @Nonnull public List getDefs() { final List defs = new ArrayList<>(); - defs.add(leftOp); + defs.add(getLeftOp()); return defs; } @Override @Nonnull public final List getUses() { - final List defsuses = leftOp.getUses(); + final List defsuses = getLeftOp().getUses(); + final Value rightOp = getRightOp(); final List uses = rightOp.getUses(); List list = new ArrayList<>(defsuses.size() + uses.size() + 1); list.addAll(defsuses); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java index 389f13edb12..20c6be7cd76 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java @@ -1,7 +1,7 @@ package sootup.core.jimple.common.stmt; /*- - * #%L + * #%Value * Soot - a J*va Optimization Framework * %% * Copyright (C) 1997-2020 Etienne Gagnon, Linghui Luo, Markus Schmidt and others @@ -19,7 +19,7 @@ * You should have received a copy of the GNU General Lesser Public * License along with this program. If not, see * . - * #L% + * #Value% */ import javax.annotation.Nonnull; @@ -35,8 +35,10 @@ import sootup.core.util.printer.StmtPrinter; /** Represents the assignment of one value to another */ -public final class JAssignStmt - extends AbstractDefinitionStmt implements Copyable { +public final class JAssignStmt extends AbstractDefinitionStmt implements Copyable { + + @Nonnull final Value leftOp; + @Nonnull final Value rightOp; /** * Instantiates a new JAssignStmt. @@ -45,8 +47,11 @@ public final class JAssignStmt * @param rValue the value on the right side of the assign statement. */ public JAssignStmt( - @Nonnull L variable, @Nonnull R rValue, @Nonnull StmtPositionInfo positionInfo) { - super(variable, rValue, positionInfo); + @Nonnull Value variable, @Nonnull Value rValue, @Nonnull StmtPositionInfo positionInfo) { + super(positionInfo); + leftOp = variable; + rightOp = rValue; + if (!validateVariable(variable)) { throw new RuntimeException( "Illegal Assignment statement. Make sure that left hand side has a valid operand."); @@ -198,18 +203,30 @@ public int equivHashCode() { } @Nonnull - public JAssignStmt withVariable(@Nonnull N variable) { - return new JAssignStmt<>(variable, getRightOp(), getPositionInfo()); + public JAssignStmt withVariable(@Nonnull Value variable) { + return new JAssignStmt(variable, getRightOp(), getPositionInfo()); + } + + @Nonnull + public JAssignStmt withRValue(@Nonnull Value rValue) { + return new JAssignStmt(getLeftOp(), rValue, getPositionInfo()); } @Nonnull - public JAssignStmt withRValue(@Nonnull N rValue) { - return new JAssignStmt<>(getLeftOp(), rValue, getPositionInfo()); + public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { + return new JAssignStmt(getLeftOp(), getRightOp(), positionInfo); } @Nonnull - public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { - return new JAssignStmt<>(getLeftOp(), getRightOp(), positionInfo); + @Override + public Value getLeftOp() { + return leftOp; + } + + @Nonnull + @Override + public Value getRightOp() { + return rightOp; } @Override diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java index 2497b433b05..ad6553ef58c 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java @@ -31,12 +31,18 @@ import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JIdentityStmt extends AbstractDefinitionStmt - implements Copyable { +public final class JIdentityStmt extends AbstractDefinitionStmt implements Copyable { + + @Nonnull final Local leftOp; + @Nonnull final IdentityRef rightOp; public JIdentityStmt( - @Nonnull Local local, @Nonnull T identityValue, @Nonnull StmtPositionInfo positionInfo) { - super(local, identityValue, positionInfo); + @Nonnull Local local, + @Nonnull IdentityRef identityValue, + @Nonnull StmtPositionInfo positionInfo) { + super(positionInfo); + leftOp = local; + rightOp = identityValue; } @Override @@ -44,6 +50,17 @@ public String toString() { return getLeftOp() + " := " + getRightOp(); } + @Nonnull + public Local getLeftOp() { + return leftOp; + } + + @Nonnull + @Override + public IdentityRef getRightOp() { + return rightOp; + } + @Override public void toString(@Nonnull StmtPrinter up) { getLeftOp().toString(up); @@ -67,18 +84,18 @@ public int equivHashCode() { } @Nonnull - public JIdentityStmt withLocal(@Nonnull Local local) { - return new JIdentityStmt<>(local, getRightOp(), getPositionInfo()); + public JIdentityStmt withLocal(@Nonnull Local local) { + return new JIdentityStmt(local, getRightOp(), getPositionInfo()); } @Nonnull - public JIdentityStmt withIdentityValue(@Nonnull N identityValue) { - return new JIdentityStmt<>(getLeftOp(), identityValue, getPositionInfo()); + public JIdentityStmt withIdentityValue(@Nonnull IdentityRef identityValue) { + return new JIdentityStmt(getLeftOp(), identityValue, getPositionInfo()); } @Nonnull - public JIdentityStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { - return new JIdentityStmt<>(getLeftOp(), getRightOp(), positionInfo); + public JIdentityStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { + return new JIdentityStmt(getLeftOp(), getRightOp(), positionInfo); } @Override diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/AbstractStmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/AbstractStmtVisitor.java index 10b63ab7c28..2390bbace15 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/AbstractStmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/AbstractStmtVisitor.java @@ -39,12 +39,12 @@ public void caseInvokeStmt(@Nonnull JInvokeStmt stmt) { } @Override - public void caseAssignStmt(@Nonnull JAssignStmt stmt) { + public void caseAssignStmt(@Nonnull JAssignStmt stmt) { defaultCaseStmt(stmt); } @Override - public void caseIdentityStmt(@Nonnull JIdentityStmt stmt) { + public void caseIdentityStmt(@Nonnull JIdentityStmt stmt) { defaultCaseStmt(stmt); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java index b81defbdeb3..0529d096eb8 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java @@ -69,7 +69,7 @@ public void caseInvokeStmt(@Nonnull JInvokeStmt stmt) { } @Override - public void caseAssignStmt(@Nonnull JAssignStmt stmt) { + public void caseAssignStmt(@Nonnull JAssignStmt stmt) { // uses on the def side.. final Value leftOp = stmt.getLeftOp(); @@ -109,7 +109,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { } @Override - public void caseIdentityStmt(@Nonnull JIdentityStmt stmt) { + public void caseIdentityStmt(@Nonnull JIdentityStmt stmt) { defaultCaseStmt(stmt); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/StmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/StmtVisitor.java index 83e47a70355..a1158bd8712 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/StmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/StmtVisitor.java @@ -36,9 +36,9 @@ public interface StmtVisitor extends Visitor { void caseInvokeStmt(JInvokeStmt stmt); - void caseAssignStmt(JAssignStmt stmt); + void caseAssignStmt(JAssignStmt stmt); - void caseIdentityStmt(JIdentityStmt stmt); + void caseIdentityStmt(JIdentityStmt stmt); void caseEnterMonitorStmt(JEnterMonitorStmt stmt); diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 68ebd82e544..8d4d682690f 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -86,7 +86,6 @@ private Body( this.graph = /* FIXME: [ms] make immutable when availabe */ new MutableBlockStmtGraph(stmtGraph).unmodifiableStmtGraph(); this.position = position; - // FIXME: [JMP] Virtual method call in constructor checkInit(); } @@ -159,7 +158,7 @@ public List getTraps() { public Stmt getThisStmt() { for (Stmt u : getStmts()) { if (u instanceof JIdentityStmt) { - if (((JIdentityStmt) u).getRightOp() instanceof JThisRef) { + if (((JIdentityStmt) u).getRightOp() instanceof JThisRef) { return u; } } else { @@ -175,23 +174,23 @@ public Stmt getThisStmt() { /** Return LHS of the first identity stmt assigning from \@this. */ @Nullable public Local getThisLocal() { - final JIdentityStmt thisStmt = (JIdentityStmt) getThisStmt(); + final JIdentityStmt thisStmt = (JIdentityStmt) getThisStmt(); if (thisStmt == null) { return null; } - return thisStmt.getLeftOp(); + return (Local) thisStmt.getLeftOp(); } /** Return LHS of the first identity stmt assigning from \@parameter i. */ - @Nullable + @Nonnull public Local getParameterLocal(int i) { for (Stmt s : getStmts()) { if (s instanceof JIdentityStmt) { - if (((JIdentityStmt) s).getRightOp() instanceof JParameterRef) { - JIdentityStmt idStmt = (JIdentityStmt) s; + if (((JIdentityStmt) s).getRightOp() instanceof JParameterRef) { + JIdentityStmt idStmt = (JIdentityStmt) s; JParameterRef pr = (JParameterRef) idStmt.getRightOp(); if (pr.getIndex() == i) { - return idStmt.getLeftOp(); + return (Local) idStmt.getLeftOp(); } } } else { @@ -199,7 +198,7 @@ public Local getParameterLocal(int i) { // break; } } - return null; + throw new IllegalArgumentException("There exists no Parameter Local with index " + i + "!"); } /** @@ -216,10 +215,10 @@ public Collection getParameterLocals() { // fixed index positions at the beginning? for (Stmt u : graph.getNodes()) { if (u instanceof JIdentityStmt) { - JIdentityStmt idStmt = (JIdentityStmt) u; + JIdentityStmt idStmt = (JIdentityStmt) u; if (idStmt.getRightOp() instanceof JParameterRef) { JParameterRef pr = (JParameterRef) idStmt.getRightOp(); - retVal.add(pr.getIndex(), idStmt.getLeftOp()); + retVal.add(pr.getIndex(), (Local) idStmt.getLeftOp()); } } /* if we restrict/define that IdentityStmts MUST be at the beginnging. @@ -439,7 +438,7 @@ public void replaceLocal(@Nonnull Local oldLocal, @Nonnull Local newLocal) { for (Value def : defs) { if (def == oldLocal || def.getUses().contains(oldLocal)) { if (currStmt instanceof AbstractDefinitionStmt) { - currStmt = ((AbstractDefinitionStmt) currStmt).withNewDef(newLocal); + currStmt = ((AbstractDefinitionStmt) currStmt).withNewDef(newLocal); } } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java index 536a6619dfb..9a171328da4 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java @@ -133,9 +133,7 @@ public class AsmMethodSource extends JSRInlinerAdapter implements BodySource { @Nonnull private final Set inlineExceptionLabels = new HashSet<>(); - @Nonnull - private final Map> - inlineExceptionHandlers = new HashMap<>(); + @Nonnull private final Map inlineExceptionHandlers = new HashMap<>(); @Nonnull private final Map labelsToStmt = new HashMap<>(); @@ -234,8 +232,9 @@ public Body resolveBody(@Nonnull Iterable modifierIt) { } // propagate position information - if (graph.getNodes().size() > 0) { - Position firstStmtPos = graph.getStartingStmt().getPositionInfo().getStmtPosition(); + final Stmt startingStmt = graph.getStartingStmt(); + if (!graph.getNodes().isEmpty() && startingStmt != null) { + Position firstStmtPos = startingStmt.getPositionInfo().getStmtPosition(); bodyBuilder.setPosition( new FullPosition( firstStmtPos.getFirstLine(), @@ -389,8 +388,7 @@ private void addReadOperandAssignments_internal(BiFunction asssignStmt = - Jimple.newAssignStmt(stackLocal, opValue, getStmtPositionInfo()); + JAssignStmt asssignStmt = Jimple.newAssignStmt(stackLocal, opValue, getStmtPositionInfo()); setStmt(operand.insn, asssignStmt); operand.updateUsages(); @@ -453,8 +451,7 @@ private void convertPutFieldInsn(@Nonnull FieldInsnNode insn) { } opr = new Operand(insn, val, this); frame.setOut(opr); - JAssignStmt as = - Jimple.newAssignStmt(val, rvalue.stackOrValue(), getStmtPositionInfo()); + JAssignStmt as = Jimple.newAssignStmt(val, rvalue.stackOrValue(), getStmtPositionInfo()); setStmt(insn, as); rvalue.addUsageInStmt(as); } else { @@ -562,8 +559,7 @@ private void convertArrayStoreInsn(@Nonnull InsnNode insn) { JArrayRef ar = JavaJimple.getInstance() .newArrayRef((Local) baseOp.stackOrValue(), (Immediate) indexOp.stackOrValue()); - JAssignStmt as = - Jimple.newAssignStmt(ar, valueOp.stackOrValue(), getStmtPositionInfo()); + JAssignStmt as = Jimple.newAssignStmt(ar, valueOp.stackOrValue(), getStmtPositionInfo()); frame.setIn(valueOp, indexOp, baseOp); setStmt(insn, as); valueOp.addUsageInStmt(as); @@ -1571,7 +1567,7 @@ private void convertVarStoreInsn(@Nonnull VarInsnNode insn) { Operand opr = dword ? operandStack.popDual() : operandStack.pop(); Local local = getOrCreateLocal(insn.var); if (!insnToStmt.containsKey(insn)) { - AbstractDefinitionStmt as = + AbstractDefinitionStmt as = Jimple.newAssignStmt(local, opr.stackOrValue(), getStmtPositionInfo()); frame.setIn(opr); setStmt(insn, as); @@ -1621,8 +1617,7 @@ private void convertLabel(@Nonnull LabelNode ln) { if (out == null) { JCaughtExceptionRef ref = JavaJimple.getInstance().newCaughtExceptionRef(); Local stack = newStackLocal(); - AbstractDefinitionStmt as = - Jimple.newIdentityStmt(stack, ref, getStmtPositionInfo()); + JIdentityStmt as = Jimple.newIdentityStmt(stack, ref, getStmtPositionInfo()); opr = new Operand(ln, ref, this); opr.stackLocal = stack; frame.setOut(opr); @@ -1702,8 +1697,7 @@ private void convert() { // Catch the exception JCaughtExceptionRef ref = JavaJimple.getInstance().newCaughtExceptionRef(); Local local = newStackLocal(); - AbstractDefinitionStmt as = - Jimple.newIdentityStmt(local, ref, getStmtPositionInfo()); + JIdentityStmt as = Jimple.newIdentityStmt(local, ref, getStmtPositionInfo()); Operand opr = new Operand(handlerNode, ref, this); opr.stackLocal = local; @@ -1882,7 +1876,7 @@ private List buildPreambleLocals(Body.BodyBuilder bodyBuilder) { if (!bodyBuilder.getModifiers().contains(MethodModifier.STATIC)) { JavaLocal thisLocal = JavaJimple.newLocal(determineLocalName(localIdx), declaringClass); locals.set(localIdx++, thisLocal); - final JIdentityStmt stmt = + final JIdentityStmt stmt = Jimple.newIdentityStmt(thisLocal, Jimple.newThisRef(declaringClass), methodPosInfo); preambleBlock.add(stmt); } @@ -1898,7 +1892,7 @@ private List buildPreambleLocals(Body.BodyBuilder bodyBuilder) { invisibleParameterAnnotations == null ? null : invisibleParameterAnnotations[i])); locals.set(localIdx, local); - final JIdentityStmt stmt = + final JIdentityStmt stmt = Jimple.newIdentityStmt(local, Jimple.newParameterRef(parameterType, i), methodPosInfo); preambleBlock.add(stmt); @@ -1979,7 +1973,7 @@ private void arrangeStmts( danglingLabel.forEach(l -> labelsToStmt.put(l, targetStmt)); if (isLabelNode) { // If the targetStmt is an exception handler, register the starting Stmt for it - JIdentityStmt identityRef = findIdentityRefInStmtContainer(stmt); + JIdentityStmt identityRef = findIdentityRefInStmtContainer(stmt); if (identityRef != null && identityRef.getRightOp() instanceof JCaughtExceptionRef) { danglingLabel.forEach(label -> trapHandler.put(label, identityRef)); } @@ -2021,10 +2015,9 @@ private void arrangeStmts( // Emit the inline exception handler blocks i.e. those that are reachable without exceptional // flow // FIXME:[ms] the following code seems odd.. we need a testcase to test inlineexceptionhandling! - for (Entry> entry : - inlineExceptionHandlers.entrySet()) { + for (Entry entry : inlineExceptionHandlers.entrySet()) { - AbstractDefinitionStmt handlerStmt = entry.getValue(); + JIdentityStmt handlerStmt = entry.getValue(); emitStmt(handlerStmt, stmtList); trapHandler.put(entry.getKey(), handlerStmt); // TODO: update handlerStmts positioninfo! @@ -2052,13 +2045,13 @@ private void emitStmt(@Nonnull Stmt handlerStmt, @Nonnull List block) { } @Nullable - private JIdentityStmt findIdentityRefInStmtContainer(@Nonnull Stmt stmt) { + private JIdentityStmt findIdentityRefInStmtContainer(@Nonnull Stmt stmt) { if (stmt instanceof JIdentityStmt) { - return (JIdentityStmt) stmt; + return (JIdentityStmt) stmt; } else if (stmt instanceof StmtContainer) { for (Stmt stmtEntry : ((StmtContainer) stmt).getStmts()) { if (stmtEntry instanceof JIdentityStmt) { - return (JIdentityStmt) stmtEntry; + return (JIdentityStmt) stmtEntry; } } } @@ -2110,11 +2103,10 @@ void replaceStmt(@Nonnull Stmt oldStmt, Stmt newStmt) { replacedStmt.put(oldStmt, newStmt); if (oldStmt instanceof BranchingStmt) { - List branchLabels = stmtsThatBranchToLabel.get((BranchingStmt) oldStmt); - if (branchLabels != null) { - branchLabels.forEach(bl -> stmtsThatBranchToLabel.put((BranchingStmt) newStmt, bl)); - stmtsThatBranchToLabel.removeAll(oldStmt); - } + final BranchingStmt branchingStmt = (BranchingStmt) oldStmt; + List branchLabels = stmtsThatBranchToLabel.get(branchingStmt); + branchLabels.forEach(bl -> stmtsThatBranchToLabel.put((BranchingStmt) newStmt, bl)); + stmtsThatBranchToLabel.removeAll(branchingStmt); } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StackFrame.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StackFrame.java index c8b91ae7278..4474eef5a7e 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StackFrame.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StackFrame.java @@ -109,14 +109,14 @@ void mergeIn(int lineNumber, @Nonnull Operand... oprs) { if (stack != null) { if (newOp.stackLocal == null) { newOp.stackLocal = stack; - JAssignStmt as = Jimple.newAssignStmt(stack, newOp.value, positionInfo); + JAssignStmt as = Jimple.newAssignStmt(stack, newOp.value, positionInfo); src.setStmt(newOp.insn, as); newOp.updateUsages(); } else { final Value rvalue = newOp.stackOrValue(); // check for self/identity assignments and ignore them if (stack != rvalue) { - JAssignStmt as = Jimple.newAssignStmt(stack, rvalue, positionInfo); + JAssignStmt as = Jimple.newAssignStmt(stack, rvalue, positionInfo); src.mergeStmts(newOp.insn, as); } } @@ -141,12 +141,12 @@ void mergeIn(int lineNumber, @Nonnull Operand... oprs) { } if (prevOp.stackLocal == null) { prevOp.stackLocal = stack; - JAssignStmt as = Jimple.newAssignStmt(stack, prevOp.value, positionInfo); + JAssignStmt as = Jimple.newAssignStmt(stack, prevOp.value, positionInfo); src.setStmt(prevOp.insn, as); } else { Stmt u = src.getStmt(prevOp.insn); - AbstractDefinitionStmt as = - (AbstractDefinitionStmt) + AbstractDefinitionStmt as = + (AbstractDefinitionStmt) (u instanceof StmtContainer ? ((StmtContainer) u).getFirstStmt() : u); Value lvb = as.getLeftOp(); assert lvb == prevOp.stackLocal : "Invalid stack local!"; @@ -157,13 +157,13 @@ void mergeIn(int lineNumber, @Nonnull Operand... oprs) { if (newOp.stackLocal != stack) { if (newOp.stackLocal == null) { newOp.stackLocal = stack; - JAssignStmt as = Jimple.newAssignStmt(stack, newOp.value, positionInfo); + JAssignStmt as = Jimple.newAssignStmt(stack, newOp.value, positionInfo); src.setStmt(newOp.insn, as); } else { Stmt u = src.getStmt(newOp.insn); if (!(u instanceof JNopStmt)) { - AbstractDefinitionStmt as = - (AbstractDefinitionStmt) + AbstractDefinitionStmt as = + (AbstractDefinitionStmt) (u instanceof StmtContainer ? ((StmtContainer) u).getFirstStmt() : u); Value lvb = as.getLeftOp(); assert lvb == newOp.stackLocal : "Invalid stack local!"; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index 651e6628d21..55ecc772878 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -75,7 +75,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (!(stmt instanceof JAssignStmt)) { continue; } - final JAssignStmt assignStmt = (JAssignStmt) stmt; + final JAssignStmt assignStmt = (JAssignStmt) stmt; Value lhs = assignStmt.getLeftOp(); if (!(lhs instanceof Local)) { continue; @@ -88,7 +88,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (!(val instanceof Local)) { continue; } - List> defs = ((Local) val).getDefsOfLocal(stmts); + List defs = ((Local) val).getDefsOfLocal(stmts); if (defs.size() != 1) { continue; } @@ -176,20 +176,18 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi continue; } - Value aggregatee = ((JAssignStmt) relevantDef).getRightOp(); - JAssignStmt newStmt = null; + Value aggregatee = ((JAssignStmt) relevantDef).getRightOp(); + JAssignStmt newStmt = null; if (assignStmt.getRightOp() instanceof AbstractBinopExpr) { AbstractBinopExpr rightOp = (AbstractBinopExpr) assignStmt.getRightOp(); if (rightOp.getOp1() == val) { AbstractBinopExpr newBinopExpr = rightOp.withOp1((Immediate) aggregatee); newStmt = - new JAssignStmt<>( - assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); + new JAssignStmt(assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); } else if (rightOp.getOp2() == val) { AbstractBinopExpr newBinopExpr = rightOp.withOp2((Immediate) aggregatee); newStmt = - new JAssignStmt<>( - assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); + new JAssignStmt(assignStmt.getLeftOp(), newBinopExpr, assignStmt.getPositionInfo()); } } else { newStmt = assignStmt.withRValue(aggregatee); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CastAndReturnInliner.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CastAndReturnInliner.java index 180c30bc9f8..465813b42c6 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CastAndReturnInliner.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CastAndReturnInliner.java @@ -77,7 +77,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (!(successorOfGoto instanceof JAssignStmt)) { continue; } - JAssignStmt assign = (JAssignStmt) successorOfGoto; + JAssignStmt assign = (JAssignStmt) successorOfGoto; if (!(assign.getRightOp() instanceof JCastExpr)) { continue; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java index 4ab42d828c0..bafad41f397 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java @@ -69,9 +69,9 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } else if (stmt instanceof JReturnStmt) { for (Value value : stmt.getUses()) { if (value instanceof Local) { - List> defsOfUse = ((Local) value).getDefsOfLocal(defs); + List defsOfUse = ((Local) value).getDefsOfLocal(defs); if (defsOfUse.size() == 1) { - AbstractDefinitionStmt definitionStmt = defsOfUse.get(0); + AbstractDefinitionStmt definitionStmt = defsOfUse.get(0); Value rhs = definitionStmt.getRightOp(); if (rhs instanceof NumericConstant || rhs instanceof StringConstant diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java index e84fcfcd4df..2ca2e0cde8f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java @@ -61,7 +61,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi List defsOfUse = ((Local) use).getDefsForLocalUse(stmtGraph, stmt); if (isPropatabable(defsOfUse)) { - AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) defsOfUse.get(0); + AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) defsOfUse.get(0); Value rhs = defStmt.getRightOp(); // if rhs is a constant, then replace use, if it is possible if (rhs instanceof Constant) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java index 4dc6a6a51b1..7f68381641e 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java @@ -83,7 +83,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi boolean isEssential = true; if (stmt instanceof JAssignStmt) { - JAssignStmt assignStmt = (JAssignStmt) stmt; + JAssignStmt assignStmt = (JAssignStmt) stmt; Value lhs = assignStmt.getLeftOp(); Value rhs = assignStmt.getRightOp(); @@ -207,10 +207,10 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (containsInvoke) { allUses = Body.collectUses(builder.getStmts()); // Eliminate dead assignments from invokes such as x = f(), where x is no longer used - List> postProcess = new ArrayList<>(); + List postProcess = new ArrayList<>(); for (Stmt stmt : stmts) { if (stmt instanceof JAssignStmt) { - JAssignStmt assignStmt = (JAssignStmt) stmt; + JAssignStmt assignStmt = (JAssignStmt) stmt; if (assignStmt.containsInvokeExpr()) { // Just find one use of local which is essential boolean deadAssignment = true; @@ -228,7 +228,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } } - for (JAssignStmt assignStmt : postProcess) { + for (JAssignStmt assignStmt : postProcess) { // Transform it into a simple invoke Stmt newInvoke = Jimple.newInvokeStmt(assignStmt.getInvokeExpr(), assignStmt.getPositionInfo()); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalPacker.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalPacker.java index 035533eb747..12ae185a45e 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalPacker.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalPacker.java @@ -115,7 +115,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi localToNewLocal.put(ori, newLocal); } } - newStmt = ((AbstractDefinitionStmt) newStmt).withNewDef(newLocal); + newStmt = ((AbstractDefinitionStmt) newStmt).withNewDef(newLocal); } if (!stmt.equals(newStmt)) { builder.replaceStmt(stmt, newStmt); @@ -144,8 +144,8 @@ private Map assignLocalsColor(Body.BodyBuilder builder) { // assign each parameter local a color (local from IdentityStmt) for (Stmt stmt : builder.getStmts()) { if (stmt instanceof JIdentityStmt) { - if (((JIdentityStmt) stmt).getLeftOp() instanceof Local) { - Local l = (Local) ((JIdentityStmt) stmt).getLeftOp(); + if (((JIdentityStmt) stmt).getLeftOp() instanceof Local) { + Local l = (Local) ((JIdentityStmt) stmt).getLeftOp(); Type type = l.getType(); int count = typeToColorCount.get(type); localToColor.put(l, count); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java index 8d0c09a143e..d7e93227693 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java @@ -110,7 +110,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi localIndex++; // create newStmt whose definition is replaced with the newLocal, - Stmt newStmt = ((AbstractDefinitionStmt) currentStmt).withNewDef(newLocal); + Stmt newStmt = ((AbstractDefinitionStmt) currentStmt).withNewDef(newLocal); // replace corresponding oldStmt with newStmt in builder replaceStmtInBuilder(builder, stmts, currentStmt, newStmt); @@ -175,7 +175,7 @@ else if (hasModifiedUse(head, oriLocal)) { if (hasModifiedDef(backStmt, oriLocal)) { if (hasHigherLocalName((Local) backStmt.getDefs().get(0), modifiedLocal)) { Stmt newBackStmt = - ((AbstractDefinitionStmt) backStmt).withNewDef(modifiedLocal); + ((AbstractDefinitionStmt) backStmt).withNewDef(modifiedLocal); replaceStmtInBuilder(builder, stmts, backStmt, newBackStmt); newLocals.remove(newLocal); } @@ -393,7 +393,7 @@ private Set traceHandlerStmts(@Nonnull StmtGraph graph, @Nonnull Stmt e while (!queue.isEmpty()) { Stmt stmt = queue.removeFirst(); if (stmt instanceof JIdentityStmt - && ((JIdentityStmt) stmt).getRightOp() instanceof JCaughtExceptionRef) { + && ((JIdentityStmt) stmt).getRightOp() instanceof JCaughtExceptionRef) { handlerStmts.add(stmt); } else { final List predecessors = graph.predecessors(stmt); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java index 6f360c5ec71..396d8445ac3 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java @@ -131,7 +131,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi newLocals.add(newDef); nextFreeIdx++; localToNameStack.get(def).push(newDef); - Stmt newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(newDef); + Stmt newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(newDef); stmtGraph.replaceNode(stmt, newStmt); if (constainsPhiExpr(newStmt)) { newPhiStmts.add(newStmt); @@ -225,7 +225,7 @@ private Map, Set> decideBlockToPhiStmts( basicBlocks.add(df); // create an empty phiStmt - JAssignStmt phiStmt = createEmptyPhiStmt(local); + JAssignStmt phiStmt = createEmptyPhiStmt(local); // store phiStmt into map if (blockToPhiStmts.containsKey(df)) { @@ -332,9 +332,9 @@ private boolean constainsPhiExpr(Stmt stmt) { return false; } - private JAssignStmt createEmptyPhiStmt(Local local) { + private JAssignStmt createEmptyPhiStmt(Local local) { JPhiExpr phi = new JPhiExpr(Collections.emptyList(), Collections.emptyMap()); - return new JAssignStmt<>(local, phi, StmtPositionInfo.createNoStmtPositionInfo()); + return new JAssignStmt(local, phi, StmtPositionInfo.createNoStmtPositionInfo()); } private Local getOriginalLocal(Local local, Set oriLocals) { @@ -363,7 +363,7 @@ private Stmt addNewArgToPhi(Stmt phiStmt, Local arg, BasicBlock block) { argToBlock.put(arg, block); newPhiExpr = newPhiExpr.withArgs(args); newPhiExpr = newPhiExpr.withArgToBlockMap(argToBlock); - newPhiStmt = ((JAssignStmt) phiStmt).withRValue(newPhiExpr); + newPhiStmt = ((JAssignStmt) phiStmt).withRValue(newPhiExpr); break; } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index 2c47ff53eb1..47be5a0d580 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -112,15 +112,14 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm old_local = generateTempLocal(evaType); builder.addLocal(old_local); typing.set(old_local, evaType); - JAssignStmt newAssign = - Jimple.newAssignStmt(old_local, value, stmt.getPositionInfo()); + JAssignStmt newAssign = Jimple.newAssignStmt(old_local, value, stmt.getPositionInfo()); builder.insertBefore(stmt, newAssign); } Local new_local = generateTempLocal(stdType); builder.addLocal(new_local); typing.set(new_local, stdType); addUpdatedValue(oriValue, new_local, oriStmt); - JAssignStmt newCast = + JAssignStmt newCast = Jimple.newAssignStmt( new_local, Jimple.newCastExpr(old_local, stdType), stmt.getPositionInfo()); builder.insertBefore(stmt, newCast); @@ -129,7 +128,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm if (stmt.getUses().contains(value)) { newStmt = stmt.withNewUse(value, new_local); } else { - newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); + newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); } builder.replaceStmt(stmt, newStmt); this.stmt2NewStmt.put(oriStmt, newStmt); @@ -146,8 +145,8 @@ private void addUpdatedValue(Value oldValue, Value newValue, Stmt stmt) { } map.put(oldValue, newValue); if (stmt instanceof JAssignStmt && stmt.containsArrayRef()) { - Value leftOp = ((JAssignStmt) stmt).getLeftOp(); - Value rightOp = ((JAssignStmt) stmt).getRightOp(); + Value leftOp = ((JAssignStmt) stmt).getLeftOp(); + Value rightOp = ((JAssignStmt) stmt).getRightOp(); if (leftOp instanceof JArrayRef) { if (oldValue == leftOp) { Local base = ((JArrayRef) oldValue).getBase(); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java index 07400de262a..92e7e39d7e5 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java @@ -77,7 +77,7 @@ public void caseInvokeStmt(@Nonnull JInvokeStmt stmt) { } @Override - public void caseAssignStmt(@Nonnull JAssignStmt stmt) { + public void caseAssignStmt(@Nonnull JAssignStmt stmt) { Value lhs = stmt.getLeftOp(); Value rhs = stmt.getRightOp(); Type type_lhs = null; @@ -103,7 +103,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { if (defStmts != null) { for (Stmt defStmt : defStmts) { if (defStmt instanceof JAssignStmt) { - Value arrExpr = ((JAssignStmt) defStmt).getRightOp(); + Value arrExpr = ((JAssignStmt) defStmt).getRightOp(); if (arrExpr instanceof JNewArrayExpr) { arrayType = (ArrayType) arrExpr.getType(); findDef = true; @@ -162,7 +162,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { Collection stmts = defs.get(pair.getLocal()); for (Stmt s : stmts) { if (s instanceof JAssignStmt) { - Value value = ((JAssignStmt) s).getRightOp(); + Value value = ((JAssignStmt) s).getRightOp(); if (value instanceof JNewArrayExpr) { sel = selectType(sel, ((JNewArrayExpr) value).getBaseType(), s); } else if (value instanceof JNewMultiArrayExpr) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java index ca3c4b405f7..692cd30b583 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java @@ -46,7 +46,7 @@ /** @author Zun Wang Algorithm: see 'Efficient Local Type Inference' at OOPSLA 08 */ public class TypeResolver { - private final ArrayList> assignments = new ArrayList<>(); + private final ArrayList assignments = new ArrayList<>(); private final Map depends = new HashMap<>(); private final JavaView view; private int castCount; @@ -101,7 +101,7 @@ public boolean resolve(@Nonnull Body.BodyBuilder builder) { private void init(Body.BodyBuilder builder) { for (Stmt stmt : builder.getStmts()) { if (stmt instanceof AbstractDefinitionStmt) { - AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) stmt; + AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) stmt; Value lhs = defStmt.getLeftOp(); if (lhs instanceof Local || lhs instanceof JArrayRef || lhs instanceof JInstanceFieldRef) { final int id = assignments.size(); @@ -176,7 +176,7 @@ private Collection applyAssignmentConstraint( workQueue.removeFirst(); } else { actualSL.clear(stmtId); - AbstractDefinitionStmt defStmt = this.assignments.get(stmtId); + AbstractDefinitionStmt defStmt = this.assignments.get(stmtId); Value lhs = defStmt.getLeftOp(); Local local = (lhs instanceof Local) ? (Local) lhs : ((JArrayRef) lhs).getBase(); Type t_old = actualTyping.getType(local); diff --git a/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java b/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java index 3d031009c1b..d492ac29c84 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java @@ -53,7 +53,7 @@ public void testCreateMethod() { .getMethodSignature("main", "dummyMain", "void", Collections.emptyList()); Body.BodyBuilder bodyBuilder = Body.builder(); - final JIdentityStmt firstStmt = + final JIdentityStmt firstStmt = Jimple.newIdentityStmt( generator.generateLocal(type), Jimple.newParameterRef(type, 0), From 443db38bcfec60f9fa7c5daeb49f18e72407f81f Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 18:56:43 +0200 Subject: [PATCH 12/54] fix MutableBlockStmtGraph.insertBefore() : indexing of Stmt->Block was missing --- .../main/java/sootup/core/graph/MutableBlockStmtGraph.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 7a264665d07..4ec00c1aff2 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -847,7 +847,11 @@ public void insertBefore( (type, handler) -> successorBlock.addExceptionalSuccessorBlock(type, getOrCreateBlock(handler))); stmts.forEach(stmt -> addNodeToBlock(block, stmt)); - tryMergeBlocks(block, successorBlock); + if (!tryMergeBlocks(block, successorBlock)) { + // update index: for splitted stmts + successorBlock.getStmts().forEach((stmt) -> stmtToBlock.put(stmt, successorBlock)); + blocks.add(successorBlock); + } } if (beforeStmt == getStartingStmt()) { From a921226bf79a10f4ba18472fdb2ecde1ed022849 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 19:27:59 +0200 Subject: [PATCH 13/54] fix Interceptors.. and Introduce a new marker interface LhsValuewhich denotes Values that can be used on the left handside of an Assignment - eliminates another runtime check for allowed variable ie leftOp types in JAssignStmt --- .../sootup/core/jimple/basic/LhsValue.java | 4 ++ .../java/sootup/core/jimple/basic/Local.java | 2 +- .../core/jimple/common/ref/JArrayRef.java | 7 +--- .../core/jimple/common/ref/JFieldRef.java | 3 +- .../jimple/common/ref/JInstanceFieldRef.java | 3 +- .../jimple/common/ref/JStaticFieldRef.java | 3 +- .../common/stmt/AbstractDefinitionStmt.java | 1 + .../core/jimple/common/stmt/JAssignStmt.java | 25 +++-------- .../jimple/common/stmt/JIdentityStmt.java | 1 + .../sootup/core/jimple/common/stmt/Stmt.java | 3 +- .../src/main/java/sootup/core/model/Body.java | 6 +-- .../core/typehierarchy/TypeHierarchy.java | 5 +++ .../core/typehierarchy/ViewTypeHierarchy.java | 5 +++ .../core/graph/MutableBlockStmtGraphTest.java | 8 ++-- .../bytecode/interceptors/Aggregator.java | 12 +++++- .../bytecode/interceptors/CopyPropagator.java | 2 +- .../DeadAssignmentEliminator.java | 17 ++++---- .../typeresolving/AugEvalFunction.java | 25 ++++++++--- .../typeresolving/BytecodeHierarchy.java | 8 +++- .../typeresolving/CastCounter.java | 9 +++- .../typeresolving/StmtLocalPair.java | 1 + .../typeresolving/TypeChecker.java | 7 +++- .../typeresolving/TypeResolver.java | 4 ++ .../bytecode/interceptors/AggregatorTest.java | 27 ++++++++++++ .../interceptors/CopyPropagatorTest.java | 11 +++-- .../test/resources/interceptor/Misuse.java | 42 +++++++++++++++++++ 26 files changed, 177 insertions(+), 64 deletions(-) create mode 100644 sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java create mode 100644 sootup.tests/src/test/resources/interceptor/Misuse.java diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java b/sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java new file mode 100644 index 00000000000..9612e7b4f72 --- /dev/null +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java @@ -0,0 +1,4 @@ +package sootup.core.jimple.basic; + +/** Marker interface for Values that can be on the left side of an Assignment */ +public interface LhsValue extends Value {} diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java index f8dfb861b3f..6458b1e8046 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java @@ -43,7 +43,7 @@ * * @author Linghui Luo */ -public class Local implements Immediate, Copyable, Acceptor { +public class Local implements Immediate, LhsValue, Copyable, Acceptor { @Nonnull private final String name; @Nonnull private final Type type; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java index ed5db22889e..a751ff6d016 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java @@ -25,16 +25,13 @@ import java.util.ArrayList; import java.util.List; import javax.annotation.Nonnull; -import sootup.core.jimple.basic.Immediate; -import sootup.core.jimple.basic.JimpleComparator; -import sootup.core.jimple.basic.Local; -import sootup.core.jimple.basic.Value; +import sootup.core.jimple.basic.*; import sootup.core.jimple.visitor.RefVisitor; import sootup.core.types.Type; import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JArrayRef implements ConcreteRef, Copyable { +public final class JArrayRef implements ConcreteRef, LhsValue, Copyable { private final Local base; private final Immediate index; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java index 8e3d2c4e996..3137d6ebe0f 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java @@ -23,10 +23,11 @@ */ import javax.annotation.Nonnull; +import sootup.core.jimple.basic.LhsValue; import sootup.core.signatures.FieldSignature; import sootup.core.types.Type; -public abstract class JFieldRef implements ConcreteRef { +public abstract class JFieldRef implements ConcreteRef, LhsValue { @Nonnull private final FieldSignature fieldSignature; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java index 1c5927ed09e..51e2e3547e2 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java @@ -30,6 +30,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.RefVisitor; @@ -37,7 +38,7 @@ import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JInstanceFieldRef extends JFieldRef implements Copyable { +public final class JInstanceFieldRef extends JFieldRef implements LhsValue, Copyable { private final Local base; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java index c4d9d1e5248..a0904fdd6db 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java @@ -26,13 +26,14 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.RefVisitor; import sootup.core.signatures.FieldSignature; import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JStaticFieldRef extends JFieldRef implements Copyable { +public final class JStaticFieldRef extends JFieldRef implements LhsValue, Copyable { public JStaticFieldRef(@Nonnull FieldSignature fieldSig) { super(fieldSig); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index e109cafb518..80df18c62f7 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -78,5 +78,6 @@ public boolean branches() { return false; } + @Nonnull public abstract Stmt withNewDef(@Nonnull Local newLocal); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java index 20c6be7cd76..de717d905e1 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java @@ -37,7 +37,7 @@ /** Represents the assignment of one value to another */ public final class JAssignStmt extends AbstractDefinitionStmt implements Copyable { - @Nonnull final Value leftOp; + @Nonnull final LhsValue leftOp; @Nonnull final Value rightOp; /** @@ -47,31 +47,17 @@ public final class JAssignStmt extends AbstractDefinitionStmt implements Copyabl * @param rValue the value on the right side of the assign statement. */ public JAssignStmt( - @Nonnull Value variable, @Nonnull Value rValue, @Nonnull StmtPositionInfo positionInfo) { + @Nonnull LhsValue variable, @Nonnull Value rValue, @Nonnull StmtPositionInfo positionInfo) { super(positionInfo); leftOp = variable; rightOp = rValue; - if (!validateVariable(variable)) { - throw new RuntimeException( - "Illegal Assignment statement. Make sure that left hand side has a valid operand."); - } if (!validateValue(rValue)) { throw new RuntimeException( "Illegal Assignment statement. Make sure that right hand side has a valid operand."); } } - /** - * returns true if variable can be on the left side of the assign statement - * - * @param variable the variable on the left side of the assign statement. - */ - private boolean validateVariable(@Nonnull Value variable) { - // i.e. not Constant, not IdentityRef, not Expr - return variable instanceof Local || variable instanceof ConcreteRef; - } - /** * returns true if rValue can be on the right side of the assign statement * @@ -203,7 +189,7 @@ public int equivHashCode() { } @Nonnull - public JAssignStmt withVariable(@Nonnull Value variable) { + public JAssignStmt withVariable(@Nonnull LhsValue variable) { return new JAssignStmt(variable, getRightOp(), getPositionInfo()); } @@ -219,7 +205,7 @@ public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { @Nonnull @Override - public Value getLeftOp() { + public LhsValue getLeftOp() { return leftOp; } @@ -229,11 +215,12 @@ public Value getRightOp() { return rightOp; } + @Nonnull @Override public Stmt withNewDef(@Nonnull Local newLocal) { // "ReplaceDefVisitor" final Value leftOp = getLeftOp(); - Value newVal; + LhsValue newVal; if (leftOp instanceof ConcreteRef) { if (leftOp instanceof JArrayRef) { newVal = ((JArrayRef) leftOp).withBase(newLocal); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java index ad6553ef58c..1d8d543766e 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java @@ -98,6 +98,7 @@ public JIdentityStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { return new JIdentityStmt(getLeftOp(), getRightOp(), positionInfo); } + @Nonnull @Override public Stmt withNewDef(@Nonnull Local newLocal) { return withLocal(newLocal); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index 4e53d63e3ff..901043e5c4b 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.List; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import sootup.core.jimple.basic.EquivTo; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -148,7 +147,7 @@ public StmtPositionInfo getPositionInfo() { * @param newUse a Value is to replace oldUse * @return a new Stmt with newUse */ - @Nullable + @Nonnull public Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse) { ReplaceUseStmtVisitor visitor = new ReplaceUseStmtVisitor(oldUse, newUse); accept(visitor); diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 8d4d682690f..99fac5713c9 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -578,12 +578,12 @@ public String toString() { * @param stmts The searched list of statements * @return A map of Locals and their using statements */ - public static Map> collectDefs(Collection stmts) { - Map> allDefs = new HashMap<>(); + public static Map> collectDefs(Collection stmts) { + Map> allDefs = new HashMap<>(); for (Stmt stmt : stmts) { List defs = stmt.getDefs(); for (Value value : defs) { - if (value instanceof Local) { + if (value instanceof LhsValue) { Collection localDefs = allDefs.get(value); if (localDefs == null) { localDefs = new ArrayList<>(); diff --git a/sootup.core/src/main/java/sootup/core/typehierarchy/TypeHierarchy.java b/sootup.core/src/main/java/sootup/core/typehierarchy/TypeHierarchy.java index 6e70a97ae20..db96e2628f4 100644 --- a/sootup.core/src/main/java/sootup/core/typehierarchy/TypeHierarchy.java +++ b/sootup.core/src/main/java/sootup/core/typehierarchy/TypeHierarchy.java @@ -184,6 +184,7 @@ default List superClassesOf(@Nonnull ClassType classType) { } return superClasses; } + /** * Returns all superclasses of classType up to java.lang.Object, which * will be the last entry in the list, or till one of the superclasses is not contained in view. @@ -213,4 +214,8 @@ default List incompleteSuperClassesOf(@Nonnull ClassType classType) { boolean isInterface(@Nonnull ClassType type); Set directlyExtendedInterfacesOf(@Nonnull ClassType type); + + // checks if a Type is contained int the TypeHierarchy - should return the equivalent to + // View.getClass(...).isPresent() + boolean contains(ClassType type); } diff --git a/sootup.core/src/main/java/sootup/core/typehierarchy/ViewTypeHierarchy.java b/sootup.core/src/main/java/sootup/core/typehierarchy/ViewTypeHierarchy.java index f04cbbefaf0..1a5b91d48ac 100644 --- a/sootup.core/src/main/java/sootup/core/typehierarchy/ViewTypeHierarchy.java +++ b/sootup.core/src/main/java/sootup/core/typehierarchy/ViewTypeHierarchy.java @@ -215,6 +215,11 @@ public Set directlyExtendedInterfacesOf(@Nonnull ClassType interfaceT .collect(Collectors.toSet()); } + @Override + public boolean contains(ClassType type) { + return lazyScanResult.get().typeToVertex.get(type) != null; + } + /** * method exists for completeness - superClassOf() / which is basically SootClass.getSuperClass() * should be more performant. diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 64c9c4bf2e7..b74b27102cb 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -80,12 +80,12 @@ public PackageName getPackageName() { }; Stmt firstHandlerStmt = - new JIdentityStmt<>( + new JIdentityStmt( new Local("ex", throwableSig), new JCaughtExceptionRef(throwableSig), StmtPositionInfo.createNoStmtPositionInfo()); Stmt secondHandlerStmt = - new JIdentityStmt<>( + new JIdentityStmt( new Local("ex2", throwableSig), new JCaughtExceptionRef(ioExceptionSig), StmtPositionInfo.createNoStmtPositionInfo()); @@ -568,12 +568,12 @@ public PackageName getPackageName() { Local exc = new Local("ex", UnknownType.getInstance()); // hint: applied types make no sense in this test! Stmt catchStmt1 = - new JIdentityStmt<>( + new JIdentityStmt( exc, new JCaughtExceptionRef(UnknownType.getInstance()), StmtPositionInfo.createNoStmtPositionInfo()); Stmt catchStmt2 = - new JIdentityStmt<>( + new JIdentityStmt( exc, new JCaughtExceptionRef(PrimitiveType.getInt()), StmtPositionInfo.createNoStmtPositionInfo()); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index 55ecc772878..9b4fc4dab80 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -93,7 +93,9 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi continue; } Stmt relevantDef = defs.get(0); - if (!graph.containsNode(relevantDef)) { + // TODO: ms: check why its possible that we can get relevantDes/stmt but they are not in the + // graph anymore + if (!graph.containsNode(relevantDef) || !graph.containsNode(stmt)) { continue; } List path = graph.getExtendedBasicBlockPathBetween(relevantDef, stmt); @@ -176,7 +178,12 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi continue; } - Value aggregatee = ((JAssignStmt) relevantDef).getRightOp(); + // cannot aggregate e.g. a JIdentityStmt + if (!(relevantDef instanceof JAssignStmt)) { + continue; + } + + Value aggregatee = ((AbstractDefinitionStmt) relevantDef).getRightOp(); JAssignStmt newStmt = null; if (assignStmt.getRightOp() instanceof AbstractBinopExpr) { AbstractBinopExpr rightOp = (AbstractBinopExpr) assignStmt.getRightOp(); @@ -192,6 +199,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } else { newStmt = assignStmt.withRValue(aggregatee); } + if (newStmt != null) { builder.replaceStmt(stmt, newStmt); if (graph.getStartingStmt() == relevantDef) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java index 2ca2e0cde8f..9a7456c2b46 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/CopyPropagator.java @@ -64,7 +64,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi AbstractDefinitionStmt defStmt = (AbstractDefinitionStmt) defsOfUse.get(0); Value rhs = defStmt.getRightOp(); // if rhs is a constant, then replace use, if it is possible - if (rhs instanceof Constant) { + if (rhs instanceof Constant && !stmt.containsInvokeExpr()) { replaceUse(builder, stmt, use, rhs); } // if rhs is a cast expr with a ref type and its op is 0 (IntConstant or LongConstant) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java index 7f68381641e..23ce9c0c04f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java @@ -25,6 +25,7 @@ import sootup.core.graph.MutableBasicBlock; import sootup.core.graph.StmtGraph; import sootup.core.jimple.Jimple; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.constant.IntConstant; @@ -62,7 +63,7 @@ public DeadAssignmentEliminator(boolean eliminateOnlyStackLocals) { this.eliminateOnlyStackLocals = eliminateOnlyStackLocals; } - Map> allDefs = new HashMap<>(); + Map> allDefs = new HashMap<>(); Map> allUses = new HashMap<>(); @Override @@ -212,13 +213,15 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (stmt instanceof JAssignStmt) { JAssignStmt assignStmt = (JAssignStmt) stmt; if (assignStmt.containsInvokeExpr()) { - // Just find one use of local which is essential + // Just find one use of Value which is essential boolean deadAssignment = true; - Local local = (Local) assignStmt.getRightOp(); - for (Stmt use : allUses.get(local)) { - if (builder.getStmts().contains(use)) { - deadAssignment = false; - break; + if (assignStmt.getRightOp() instanceof Local) { + Local value = (Local) assignStmt.getRightOp(); + for (Stmt use : allUses.get(value)) { + if (builder.getStmts().contains(use)) { + deadAssignment = false; + break; + } } } if (deadAssignment) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index 33fb8559571..eedd635b7dd 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet; import java.util.*; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import sootup.core.IdentifierFactory; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Immediate; @@ -76,6 +77,7 @@ public AugEvalFunction(View> view) { * This method is used to evaluate the type of the given value which the given stmt and body * belongs to. */ + @Nullable public Type evaluate( @Nonnull Typing typing, @Nonnull Value value, @@ -121,12 +123,17 @@ public Type evaluate( } else if (value.getClass() == MethodType.class) { return methodTypeClassType; } else { - throw new RuntimeException("Invaluable constant in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + + // "'."); } } } else if (value instanceof Expr) { if (value instanceof AbstractBinopExpr) { Type tl = evaluate(typing, ((AbstractBinopExpr) value).getOp1(), stmt, graph); + if (tl == null) { + return null; + } Type tr = evaluate(typing, ((AbstractBinopExpr) value).getOp2(), stmt, graph); if (value instanceof AbstractIntBinopExpr) { if (value instanceof AbstractConditionExpr) { @@ -183,7 +190,10 @@ public Type evaluate( if (exceptionClassOp.isPresent()) { exceptionClass = (SootClass) exceptionClassOp.get(); } else { - throw new RuntimeException("ExceptionType '" + exceptionType + "' is not in the view"); + // Logger.info("ExceptionType '" + exceptionType + "' is not in the view"); + return null; + // throw new RuntimeException("ExceptionType '" + exceptionType + "' is not in the + // view"); } if (exceptionClass.isPhantomClass()) { return throwableClassType; @@ -193,9 +203,12 @@ public Type evaluate( type = getLeastCommonExceptionType(type, exceptionType); } } + + /* if (type == null) { - throw new RuntimeException("Invaluable reference in AugEvalFunction '" + value + "'."); - } + return null; + throw new RuntimeException("inevaluatable reference in AugEvalFunction '" + value +"'."); + }*/ return type; } else if (value instanceof JArrayRef) { Type type = typing.getType(((JArrayRef) value).getBase()); @@ -213,7 +226,9 @@ public Type evaluate( || value instanceof JFieldRef) { return value.getType(); } else { - throw new RuntimeException("Invaluable reference in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("Inevaluatable reference in AugEvalFunction '" + value + + // "'."); } } return null; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java index bc154fcd05a..413715879f8 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java @@ -33,7 +33,6 @@ /** @author Zun Wang */ public class BytecodeHierarchy { - private final TypeHierarchy typeHierarchy; public final ClassType objectClassType; public final ClassType throwableClassType; @@ -49,6 +48,10 @@ public BytecodeHierarchy(View> view) { cloneableClassType = factory.getClassType("java.lang.Cloneable"); } + boolean contains(ClassType type) { + return typeHierarchy.contains(type); + } + public boolean isAncestor(@Nonnull Type ancestor, @Nonnull Type child) { if (PrimitiveHierarchy.isAncestor(ancestor, child)) { return true; @@ -198,7 +201,8 @@ public Collection getLeastCommonAncestor(Type a, Type b) { } private boolean canStoreType(ClassType ancestor, ClassType child) { - return ancestor == objectClassType || typeHierarchy.subtypesOf(ancestor).contains(child); + return ancestor == objectClassType + || (typeHierarchy.contains(ancestor) && typeHierarchy.subtypesOf(ancestor).contains(child)); } private Set buildAncestryPaths(ClassType type) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index 47be5a0d580..d421681602a 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -80,6 +80,9 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm Typing typing = getTyping(); if (countOnly) { Type evaType = evalFunction.evaluate(typing, value, stmt, graph); + if (evaType == null) { + return; + } if (hierarchy.isAncestor(stdType, evaType)) { return; } @@ -130,8 +133,10 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm } else { newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); } - builder.replaceStmt(stmt, newStmt); - this.stmt2NewStmt.put(oriStmt, newStmt); + if (graph.containsNode(stmt)) { + builder.replaceStmt(stmt, newStmt); + this.stmt2NewStmt.put(oriStmt, newStmt); + } } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/StmtLocalPair.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/StmtLocalPair.java index bb6c8d8f074..a9ed1331bdd 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/StmtLocalPair.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/StmtLocalPair.java @@ -25,6 +25,7 @@ import sootup.core.jimple.basic.Local; import sootup.core.jimple.common.stmt.Stmt; +// FIXME: ms: refactor to Pair public class StmtLocalPair { private final Stmt stmt; private final Local local; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java index 92e7e39d7e5..e91a9bd3a40 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java @@ -28,6 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import sootup.core.graph.StmtGraph; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.constant.Constant; @@ -97,7 +98,8 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { // allocation site. if (Type.isObjectLikeType(type_base) || (Type.isObject(type_base) && type_rhs instanceof PrimitiveType)) { - Map> defs = Body.collectDefs(builder.getStmtGraph().getNodes()); + Map> defs = + Body.collectDefs(builder.getStmtGraph().getNodes()); Collection defStmts = defs.get(base); boolean findDef = false; if (defStmts != null) { @@ -149,7 +151,8 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { arrayType = (ArrayType) type_base; } else { if (type_base instanceof NullType || Type.isObjectLikeType(type_base)) { - Map> defs = Body.collectDefs(builder.getStmtGraph().getNodes()); + Map> defs = + Body.collectDefs(builder.getStmtGraph().getNodes()); Deque worklist = new ArrayDeque<>(); Set visited = new HashSet<>(); worklist.add(new StmtLocalPair(stmt, base)); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java index 692cd30b583..4abdf52aeb6 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java @@ -181,6 +181,10 @@ private Collection applyAssignmentConstraint( Local local = (lhs instanceof Local) ? (Local) lhs : ((JArrayRef) lhs).getBase(); Type t_old = actualTyping.getType(local); Type t_right = evalFunction.evaluate(actualTyping, defStmt.getRightOp(), defStmt, graph); + if (t_right == null) { + workQueue.removeFirst(); + continue; + } if (lhs instanceof JArrayRef) { t_right = Type.createArrayType(t_right, 1); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index ca6666af81c..c38d7a116c3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -6,6 +6,7 @@ import java.util.List; import java.util.Set; import org.junit.Test; +import sootup.core.inputlocation.AnalysisInputLocation; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; @@ -13,10 +14,17 @@ import sootup.core.jimple.common.expr.JAddExpr; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; +import sootup.core.model.SootMethod; import sootup.core.types.PrimitiveType; import sootup.core.util.ImmutableUtils; +import sootup.java.bytecode.inputlocation.BytecodeClassLoadingOptions; +import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; import sootup.java.core.JavaIdentifierFactory; +import sootup.java.core.JavaProject; +import sootup.java.core.JavaSootClass; import sootup.java.core.language.JavaJimple; +import sootup.java.core.language.JavaLanguage; +import sootup.java.core.views.JavaView; public class AggregatorTest { @@ -97,4 +105,23 @@ private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { return builder; } + + @Test + public void testResource_Misuse() { + + String classPath = "../sootup.tests/src/test/resources/interceptor/"; + AnalysisInputLocation inputLocation = + new JavaClassPathAnalysisInputLocation(classPath); + JavaLanguage language = new JavaLanguage(8); + + JavaProject project = JavaProject.builder(language).addInputLocation(inputLocation).build(); + JavaView view = project.createView(); + view.configBodyInterceptors((analysisInputLocation) -> BytecodeClassLoadingOptions.Default); + + final SootMethod sootMethod = + view.getMethod(view.getIdentifierFactory().parseMethodSignature("")) + .get(); + + sootMethod.getBody(); + } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java index 8b6aa7a01e5..7f6746222b1 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java @@ -10,7 +10,6 @@ import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.common.constant.Constant; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.constant.LongConstant; import sootup.core.jimple.common.constant.NullConstant; @@ -114,9 +113,9 @@ public class CopyPropagatorTest { // r6 = r3 Stmt stmt14 = JavaJimple.newAssignStmt(r6, r3, noStmtPositionInfo); - JAssignStmt eestmt4 = + JAssignStmt eestmt4 = JavaJimple.newAssignStmt(r4, NullConstant.getInstance(), noStmtPositionInfo); - JAssignStmt estmt13 = + JAssignStmt estmt13 = JavaJimple.newAssignStmt(r5, NullConstant.getInstance(), noStmtPositionInfo); @Test @@ -124,8 +123,8 @@ public void testEqualStmt() { assertTrue(eestmt4.equivTo(eestmt4.withRValue(NullConstant.getInstance()))); } - @Test /** Test the copy propagation's chain */ + @Test public void testChainBody() { Body body = createChainBody(); @@ -137,8 +136,8 @@ public void testChainBody() { AssertUtils.assertStmtGraphEquiv(expectedBody, builder.build()); } - @Test /** Test the copy propagation for loop */ + @Test public void testLoopBody() { Body.BodyBuilder builder = createLoopBody(); @@ -150,8 +149,8 @@ public void testLoopBody() { AssertUtils.assertStmtGraphEquiv(expectedBody, builder.build()); } + /** Test the copy propagation for castExpr */ @Test - /* Test the copy propagation for castExpr */ public void testCastExprBody() { Body body = createCastExprBody(); diff --git a/sootup.tests/src/test/resources/interceptor/Misuse.java b/sootup.tests/src/test/resources/interceptor/Misuse.java new file mode 100644 index 00000000000..71e69a1f82a --- /dev/null +++ b/sootup.tests/src/test/resources/interceptor/Misuse.java @@ -0,0 +1,42 @@ + +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; + +import javax.crypto.BadPaddingException; +import javax.crypto.Cipher; +import javax.crypto.IllegalBlockSizeException; +import javax.crypto.KeyGenerator; +import javax.crypto.NoSuchPaddingException; +import javax.crypto.SecretKey; + +public class Misuse { + + /** + * Misuse: "AES" is not secure, "AES/GCM/PKCS5Padding" should be used to get the cipher + */ + public void test() { + try { + String plainText = "Sensitive information"; + int keySize = 128; + // Generate a key for AES + KeyGenerator keygenerator = KeyGenerator.getInstance("AES"); + keygenerator.init(keySize); + SecretKey key = keygenerator.generateKey(); + // Encrypt the plain text with AES + Cipher aesChipher; + aesChipher = Cipher.getInstance("AES"); + aesChipher.init(Cipher.ENCRYPT_MODE, key); + byte[] encrypted= aesChipher.doFinal(plainText.getBytes()); + } catch (NoSuchAlgorithmException e) { + e.printStackTrace(); + } catch (NoSuchPaddingException e) { + e.printStackTrace(); + } catch (InvalidKeyException e) { + e.printStackTrace(); + } catch (IllegalBlockSizeException e) { + e.printStackTrace(); + } catch (BadPaddingException e) { + e.printStackTrace(); + } + } +} From a2782cc4277db2be43d154c2a2921a09b1646dcc Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 19:43:47 +0200 Subject: [PATCH 14/54] fix LhsValue leftovers --- .../main/java/sootup/core/jimple/Jimple.java | 3 +- .../jimple/visitor/ReplaceUseStmtVisitor.java | 3 +- .../frontend/InstructionConverter.java | 30 ++++++++++--------- .../sootup/jimple/parser/JimpleConverter.java | 6 ++-- 4 files changed, 23 insertions(+), 19 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java index 11f3d69f4ce..f44812c1233 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java +++ b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java @@ -525,7 +525,8 @@ public static JIdentityStmt newIdentityStmt( } /** Constructs a AssignStmt(Variable, RValue) grammar chunk. */ - public static JAssignStmt newAssignStmt(Value variable, Value rvalue, StmtPositionInfo posInfo) { + public static JAssignStmt newAssignStmt( + LhsValue variable, Value rvalue, StmtPositionInfo posInfo) { return new JAssignStmt(variable, rvalue, posInfo); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java index 0529d096eb8..82ba5f5da3b 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java @@ -24,6 +24,7 @@ import javax.annotation.Nonnull; import sootup.core.jimple.basic.Immediate; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractConditionExpr; import sootup.core.jimple.common.expr.AbstractInvokeExpr; @@ -77,7 +78,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { refVisitor.init(oldUse, newUse); ((Ref) leftOp).accept(refVisitor); if (refVisitor.getResult() != leftOp) { - setResult(stmt.withVariable(refVisitor.getResult())); + setResult(stmt.withVariable((LhsValue) refVisitor.getResult())); } } diff --git a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java index 37606aac35d..7d70caad1ee 100644 --- a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java +++ b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java @@ -230,9 +230,8 @@ private Stmt convertArrayLoadInstruction(SSAArrayLoadInstruction inst) { index = getLocal(PrimitiveType.getInt(), i); } JArrayRef arrayRef = JavaJimple.getInstance().newArrayRef(base, index); - Value left = null; int def = inst.getDef(); - left = getLocal(base.getType(), def); + LhsValue left = getLocal(base.getType(), def); Position[] operandPos = new Position[1]; // TODO: loaded arrayindex position info is missing @@ -415,7 +414,7 @@ private List convertAstLexicalWrite(AstLexicalWrite inst) { } JavaClassType cSig = (JavaClassType) methodSignature.getDeclClassType(); // TODO check modifier - Value left; + LhsValue left; if (!walaMethod.isStatic()) { FieldSignature fieldSig = identifierFactory.getFieldSignature( @@ -475,7 +474,7 @@ private List convertAstLexicalRead(AstLexicalRead inst) { private Stmt convertEnclosingObjectReference(EnclosingObjectReference inst) { Type enclosingType = converter.convertType(inst.getEnclosingType()); - Value variable = getLocal(enclosingType, inst.getDef()); + LhsValue variable = getLocal(enclosingType, inst.getDef()); JavaClassType cSig = (JavaClassType) methodSignature.getDeclClassType(); // TODO check modifier @@ -634,7 +633,7 @@ private Stmt convertPutInstruction(SSAPutInstruction inst) { FieldSignature fieldSig = identifierFactory.getFieldSignature( fieldRef.getName().toString(), classSig, fieldType.toString()); - Value fieldValue; + LhsValue fieldValue; if (inst.isStatic()) { fieldValue = Jimple.newStaticFieldRef(fieldSig); } else { @@ -664,7 +663,7 @@ private Stmt convertPutInstruction(SSAPutInstruction inst) { private Stmt convertNewInstruction(SSANewInstruction inst) { int result = inst.getDef(); Type type = converter.convertType(inst.getNewSite().getDeclaredType()); - Value var = getLocal(type, result); + LhsValue var = getLocal(type, result); Value rvalue; if (type instanceof ArrayType) { int use = inst.getUse(0); @@ -708,7 +707,7 @@ private Stmt convertInstanceofInstruction(SSAInstanceofInstruction inst) { // TODO. how to get type of ref? Local op = getLocal(UnknownType.getInstance(), ref); JInstanceOfExpr expr = Jimple.newInstanceOfExpr(op, checkedType); - Value left = getLocal(PrimitiveType.getBoolean(), result); + LhsValue left = getLocal(PrimitiveType.getBoolean(), result); Position[] operandPos = new Position[2]; // TODO: has no operand positions yet for checked and expected side @@ -727,7 +726,7 @@ private Stmt convertConversionInstruction(SSAConversionInstruction inst) { Type toType = converter.convertType(inst.getToType()); int def = inst.getDef(); int use = inst.getUse(0); - Value lvalue = getLocal(toType, def); + LhsValue lvalue = getLocal(toType, def); Immediate rvalue; if (symbolTable.isConstant(use)) { rvalue = getConstant(use); @@ -877,6 +876,9 @@ private Immediate extractValueAndAddAssignStmt( Object constant = null; if (symbolTable.isZero(val)) { value = IntConstant.getInstance(0); + // FIXME: type safety found an issue..fix it as value would have been a constant on the + // lefthandside + throw new IllegalStateException("ms: this execution path needs to be fixed."); } else { if (symbolTable.isConstant(val)) { constant = symbolTable.getConstantValue(val); @@ -885,7 +887,7 @@ private Immediate extractValueAndAddAssignStmt( } if (constant != null) { JAssignStmt assignStmt = - Jimple.newAssignStmt(value, ConstantUtil.fromObject(constant), posInfo); + Jimple.newAssignStmt((LhsValue) value, ConstantUtil.fromObject(constant), posInfo); addTo.add(assignStmt); } return value; @@ -918,7 +920,7 @@ private Stmt convertReturnInstruction(SSAReturnInstruction inst) { private List convertStringAddition( Immediate op1, Immediate op2, - Immediate result, + LhsValue result, Type type, int iindex, AstMethod.DebuggingInformation debugInfo) { @@ -1013,7 +1015,7 @@ private List convertBinaryOpInstruction(SSABinaryOpInstruction binOpInst) if (operator.equals(IBinaryOpInstruction.Operator.ADD)) { if (type.toString().equals("java.lang.String")) { // from wala java source code frontend we get also string addition(concatenation). - Immediate result = getLocal(type, def); + Local result = getLocal(type, def); return convertStringAddition(op1, op2, result, type, binOpInst.iIndex(), debugInfo); } binExpr = Jimple.newAddExpr(op1, op2); @@ -1063,7 +1065,7 @@ private List convertBinaryOpInstruction(SSABinaryOpInstruction binOpInst) operandPos[0] = p1; Position p2 = debugInfo.getOperandPosition(binOpInst.iIndex(), 1); operandPos[1] = p2; - Value result = getLocal(type, def); + Local result = getLocal(type, def); ret.add( Jimple.newAssignStmt( result, @@ -1105,7 +1107,7 @@ private Stmt convertGetInstruction(SSAGetInstruction inst) { Position[] operandPos = new Position[1]; operandPos[0] = debugInfo.getOperandPosition(inst.iIndex(), 0); - Value var = getLocal(fieldType, def); + Local var = getLocal(fieldType, def); return Jimple.newAssignStmt( var, rvalue, @@ -1133,7 +1135,7 @@ private Constant getConstant(int valueNumber) { } else if (symbolTable.isNullConstant(valueNumber)) { return NullConstant.getInstance(); } else { - throw new RuntimeException("Unsupported constant type: " + value.getClass().toString()); + throw new RuntimeException("Unsupported constant type: " + value.getClass()); } } diff --git a/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java b/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java index fea8fa1bc96..7b5cef61b99 100644 --- a/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java +++ b/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java @@ -487,10 +487,10 @@ public Stmt visitStmt(JimpleParser.StmtContext ctx) { return Jimple.newIdentityStmt(left, ref, pos); } else if (assignments.EQUALS() != null) { - Value left = + LhsValue left = assignments.local != null ? getLocal(assignments.local.getText()) - : valueVisitor.visitReference(assignments.reference()); + : (LhsValue) valueVisitor.visitReference(assignments.reference()); final Value right = valueVisitor.visitValue(assignments.value()); return Jimple.newAssignStmt(left, right, pos); @@ -569,7 +569,7 @@ public Value visitValue(JimpleParser.ValueContext ctx) { List sizes = ctx.immediate().stream().map(this::visitImmediate).collect(Collectors.toList()); - if (sizes.size() < 1) { + if (sizes.isEmpty()) { throw new ResolveException( "The Size list must have at least one Element.", path, From cdd7c10b246968e483639e89e4b36f98a6ec222d Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 20:12:38 +0200 Subject: [PATCH 15/54] fix TypeResolver handling of incomplete information in View --- .../typeresolving/BytecodeHierarchy.java | 12 ++++++++---- .../interceptors/typeresolving/TypeResolver.java | 12 +++++++++++- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java index 413715879f8..e86c92abfed 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java @@ -211,6 +211,9 @@ private Set buildAncestryPaths(ClassType type) { Set paths = new HashSet<>(); while (!pathNodes.isEmpty()) { AncestryPath node = pathNodes.removeFirst(); + if (!typeHierarchy.contains(node.type)) { + break; + } if (node.type == objectClassType) { paths.add(node); } else { @@ -231,10 +234,11 @@ private Set buildAncestryPaths(ClassType type) { pathNodes.add(superNode); } ClassType superClass = typeHierarchy.superClassOf(node.type); - // only java.lang.Object can have no SuperClass i.e. is null - this is already filtered - // above - AncestryPath superNode = new AncestryPath(superClass, node); - pathNodes.add(superNode); + // only java.lang.Object can have no SuperClass i.e. is null + if (superClass != null) { + AncestryPath superNode = new AncestryPath(superClass, node); + pathNodes.add(superNode); + } } } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java index 4abdf52aeb6..e6fa6238d39 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java @@ -178,10 +178,20 @@ private Collection applyAssignmentConstraint( actualSL.clear(stmtId); AbstractDefinitionStmt defStmt = this.assignments.get(stmtId); Value lhs = defStmt.getLeftOp(); - Local local = (lhs instanceof Local) ? (Local) lhs : ((JArrayRef) lhs).getBase(); + Local local; + if (lhs instanceof Local) { + local = (Local) lhs; + } else if (lhs instanceof JArrayRef) { + local = ((JArrayRef) lhs).getBase(); + } else if (lhs instanceof JInstanceFieldRef) { + local = ((JInstanceFieldRef) lhs).getBase(); + } else { + throw new IllegalStateException("can not handle " + lhs.getClass()); + } Type t_old = actualTyping.getType(local); Type t_right = evalFunction.evaluate(actualTyping, defStmt.getRightOp(), defStmt, graph); if (t_right == null) { + // TODO: ms: is this correct to handle: null? workQueue.removeFirst(); continue; } From b789f9a153ffb9ba42a64d686be7863a9cdbee2d Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 20:37:22 +0200 Subject: [PATCH 16/54] fix wrong assumption - and simplified the code. --- .../frontend/InstructionConverter.java | 21 +++++++------------ 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java index 7d70caad1ee..ed35609048f 100644 --- a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java +++ b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java @@ -872,25 +872,18 @@ private List convertBranchInstruction(SSAConditionalBranchInstruction cond private Immediate extractValueAndAddAssignStmt( StmtPositionInfo posInfo, List addTo, int val) { - Immediate value; - Object constant = null; if (symbolTable.isZero(val)) { - value = IntConstant.getInstance(0); - // FIXME: type safety found an issue..fix it as value would have been a constant on the - // lefthandside - throw new IllegalStateException("ms: this execution path needs to be fixed."); + return IntConstant.getInstance(0); } else { + Local value = getLocal(PrimitiveType.getInt(), val); if (symbolTable.isConstant(val)) { - constant = symbolTable.getConstantValue(val); + Object constant = symbolTable.getConstantValue(val); + JAssignStmt assignStmt = + Jimple.newAssignStmt(value, ConstantUtil.fromObject(constant), posInfo); + addTo.add(assignStmt); } - value = getLocal(PrimitiveType.getInt(), val); - } - if (constant != null) { - JAssignStmt assignStmt = - Jimple.newAssignStmt((LhsValue) value, ConstantUtil.fromObject(constant), posInfo); - addTo.add(assignStmt); + return value; } - return value; } private Stmt convertReturnInstruction(SSAReturnInstruction inst) { From 6a1b86c44a88a111cefae227226aa1a725f40a36 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 22:45:05 +0200 Subject: [PATCH 17/54] more LhsValue typesafety.. --- .../common/stmt/AbstractDefinitionStmt.java | 7 +++--- .../sootup/core/jimple/common/stmt/Stmt.java | 5 ++-- .../src/main/java/sootup/core/model/Body.java | 25 ++++++++----------- .../java/bytecode/frontend/StmtContainer.java | 3 ++- .../interceptors/LocalLivenessAnalyser.java | 3 ++- .../interceptors/LocalNameStandardizer.java | 3 ++- .../bytecode/interceptors/LocalSplitter.java | 9 ++++--- .../StaticSingleAssignmentFormer.java | 3 ++- 8 files changed, 30 insertions(+), 28 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index 80df18c62f7..ba5be1b530e 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import javax.annotation.Nonnull; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -37,7 +38,7 @@ public abstract class AbstractDefinitionStmt extends Stmt { } @Nonnull - public abstract Value getLeftOp(); + public abstract LhsValue getLeftOp(); @Nonnull public abstract Value getRightOp(); @@ -49,8 +50,8 @@ public Type getType() { @Override @Nonnull - public List getDefs() { - final List defs = new ArrayList<>(); + public List getDefs() { + final List defs = new ArrayList<>(); defs.add(getLeftOp()); return defs; } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index 901043e5c4b..ff5512962a5 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -27,6 +27,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.EquivTo; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractInvokeExpr; @@ -60,7 +61,7 @@ public List getUses() { * types/assignments so we return a List */ @Nonnull - public List getDefs() { + public List getDefs() { return Collections.emptyList(); } @@ -68,7 +69,7 @@ public List getDefs() { @Nonnull public List getUsesAndDefs() { List uses = getUses(); - List defs = getDefs(); + List defs = new ArrayList<>(getDefs()); if (uses.isEmpty()) { return defs; } else if (defs.isEmpty()) { diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 99fac5713c9..e6c60952da0 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -429,13 +429,10 @@ public void replaceLocal(@Nonnull Local oldLocal, @Nonnull Local newLocal) { for (Stmt currStmt : Lists.newArrayList(getStmtGraph().getNodes())) { final Stmt stmt = currStmt; if (currStmt.getUses().contains(oldLocal)) { - final Stmt newStmt = currStmt.withNewUse(oldLocal, newLocal); - if (newStmt != null) { - currStmt = newStmt; - } + currStmt = currStmt.withNewUse(oldLocal, newLocal); } - final List defs = currStmt.getDefs(); - for (Value def : defs) { + final List defs = currStmt.getDefs(); + for (LhsValue def : defs) { if (def == oldLocal || def.getUses().contains(oldLocal)) { if (currStmt instanceof AbstractDefinitionStmt) { currStmt = ((AbstractDefinitionStmt) currStmt).withNewDef(newLocal); @@ -581,16 +578,14 @@ public String toString() { public static Map> collectDefs(Collection stmts) { Map> allDefs = new HashMap<>(); for (Stmt stmt : stmts) { - List defs = stmt.getDefs(); - for (Value value : defs) { - if (value instanceof LhsValue) { - Collection localDefs = allDefs.get(value); - if (localDefs == null) { - localDefs = new ArrayList<>(); - } - localDefs.add(stmt); - allDefs.put((Local) value, localDefs); + List defs = stmt.getDefs(); + for (LhsValue value : defs) { + Collection localDefs = allDefs.get(value); + if (localDefs == null) { + localDefs = new ArrayList<>(); } + localDefs.add(stmt); + allDefs.put(value, localDefs); } } return allDefs; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java index b26914dfc1d..3654abdcc9f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java @@ -25,6 +25,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.stmt.Stmt; @@ -83,7 +84,7 @@ public List getUses() { @Nonnull @Override - public List getDefs() { + public List getDefs() { throw new UnsupportedOperationException(); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java index c07b872b9ae..165eb220c19 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java @@ -25,6 +25,7 @@ import java.util.*; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.stmt.Stmt; @@ -77,7 +78,7 @@ public LocalLivenessAnalyser(@Nonnull StmtGraph graph) { in.add((Local) use); } } - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty()) { final Value value = defs.get(0); if (value instanceof Local) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java index 465eca720a5..fc885674399 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java @@ -23,6 +23,7 @@ import java.util.*; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.LocalGenerator; import sootup.core.jimple.basic.Value; @@ -44,7 +45,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi Map localToFirstOccurrence = new HashMap<>(); int defsCount = 0; for (Stmt stmt : graph) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); for (Value def : defs) { if (def instanceof Local) { final Local localDef = (Local) def; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java index d7e93227693..11b592d4209 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java @@ -26,6 +26,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import sootup.core.graph.StmtGraph; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.ref.JCaughtExceptionRef; @@ -79,7 +80,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi Set visitedLocals = new LinkedHashSet<>(); Set toSplitLocals = new LinkedHashSet<>(); for (Stmt stmt : stmts) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty()) { Value def = defs.get(0); if (def instanceof Local) { @@ -101,7 +102,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi while (!stmts.isEmpty()) { Stmt currentStmt = stmts.remove(0); // At first Check the definition(left side) of the currentStmt is a local which must be split: - final List defs = currentStmt.getDefs(); + final List defs = currentStmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local && toSplitLocals.contains(defs.get(0))) { // then assign a new name to the oriLocal to get a new local which is called newLocal Local oriLocal = (Local) defs.get(0); @@ -204,7 +205,7 @@ else if (hasModifiedUse(backStmt, oriLocal)) { // then add all successors of head which are not in forwardsQueue and visitedStmts, // into the forwardsQueue. else { - final List headDefs = head.getDefs(); + final List headDefs = head.getDefs(); if (headDefs.isEmpty() || !headDefs.get(0).equivTo(oriLocal)) { for (Stmt succ : graph.successors(head)) { if (!visitedStmts.contains(succ) && !forwardsQueue.contains(succ)) { @@ -351,7 +352,7 @@ private boolean isLocalFromSameOrigin(@Nonnull Local oriLocal, Value local) { * @return if so, return true, else return false */ private boolean hasModifiedDef(@Nonnull Stmt stmt, @Nonnull Local oriLocal) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local) { return isLocalFromSameOrigin(oriLocal, defs.get(0)); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java index 396d8445ac3..9a409cd8e57 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java @@ -28,6 +28,7 @@ import sootup.core.graph.DominanceFinder; import sootup.core.graph.DominanceTree; import sootup.core.graph.MutableStmtGraph; +import sootup.core.jimple.basic.LhsValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -124,7 +125,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } } // generate new def and replace with new def - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local) { Local def = (Local) defs.get(0); Local newDef = def.withName(def.getName() + "#" + nextFreeIdx); From 46069c3ca08c7290defd4de120c7c38dcd9e00e4 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 15 Sep 2023 22:45:39 +0200 Subject: [PATCH 18/54] fix porting bug: soot used getUses() and not getDefs() --- .../bytecode/interceptors/Aggregator.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index 9b4fc4dab80..f6e9fbf11de 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -23,7 +23,9 @@ import com.google.common.collect.Lists; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Immediate; @@ -108,19 +110,18 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi boolean propagatingArrayRef = false; List fieldRefList = new ArrayList<>(); - List localsUsed = new ArrayList<>(); + Set localsUsed = new HashSet<>(); for (Stmt pathStmt : path) { - List allDefs = pathStmt.getDefs(); - for (Value def : allDefs) { - if (def instanceof Local) { - localsUsed.add(def); - } else if (def instanceof AbstractInstanceInvokeExpr) { + for (Value use : pathStmt.getUses()) { + if (use instanceof Local) { + localsUsed.add(use); + } else if (use instanceof AbstractInstanceInvokeExpr) { propagatingInvokeExpr = true; - } else if (def instanceof JArrayRef) { + } else if (use instanceof JArrayRef) { propagatingArrayRef = true; - } else if (def instanceof JFieldRef) { + } else if (use instanceof JFieldRef) { propagatingFieldRef = true; - fieldRefList.add((JFieldRef) def); + fieldRefList.add((JFieldRef) use); } } } From 01368ed4ee131ab8a4f88816a4360f5c0e823e49 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Thu, 21 Sep 2023 13:59:28 +0200 Subject: [PATCH 19/54] refactor: LhsValue to LValue; Local.getDefsOfLocal(...) to Local.getDefs(...) --- .../core/graph/MutableBlockStmtGraph.java | 2 +- .../main/java/sootup/core/jimple/Jimple.java | 4 +- .../basic/{LhsValue.java => LValue.java} | 2 +- .../java/sootup/core/jimple/basic/Local.java | 4 +- .../core/jimple/common/ref/JArrayRef.java | 2 +- .../core/jimple/common/ref/JFieldRef.java | 4 +- .../jimple/common/ref/JInstanceFieldRef.java | 4 +- .../jimple/common/ref/JStaticFieldRef.java | 4 +- .../common/stmt/AbstractDefinitionStmt.java | 8 +-- .../core/jimple/common/stmt/JAssignStmt.java | 10 ++-- .../sootup/core/jimple/common/stmt/Stmt.java | 4 +- .../jimple/visitor/ReplaceUseStmtVisitor.java | 4 +- .../src/main/java/sootup/core/model/Body.java | 12 ++--- .../java/bytecode/frontend/StmtContainer.java | 4 +- .../bytecode/interceptors/Aggregator.java | 4 +- .../ConstantPropagatorAndFolder.java | 2 +- .../DeadAssignmentEliminator.java | 4 +- .../interceptors/LocalLivenessAnalyser.java | 4 +- .../interceptors/LocalNameStandardizer.java | 4 +- .../bytecode/interceptors/LocalSplitter.java | 10 ++-- .../StaticSingleAssignmentFormer.java | 4 +- .../typeresolving/TypeChecker.java | 7 ++- .../bytecode/interceptors/AggregatorTest.java | 46 +++++++++++++++--- .../frontend/InstructionConverter.java | 16 +++--- .../sootup/jimple/parser/JimpleConverter.java | 4 +- .../src/test/resources/interceptor/File.java | 22 +++++++++ .../test/resources/interceptor/Misuse.class | Bin 0 -> 1290 bytes .../test/resources/interceptor/Misuse.java | 7 +++ 28 files changed, 130 insertions(+), 72 deletions(-) rename sootup.core/src/main/java/sootup/core/jimple/basic/{LhsValue.java => LValue.java} (72%) create mode 100644 sootup.tests/src/test/resources/interceptor/File.java create mode 100644 sootup.tests/src/test/resources/interceptor/Misuse.class diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 4ec00c1aff2..5039e1a81c2 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -28,7 +28,7 @@ * */ public class MutableBlockStmtGraph extends MutableStmtGraph { @Nullable private Stmt startingStmt = null; - @Nonnull private final Map stmtToBlock = new HashMap<>(); + @Nonnull private final Map stmtToBlock = new IdentityHashMap<>(); @Nonnull private final Set blocks = new HashSet<>(); diff --git a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java index f44812c1233..cea0694f7bb 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/Jimple.java +++ b/sootup.core/src/main/java/sootup/core/jimple/Jimple.java @@ -42,7 +42,6 @@ import sootup.core.jimple.javabytecode.stmt.*; import sootup.core.signatures.FieldSignature; import sootup.core.signatures.MethodSignature; -import sootup.core.types.*; import sootup.core.types.ArrayType; import sootup.core.types.ClassType; import sootup.core.types.Type; @@ -525,8 +524,7 @@ public static JIdentityStmt newIdentityStmt( } /** Constructs a AssignStmt(Variable, RValue) grammar chunk. */ - public static JAssignStmt newAssignStmt( - LhsValue variable, Value rvalue, StmtPositionInfo posInfo) { + public static JAssignStmt newAssignStmt(LValue variable, Value rvalue, StmtPositionInfo posInfo) { return new JAssignStmt(variable, rvalue, posInfo); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java b/sootup.core/src/main/java/sootup/core/jimple/basic/LValue.java similarity index 72% rename from sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java rename to sootup.core/src/main/java/sootup/core/jimple/basic/LValue.java index 9612e7b4f72..e7ff23de3c2 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/LhsValue.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/LValue.java @@ -1,4 +1,4 @@ package sootup.core.jimple.basic; /** Marker interface for Values that can be on the left side of an Assignment */ -public interface LhsValue extends Value {} +public interface LValue extends Value {} diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java index 6458b1e8046..07cf93cf5eb 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/Local.java @@ -43,7 +43,7 @@ * * @author Linghui Luo */ -public class Local implements Immediate, LhsValue, Copyable, Acceptor { +public class Local implements Immediate, LValue, Copyable, Acceptor { @Nonnull private final String name; @Nonnull private final Type type; @@ -124,7 +124,7 @@ public Position getPosition() { } /** returns the returned List can contain: Locals, JFieldRefs, JArrayRefs */ - public List getDefsOfLocal(List defs) { + public List getDefs(Collection defs) { List localDefs = new ArrayList<>(); for (Stmt stmt : defs) { if (stmt instanceof AbstractDefinitionStmt diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java index a751ff6d016..43e4d641590 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JArrayRef.java @@ -31,7 +31,7 @@ import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JArrayRef implements ConcreteRef, LhsValue, Copyable { +public final class JArrayRef implements ConcreteRef, LValue, Copyable { private final Local base; private final Immediate index; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java index 3137d6ebe0f..9560c2cd02c 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JFieldRef.java @@ -23,11 +23,11 @@ */ import javax.annotation.Nonnull; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.signatures.FieldSignature; import sootup.core.types.Type; -public abstract class JFieldRef implements ConcreteRef, LhsValue { +public abstract class JFieldRef implements ConcreteRef, LValue { @Nonnull private final FieldSignature fieldSignature; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java index 51e2e3547e2..3c6ac8f284d 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JInstanceFieldRef.java @@ -30,7 +30,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.RefVisitor; @@ -38,7 +38,7 @@ import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JInstanceFieldRef extends JFieldRef implements LhsValue, Copyable { +public final class JInstanceFieldRef extends JFieldRef implements LValue, Copyable { private final Local base; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java index a0904fdd6db..8f471346261 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/ref/JStaticFieldRef.java @@ -26,14 +26,14 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.RefVisitor; import sootup.core.signatures.FieldSignature; import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JStaticFieldRef extends JFieldRef implements LhsValue, Copyable { +public final class JStaticFieldRef extends JFieldRef implements LValue, Copyable { public JStaticFieldRef(@Nonnull FieldSignature fieldSig) { super(fieldSig); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index ba5be1b530e..3ef0a94d4c5 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -25,7 +25,7 @@ import java.util.ArrayList; import java.util.List; import javax.annotation.Nonnull; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -38,7 +38,7 @@ public abstract class AbstractDefinitionStmt extends Stmt { } @Nonnull - public abstract LhsValue getLeftOp(); + public abstract LValue getLeftOp(); @Nonnull public abstract Value getRightOp(); @@ -50,8 +50,8 @@ public Type getType() { @Override @Nonnull - public List getDefs() { - final List defs = new ArrayList<>(); + public List getDefs() { + final List defs = new ArrayList<>(); defs.add(getLeftOp()); return defs; } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java index de717d905e1..2b60821d540 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java @@ -37,7 +37,7 @@ /** Represents the assignment of one value to another */ public final class JAssignStmt extends AbstractDefinitionStmt implements Copyable { - @Nonnull final LhsValue leftOp; + @Nonnull final LValue leftOp; @Nonnull final Value rightOp; /** @@ -47,7 +47,7 @@ public final class JAssignStmt extends AbstractDefinitionStmt implements Copyabl * @param rValue the value on the right side of the assign statement. */ public JAssignStmt( - @Nonnull LhsValue variable, @Nonnull Value rValue, @Nonnull StmtPositionInfo positionInfo) { + @Nonnull LValue variable, @Nonnull Value rValue, @Nonnull StmtPositionInfo positionInfo) { super(positionInfo); leftOp = variable; rightOp = rValue; @@ -189,7 +189,7 @@ public int equivHashCode() { } @Nonnull - public JAssignStmt withVariable(@Nonnull LhsValue variable) { + public JAssignStmt withVariable(@Nonnull LValue variable) { return new JAssignStmt(variable, getRightOp(), getPositionInfo()); } @@ -205,7 +205,7 @@ public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { @Nonnull @Override - public LhsValue getLeftOp() { + public LValue getLeftOp() { return leftOp; } @@ -220,7 +220,7 @@ public Value getRightOp() { public Stmt withNewDef(@Nonnull Local newLocal) { // "ReplaceDefVisitor" final Value leftOp = getLeftOp(); - LhsValue newVal; + LValue newVal; if (leftOp instanceof ConcreteRef) { if (leftOp instanceof JArrayRef) { newVal = ((JArrayRef) leftOp).withBase(newLocal); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index ff5512962a5..33bbe1e00ef 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -27,7 +27,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.EquivTo; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractInvokeExpr; @@ -61,7 +61,7 @@ public List getUses() { * types/assignments so we return a List */ @Nonnull - public List getDefs() { + public List getDefs() { return Collections.emptyList(); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java index 82ba5f5da3b..7e26027d8f6 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java @@ -24,7 +24,7 @@ import javax.annotation.Nonnull; import sootup.core.jimple.basic.Immediate; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractConditionExpr; import sootup.core.jimple.common.expr.AbstractInvokeExpr; @@ -78,7 +78,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { refVisitor.init(oldUse, newUse); ((Ref) leftOp).accept(refVisitor); if (refVisitor.getResult() != leftOp) { - setResult(stmt.withVariable((LhsValue) refVisitor.getResult())); + setResult(stmt.withVariable((LValue) refVisitor.getResult())); } } diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index e6c60952da0..8e4135a7fea 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -431,8 +431,8 @@ public void replaceLocal(@Nonnull Local oldLocal, @Nonnull Local newLocal) { if (currStmt.getUses().contains(oldLocal)) { currStmt = currStmt.withNewUse(oldLocal, newLocal); } - final List defs = currStmt.getDefs(); - for (LhsValue def : defs) { + final List defs = currStmt.getDefs(); + for (LValue def : defs) { if (def == oldLocal || def.getUses().contains(oldLocal)) { if (currStmt instanceof AbstractDefinitionStmt) { currStmt = ((AbstractDefinitionStmt) currStmt).withNewDef(newLocal); @@ -575,11 +575,11 @@ public String toString() { * @param stmts The searched list of statements * @return A map of Locals and their using statements */ - public static Map> collectDefs(Collection stmts) { - Map> allDefs = new HashMap<>(); + public static Map> collectDefs(Collection stmts) { + Map> allDefs = new HashMap<>(); for (Stmt stmt : stmts) { - List defs = stmt.getDefs(); - for (LhsValue value : defs) { + List defs = stmt.getDefs(); + for (LValue value : defs) { Collection localDefs = allDefs.get(value); if (localDefs == null) { localDefs = new ArrayList<>(); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java index 3654abdcc9f..82f6da384e9 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java @@ -25,7 +25,7 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.JimpleComparator; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.stmt.Stmt; @@ -84,7 +84,7 @@ public List getUses() { @Nonnull @Override - public List getDefs() { + public List getDefs() { throw new UnsupportedOperationException(); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index f6e9fbf11de..e5ca51382e8 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -90,13 +90,11 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (!(val instanceof Local)) { continue; } - List defs = ((Local) val).getDefsOfLocal(stmts); + List defs = ((Local) val).getDefs(stmts); if (defs.size() != 1) { continue; } Stmt relevantDef = defs.get(0); - // TODO: ms: check why its possible that we can get relevantDes/stmt but they are not in the - // graph anymore if (!graph.containsNode(relevantDef) || !graph.containsNode(stmt)) { continue; } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java index bafad41f397..63b4b7e0a06 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolder.java @@ -69,7 +69,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } else if (stmt instanceof JReturnStmt) { for (Value value : stmt.getUses()) { if (value instanceof Local) { - List defsOfUse = ((Local) value).getDefsOfLocal(defs); + List defsOfUse = ((Local) value).getDefs(defs); if (defsOfUse.size() == 1) { AbstractDefinitionStmt definitionStmt = defsOfUse.get(0); Value rhs = definitionStmt.getRightOp(); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java index 23ce9c0c04f..229cc15f0ad 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java @@ -25,7 +25,7 @@ import sootup.core.graph.MutableBasicBlock; import sootup.core.graph.StmtGraph; import sootup.core.jimple.Jimple; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.constant.IntConstant; @@ -63,7 +63,7 @@ public DeadAssignmentEliminator(boolean eliminateOnlyStackLocals) { this.eliminateOnlyStackLocals = eliminateOnlyStackLocals; } - Map> allDefs = new HashMap<>(); + Map> allDefs = new HashMap<>(); Map> allUses = new HashMap<>(); @Override diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java index 165eb220c19..c4f91c91361 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyser.java @@ -25,7 +25,7 @@ import java.util.*; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.stmt.Stmt; @@ -78,7 +78,7 @@ public LocalLivenessAnalyser(@Nonnull StmtGraph graph) { in.add((Local) use); } } - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty()) { final Value value = defs.get(0); if (value instanceof Local) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java index fc885674399..8f7251ae06e 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalNameStandardizer.java @@ -23,7 +23,7 @@ import java.util.*; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.LocalGenerator; import sootup.core.jimple.basic.Value; @@ -45,7 +45,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi Map localToFirstOccurrence = new HashMap<>(); int defsCount = 0; for (Stmt stmt : graph) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); for (Value def : defs) { if (def instanceof Local) { final Local localDef = (Local) def; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java index 11b592d4209..c15d33208b8 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/LocalSplitter.java @@ -26,7 +26,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import sootup.core.graph.StmtGraph; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.ref.JCaughtExceptionRef; @@ -80,7 +80,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi Set visitedLocals = new LinkedHashSet<>(); Set toSplitLocals = new LinkedHashSet<>(); for (Stmt stmt : stmts) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty()) { Value def = defs.get(0); if (def instanceof Local) { @@ -102,7 +102,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi while (!stmts.isEmpty()) { Stmt currentStmt = stmts.remove(0); // At first Check the definition(left side) of the currentStmt is a local which must be split: - final List defs = currentStmt.getDefs(); + final List defs = currentStmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local && toSplitLocals.contains(defs.get(0))) { // then assign a new name to the oriLocal to get a new local which is called newLocal Local oriLocal = (Local) defs.get(0); @@ -205,7 +205,7 @@ else if (hasModifiedUse(backStmt, oriLocal)) { // then add all successors of head which are not in forwardsQueue and visitedStmts, // into the forwardsQueue. else { - final List headDefs = head.getDefs(); + final List headDefs = head.getDefs(); if (headDefs.isEmpty() || !headDefs.get(0).equivTo(oriLocal)) { for (Stmt succ : graph.successors(head)) { if (!visitedStmts.contains(succ) && !forwardsQueue.contains(succ)) { @@ -352,7 +352,7 @@ private boolean isLocalFromSameOrigin(@Nonnull Local oriLocal, Value local) { * @return if so, return true, else return false */ private boolean hasModifiedDef(@Nonnull Stmt stmt, @Nonnull Local oriLocal) { - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local) { return isLocalFromSameOrigin(oriLocal, defs.get(0)); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java index 9a409cd8e57..c85f6962b5d 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java @@ -28,7 +28,7 @@ import sootup.core.graph.DominanceFinder; import sootup.core.graph.DominanceTree; import sootup.core.graph.MutableStmtGraph; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -125,7 +125,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } } // generate new def and replace with new def - final List defs = stmt.getDefs(); + final List defs = stmt.getDefs(); if (!defs.isEmpty() && defs.get(0) instanceof Local) { Local def = (Local) defs.get(0); Local newDef = def.withName(def.getName() + "#" + nextFreeIdx); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java index e91a9bd3a40..9e1aa993ac4 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java @@ -28,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import sootup.core.graph.StmtGraph; -import sootup.core.jimple.basic.LhsValue; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.constant.Constant; @@ -98,7 +98,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { // allocation site. if (Type.isObjectLikeType(type_base) || (Type.isObject(type_base) && type_rhs instanceof PrimitiveType)) { - Map> defs = + Map> defs = Body.collectDefs(builder.getStmtGraph().getNodes()); Collection defStmts = defs.get(base); boolean findDef = false; @@ -151,8 +151,7 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { arrayType = (ArrayType) type_base; } else { if (type_base instanceof NullType || Type.isObjectLikeType(type_base)) { - Map> defs = - Body.collectDefs(builder.getStmtGraph().getNodes()); + Map> defs = Body.collectDefs(builder.getStmtGraph().getNodes()); Deque worklist = new ArrayDeque<>(); Set visited = new HashSet<>(); worklist.add(new StmtLocalPair(stmt, base)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index c38d7a116c3..4f4c81c70dd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -2,11 +2,14 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Set; +import javax.annotation.Nonnull; import org.junit.Test; import sootup.core.inputlocation.AnalysisInputLocation; +import sootup.core.inputlocation.ClassLoadingOptions; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; @@ -15,9 +18,9 @@ import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.SootMethod; +import sootup.core.transform.BodyInterceptor; import sootup.core.types.PrimitiveType; import sootup.core.util.ImmutableUtils; -import sootup.java.bytecode.inputlocation.BytecodeClassLoadingOptions; import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; import sootup.java.core.JavaIdentifierFactory; import sootup.java.core.JavaProject; @@ -116,12 +119,43 @@ public void testResource_Misuse() { JavaProject project = JavaProject.builder(language).addInputLocation(inputLocation).build(); JavaView view = project.createView(); - view.configBodyInterceptors((analysisInputLocation) -> BytecodeClassLoadingOptions.Default); - final SootMethod sootMethod = - view.getMethod(view.getIdentifierFactory().parseMethodSignature("")) - .get(); + view.configBodyInterceptors( + (analysisInputLocation) -> + new ClassLoadingOptions() { + @Nonnull + @Override + public List getBodyInterceptors() { + return Arrays.asList( + new CastAndReturnInliner(), + // new UnreachableCodeEliminator(), + // new LocalSplitter(), + new Aggregator() + // new TypeAssigner(), + // ms: is already called from typeassigner? new LocalNameStandardizer(), + /*new CopyPropagator(), + new DeadAssignmentEliminator(), + new ConditionalBranchFolder(), + new EmptySwitchEliminator(), + new NopEliminator(), + new UnusedLocalEliminator(), + new UnreachableCodeEliminator()*/ ); + } + }); + + { + final SootMethod sootMethod = + view.getMethod(view.getIdentifierFactory().parseMethodSignature("")) + .get(); + + sootMethod.getBody(); + } + { + final SootMethod sootMethod = + view.getMethod(view.getIdentifierFactory().parseMethodSignature("")) + .get(); - sootMethod.getBody(); + System.out.println(sootMethod.getBody()); + } } } diff --git a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java index ed35609048f..f10dda0056e 100644 --- a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java +++ b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java @@ -231,7 +231,7 @@ private Stmt convertArrayLoadInstruction(SSAArrayLoadInstruction inst) { } JArrayRef arrayRef = JavaJimple.getInstance().newArrayRef(base, index); int def = inst.getDef(); - LhsValue left = getLocal(base.getType(), def); + LValue left = getLocal(base.getType(), def); Position[] operandPos = new Position[1]; // TODO: loaded arrayindex position info is missing @@ -414,7 +414,7 @@ private List convertAstLexicalWrite(AstLexicalWrite inst) { } JavaClassType cSig = (JavaClassType) methodSignature.getDeclClassType(); // TODO check modifier - LhsValue left; + LValue left; if (!walaMethod.isStatic()) { FieldSignature fieldSig = identifierFactory.getFieldSignature( @@ -474,7 +474,7 @@ private List convertAstLexicalRead(AstLexicalRead inst) { private Stmt convertEnclosingObjectReference(EnclosingObjectReference inst) { Type enclosingType = converter.convertType(inst.getEnclosingType()); - LhsValue variable = getLocal(enclosingType, inst.getDef()); + LValue variable = getLocal(enclosingType, inst.getDef()); JavaClassType cSig = (JavaClassType) methodSignature.getDeclClassType(); // TODO check modifier @@ -633,7 +633,7 @@ private Stmt convertPutInstruction(SSAPutInstruction inst) { FieldSignature fieldSig = identifierFactory.getFieldSignature( fieldRef.getName().toString(), classSig, fieldType.toString()); - LhsValue fieldValue; + LValue fieldValue; if (inst.isStatic()) { fieldValue = Jimple.newStaticFieldRef(fieldSig); } else { @@ -663,7 +663,7 @@ private Stmt convertPutInstruction(SSAPutInstruction inst) { private Stmt convertNewInstruction(SSANewInstruction inst) { int result = inst.getDef(); Type type = converter.convertType(inst.getNewSite().getDeclaredType()); - LhsValue var = getLocal(type, result); + LValue var = getLocal(type, result); Value rvalue; if (type instanceof ArrayType) { int use = inst.getUse(0); @@ -707,7 +707,7 @@ private Stmt convertInstanceofInstruction(SSAInstanceofInstruction inst) { // TODO. how to get type of ref? Local op = getLocal(UnknownType.getInstance(), ref); JInstanceOfExpr expr = Jimple.newInstanceOfExpr(op, checkedType); - LhsValue left = getLocal(PrimitiveType.getBoolean(), result); + LValue left = getLocal(PrimitiveType.getBoolean(), result); Position[] operandPos = new Position[2]; // TODO: has no operand positions yet for checked and expected side @@ -726,7 +726,7 @@ private Stmt convertConversionInstruction(SSAConversionInstruction inst) { Type toType = converter.convertType(inst.getToType()); int def = inst.getDef(); int use = inst.getUse(0); - LhsValue lvalue = getLocal(toType, def); + LValue lvalue = getLocal(toType, def); Immediate rvalue; if (symbolTable.isConstant(use)) { rvalue = getConstant(use); @@ -913,7 +913,7 @@ private Stmt convertReturnInstruction(SSAReturnInstruction inst) { private List convertStringAddition( Immediate op1, Immediate op2, - LhsValue result, + LValue result, Type type, int iindex, AstMethod.DebuggingInformation debugInfo) { diff --git a/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java b/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java index 7b5cef61b99..947f3be874b 100644 --- a/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java +++ b/sootup.jimple.parser/src/main/java/sootup/jimple/parser/JimpleConverter.java @@ -487,10 +487,10 @@ public Stmt visitStmt(JimpleParser.StmtContext ctx) { return Jimple.newIdentityStmt(left, ref, pos); } else if (assignments.EQUALS() != null) { - LhsValue left = + LValue left = assignments.local != null ? getLocal(assignments.local.getText()) - : (LhsValue) valueVisitor.visitReference(assignments.reference()); + : (LValue) valueVisitor.visitReference(assignments.reference()); final Value right = valueVisitor.visitValue(assignments.value()); return Jimple.newAssignStmt(left, right, pos); diff --git a/sootup.tests/src/test/resources/interceptor/File.java b/sootup.tests/src/test/resources/interceptor/File.java new file mode 100644 index 00000000000..de8db2a6fc4 --- /dev/null +++ b/sootup.tests/src/test/resources/interceptor/File.java @@ -0,0 +1,22 @@ +import java.util.Random; + +public class File { + + public File() + { + // stub constructor + } + + public void open() { + // stub to open a file + } + + public void close() { + // stub to close a file + } + + public int size() { + Random random=new Random(); + return random.nextInt(); + } +} diff --git a/sootup.tests/src/test/resources/interceptor/Misuse.class b/sootup.tests/src/test/resources/interceptor/Misuse.class new file mode 100644 index 0000000000000000000000000000000000000000..e75f0dfa6ebb2b3ea7a15e67f8f8a21a616521ae GIT binary patch literal 1290 zcmah|T~iZD6g>@-Og0Vz0fi_^5XFfIVPR#LKv04Kqe--kQnm6lOnSrUFq4{0DA}j| zH+l9YZtYg>57-~I<>`qZWVOmvb>GhId+#~-^qqhI`|BS71=Nxl!mtj_Kn!t#vDenA zm2)kxnR|NlTH2vNe9Q5ia9d#5%pN3=5V%yAUfT(sQ<-tR-~8aj3LW1|Ajz1WQauR+ zBRVb`7{!=CT9tg8vxBo%=;xluvwi8wzzY39;6hV|Wv?Ato-GA(X5|ZfJq#SLxtWbz z-%Q_BKq&(gm=w?yNgA`9Jy8D32Bt6_Aw+1Bzyepy|JAEYJCGsoSnLX}>X=ciuL(>H zu-u*gdtcLf$A1wRAT{CdYe-f^2gagR?*Z*AF9 zkCntCmJQ_8n~%1rbXX59`^}5MvZan61?C6(@0WRIH5z=Ab46CLtm7wvxk1#jcWSv# zgI%7BT~o0?e`NUiLCdZyo0eO2efv$_`CXn%SXT*ofy@AvqSg4C$A$_mP^3M_l}T)2 zSZxms?5GAcza`o67IqChRF0VK`fc{PNA0RkyVK_IX}f-dX0+mXver2{lEDk>$W<4V zpOZBb2~@ROAKfwKrclYc-wAA~fH`gbYRif-=1}6trGCR8CgR-p`R^L7ppD&m5Alu% zfk)icH8{Bfjz0>iAO6IJ+R{u9<4ZkU%4^Dz&d1U*Ev@x%Jsn%@ zVXlYU`FIy*g@2a1$SAwo#k8{bx)@XTmo5^@ZoVTkU={Z`5y}>@64?^gB3s3JWM5)~ z-6*0*nC3n~YLg^)86*7CQuIyZDrPZ*+w5=^CM#ZN?kw^=7m&d=6)Eyu;@IzD9*=ph z;wEYw|K~iv!~zbO{fhJbmsaAh<67t1(5TRpPYMxaC=z-6Z*2Yg9^0y? c`&`8@!lt0H+7c32ro1_fu|t6heM6{z0E3qsI{*Lx literal 0 HcmV?d00001 diff --git a/sootup.tests/src/test/resources/interceptor/Misuse.java b/sootup.tests/src/test/resources/interceptor/Misuse.java index 71e69a1f82a..39ad8f22177 100644 --- a/sootup.tests/src/test/resources/interceptor/Misuse.java +++ b/sootup.tests/src/test/resources/interceptor/Misuse.java @@ -39,4 +39,11 @@ public void test() { e.printStackTrace(); } } + + + public void test1() { + File file = new File(); + file.open(); + file.close(); + } } From 94659fd3c8d81615fe0368d9b00ce6f9cc40a11e Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Tue, 26 Sep 2023 16:50:49 +0200 Subject: [PATCH 20/54] progress --- .../java/sootup/core/graph/StmtGraph.java | 6 +- .../src/main/java/sootup/core/model/Body.java | 12 ++-- .../java/sootup/core/types/ArrayType.java | 2 +- .../bytecode/frontend/AsmMethodSource.java | 11 +-- .../DeadAssignmentEliminator.java | 9 ++- .../typeresolving/AugEvalFunction.java | 31 ++++---- .../typeresolving/BytecodeHierarchy.java | 2 +- .../typeresolving/CastCounter.java | 3 +- .../interceptors/typeresolving/Typing.java | 71 +++++++++++-------- .../typeresolving/types/BottomType.java | 7 +- .../bytecode/interceptors/AggregatorTest.java | 12 ++-- 11 files changed, 91 insertions(+), 75 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java index 5a1e6237817..f9c27e8d366 100644 --- a/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/StmtGraph.java @@ -206,19 +206,19 @@ public void validateStmtConnectionsInGraph() { stmt + ": size of outgoing flows (i.e. " + successorCount - + ") does not match the amount of switch statements case labels (i.e. " + + ") does not match the amount of JSwitchStmts case labels (i.e. " + ((JSwitchStmt) stmt).getValueCount() + ")."); } } else if (stmt instanceof JIfStmt) { if (successorCount != 2) { throw new IllegalStateException( - stmt + ": If must have '2' outgoing flow but has '" + successorCount + "'."); + stmt + ": JIfStmt must have '2' outgoing flow but has '" + successorCount + "'."); } } else if (stmt instanceof JGotoStmt) { if (successorCount != 1) { throw new IllegalStateException( - stmt + ": Goto must have '1' outgoing flow but has '" + successorCount + "'."); + stmt + ": JGoto must have '1' outgoing flow but has '" + successorCount + "'."); } } diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 8e4135a7fea..c85b4f129bb 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -317,8 +317,8 @@ public Collection getUses() { * * @return a List of all the Values for Values defined by this Body's Stmts. */ - public Collection getDefs() { - ArrayList defList = new ArrayList<>(); + public Collection getDefs() { + ArrayList defList = new ArrayList<>(); for (Stmt stmt : graph.getNodes()) { defList.addAll(stmt.getDefs()); @@ -597,18 +597,18 @@ public static Map> collectDefs(Collection stmts) * @param stmts The searched list of statements * @return A map of Locals and their using statements */ - public static Map> collectUses(List stmts) { - Map> allUses = new HashMap<>(); + public static Map> collectUses(Collection stmts) { + Map> allUses = new HashMap<>(); for (Stmt stmt : stmts) { Collection uses = stmt.getUses(); for (Value value : uses) { - if (value instanceof Local) { + if (value instanceof LValue) { Collection localUses = allUses.get(value); if (localUses == null) { localUses = new ArrayList<>(); } localUses.add(stmt); - allUses.put((Local) value, localUses); + allUses.put((LValue) value, localUses); } } } diff --git a/sootup.core/src/main/java/sootup/core/types/ArrayType.java b/sootup.core/src/main/java/sootup/core/types/ArrayType.java index 4a664a01fb5..61111d25218 100644 --- a/sootup.core/src/main/java/sootup/core/types/ArrayType.java +++ b/sootup.core/src/main/java/sootup/core/types/ArrayType.java @@ -42,7 +42,7 @@ public ArrayType(Type baseType, int dimension) { || baseType instanceof ClassType || baseType instanceof NullType)) { throw new RuntimeException( - "The type: " + baseType + "can not be as a base type of an ArrayType."); + "The type: '" + baseType + "' can not be as a base type of an ArrayType."); } if (dimension < 1) { throw new RuntimeException("The dimension of array type should be at least 1."); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java index 9a171328da4..3431476513c 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java @@ -289,7 +289,7 @@ private JavaLocal getOrCreateLocal(int idx) { } JavaLocal local = locals.get(idx); if (local == null) { - String name = determineLocalName(idx); + String name = determineLocalName(idx, false); // FIXME: isField local = JavaJimple.newLocal(name, UnknownType.getInstance(), Collections.emptyList()); locals.set(idx, local); } @@ -297,7 +297,7 @@ private JavaLocal getOrCreateLocal(int idx) { } @Nonnull - private String determineLocalName(int idx) { + private String determineLocalName(int idx, boolean isField) { String name; if (localVariables != null) { name = null; @@ -314,7 +314,7 @@ private String determineLocalName(int idx) { } else { name = "l" + idx; } - return name; + return isField ? name : "$" + name; } void setStmt(@Nonnull AbstractInsnNode insn, @Nonnull Stmt stmt) { @@ -1874,7 +1874,8 @@ private List buildPreambleLocals(Body.BodyBuilder bodyBuilder) { int localIdx = 0; // create this Local if necessary ( i.e. not static ) if (!bodyBuilder.getModifiers().contains(MethodModifier.STATIC)) { - JavaLocal thisLocal = JavaJimple.newLocal(determineLocalName(localIdx), declaringClass); + JavaLocal thisLocal = + JavaJimple.newLocal(determineLocalName(localIdx, false), declaringClass); locals.set(localIdx++, thisLocal); final JIdentityStmt stmt = Jimple.newIdentityStmt(thisLocal, Jimple.newThisRef(declaringClass), methodPosInfo); @@ -1886,7 +1887,7 @@ private List buildPreambleLocals(Body.BodyBuilder bodyBuilder) { // [BH] parameterlocals do not exist yet -> create with annotation JavaLocal local = JavaJimple.newLocal( - determineLocalName(localIdx), + determineLocalName(localIdx, false), parameterType, AsmUtil.createAnnotationUsage( invisibleParameterAnnotations == null ? null : invisibleParameterAnnotations[i])); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java index 229cc15f0ad..4cd09e51b5a 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminator.java @@ -64,7 +64,7 @@ public DeadAssignmentEliminator(boolean eliminateOnlyStackLocals) { } Map> allDefs = new HashMap<>(); - Map> allUses = new HashMap<>(); + Map> allUses = new HashMap<>(); @Override public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View view) { @@ -98,7 +98,6 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi && (!eliminateOnlyStackLocals || ((Local) lhs).getName().startsWith("$") || lhs.getType() instanceof NullType)) { - // FIXME: [ms] inspection says right side of && is always true isEssential = false; if (!containsInvoke) { @@ -178,7 +177,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (containsInvoke || !allEssential) { // Add all the statements which are used to compute values for the essential statements, // recursively - allDefs = Body.collectDefs(builder.getStmts()); + allDefs = Body.collectDefs(builder.getStmtGraph().getNodes()); if (!allEssential) { Set essentialStmts = new HashSet<>(stmts.size()); @@ -206,7 +205,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi } if (containsInvoke) { - allUses = Body.collectUses(builder.getStmts()); + allUses = Body.collectUses(builder.getStmtGraph().getNodes()); // Eliminate dead assignments from invokes such as x = f(), where x is no longer used List postProcess = new ArrayList<>(); for (Stmt stmt : stmts) { @@ -218,7 +217,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (assignStmt.getRightOp() instanceof Local) { Local value = (Local) assignStmt.getRightOp(); for (Stmt use : allUses.get(value)) { - if (builder.getStmts().contains(use)) { + if (builder.getStmtGraph().containsNode(use)) { deadAssignment = false; break; } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index eedd635b7dd..6a3929e0271 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -84,7 +84,7 @@ public Type evaluate( @Nonnull Stmt stmt, @Nonnull StmtGraph graph) { - // TODO: [ms] make use of the ValueVisitor.. + // TODO: [ms] make use of the ValueVisitor if (value instanceof Immediate) { if (value instanceof Local) { @@ -123,18 +123,22 @@ public Type evaluate( } else if (value.getClass() == MethodType.class) { return methodTypeClassType; } else { - return null; - // throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + - // "'."); + // return null; + throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); } } } else if (value instanceof Expr) { if (value instanceof AbstractBinopExpr) { Type tl = evaluate(typing, ((AbstractBinopExpr) value).getOp1(), stmt, graph); if (tl == null) { - return null; + // return null; + throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); } Type tr = evaluate(typing, ((AbstractBinopExpr) value).getOp2(), stmt, graph); + if (tr == null) { + // return null; + throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); + } if (value instanceof AbstractIntBinopExpr) { if (value instanceof AbstractConditionExpr) { return PrimitiveType.getBoolean(); @@ -191,9 +195,8 @@ public Type evaluate( exceptionClass = (SootClass) exceptionClassOp.get(); } else { // Logger.info("ExceptionType '" + exceptionType + "' is not in the view"); - return null; - // throw new RuntimeException("ExceptionType '" + exceptionType + "' is not in the - // view"); + // return null; + throw new RuntimeException("ExceptionType '" + exceptionType + "' is not in the view"); } if (exceptionClass.isPhantomClass()) { return throwableClassType; @@ -204,11 +207,10 @@ public Type evaluate( } } - /* if (type == null) { - return null; - throw new RuntimeException("inevaluatable reference in AugEvalFunction '" + value +"'."); - }*/ + // return null; + throw new RuntimeException("inevaluatable reference in AugEvalFunction '" + value + "'."); + } return type; } else if (value instanceof JArrayRef) { Type type = typing.getType(((JArrayRef) value).getBase()); @@ -226,9 +228,8 @@ public Type evaluate( || value instanceof JFieldRef) { return value.getType(); } else { - return null; - // throw new RuntimeException("Inevaluatable reference in AugEvalFunction '" + value + - // "'."); + // return null; + throw new RuntimeException("Inevaluatable reference in AugEvalFunction '" + value + "'."); } } return null; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java index e86c92abfed..49f741b086d 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchy.java @@ -97,7 +97,7 @@ public boolean isAncestor(@Nonnull Type ancestor, @Nonnull Type child) { return canStoreType((ClassType) ancestorBase, (ClassType) childBase); } } else if (ancestorArr.getDimension() < childArr.getDimension()) { - // TODO: [ms] check: the dimension condition check seems weird? + // TODO: [ms] check: the dimension condition check as it seems weird? return ancestorBase == objectClassType || ancestorBase == serializableClassType || ancestorBase == cloneableClassType; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index d421681602a..a6391fc6146 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -35,6 +35,7 @@ import sootup.core.model.Body; import sootup.core.types.Type; +/** FIXME: outline what this class does */ public class CastCounter extends TypeChecker { private int castCount = 0; @@ -133,7 +134,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm } else { newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); } - if (graph.containsNode(stmt)) { + if (newStmt != null && graph.containsNode(stmt)) { builder.replaceStmt(stmt, newStmt); this.stmt2NewStmt.put(oriStmt, newStmt); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/Typing.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/Typing.java index e7d1b20c7dd..da049bc4877 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/Typing.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/Typing.java @@ -20,20 +20,20 @@ * . * #L% */ -import java.util.BitSet; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; +import java.util.*; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import sootup.core.jimple.basic.Local; import sootup.core.types.Type; import sootup.java.bytecode.interceptors.typeresolving.types.BottomType; public class Typing { - private HashMap local2Type = new HashMap<>(); - private BitSet stmtsIDList; + @Nonnull private Map local2Type; + @Nonnull private BitSet stmtsIDList; public Typing(@Nonnull Collection locals) { + // initialize + local2Type = new IdentityHashMap<>(locals.size()); for (Local local : locals) { local2Type.put(local, BottomType.getInstance()); } @@ -41,11 +41,12 @@ public Typing(@Nonnull Collection locals) { } public Typing(@Nonnull Typing typing, @Nonnull BitSet stmtsIDList) { - this.local2Type = new HashMap<>(typing.local2Type); + this.local2Type = new IdentityHashMap<>(typing.local2Type); this.stmtsIDList = stmtsIDList; } - public Type getType(Local local) { + @Nullable + protected Type getType(@Nonnull Local local) { return local2Type.get(local); } @@ -61,10 +62,11 @@ public Map getMap() { return this.local2Type; } - public void setStmtsIDList(BitSet bitSet) { + public void setStmtsIDList(@Nonnull BitSet bitSet) { this.stmtsIDList = bitSet; } + @Nonnull public BitSet getStmtsIDList() { return this.stmtsIDList; } @@ -80,33 +82,40 @@ public BitSet getStmtsIDList() { * more specific than the given typing, but another local's type is more general than the * given typing. */ - public int compare(Typing typing, BytecodeHierarchy hierarchy, Collection localsToIgnore) { - if (!typing.getLocals().equals(this.getLocals())) { + public int compare( + @Nonnull Typing typing, + @Nonnull BytecodeHierarchy hierarchy, + @Nonnull Collection localsToIgnore) { + + if (!typing.getLocals().equals(this.getLocals())) { // TODO: ms: check isnt it even == then? throw new RuntimeException("The compared typings should have the same locals' set!"); } + int ret = 0; - for (Local local : this.local2Type.keySet()) { - if (!localsToIgnore.contains(local)) { - Type ta = getType(local); - Type tb = typing.getType(local); + for (Map.Entry local : this.local2Type.entrySet()) { + if (localsToIgnore.contains(local.getKey())) { + continue; + } - int cmp; - if (ta.equals(tb)) { - cmp = 0; - } else if (hierarchy.isAncestor(ta, tb)) { - cmp = 1; - } else if (hierarchy.isAncestor(tb, ta)) { - cmp = -1; - } else { - return -2; - } + Type ta = local.getValue(); + Type tb = typing.getType(local.getKey()); - if ((cmp == 1 && ret == -1) || (cmp == -1 && ret == 1)) { - return 2; - } - if (ret == 0) { - ret = cmp; - } + int cmp; + if (ta.equals(tb)) { + cmp = 0; + } else if (hierarchy.isAncestor(ta, tb)) { + cmp = 1; + } else if (hierarchy.isAncestor(tb, ta)) { + cmp = -1; + } else { + return -2; + } + + if ((cmp == 1 && ret == -1) || (cmp == -1 && ret == 1)) { + return 2; + } + if (ret == 0) { + ret = cmp; } } return ret; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/BottomType.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/BottomType.java index 13ac0def779..aa71c942a9a 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/BottomType.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/BottomType.java @@ -26,7 +26,7 @@ import sootup.core.types.Type; /** - * This type is imaginary type, and used for Type Inference + * This type is an imaginary/intermediate type which is used to calculate Type Inference * * @author Zun Wang */ @@ -45,4 +45,9 @@ private BottomType() {} public void accept(@Nonnull TypeVisitor v) { // todo: add bottom type case } + + @Override + public String toString() { + return "BottomType"; + } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 4f4c81c70dd..115fdeca057 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -128,18 +128,18 @@ public void testResource_Misuse() { public List getBodyInterceptors() { return Arrays.asList( new CastAndReturnInliner(), - // new UnreachableCodeEliminator(), - // new LocalSplitter(), - new Aggregator() - // new TypeAssigner(), + new UnreachableCodeEliminator(), + new LocalSplitter(), + // FIXME new Aggregator() + new TypeAssigner(), // ms: is already called from typeassigner? new LocalNameStandardizer(), - /*new CopyPropagator(), + new CopyPropagator(), new DeadAssignmentEliminator(), new ConditionalBranchFolder(), new EmptySwitchEliminator(), new NopEliminator(), new UnusedLocalEliminator(), - new UnreachableCodeEliminator()*/ ); + new UnreachableCodeEliminator()); } }); From a374cd4293536ac8ebc199c27d6da8b5b996a74c Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 27 Sep 2023 16:25:01 +0200 Subject: [PATCH 21/54] colon without a mandatory whitespace should be parsed as well --- .../src/main/java/sootup/java/core/JavaIdentifierFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java b/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java index c9742affc49..1b9f7a8f7c3 100644 --- a/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java +++ b/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java @@ -349,7 +349,7 @@ private static final class MethodSignatureParserPatternHolder { @Nonnull private static final Pattern SOOT_METHOD_SIGNATURE_PATTERN = Pattern.compile( - "^<(?[^:]+):\\s+(?[^\\s]+)\\s+(?[^(]+)\\((?[^)]+)?\\)>$"); + "^<(?[^:]+):\\s*(?[^\\s]+)\\s+(?[^(]+)\\((?[^)]+)?\\)>$"); @Nonnull private static final Pattern JAVADOCLIKE_METHOD_SIGNATURE_PATTERN = From 5d3c01dcbefdedc73ffa0e357df98e1382c38295 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 4 Oct 2023 11:00:30 +0200 Subject: [PATCH 22/54] remove SourceType.Phantom as it does not make sence in SootUp anymore; remove leftover from port in TypeAssigner --- .../java/sootup/core/model/SootClass.java | 5 ---- .../java/sootup/core/model/SourceType.java | 3 --- .../BytecodeBodyInterceptors.java | 27 ++++++++++--------- .../typeresolving/AugEvalFunction.java | 15 +++-------- 4 files changed, 19 insertions(+), 31 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/model/SootClass.java b/sootup.core/src/main/java/sootup/core/model/SootClass.java index afa988069e8..a3e8155d431 100644 --- a/sootup.core/src/main/java/sootup/core/model/SootClass.java +++ b/sootup.core/src/main/java/sootup/core/model/SootClass.java @@ -233,11 +233,6 @@ public boolean isLibraryClass() { return sourceType.equals(SourceType.Library); } - /** Returns true if this class is a phantom class. */ - public boolean isPhantomClass() { - return sourceType.equals(SourceType.Phantom); - } - /** Convenience method returning true if this class is private. */ public boolean isPrivate() { return ClassModifier.isPrivate(this.getModifiers()); diff --git a/sootup.core/src/main/java/sootup/core/model/SourceType.java b/sootup.core/src/main/java/sootup/core/model/SourceType.java index 95afd02f847..c342c7fe00d 100644 --- a/sootup.core/src/main/java/sootup/core/model/SourceType.java +++ b/sootup.core/src/main/java/sootup/core/model/SourceType.java @@ -23,9 +23,6 @@ */ public enum SourceType { - Phantom, // in soot: code that we have no access to and only can assume the class layout.. TODO: - // which seems to makes no sense anymore as we now reference indirectly to SootClasses - // and dont need a "catch all" Phantom SootClass? Application, // code that we want to analyze e.g. in call graph generation its traversed Library // code that is given but should not be analyzed e.g. in call graph generation its not // traversed diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java index 2eab5cc898f..944a91b084c 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java @@ -29,19 +29,22 @@ /** Built-in sets of {@link BodyInterceptor}s for the bytecode frontend */ public enum BytecodeBodyInterceptors { Default( - new CastAndReturnInliner(), - new UnreachableCodeEliminator(), - new LocalSplitter(), - new Aggregator(), - new TypeAssigner(), + /* new CastAndReturnInliner(), + new UnreachableCodeEliminator(), + new LocalSplitter(), + new Aggregator(), + + */ + new TypeAssigner() // ms: is already called from typeassigner? new LocalNameStandardizer(), - new CopyPropagator(), - new DeadAssignmentEliminator(), - new ConditionalBranchFolder(), - new EmptySwitchEliminator(), - new NopEliminator(), - new UnusedLocalEliminator(), - new UnreachableCodeEliminator()); + /* new CopyPropagator(), + new DeadAssignmentEliminator(), + new ConditionalBranchFolder(), + new EmptySwitchEliminator(), + new NopEliminator(), + new UnusedLocalEliminator(), + new UnreachableCodeEliminator() + */ ); @Nonnull private final List bodyInterceptors; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index 6a3929e0271..aaee4c6e4fc 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -189,18 +189,11 @@ public Type evaluate( Set exceptionTypes = getExceptionTypeCandidates(stmt, graph); ClassType type = null; for (ClassType exceptionType : exceptionTypes) { - Optional exceptionClassOp = view.getClass(exceptionType); - SootClass exceptionClass; - if (exceptionClassOp.isPresent()) { - exceptionClass = (SootClass) exceptionClassOp.get(); - } else { - // Logger.info("ExceptionType '" + exceptionType + "' is not in the view"); - // return null; - throw new RuntimeException("ExceptionType '" + exceptionType + "' is not in the view"); - } - if (exceptionClass.isPhantomClass()) { + Optional exceptionClassOpt = view.getClass(exceptionType); + if (!exceptionClassOpt.isPresent()) { return throwableClassType; - } else if (type == null) { + } + if (type == null) { type = exceptionType; } else { type = getLeastCommonExceptionType(type, exceptionType); From 366b3e84daa0ccd4ab8c7d4dca6f84e125038b66 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 4 Oct 2023 14:33:03 +0200 Subject: [PATCH 23/54] add FallsThroughStmt interface; make BranchingStmt and Stmt an interface as well; --- .../icfg/BiDiInterproceduralCFG.java | 3 +- .../core/graph/MutableBlockStmtGraph.java | 13 +- .../core/jimple/basic/JimpleComparator.java | 10 +- .../common/stmt/AbstractDefinitionStmt.java | 2 +- .../jimple/common/stmt/AbstractOpStmt.java | 59 ------- .../core/jimple/common/stmt/AbstractStmt.java | 150 ++++++++++++++++++ .../jimple/common/stmt/BranchingStmt.java | 12 +- .../jimple/common/stmt/FallsThroughStmt.java | 14 ++ .../core/jimple/common/stmt/JAssignStmt.java | 47 +++--- .../core/jimple/common/stmt/JGotoStmt.java | 8 +- .../jimple/common/stmt/JIdentityStmt.java | 13 +- .../core/jimple/common/stmt/JIfStmt.java | 8 +- .../core/jimple/common/stmt/JInvokeStmt.java | 3 +- .../core/jimple/common/stmt/JNopStmt.java | 3 +- .../core/jimple/common/stmt/JReturnStmt.java | 30 +++- .../jimple/common/stmt/JReturnVoidStmt.java | 3 +- .../core/jimple/common/stmt/JThrowStmt.java | 30 +++- .../sootup/core/jimple/common/stmt/Stmt.java | 130 ++------------- .../javabytecode/stmt/JBreakpointStmt.java | 6 +- .../javabytecode/stmt/JEnterMonitorStmt.java | 33 +++- .../javabytecode/stmt/JExitMonitorStmt.java | 33 +++- .../jimple/javabytecode/stmt/JRetStmt.java | 7 +- .../jimple/javabytecode/stmt/JSwitchStmt.java | 10 +- .../bytecode/frontend/AsmMethodSource.java | 4 +- .../java/bytecode/frontend/StmtContainer.java | 3 +- .../typeresolving/AugEvalFunction.java | 30 ++-- .../typeresolving/CastCounter.java | 7 +- .../typeresolving/TypeResolver.java | 10 +- 28 files changed, 414 insertions(+), 267 deletions(-) delete mode 100644 sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractOpStmt.java create mode 100644 sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java create mode 100644 sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java diff --git a/sootup.analysis/src/main/java/sootup/analysis/interprocedural/icfg/BiDiInterproceduralCFG.java b/sootup.analysis/src/main/java/sootup/analysis/interprocedural/icfg/BiDiInterproceduralCFG.java index 6efd0fcf439..9f809265c89 100644 --- a/sootup.analysis/src/main/java/sootup/analysis/interprocedural/icfg/BiDiInterproceduralCFG.java +++ b/sootup.analysis/src/main/java/sootup/analysis/interprocedural/icfg/BiDiInterproceduralCFG.java @@ -28,9 +28,10 @@ import java.util.Set; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Value; +import sootup.core.jimple.common.stmt.AbstractStmt; /** - * @param Nodes in the CFG, e.g. {@link sootup.core.jimple.common.stmt.Stmt} + * @param Nodes in the CFG, e.g. {@link AbstractStmt} * @param Method representation, e.g. {@link sootup.core.model.SootMethod} */ public interface BiDiInterproceduralCFG extends InterproceduralCFG { diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 5039e1a81c2..45dbcdebdeb 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -1,6 +1,7 @@ package sootup.core.graph; import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Lists; import java.util.*; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -653,7 +654,8 @@ protected boolean isMergeable( return false; } final List fBlocksuccessors = firstBlock.getSuccessors(); - if (fBlocksuccessors.size() != 1 || fBlocksuccessors.get(0) != followingBlock) { + if (fBlocksuccessors.size() > 1 + || (fBlocksuccessors.size() == 1 && fBlocksuccessors.get(0) != followingBlock)) { return false; } // if we are here the datastructure should have managed that the next if is true.. @@ -833,14 +835,19 @@ public void insertBefore( // merge blocks - performance hint: if exceptionMap equals the current blocks exception and // the stmts have only fallsthrough Stmts there could be some allocation/deallocation be saved final MutableBasicBlock predecessorBlock = addBlockInternal(stmts, exceptionMap); - for (MutableBasicBlock predecessor : block.getPredecessors()) { + for (MutableBasicBlock predecessor : Lists.newArrayList(block.getPredecessors())) { // cleanup old predecessor.removeSuccessorBlock(block); block.removePredecessorBlock(predecessor); // add new link linkBlocks(predecessor, predecessorBlock); } - tryMergeBlocks(predecessorBlock, block); + + if (!tryMergeBlocks(predecessorBlock, block)) { + // hint: ms: this could be bad/unintuitive behaviour for a branching stmt for branching + predecessorBlock.addSuccessorBlock(block); + block.addPredecessorBlock(predecessorBlock); + } } else { final MutableBasicBlock successorBlock = block.splitBlockLinked(beforeStmt, true); exceptionMap.forEach( diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java b/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java index a4d2017a730..350f4e234ad 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java @@ -102,7 +102,7 @@ public boolean caseBlock(BasicBlock block, Object o) { } public boolean caseStmt(Stmt stmt, Object o) { - if (!(o instanceof Stmt)) { + if (!(o instanceof AbstractStmt)) { return false; } else if (stmt instanceof JBreakpointStmt) { return caseBreakpointStmt((JBreakpointStmt) stmt, o); @@ -168,14 +168,14 @@ public boolean caseEnterMonitorStmt(JEnterMonitorStmt stmt, Object o) { if (!(o instanceof JEnterMonitorStmt)) { return false; } - return stmt.getOp().equivTo(((AbstractOpStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); } public boolean caseExitMonitorStmt(JExitMonitorStmt stmt, Object o) { if (!(o instanceof JExitMonitorStmt)) { return false; } - return stmt.getOp().equivTo(((AbstractOpStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); } public boolean caseGotoStmt(JGotoStmt stmt, Object o) { @@ -233,7 +233,7 @@ public boolean caseReturnStmt(JReturnStmt stmt, Object o) { if (!(o instanceof JReturnStmt)) { return false; } - return stmt.getOp().equivTo(((AbstractOpStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); } public boolean caseReturnVoidStmt(JReturnVoidStmt stmt, Object o) { @@ -244,7 +244,7 @@ public boolean caseThrowStmt(JThrowStmt stmt, Object o) { if (!(o instanceof JThrowStmt)) { return false; } - return stmt.getOp().equivTo(((AbstractOpStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); } public boolean caseAbstractBinopExpr(AbstractBinopExpr obj, Object o) { diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index 3ef0a94d4c5..e9236b3169f 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -31,7 +31,7 @@ import sootup.core.jimple.basic.Value; import sootup.core.types.Type; -public abstract class AbstractDefinitionStmt extends Stmt { +public abstract class AbstractDefinitionStmt extends AbstractStmt { AbstractDefinitionStmt(@Nonnull StmtPositionInfo positionInfo) { super(positionInfo); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractOpStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractOpStmt.java deleted file mode 100644 index fa788ece3e7..00000000000 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractOpStmt.java +++ /dev/null @@ -1,59 +0,0 @@ -package sootup.core.jimple.common.stmt; - -/*- - * #%L - * Soot - a J*va Optimization Framework - * %% - * Copyright (C) 1997-2020 Etienne Gagnon, Linghui Luo, Markus Schmidt and others - * %% - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Lesser General Public License as - * published by the Free Software Foundation, either version 2.1 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Lesser Public License for more details. - * - * You should have received a copy of the GNU General Lesser Public - * License along with this program. If not, see - * . - * #L% - */ - -import java.util.ArrayList; -import java.util.List; -import javax.annotation.Nonnull; -import sootup.core.jimple.basic.Immediate; -import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.basic.Value; - -public abstract class AbstractOpStmt extends Stmt { - - protected final Immediate op; - - protected AbstractOpStmt(@Nonnull Immediate op, @Nonnull StmtPositionInfo positionInfo) { - super(positionInfo); - this.op = op; - } - - @Nonnull - public Immediate getOp() { - return op; - } - - @Override - @Nonnull - public final List getUses() { - final List uses = op.getUses(); - List list = new ArrayList<>(uses.size() + 1); - list.add(op); - return list; - } - - @Override - public int equivHashCode() { - return op.equivHashCode(); - } -} diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java new file mode 100644 index 00000000000..49d2534b746 --- /dev/null +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java @@ -0,0 +1,150 @@ +package sootup.core.jimple.common.stmt; + +/*- + * #%L + * Soot - a J*va Optimization Framework + * %% + * Copyright (C) 2018-2020 Linghui Luo, Christian Brüggemann, Markus Schmidt + * %% + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Lesser General Public License as + * published by the Free Software Foundation, either version 2.1 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Lesser Public License for more details. + * + * You should have received a copy of the GNU General Lesser Public + * License along with this program. If not, see + * . + * #L% + */ + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import javax.annotation.Nonnull; +import sootup.core.jimple.basic.LValue; +import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.basic.Value; +import sootup.core.jimple.common.expr.AbstractInvokeExpr; +import sootup.core.jimple.common.ref.JArrayRef; +import sootup.core.jimple.common.ref.JFieldRef; +import sootup.core.jimple.visitor.ReplaceUseStmtVisitor; + +public abstract class AbstractStmt implements Stmt { + + protected final StmtPositionInfo positionInfo; + + public AbstractStmt(@Nonnull StmtPositionInfo positionInfo) { + this.positionInfo = positionInfo; + } + + /** + * Returns a list of Values used in this Stmt. Note that they are returned in usual evaluation + * order. + */ + @Override + @Nonnull + public List getUses() { + return Collections.emptyList(); + } + + /** + * Returns a list of Values defined in this Stmt. There are languages which allow multiple return + * types/assignments so we return a List + */ + @Override + @Nonnull + public List getDefs() { + return Collections.emptyList(); + } + + /** Returns a list of Values, either used or defined or both in this Stmt. */ + @Override + @Nonnull + public List getUsesAndDefs() { + List uses = getUses(); + List defs = new ArrayList<>(getDefs()); + if (uses.isEmpty()) { + return defs; + } else if (defs.isEmpty()) { + return uses; + } else { + List values = new ArrayList<>(); + values.addAll(defs); + values.addAll(uses); + return values; + } + } + + /** Returns the amount of unexceptional successors the Stmt needs to have in the StmtGraph. */ + @Override + public int getExpectedSuccessorCount() { + return 1; + } + + @Override + public boolean containsInvokeExpr() { + return false; + } + + /** + * This method must only be used for Stmts which contain an InvokeExpr (JInvokeStmt; possible in + * JAssignStmt) check via containsInvokExpr(). + */ + @Override + public AbstractInvokeExpr getInvokeExpr() { + throw new RuntimeException("getInvokeExpr() called with no invokeExpr present!"); + } + + @Override + public boolean containsArrayRef() { + return false; + } + + /** + * This method must only be used for Stmts which contain an ArrayRef - possible with JAssignStmts. + * check via containsArrayRef(). + */ + @Override + public JArrayRef getArrayRef() { + throw new RuntimeException("getArrayRef() called with no ArrayRef present!"); + } + + @Override + public boolean containsFieldRef() { + return false; + } + + /** + * This method must only be used for Stmts which contain an FieldRef - possible with JAssignStmts. + * check via containsFieldRef(). + */ + @Override + public JFieldRef getFieldRef() { + throw new RuntimeException("getFieldRef() called with no JFieldRef present!"); + } + + @Override + public StmtPositionInfo getPositionInfo() { + return positionInfo; + } + + /** + * Use newUse to replace the oldUse in oldStmt. + * + * @param oldUse a Value in the useList of oldStmt. + * @param newUse a Value is to replace oldUse + * @return a new Stmt with newUse + */ + @Override + @Nonnull + public Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse) { + ReplaceUseStmtVisitor visitor = new ReplaceUseStmtVisitor(oldUse, newUse); + accept(visitor); + return visitor.getResult(); + } +} diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/BranchingStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/BranchingStmt.java index 06df9c8e350..a0f74b3577b 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/BranchingStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/BranchingStmt.java @@ -23,8 +23,6 @@ */ import java.util.List; -import javax.annotation.Nonnull; -import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.model.Body; /** @@ -33,15 +31,11 @@ * *

mandatory: branches() needs to be true! */ -public abstract class BranchingStmt extends Stmt { - public BranchingStmt(@Nonnull StmtPositionInfo positionInfo) { - super(positionInfo); - } - - public abstract List getTargetStmts(Body body); +public interface BranchingStmt extends Stmt { + List getTargetStmts(Body body); @Override - public final boolean branches() { + default boolean branches() { return true; } } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java new file mode 100644 index 00000000000..04f58bd22ec --- /dev/null +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java @@ -0,0 +1,14 @@ +package sootup.core.jimple.common.stmt; + + + +/** as an equivalent to BranchingStmt */ +public interface FallsThroughStmt extends Stmt { + + // has to return true in subclasses! + // hint: this class can't be abstract and method final because of JIfStmt which would need + // FallsThrough and BranchingStmt as parent. + default boolean fallsThrough() { + return true; + } +} diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java index 2b60821d540..871bbc91dd8 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JAssignStmt.java @@ -31,11 +31,10 @@ import sootup.core.jimple.common.ref.JFieldRef; import sootup.core.jimple.common.ref.JInstanceFieldRef; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** Represents the assignment of one value to another */ -public final class JAssignStmt extends AbstractDefinitionStmt implements Copyable { +public final class JAssignStmt extends AbstractDefinitionStmt implements FallsThroughStmt { @Nonnull final LValue leftOp; @Nonnull final Value rightOp; @@ -188,21 +187,6 @@ public int equivHashCode() { return getLeftOp().equivHashCode() + 31 * getRightOp().equivHashCode(); } - @Nonnull - public JAssignStmt withVariable(@Nonnull LValue variable) { - return new JAssignStmt(variable, getRightOp(), getPositionInfo()); - } - - @Nonnull - public JAssignStmt withRValue(@Nonnull Value rValue) { - return new JAssignStmt(getLeftOp(), rValue, getPositionInfo()); - } - - @Nonnull - public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { - return new JAssignStmt(getLeftOp(), getRightOp(), positionInfo); - } - @Nonnull @Override public LValue getLeftOp() { @@ -215,9 +199,19 @@ public Value getRightOp() { return rightOp; } + @Override + public boolean fallsThrough() { + return true; + } + + @Override + public boolean branches() { + return false; + } + @Nonnull @Override - public Stmt withNewDef(@Nonnull Local newLocal) { + public JAssignStmt withNewDef(@Nonnull Local newLocal) { // "ReplaceDefVisitor" final Value leftOp = getLeftOp(); LValue newVal; @@ -231,9 +225,24 @@ public Stmt withNewDef(@Nonnull Local newLocal) { return this; } } else { - // its a Local.. + // it's a Local newVal = newLocal; } return withVariable(newVal); } + + @Nonnull + public JAssignStmt withVariable(@Nonnull LValue variable) { + return new JAssignStmt(variable, getRightOp(), getPositionInfo()); + } + + @Nonnull + public JAssignStmt withRValue(@Nonnull Value rValue) { + return new JAssignStmt(getLeftOp(), rValue, getPositionInfo()); + } + + @Nonnull + public JAssignStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { + return new JAssignStmt(getLeftOp(), getRightOp(), positionInfo); + } } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java index 9a5a2f00810..6e7bdf798bc 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java @@ -29,11 +29,10 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.visitor.StmtVisitor; import sootup.core.model.Body; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** Unconditionally jumps to a target Stmt */ -public class JGotoStmt extends BranchingStmt implements Copyable { +public class JGotoStmt extends AbstractStmt implements BranchingStmt { public JGotoStmt(StmtPositionInfo positionInfo) { super(positionInfo); @@ -73,6 +72,11 @@ public boolean fallsThrough() { return false; } + @Override + public boolean branches() { + return true; + } + @Override public boolean equivTo(@Nonnull Object o, @Nonnull JimpleComparator comparator) { return comparator.caseGotoStmt(this, o); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java index 1d8d543766e..d0e57427df5 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java @@ -28,10 +28,9 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.ref.IdentityRef; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JIdentityStmt extends AbstractDefinitionStmt implements Copyable { +public final class JIdentityStmt extends AbstractDefinitionStmt implements FallsThroughStmt { @Nonnull final Local leftOp; @Nonnull final IdentityRef rightOp; @@ -61,6 +60,16 @@ public IdentityRef getRightOp() { return rightOp; } + @Override + public boolean fallsThrough() { + return true; + } + + @Override + public boolean branches() { + return false; + } + @Override public void toString(@Nonnull StmtPrinter up) { getLeftOp().toString(up); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java index 36eecb1a366..be6baa406e1 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java @@ -32,7 +32,6 @@ import sootup.core.jimple.common.expr.AbstractConditionExpr; import sootup.core.jimple.visitor.StmtVisitor; import sootup.core.model.Body; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** @@ -40,7 +39,7 @@ * successor (index=0) is the fallsthrough Stmt and the second successor (index=1) is the rbanching * one. */ -public final class JIfStmt extends BranchingStmt implements Copyable { +public final class JIfStmt extends AbstractStmt implements BranchingStmt, FallsThroughStmt { @Nonnull private final AbstractConditionExpr condition; @@ -101,6 +100,11 @@ public boolean fallsThrough() { return true; } + @Override + public boolean branches() { + return true; + } + @Override public boolean equivTo(@Nonnull Object o, @Nonnull JimpleComparator comparator) { return comparator.caseIfStmt(this, o); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JInvokeStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JInvokeStmt.java index 1832f534c67..cf2fa3e3653 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JInvokeStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JInvokeStmt.java @@ -29,11 +29,10 @@ import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractInvokeExpr; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A method call */ -public final class JInvokeStmt extends Stmt implements Copyable { +public final class JInvokeStmt extends AbstractStmt implements FallsThroughStmt { @Nonnull private final AbstractInvokeExpr invokeExpr; diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JNopStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JNopStmt.java index 6220eb27942..4b71e27e10c 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JNopStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JNopStmt.java @@ -27,11 +27,10 @@ import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A no-operation statement. */ -public final class JNopStmt extends Stmt implements Copyable { +public final class JNopStmt extends AbstractStmt implements FallsThroughStmt { public JNopStmt(@Nonnull StmtPositionInfo positionInfo) { super(positionInfo); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnStmt.java index cdf5a9710c2..eb8906c1249 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnStmt.java @@ -22,20 +22,25 @@ * #L% */ +import java.util.ArrayList; +import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Immediate; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A statement that ends the method, returning a value. */ -public final class JReturnStmt extends AbstractOpStmt implements Copyable { +public final class JReturnStmt extends AbstractStmt { + + protected final Immediate op; public JReturnStmt(@Nonnull Immediate returnValue, @Nonnull StmtPositionInfo positionInfo) { - super(returnValue, positionInfo); + super(positionInfo); + this.op = returnValue; } @Override @@ -70,6 +75,25 @@ public int getExpectedSuccessorCount() { return 0; } + @Nonnull + public Immediate getOp() { + return op; + } + + @Override + @Nonnull + public final List getUses() { + final List uses = op.getUses(); + List list = new ArrayList<>(uses.size() + 1); + list.add(op); + return list; + } + + @Override + public int equivHashCode() { + return op.equivHashCode(); + } + @Override public boolean equivTo(@Nonnull Object o, @Nonnull JimpleComparator comparator) { return comparator.caseReturnStmt(this, o); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnVoidStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnVoidStmt.java index 1ed0b23ec59..498d73985a6 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnVoidStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JReturnVoidStmt.java @@ -27,11 +27,10 @@ import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A statement that ends the method, returning no value. */ -public final class JReturnVoidStmt extends Stmt implements Copyable { +public final class JReturnVoidStmt extends AbstractStmt { public JReturnVoidStmt(@Nonnull StmtPositionInfo positionInfo) { super(positionInfo); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JThrowStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JThrowStmt.java index 2ec0ffd94db..95b371a3f5f 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JThrowStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JThrowStmt.java @@ -22,20 +22,25 @@ * #L% */ +import java.util.ArrayList; +import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Immediate; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.basic.Value; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A statement that throws an Exception */ -public final class JThrowStmt extends AbstractOpStmt implements Copyable { +public final class JThrowStmt extends AbstractStmt { + + protected final Immediate op; public JThrowStmt(@Nonnull Immediate op, @Nonnull StmtPositionInfo positionInfo) { - super(op, positionInfo); + super(positionInfo); + this.op = op; } @Override @@ -84,4 +89,23 @@ public JThrowStmt withOp(@Nonnull Immediate op) { public JThrowStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { return new JThrowStmt(getOp(), positionInfo); } + + @Nonnull + public Immediate getOp() { + return op; + } + + @Override + @Nonnull + public final List getUses() { + final List uses = op.getUses(); + List list = new ArrayList<>(uses.size() + 1); + list.add(op); + return list; + } + + @Override + public int equivHashCode() { + return op.equivHashCode(); + } } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index 33bbe1e00ef..d74b0f99f23 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -1,29 +1,5 @@ package sootup.core.jimple.common.stmt; -/*- - * #%L - * Soot - a J*va Optimization Framework - * %% - * Copyright (C) 2018-2020 Linghui Luo, Christian Brüggemann, Markus Schmidt - * %% - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Lesser General Public License as - * published by the Free Software Foundation, either version 2.1 of the - * License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Lesser Public License for more details. - * - * You should have received a copy of the GNU General Lesser Public - * License along with this program. If not, see - * . - * #L% - */ - -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.EquivTo; @@ -34,124 +10,50 @@ import sootup.core.jimple.common.ref.JArrayRef; import sootup.core.jimple.common.ref.JFieldRef; import sootup.core.jimple.visitor.Acceptor; -import sootup.core.jimple.visitor.ReplaceUseStmtVisitor; import sootup.core.jimple.visitor.StmtVisitor; import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public abstract class Stmt implements EquivTo, Acceptor, Copyable { - - protected final StmtPositionInfo positionInfo; - - public Stmt(@Nonnull StmtPositionInfo positionInfo) { - this.positionInfo = positionInfo; - } - - /** - * Returns a list of Values used in this Stmt. Note that they are returned in usual evaluation - * order. - */ +public interface Stmt extends EquivTo, Acceptor, Copyable { @Nonnull - public List getUses() { - return Collections.emptyList(); - } + List getUses(); - /** - * Returns a list of Values defined in this Stmt. There are languages which allow multiple return - * types/assignments so we return a List - */ @Nonnull - public List getDefs() { - return Collections.emptyList(); - } + List getDefs(); - /** Returns a list of Values, either used or defined or both in this Stmt. */ @Nonnull - public List getUsesAndDefs() { - List uses = getUses(); - List defs = new ArrayList<>(getDefs()); - if (uses.isEmpty()) { - return defs; - } else if (defs.isEmpty()) { - return uses; - } else { - List values = new ArrayList<>(); - values.addAll(defs); - values.addAll(uses); - return values; - } - } + List getUsesAndDefs(); /** * Returns true if execution after this statement may continue at the following statement. (e.g. * GotoStmt will return false and e.g. IfStmt will return true). */ - public abstract boolean fallsThrough(); + boolean fallsThrough(); /** * Returns true if execution after this statement does not necessarily continue at the following * statement. The {@link BranchingStmt}'s GotoStmt, JSwitchStmt and IfStmt will return true. */ - public abstract boolean branches(); + boolean branches(); - /** Returns the amount of unexceptional successors the Stmt needs to have in the StmtGraph. */ - public int getExpectedSuccessorCount() { - return 1; - } + int getExpectedSuccessorCount(); - public abstract void toString(@Nonnull StmtPrinter up); + void toString(@Nonnull StmtPrinter up); - public boolean containsInvokeExpr() { - return false; - } + boolean containsInvokeExpr(); - /** - * This method must only be used for Stmts which contain an InvokeExpr (JInvokeStmt; possible in - * JAssignStmt) check via containsInvokExpr(). - */ - public AbstractInvokeExpr getInvokeExpr() { - throw new RuntimeException("getInvokeExpr() called with no invokeExpr present!"); - } + AbstractInvokeExpr getInvokeExpr(); - public boolean containsArrayRef() { - return false; - } + boolean containsArrayRef(); - /** - * This method must only be used for Stmts which contain an ArrayRef - possible with JAssignStmts. - * check via containsArrayRef(). - */ - public JArrayRef getArrayRef() { - throw new RuntimeException("getArrayRef() called with no ArrayRef present!"); - } + JArrayRef getArrayRef(); - public boolean containsFieldRef() { - return false; - } + boolean containsFieldRef(); - /** - * This method must only be used for Stmts which contain an FieldRef - possible with JAssignStmts. - * check via containsFieldRef(). - */ - public JFieldRef getFieldRef() { - throw new RuntimeException("getFieldRef() called with no JFieldRef present!"); - } + JFieldRef getFieldRef(); - public StmtPositionInfo getPositionInfo() { - return positionInfo; - } + StmtPositionInfo getPositionInfo(); - /** - * Use newUse to replace the oldUse in oldStmt. - * - * @param oldUse a Value in the useList of oldStmt. - * @param newUse a Value is to replace oldUse - * @return a new Stmt with newUse - */ @Nonnull - public Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse) { - ReplaceUseStmtVisitor visitor = new ReplaceUseStmtVisitor(oldUse, newUse); - accept(visitor); - return visitor.getResult(); - } + Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JBreakpointStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JBreakpointStmt.java index 51ca9c0a58c..78b217bda81 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JBreakpointStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JBreakpointStmt.java @@ -26,12 +26,12 @@ import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.common.stmt.Stmt; +import sootup.core.jimple.common.stmt.AbstractStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; -public final class JBreakpointStmt extends Stmt implements Copyable { +public final class JBreakpointStmt extends AbstractStmt implements FallsThroughStmt { public JBreakpointStmt(@Nonnull StmtPositionInfo positionInfo) { super(positionInfo); diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JEnterMonitorStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JEnterMonitorStmt.java index c62951d9102..7341c246f99 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JEnterMonitorStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JEnterMonitorStmt.java @@ -22,21 +22,27 @@ * #L% */ +import java.util.ArrayList; +import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Immediate; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.common.stmt.AbstractOpStmt; +import sootup.core.jimple.basic.Value; +import sootup.core.jimple.common.stmt.AbstractStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A statement that enters a JVM monitor, thereby synchronizing its following statements. */ -public final class JEnterMonitorStmt extends AbstractOpStmt implements Copyable { +public final class JEnterMonitorStmt extends AbstractStmt implements FallsThroughStmt { + + protected final Immediate op; public JEnterMonitorStmt(@Nonnull Immediate op, @Nonnull StmtPositionInfo positionInfo) { - super(op, positionInfo); + super(positionInfo); + this.op = op; } @Override @@ -66,6 +72,25 @@ public boolean branches() { return false; } + @Nonnull + public Immediate getOp() { + return op; + } + + @Override + @Nonnull + public List getUses() { + final List uses = op.getUses(); + List list = new ArrayList<>(uses.size() + 1); + list.add(op); + return list; + } + + @Override + public int equivHashCode() { + return op.equivHashCode(); + } + @Override public boolean equivTo(@Nonnull Object o, @Nonnull JimpleComparator comparator) { return comparator.caseEnterMonitorStmt(this, o); diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JExitMonitorStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JExitMonitorStmt.java index 213f87e44ea..351371201da 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JExitMonitorStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JExitMonitorStmt.java @@ -22,21 +22,27 @@ * #L% */ +import java.util.ArrayList; +import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Immediate; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.common.stmt.AbstractOpStmt; +import sootup.core.jimple.basic.Value; +import sootup.core.jimple.common.stmt.AbstractStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** A statement that exits a JVM monitor, thereby ending synchronization. */ -public final class JExitMonitorStmt extends AbstractOpStmt implements Copyable { +public final class JExitMonitorStmt extends AbstractStmt implements FallsThroughStmt { + + protected final Immediate op; public JExitMonitorStmt(@Nonnull Immediate op, @Nonnull StmtPositionInfo positionInfo) { - super(op, positionInfo); + super(positionInfo); + this.op = op; } @Override @@ -80,4 +86,23 @@ public JExitMonitorStmt withOp(@Nonnull Immediate op) { public JExitMonitorStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { return new JExitMonitorStmt(getOp(), positionInfo); } + + @Nonnull + public Immediate getOp() { + return op; + } + + @Override + @Nonnull + public final List getUses() { + final List uses = op.getUses(); + List list = new ArrayList<>(uses.size() + 1); + list.add(op); + return list; + } + + @Override + public int equivHashCode() { + return op.equivHashCode(); + } } diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JRetStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JRetStmt.java index 7d7594a2e5b..bf81414cc3e 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JRetStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JRetStmt.java @@ -26,20 +26,19 @@ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; -import sootup.core.jimple.basic.*; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; -import sootup.core.jimple.common.stmt.Stmt; +import sootup.core.jimple.common.stmt.AbstractStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.visitor.StmtVisitor; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /** * Represents the deprecated JVM ret statement - which is used in JSR Context - which * is deprecated as well */ -public final class JRetStmt extends Stmt implements Copyable { +public final class JRetStmt extends AbstractStmt implements FallsThroughStmt { @Nonnull private final Value stmtAddress; diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java index efeb67cdc62..b6911287a72 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java @@ -25,24 +25,23 @@ import java.util.*; import javax.annotation.Nonnull; import sootup.core.jimple.Jimple; -import sootup.core.jimple.basic.*; import sootup.core.jimple.basic.Immediate; import sootup.core.jimple.basic.JimpleComparator; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.AbstractStmt; import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.jimple.visitor.StmtVisitor; import sootup.core.model.Body; -import sootup.core.util.Copyable; import sootup.core.util.printer.StmtPrinter; /* * Switch Statements (combining LookupSwitch/TableSwitch) * @author Markus Schmidt */ -public class JSwitchStmt extends BranchingStmt implements Copyable { +public class JSwitchStmt extends AbstractStmt implements BranchingStmt { private final Immediate key; private List values; @@ -106,6 +105,11 @@ public boolean fallsThrough() { return false; } + @Override + public boolean branches() { + return true; + } + @Override public void accept(@Nonnull StmtVisitor sw) { sw.caseSwitchStmt(this); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java index 3431476513c..880c00766e6 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java @@ -343,7 +343,7 @@ Local newStackLocal() { } @SuppressWarnings("unchecked") - A getStmt(@Nonnull AbstractInsnNode insn) { + A getStmt(@Nonnull AbstractInsnNode insn) { return (A) insnToStmt.get(insn); } @@ -919,7 +919,7 @@ private void convertInsn(@Nonnull InsnNode insn) { StackFrame frame = operandStack.getOrCreateStackframe(insn); if (!insnToStmt.containsKey(insn)) { Operand opr = operandStack.popStackConst(); - AbstractOpStmt ts = + Stmt ts = op == MONITORENTER ? Jimple.newEnterMonitorStmt((Immediate) opr.stackOrValue(), getStmtPositionInfo()) : Jimple.newExitMonitorStmt((Immediate) opr.stackOrValue(), getStmtPositionInfo()); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java index 82f6da384e9..15c10f65210 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/StmtContainer.java @@ -28,6 +28,7 @@ import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; +import sootup.core.jimple.common.stmt.AbstractStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.jimple.visitor.StmtVisitor; import sootup.core.util.printer.StmtPrinter; @@ -41,7 +42,7 @@ * @author Aaloan Miftah * @author Markus Schmidt */ -class StmtContainer extends Stmt { +class StmtContainer extends AbstractStmt { @Nonnull private final List stmts = new LinkedList<>(); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index aaee4c6e4fc..9d6919b7710 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -124,20 +124,23 @@ public Type evaluate( return methodTypeClassType; } else { // return null; - throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); + throw new RuntimeException( + "can't evaluatable constant in AugEvalFunction '" + value + "'."); } } } else if (value instanceof Expr) { if (value instanceof AbstractBinopExpr) { Type tl = evaluate(typing, ((AbstractBinopExpr) value).getOp1(), stmt, graph); if (tl == null) { - // return null; - throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("can't evaluatable constant in AugEvalFunction '" + value + + // "'."); } Type tr = evaluate(typing, ((AbstractBinopExpr) value).getOp2(), stmt, graph); if (tr == null) { - // return null; - throw new RuntimeException("Inevaluatable constant in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("can't evaluatable constant in AugEvalFunction '" + value + + // "'."); } if (value instanceof AbstractIntBinopExpr) { if (value instanceof AbstractConditionExpr) { @@ -162,8 +165,9 @@ public Type evaluate( } else { Collection lca = PrimitiveHierarchy.getLeastCommonAncestor(tl, tr); if (lca.isEmpty()) { - throw new RuntimeException( - "Invaluable expression by using AugEvalFunction '" + value + "'."); + // throw new RuntimeException("can't evaluate expression by using AugEvalFunction + // '" + value + "'."); + return null; } return lca.iterator().next(); } @@ -201,8 +205,9 @@ public Type evaluate( } if (type == null) { - // return null; - throw new RuntimeException("inevaluatable reference in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("can't evaluatable reference in AugEvalFunction '" + value + + // "'."); } return type; } else if (value instanceof JArrayRef) { @@ -221,8 +226,9 @@ public Type evaluate( || value instanceof JFieldRef) { return value.getType(); } else { - // return null; - throw new RuntimeException("Inevaluatable reference in AugEvalFunction '" + value + "'."); + return null; + // throw new RuntimeException("can't evaluatable reference in AugEvalFunction '" + value + + // "'."); } } return null; @@ -241,6 +247,7 @@ private Set getExceptionTypeCandidates( * This function is used to retrieve the path from the type "Throwable" to the given exception * type */ + // TODO: ms: simplify - use the typehiararchy directly! private Deque getExceptionPath(@Nonnull ClassType exceptionType) { Deque path = new ArrayDeque<>(); path.push(exceptionType); @@ -249,6 +256,7 @@ private Deque getExceptionPath(@Nonnull ClassType exceptionType) { final Optional superclassOpt = view.getClass(exceptionType).flatMap(SootClass::getSuperclass); if (!superclassOpt.isPresent()) { + // TODO: ms: don't fail completely.. work as far as information exists and warn. throw new IllegalStateException( "The path from '" + exceptionType + "' to java.lang.Throwable cannot be found!"); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index a6391fc6146..6fc6c097a3d 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -22,6 +22,7 @@ * #L% */ +import com.google.common.collect.Lists; import java.util.HashMap; import java.util.Map; import javax.annotation.Nonnull; @@ -69,7 +70,7 @@ public void insertCastStmts(@Nonnull Typing typing) { this.castCount = 0; this.countOnly = false; setTyping(typing); - for (Stmt stmt : builder.getStmts()) { + for (Stmt stmt : Lists.newArrayList(builder.getStmts())) { stmt.accept(this); } } @@ -103,7 +104,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm } } Type evaType = evalFunction.evaluate(typing, value, stmt, graph); - if (hierarchy.isAncestor(stdType, evaType)) { + if (evaType == null || hierarchy.isAncestor(stdType, evaType)) { return; } this.castCount++; @@ -134,7 +135,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm } else { newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); } - if (newStmt != null && graph.containsNode(stmt)) { + if (graph.containsNode(stmt)) { builder.replaceStmt(stmt, newStmt); this.stmt2NewStmt.put(oriStmt, newStmt); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java index e6fa6238d39..ad906f1a809 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java @@ -67,7 +67,7 @@ public boolean resolve(@Nonnull Body.BodyBuilder builder) { return false; } Typing minCastsTyping = getMinCastsTyping(builder, typings, evalFunction, hierarchy); - if (this.castCount != 0) { + if (this.castCount > 0) { CastCounter castCounter = new CastCounter(builder, evalFunction, hierarchy); castCounter.insertCastStmts(minCastsTyping); } @@ -78,7 +78,11 @@ public boolean resolve(@Nonnull Body.BodyBuilder builder) { return false; } else { for (Local local : locals) { - Type convertedType = convertType(promotedTyping.getType(local)); + final Type type = promotedTyping.getType(local); + if (type == null) { + continue; + } + Type convertedType = convertType(type); if (convertedType != null) { promotedTyping.set(local, convertedType); } @@ -88,7 +92,7 @@ public boolean resolve(@Nonnull Body.BodyBuilder builder) { for (Local local : locals) { Type oldType = local.getType(); Type newType = promotedTyping.getType(local); - if (oldType.equals(newType)) { + if (newType == null || oldType.equals(newType)) { continue; } Local newLocal = local.withType(newType); From 0cd3e40a6c5baf645f02f36ded639bbe8fc1e084 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 4 Oct 2023 17:01:10 +0200 Subject: [PATCH 24/54] handle nullType Local naming --- .../src/main/java/sootup/core/jimple/basic/LocalGenerator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/LocalGenerator.java b/sootup.core/src/main/java/sootup/core/jimple/basic/LocalGenerator.java index 3aead19796b..1bd49ef733c 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/LocalGenerator.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/LocalGenerator.java @@ -174,7 +174,8 @@ public void caseClassType(@Nonnull ClassType classType) { @Override public void caseNullType() { - defaultCaseType(); + // could be ClassType (and ArrayType?) which is the same here so.. + result.append("r").append(tempRefLikeType++); } @Override From 4e213ce7a6aac40ccf8bf1c592947ed8bbf43748 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 4 Oct 2023 18:25:46 +0200 Subject: [PATCH 25/54] bugfix input --- .../org/apache/jsp/java_detection_samples.class | Bin 0 -> 6199 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples.class diff --git a/sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples.class b/sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples.class new file mode 100644 index 0000000000000000000000000000000000000000..57b85a98e70129da767b1e0ab7772182ea86fba9 GIT binary patch literal 6199 zcma)A349dQ9sXXj$!si=+pPsNSMy^i)hwSqi)Yp$~AQEtyFhmMNh2 zkk!EzTGUD#+XZH}ORfjQrfyu2&@EdIlF6Po2~4u#DVC(Dr6h`p_MPy66;FiMXc+>P zMdE!a%^EOtfra^}OP$w*mg);fEF+%kYjW+hr}=&wF&yiVi92leP^nwBKD~fgM7Icf zl8XR$#vvdPPR&OcO=zYm@lEZ~6FvH1#?VcZR_Yy5#0fUm!LXhPx8_bq)i6b(>xrjK zOG`!dq?RIghUCSidn=ucYF4-{3$(@oA}`*h(L_8Iw@wo%uBu)y;8~uI={_8f846Ai zIN?Y(P%#s;1Sa?CR*T-N4J53b@q$&=S*{ObzA^}$sGJ-e&?mM`MM@1O(DV-LJQN>j*3wLbZ z!gy$^-sDBSfIkmgT|a%_i&L_k4USN$qpl?zi&gj~8%qSH$cuIHbX~`Ql^L+;NxGKw zqETRieIlK)u03vAUMv+jW(2yeqjfND)qxC!$J@|;gG&0B}*__EF;*5AY7x{GXv^l;% znotYS9427LqM}Hf5mOMQOXZ6sooZa_Ds*%*RK_<&MZ}^TFkjnFtDryY^kF+SclR6V zZH|i{H)79%M8+ycD;z!FNuwrIOvSN2q>xdN7MSn4wk5x)6;R10Hx=iD`PENZ3IGI_ z9bU(DmZlHFBXhEpvW-bupj<9nbkj1@tcdoTv|2pMc<(BtB~zANv$KZn($62@Ri~(p zntd#onMtQiy-B8jUoI)JJ0M9rD~03fuq$0DxQOACZ}}|A-0d=4f{!Y=bW8!Id<3KJ~F8*^;aHI6mP7RX(Qif+`<^ zL3IhPQE{!T9i>#HmM|sx>s8!<8`)OS`fGHnKh2^tWh6hjI|(mt7En4myW2X}Mtryx z*C_ZT6NqHo@oT5$@ZxsL-<(LKw|Vg?2Jp((?%Lq0*5;PlU}r~!>nx6P+=aUp><~El zXu>=8RB;bJO-@bSTBU0-*%Qolm%p(sPj;tzy%+bfVv-Xt9$@*9e0cF$t|VVxd|u%6 z)sap!m`)|O2iY+u<5tk-TS5lYhRuInaJ3yyL8>mDg)-gJ>{Rgue9`7G6>Bw&G&wxB zs?Ak2Tnv3#;5eDb!u?h<(G>0149a!tS=}qh}s*11SA(mY3+8HlpN&hOS_&Izs zW$se38;|e`P1(twDVHd(Eex!za~B5Ek;5!^nvJfpVr^>MG+xei@j@@$Ba6q7z>IvO z%S`J6lyKehXIRB!*v~koP!Xq(6qr4xM7aXS?lo;fUsv%2o}|0;E_<2rN}D9R+RSV) z;+AeSdGQpXea;^a;28zq98>1;}j}P0V z%xU)FAS%6ho{kz##$;@MU&RmbL(UPlK@Ba*d511_@;FZK=-nNMO=ZOQBNfl%CD}?9 zhN*r06BR#|#~zDm)rXhy3k5&Vc8CtHE@yI5@k{*5F+z4nI-;9scAI*C8NXKX8~m1N z^+8?M)T8j*X{4jHY?FfDv&$dr7Ll#-jC74$E|z+JRmC6iC#SPpHqhIv8;w@qfHcGZ;Ftn>LnQbj~h} z&Vo>|%a~#~wU64gO?jm1=M+hnUCv#0A>@#8)Jxf*9vNDD=xcT@D9^Yv$*clAg{)G; zX&GoU#(^6RGiFHH1`PJtuzqJP>GVWLdOM*B^BDi!g!v9Pe`Plljg$%k{oYw&a9eCc8NMz8A+~8X*FFJ3t)r z&)kRUk07`ovp4R=oJTNsH|F}Qhf!064RX0)bh*%3=0F$i#v*^i$oV4whenr8qsvoA zm!~_+T_jtCX8ta=g_%WRXHw8AvR_T74$-@7sDxTHVIEeX4(m9Xo`d=5gCDAzgTxDyeE{nvo8*L90#$nUJ zvBZVLlf!W)+H*M86ylg$h-2xvIMyBs$D%?UXN-%Z1D)huT6Tx+J3vF|QLK9ob)k*z zPdm%wL%R_S4Iwg&4FW?rcNlcbJy=(BU?)mz4m>Hr9;P_*3qhEu!qm5X7TeaEP8)%O zD^y=O?c%5ULR{D5OczTHS=ICsb|Q-3Mo58ZQP6jYhy~)zRLbWdQ#%b<^bWi-;KkXyadFKsE|;dd zdIvhVzD{7R|E76ExaBTPsP*6GzhfA87GszHUb(jeHU9gZ#ccm)oJC+1s&=xt-T%2^ zJje^8(IXm5L}O{K|4YO8N)a2pV5p{6cqRwo8N$PB=QWlR!J`}avseD?lRu9)dIFxt zl7PqmjUhZ8@J#Xz;ko@d$kg%OMkS!s_`f%d7mDaoFOKYb1Kxo0C|=xy9|x31?>_u2 z;N|NOUfGbnzY}qJ_bL2t2UZ43B#CEH86X#bxC=9Chw++V*)5kFf7=OFKL5EBo(+N0 zXHgL-4Jbo+eJ3jR0$;o z9}^8iW);Vso@TBANB4<{GIhoAzt?bhyAhY+He8N7xpN<`!h^UP5Ao!2PEOC@I=q1E@ghg2S8x+~ zyO~YKEqI;N&|4gaJRE&0aEI`7?wN`^#dHogGjX?=hkHZ=Cy-^hSG4fij{8LipBwOi z?Qa*5%7w~38TF^4+>ZKN4`Kd$lt^KeDY!zR6%`z?|Lu`G zeQ%?Nzbh5$Yr-Lvt>%lWpqQePH8DX<q@gw*KuT z3q^wVV}-9KI6oNElWF-m(wAF=+I Date: Wed, 4 Oct 2023 18:30:32 +0200 Subject: [PATCH 26/54] progress; add FallsThroughStmt and refactored Branching/-Stmt --- .../core/jimple/common/stmt/AbstractStmt.java | 5 +- .../jimple/common/stmt/FallsThroughStmt.java | 2 - .../sootup/core/jimple/common/stmt/Stmt.java | 1 - .../jimple/visitor/ReplaceUseStmtVisitor.java | 6 +- .../bytecode/frontend/AsmMethodSource.java | 2 +- .../typeresolving/AugEvalFunction.java | 8 +- .../typeresolving/CastCounter.java | 102 +++++++++--------- .../typeresolving/PrimitiveHierarchy.java | 40 +++---- .../typeresolving/TypePromotionVisitor.java | 26 ++--- .../typeresolving/TypeResolver.java | 8 +- ...gerTypes.java => AugmentIntegerTypes.java} | 2 +- .../bytecode/interceptors/AggregatorTest.java | 6 +- .../typeresolving/AugEvalFunctionTest.java | 10 +- .../typeresolving/CastCounterTest.java | 4 +- .../typeresolving/PrimitiveHierarchyTest.java | 8 +- .../java9/DynamicInvokeTest.java | 2 +- .../java_detection_samples$interfacer.class | Bin 0 -> 291 bytes 17 files changed, 119 insertions(+), 113 deletions(-) rename sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/{AugIntegerTypes.java => AugmentIntegerTypes.java} (98%) create mode 100644 sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples$interfacer.class diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java index 49d2534b746..79fb6c7e39b 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractStmt.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.basic.Value; @@ -138,10 +139,10 @@ public StmtPositionInfo getPositionInfo() { * * @param oldUse a Value in the useList of oldStmt. * @param newUse a Value is to replace oldUse - * @return a new Stmt with newUse + * @return a new Stmt with newUse or null if oldUse was not found/replaced in the Stmt */ @Override - @Nonnull + @Nullable public Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse) { ReplaceUseStmtVisitor visitor = new ReplaceUseStmtVisitor(oldUse, newUse); accept(visitor); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java index 04f58bd22ec..6bab22a5b62 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/FallsThroughStmt.java @@ -1,7 +1,5 @@ package sootup.core.jimple.common.stmt; - - /** as an equivalent to BranchingStmt */ public interface FallsThroughStmt extends Stmt { diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index d74b0f99f23..ccbc9f1b331 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -54,6 +54,5 @@ public interface Stmt extends EquivTo, Acceptor, Copyable { StmtPositionInfo getPositionInfo(); - @Nonnull Stmt withNewUse(@Nonnull Value oldUse, @Nonnull Value newUse); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java index 7e26027d8f6..59623ce9a66 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java +++ b/sootup.core/src/main/java/sootup/core/jimple/visitor/ReplaceUseStmtVisitor.java @@ -72,7 +72,7 @@ public void caseInvokeStmt(@Nonnull JInvokeStmt stmt) { @Override public void caseAssignStmt(@Nonnull JAssignStmt stmt) { - // uses on the def side.. + // uses on the def side.. e.g. a base in an JArrayRef but NOT with a simple Local! final Value leftOp = stmt.getLeftOp(); if (leftOp instanceof Ref) { refVisitor.init(oldUse, newUse); @@ -84,7 +84,9 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { // rhs Value rValue = stmt.getRightOp(); - if (rValue instanceof Immediate) { + if (rValue == oldUse) { + setResult(stmt.withRValue(newUse)); + } else if (rValue instanceof Immediate) { if (rValue == oldUse) { setResult(stmt.withRValue(newUse)); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java index 880c00766e6..c701a30c82a 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java @@ -343,7 +343,7 @@ Local newStackLocal() { } @SuppressWarnings("unchecked") - A getStmt(@Nonnull AbstractInsnNode insn) { + A getStmt(@Nonnull AbstractInsnNode insn) { return (A) insnToStmt.get(insn); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index 9d6919b7710..7cefd49af12 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -40,7 +40,7 @@ import sootup.core.types.PrimitiveType; import sootup.core.types.Type; import sootup.core.views.View; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; import sootup.java.bytecode.interceptors.typeresolving.types.BottomType; /** @author Zun Wang */ @@ -94,13 +94,13 @@ public Type evaluate( if (value.getClass() == IntConstant.class) { int val = ((IntConstant) value).getValue(); if (val >= 0 && val < 2) { - return AugIntegerTypes.getInteger1(); + return AugmentIntegerTypes.getInteger1(); } else if (val >= 2 && val < 128) { - return AugIntegerTypes.getInteger127(); + return AugmentIntegerTypes.getInteger127(); } else if (val >= -128 && val < 0) { return PrimitiveType.getByte(); } else if (val >= 128 && val < 32768) { - return AugIntegerTypes.getInteger32767(); + return AugmentIntegerTypes.getInteger32767(); } else if (val >= -32768 && val < -128) { return PrimitiveType.getShort(); } else if (val >= 32768 && val < 65536) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index 6fc6c097a3d..5ed1a1a024f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -80,6 +80,8 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm AugEvalFunction evalFunction = getFuntion(); BytecodeHierarchy hierarchy = getHierarchy(); Typing typing = getTyping(); + + // TODO: ms: move into a Subclass instead of the countOnly option/field? if (countOnly) { Type evaType = evalFunction.evaluate(typing, value, stmt, graph); if (evaType == null) { @@ -89,57 +91,59 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm return; } this.castCount++; - } else { - Stmt oriStmt = stmt; - Value oriValue = value; - Stmt updatedStmt = stmt2NewStmt.get(stmt); - if (updatedStmt != null) { - stmt = stmt2NewStmt.get(stmt); - } - Map m = changedValues.get(oriStmt); - if (m != null) { - Value updatedValue = m.get(value); - if (updatedValue != null) { - value = updatedValue; - } - } - Type evaType = evalFunction.evaluate(typing, value, stmt, graph); - if (evaType == null || hierarchy.isAncestor(stdType, evaType)) { - return; - } - this.castCount++; - // TODO: modifiers later must be added - - Local old_local; - if (value instanceof Local) { - old_local = (Local) value; - } else { - old_local = generateTempLocal(evaType); - builder.addLocal(old_local); - typing.set(old_local, evaType); - JAssignStmt newAssign = Jimple.newAssignStmt(old_local, value, stmt.getPositionInfo()); - builder.insertBefore(stmt, newAssign); - } - Local new_local = generateTempLocal(stdType); - builder.addLocal(new_local); - typing.set(new_local, stdType); - addUpdatedValue(oriValue, new_local, oriStmt); - JAssignStmt newCast = - Jimple.newAssignStmt( - new_local, Jimple.newCastExpr(old_local, stdType), stmt.getPositionInfo()); - builder.insertBefore(stmt, newCast); - - Stmt newStmt; - if (stmt.getUses().contains(value)) { - newStmt = stmt.withNewUse(value, new_local); - } else { - newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); - } - if (graph.containsNode(stmt)) { - builder.replaceStmt(stmt, newStmt); - this.stmt2NewStmt.put(oriStmt, newStmt); + return; + } + + Stmt oriStmt = stmt; + Value oriValue = value; + Stmt updatedStmt = stmt2NewStmt.get(stmt); + if (updatedStmt != null) { + stmt = stmt2NewStmt.get(stmt); + } + Map m = changedValues.get(oriStmt); + if (m != null) { + Value updatedValue = m.get(value); + if (updatedValue != null) { + value = updatedValue; } } + Type evaType = evalFunction.evaluate(typing, value, stmt, graph); + if (evaType == null || hierarchy.isAncestor(stdType, evaType)) { + return; + } + this.castCount++; + // TODO: modifiers later must be added + + Local old_local; + if (value instanceof Local) { + old_local = (Local) value; + } else { + old_local = generateTempLocal(evaType); + builder.addLocal(old_local); + typing.set(old_local, evaType); + JAssignStmt newAssign = Jimple.newAssignStmt(old_local, value, stmt.getPositionInfo()); + builder.insertBefore(stmt, newAssign); + } + + Local new_local = generateTempLocal(stdType); + builder.addLocal(new_local); + typing.set(new_local, stdType); + addUpdatedValue(oriValue, new_local, oriStmt); + JAssignStmt newCast = + Jimple.newAssignStmt( + new_local, Jimple.newCastExpr(old_local, stdType), stmt.getPositionInfo()); + builder.insertBefore(stmt, newCast); + + Stmt newStmt; + if (stmt.getUses().contains(value)) { + newStmt = stmt.withNewUse(value, new_local); + } else { + newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(new_local); + } + if (graph.containsNode(stmt)) { + builder.replaceStmt(stmt, newStmt); + this.stmt2NewStmt.put(oriStmt, newStmt); + } } private void addUpdatedValue(Value oldValue, Value newValue, Stmt stmt) { diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchy.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchy.java index 30db578be93..b44f483ee43 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchy.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchy.java @@ -27,7 +27,7 @@ import sootup.core.types.ArrayType; import sootup.core.types.PrimitiveType; import sootup.core.types.Type; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; import sootup.java.bytecode.interceptors.typeresolving.types.BottomType; /** @author Zun Wang */ @@ -55,7 +55,7 @@ public static Collection getLeastCommonAncestor(@Nonnull Type a, @Nonnull if (a.getClass() == PrimitiveType.ByteType.class) { if (b.getClass() == PrimitiveType.ShortType.class || b.getClass() == PrimitiveType.CharType.class - || b.getClass() == AugIntegerTypes.Integer32767Type.class) { + || b.getClass() == AugmentIntegerTypes.Integer32767Type.class) { return Collections.singleton(PrimitiveType.getInt()); } return Collections.emptySet(); @@ -91,25 +91,25 @@ public static boolean isAncestor(@Nonnull Type ancestor, @Nonnull Type child) { } if (arePrimitives(ancestor, child)) { - if (ancestor.getClass() == AugIntegerTypes.Integer1Type.class) { + if (ancestor.getClass() == AugmentIntegerTypes.Integer1Type.class) { return child.getClass() == BottomType.class; } if (ancestor.getClass() == PrimitiveType.BooleanType.class - || ancestor.getClass() == AugIntegerTypes.Integer127Type.class) { - return child.getClass() == AugIntegerTypes.Integer1Type.class + || ancestor.getClass() == AugmentIntegerTypes.Integer127Type.class) { + return child.getClass() == AugmentIntegerTypes.Integer1Type.class || child.getClass() == BottomType.class; } if (ancestor.getClass() == PrimitiveType.ByteType.class - || ancestor.getClass() == AugIntegerTypes.Integer32767Type.class) { - return child.getClass() == AugIntegerTypes.Integer127Type.class - || child.getClass() == AugIntegerTypes.Integer1Type.class + || ancestor.getClass() == AugmentIntegerTypes.Integer32767Type.class) { + return child.getClass() == AugmentIntegerTypes.Integer127Type.class + || child.getClass() == AugmentIntegerTypes.Integer1Type.class || child.getClass() == BottomType.class; } if (ancestor.getClass() == PrimitiveType.CharType.class || ancestor.getClass() == PrimitiveType.ShortType.class) { - return child.getClass() == AugIntegerTypes.Integer32767Type.class - || child.getClass() == AugIntegerTypes.Integer127Type.class - || child.getClass() == AugIntegerTypes.Integer1Type.class + return child.getClass() == AugmentIntegerTypes.Integer32767Type.class + || child.getClass() == AugmentIntegerTypes.Integer127Type.class + || child.getClass() == AugmentIntegerTypes.Integer1Type.class || child.getClass() == BottomType.class; } if (ancestor instanceof PrimitiveType.IntType) { @@ -129,26 +129,26 @@ public static boolean isAncestor(@Nonnull Type ancestor, @Nonnull Type child) { if (ancestorDim == childDim && arePrimitives(ancestorBase, childBase)) { // TODO: [ms] dry? looks quite similar to the if-else-tree above.. why are they differing in // structure? - if (ancestorBase.getClass() == AugIntegerTypes.Integer1Type.class) { + if (ancestorBase.getClass() == AugmentIntegerTypes.Integer1Type.class) { return childBase.getClass() == BottomType.class; } if (ancestorBase.getClass() == PrimitiveType.BooleanType.class - || ancestorBase.getClass() == AugIntegerTypes.Integer127Type.class) { - return childBase.getClass() == AugIntegerTypes.Integer1Type.class + || ancestorBase.getClass() == AugmentIntegerTypes.Integer127Type.class) { + return childBase.getClass() == AugmentIntegerTypes.Integer1Type.class || childBase.getClass() == BottomType.class; } if (ancestorBase.getClass() == PrimitiveType.ByteType.class - || ancestorBase.getClass() == AugIntegerTypes.Integer32767Type.class) { - return childBase.getClass() == AugIntegerTypes.Integer127Type.class - || childBase.getClass() == AugIntegerTypes.Integer1Type.class + || ancestorBase.getClass() == AugmentIntegerTypes.Integer32767Type.class) { + return childBase.getClass() == AugmentIntegerTypes.Integer127Type.class + || childBase.getClass() == AugmentIntegerTypes.Integer1Type.class || childBase.getClass() == BottomType.class; } if (ancestorBase.getClass() == PrimitiveType.CharType.class || ancestorBase.getClass() == PrimitiveType.ShortType.class || ancestorBase instanceof PrimitiveType.IntType) { - return childBase.getClass() == AugIntegerTypes.Integer32767Type.class - || childBase.getClass() == AugIntegerTypes.Integer127Type.class - || childBase.getClass() == AugIntegerTypes.Integer1Type.class + return childBase.getClass() == AugmentIntegerTypes.Integer32767Type.class + || childBase.getClass() == AugmentIntegerTypes.Integer127Type.class + || childBase.getClass() == AugmentIntegerTypes.Integer1Type.class || childBase.getClass() == BottomType.class; } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java index 628678e1c5d..8f296bc603f 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java @@ -31,7 +31,7 @@ import sootup.core.model.Body; import sootup.core.types.PrimitiveType; import sootup.core.types.Type; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; public class TypePromotionVisitor extends TypeChecker { @@ -63,9 +63,9 @@ public Typing getPromotedTyping(Typing typing) { } public static boolean isIntermediateType(Type type) { - return type.equals(AugIntegerTypes.getInteger1()) - || type.equals(AugIntegerTypes.getInteger127()) - || type.equals(AugIntegerTypes.getInteger32767()); + return type.equals(AugmentIntegerTypes.getInteger1()) + || type.equals(AugmentIntegerTypes.getInteger127()) + || type.equals(AugmentIntegerTypes.getInteger32767()); } public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stmt) { @@ -98,37 +98,37 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm private Type promote(Type low, Type high) { Class lowClass = low.getClass(); Class highClass = high.getClass(); - if (lowClass == AugIntegerTypes.Integer1Type.class) { + if (lowClass == AugmentIntegerTypes.Integer1Type.class) { if (highClass == PrimitiveType.IntType.class) { - return AugIntegerTypes.getInteger127(); + return AugmentIntegerTypes.getInteger127(); } else if (highClass == PrimitiveType.ShortType.class) { return PrimitiveType.getByte(); } else if (highClass == PrimitiveType.BooleanType.class || highClass == PrimitiveType.ByteType.class || highClass == PrimitiveType.CharType.class - || highClass == AugIntegerTypes.Integer127Type.class - || highClass == AugIntegerTypes.Integer32767Type.class) { + || highClass == AugmentIntegerTypes.Integer127Type.class + || highClass == AugmentIntegerTypes.Integer32767Type.class) { return high; } else { logger.error(low + " cannot be promoted with the supertype " + high + "!"); return null; } - } else if (lowClass == AugIntegerTypes.Integer127Type.class) { + } else if (lowClass == AugmentIntegerTypes.Integer127Type.class) { if (highClass == PrimitiveType.ShortType.class) { return PrimitiveType.getByte(); } else if (highClass == PrimitiveType.IntType.class) { - return AugIntegerTypes.getInteger127(); + return AugmentIntegerTypes.getInteger127(); } else if (highClass == PrimitiveType.ByteType.class || highClass == PrimitiveType.CharType.class - || highClass == AugIntegerTypes.Integer32767Type.class) { + || highClass == AugmentIntegerTypes.Integer32767Type.class) { return high; } else { logger.error(low + " cannot be promoted with the supertype " + high + "!"); return null; } - } else if (lowClass == AugIntegerTypes.Integer32767Type.class) { + } else if (lowClass == AugmentIntegerTypes.Integer32767Type.class) { if (highClass == PrimitiveType.IntType.class) { - return AugIntegerTypes.getInteger32767(); + return AugmentIntegerTypes.getInteger32767(); } else if (highClass == PrimitiveType.ShortType.class || highClass == PrimitiveType.CharType.class) { return high; diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java index ad906f1a809..be3358008ae 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolver.java @@ -41,7 +41,7 @@ import sootup.core.types.ClassType; import sootup.core.types.PrimitiveType; import sootup.core.types.Type; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; import sootup.java.core.views.JavaView; /** @author Zun Wang Algorithm: see 'Efficient Local Type Inference' at OOPSLA 08 */ @@ -299,11 +299,11 @@ private Typing getMinCastsTyping( } private Type convertType(@Nonnull Type type) { - if (type instanceof AugIntegerTypes.Integer1Type) { + if (type instanceof AugmentIntegerTypes.Integer1Type) { return PrimitiveType.getBoolean(); - } else if (type instanceof AugIntegerTypes.Integer127Type) { + } else if (type instanceof AugmentIntegerTypes.Integer127Type) { return PrimitiveType.getByte(); - } else if (type instanceof AugIntegerTypes.Integer32767Type) { + } else if (type instanceof AugmentIntegerTypes.Integer32767Type) { return PrimitiveType.getShort(); } else if (type instanceof ArrayType) { Type eleType = convertType(((ArrayType) type).getElementType()); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugIntegerTypes.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugmentIntegerTypes.java similarity index 98% rename from sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugIntegerTypes.java rename to sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugmentIntegerTypes.java index bf7a4aa94e4..ac14998dbfa 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugIntegerTypes.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/types/AugmentIntegerTypes.java @@ -25,7 +25,7 @@ import sootup.core.jimple.visitor.TypeVisitor; import sootup.core.types.PrimitiveType; -public abstract class AugIntegerTypes { +public abstract class AugmentIntegerTypes { @Nonnull public static Integer1Type getInteger1() { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 115fdeca057..7f858dafcf3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -112,6 +112,8 @@ private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { @Test public void testResource_Misuse() { + // String classPath = + // "../sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache"; String classPath = "../sootup.tests/src/test/resources/interceptor/"; AnalysisInputLocation inputLocation = new JavaClassPathAnalysisInputLocation(classPath); @@ -129,12 +131,12 @@ public List getBodyInterceptors() { return Arrays.asList( new CastAndReturnInliner(), new UnreachableCodeEliminator(), - new LocalSplitter(), + new LocalSplitter(), // FIXME: // FIXME new Aggregator() new TypeAssigner(), // ms: is already called from typeassigner? new LocalNameStandardizer(), new CopyPropagator(), - new DeadAssignmentEliminator(), + new DeadAssignmentEliminator(), // FIXME: removes a branch wrongfully new ConditionalBranchFolder(), new EmptySwitchEliminator(), new NopEliminator(), diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java index 7d01f4e8046..47f15fbb2f1 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java @@ -17,7 +17,7 @@ import sootup.core.types.PrimitiveType; import sootup.core.types.Type; import sootup.core.types.VoidType; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; /** @author Zun Wang */ @Category(Java8Test.class) @@ -48,14 +48,14 @@ public void testImmediate() { case "l1 = 127": value = s.getUses().get(0); stmt = s; - expected = AugIntegerTypes.getInteger127(); + expected = AugmentIntegerTypes.getInteger127(); actual = evalFunction.evaluate(typing, value, stmt, graph); Assert.assertEquals(expected, actual); break; case "l1 = 32111": value = s.getUses().get(0); stmt = s; - expected = AugIntegerTypes.getInteger32767(); + expected = AugmentIntegerTypes.getInteger32767(); actual = evalFunction.evaluate(typing, value, stmt, graph); Assert.assertEquals(expected, actual); break; @@ -189,7 +189,7 @@ public void testExpr() { } map.clear(); - map.put("l1", AugIntegerTypes.getInteger1()); + map.put("l1", AugmentIntegerTypes.getInteger1()); map.put("l2", PrimitiveType.getByte()); specTyping = createTyping(builder.getLocals(), map); @@ -211,7 +211,7 @@ public void testExpr() { StmtGraph graph4 = builder4.getStmtGraph(); map.clear(); - map.put("l1", AugIntegerTypes.getInteger1()); + map.put("l1", AugmentIntegerTypes.getInteger1()); map.put("l2", PrimitiveType.getFloat()); specTyping = createTyping(builder4.getLocals(), map); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java index 21ef1ee8ddd..63b9a2fae65 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java @@ -11,7 +11,7 @@ import sootup.core.model.Body; import sootup.core.types.*; import sootup.core.util.Utils; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; @Category(Java8Test.class) public class CastCounterTest extends TypeAssignerTestSuite { @@ -59,7 +59,7 @@ public void testInvokeStmt() { count = counter.getCastCount(typing); Assert.assertEquals(3, count); - map.replace("l2", AugIntegerTypes.getInteger127()); + map.replace("l2", AugmentIntegerTypes.getInteger127()); typing = createTyping(builder.getLocals(), map); count = counter.getCastCount(typing); Assert.assertEquals(1, count); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchyTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchyTest.java index 1f13ac93a22..1c9e5572ead 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchyTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/PrimitiveHierarchyTest.java @@ -8,7 +8,7 @@ import org.junit.experimental.categories.Category; import sootup.core.types.*; import sootup.core.util.ImmutableUtils; -import sootup.java.bytecode.interceptors.typeresolving.types.AugIntegerTypes; +import sootup.java.bytecode.interceptors.typeresolving.types.AugmentIntegerTypes; import sootup.java.bytecode.interceptors.typeresolving.types.BottomType; /** @author Zun Wang */ @@ -19,9 +19,9 @@ public class PrimitiveHierarchyTest { private Type bt2 = BottomType.getInstance(); private Type boo = PrimitiveType.getBoolean(); private Type i = PrimitiveType.getInt(); - private Type i1 = AugIntegerTypes.getInteger1(); - private Type i127 = AugIntegerTypes.getInteger127(); - private Type i32767 = AugIntegerTypes.getInteger32767(); + private Type i1 = AugmentIntegerTypes.getInteger1(); + private Type i127 = AugmentIntegerTypes.getInteger127(); + private Type i32767 = AugmentIntegerTypes.getInteger32767(); private Type by = PrimitiveType.getByte(); private Type s = PrimitiveType.getShort(); private Type c = PrimitiveType.getChar(); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java index 67932fb5fa6..a12c8732dba 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java @@ -27,7 +27,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 = \"This test\"", + "$l0 = \"This test\"", "l0 = dynamicinvoke \"makeConcatWithConstants\" (l0) (\"\\u0001 is cool\")", "$stack1 = ", "virtualinvoke $stack1.(l0)", diff --git a/sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples$interfacer.class b/sootup.tests/src/test/resources/bugs/664_struce-compiled/org/apache/jsp/java_detection_samples$interfacer.class new file mode 100644 index 0000000000000000000000000000000000000000..c3c0a29831fb024cc78bce38199661641ae3e729 GIT binary patch literal 291 zcma)%y$%6U5QWe9*=3RFwFr%?Q7I)7iA3Q6YznqV9kTUNBXmof zO0|j9W}k?cK#njRtv`!xVPg}`$KMhm*k%_S@-j}D(EFV^DP=-?WenRzs-0u!qeU3} ig)AW-XixZ1mA3&B{5m|DD)8U6fu{689CCxU*L(pu*isk( literal 0 HcmV?d00001 From 2264537aa26efa48b30239d2ac1acbc9616f5184 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Thu, 5 Oct 2023 14:26:39 +0200 Subject: [PATCH 27/54] refactor methodname --- .../src/main/java/sootup/core/types/Type.java | 1 + .../typeresolving/TypeChecker.java | 16 ++++++-------- .../bytecode/interceptors/AggregatorTest.java | 21 ++++++++++--------- 3 files changed, 18 insertions(+), 20 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/types/Type.java b/sootup.core/src/main/java/sootup/core/types/Type.java index 5415f91b9bc..be513ee83f7 100644 --- a/sootup.core/src/main/java/sootup/core/types/Type.java +++ b/sootup.core/src/main/java/sootup/core/types/Type.java @@ -61,6 +61,7 @@ public static ArrayType createArrayType(@Nonnull Type type, int dim) { } public static int getValueBitSize(Type type) { + // TODO: ms: make use of the typevisitor to get O(1) if (type instanceof PrimitiveType.BooleanType) { return 1; } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java index 9e1aa993ac4..1e895254001 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypeChecker.java @@ -166,9 +166,9 @@ public void caseAssignStmt(@Nonnull JAssignStmt stmt) { if (s instanceof JAssignStmt) { Value value = ((JAssignStmt) s).getRightOp(); if (value instanceof JNewArrayExpr) { - sel = selectType(sel, ((JNewArrayExpr) value).getBaseType(), s); + sel = selectArrayType(sel, ((JNewArrayExpr) value).getBaseType(), s); } else if (value instanceof JNewMultiArrayExpr) { - sel = selectType(sel, ((JNewMultiArrayExpr) value).getBaseType(), s); + sel = selectArrayType(sel, ((JNewMultiArrayExpr) value).getBaseType(), s); } else if (value instanceof Local) { worklist.add(new StmtLocalPair(s, (Local) value)); } else if (value instanceof JCastExpr) { @@ -311,16 +311,12 @@ private void handleBinopExpr(AbstractBinopExpr expr, Type type, Stmt stmt) { } // select the type with bigger bit size - public Type selectType(@Nullable Type preType, @Nonnull Type newType, @Nonnull Stmt stmt) { + public Type selectArrayType(@Nullable Type preType, @Nonnull Type newType, @Nonnull Stmt stmt) { if (preType == null || preType.equals(newType)) { return newType; } - Type sel; - if (Type.getValueBitSize(newType) > Type.getValueBitSize(preType)) { - sel = newType; - } else { - sel = preType; - } + Type sel = Type.getValueBitSize(newType) > Type.getValueBitSize(preType) ? newType : preType; + logger.warn( "Conflicting array types at " + stmt @@ -330,7 +326,7 @@ public Type selectType(@Nullable Type preType, @Nonnull Type newType, @Nonnull S + preType + " or " + newType - + ". Select: " + + ". Selecting: " + sel); return sel; } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 7f858dafcf3..3d87a04cfa4 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -129,19 +129,20 @@ public void testResource_Misuse() { @Override public List getBodyInterceptors() { return Arrays.asList( - new CastAndReturnInliner(), + /* new CastAndReturnInliner(), new UnreachableCodeEliminator(), new LocalSplitter(), // FIXME: - // FIXME new Aggregator() - new TypeAssigner(), + // FIXME new Aggregator() */ + new TypeAssigner() // ms: is already called from typeassigner? new LocalNameStandardizer(), - new CopyPropagator(), - new DeadAssignmentEliminator(), // FIXME: removes a branch wrongfully - new ConditionalBranchFolder(), - new EmptySwitchEliminator(), - new NopEliminator(), - new UnusedLocalEliminator(), - new UnreachableCodeEliminator()); + /* new CopyPropagator(), + new DeadAssignmentEliminator(), // FIXME: removes a branch wrongfully + new ConditionalBranchFolder(), + new EmptySwitchEliminator(), + new NopEliminator(), + new UnusedLocalEliminator(), + new UnreachableCodeEliminator() + */ ); } }); From 08b6ee49c72859ed93df806fc7708b0cdd5c9d3a Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 6 Oct 2023 11:01:35 +0200 Subject: [PATCH 28/54] fix missing mapping info --- .../typeresolving/CastCounterTest.java | 34 +++++++++++-------- .../typeresolving/TypeAssignerTestSuite.java | 9 +++++ 2 files changed, 28 insertions(+), 15 deletions(-) diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java index 63b9a2fae65..4fa62fb9c11 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java @@ -8,6 +8,7 @@ import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.jimple.basic.Local; import sootup.core.model.Body; import sootup.core.types.*; import sootup.core.util.Utils; @@ -38,10 +39,10 @@ public void setup() { public void testInvokeStmt() { final Body.BodyBuilder builder = createMethodsBuilder("invokeStmt", "void"); Map map = new HashMap<>(); - map.put("l0", classType); - map.put("l1", super1); - map.put("l2", PrimitiveType.getInt()); - map.put("l3", sub2); + map.put("$l0", classType); + map.put("$l1", super1); + map.put("$l2", PrimitiveType.getInt()); + map.put("$l3", sub2); map.put("$stack4", sub1); map.put("$stack5", sub2); Typing typing = createTyping(builder.getLocals(), map); @@ -69,9 +70,9 @@ public void testInvokeStmt() { public void testAssignStmt() { final Body.BodyBuilder builder = createMethodsBuilder("assignStmt", "void"); Map map = new HashMap<>(); - map.put("l0", classType); - map.put("l1", Type.createArrayType(super1, 1)); - map.put("l2", super1); + map.put("$l0", classType); + map.put("$l1", Type.createArrayType(super1, 1)); + map.put("$l2", super1); map.put("$stack3", sub1); Typing typing = createTyping(builder.getLocals(), map); CastCounter counter = new CastCounter(builder, function, hierarchy); @@ -88,13 +89,16 @@ public void testAssignStmt() { public void testInvokeStmtWithNewCasts() { final Body.BodyBuilder builder = createMethodsBuilder("invokeStmt", "void"); Map map = new HashMap<>(); - map.put("l0", classType); - map.put("l1", super1); - map.put("l2", PrimitiveType.getLong()); - map.put("l3", super2); + map.put("$l0", classType); + map.put("$l1", super1); + map.put("$l2", PrimitiveType.getLong()); + map.put("$l3", super2); map.put("$stack4", sub1); map.put("$stack5", sub2); - Typing typing = createTyping(builder.getLocals(), map); + + final Set locals = builder.getLocals(); + + Typing typing = createTyping(locals, map); CastCounter counter = new CastCounter(builder, function, hierarchy); Assert.assertEquals(3, counter.getCastCount(typing)); counter.insertCastStmts(typing); @@ -129,9 +133,9 @@ public void testInvokeStmtWithNewCasts() { public void testAssignStmtWithNewCasts() { final Body.BodyBuilder builder = createMethodsBuilder("assignStmt", "void"); Map map = new HashMap<>(); - map.put("l0", classType); - map.put("l1", object); - map.put("l2", super1); + map.put("$l0", classType); + map.put("$l1", object); + map.put("$l2", super1); map.put("$stack3", sub1); Typing typing = createTyping(builder.getLocals(), map); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java index 04b1859ea70..7354e1655bd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java @@ -4,6 +4,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import org.junit.Assert; import sootup.core.jimple.basic.Local; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -51,6 +52,14 @@ public Body.BodyBuilder createMethodsBuilder(String methodName, String returnTyp } public Typing createTyping(Set locals, Map map) { + + final Optional foundOpt = + locals.stream() + .filter(local -> !map.containsKey(local.toString())) + .peek(i -> System.out.println("TEST: missing mapping for: " + i)) + .findAny(); + Assert.assertFalse(foundOpt.isPresent()); + Typing typing = new Typing(locals); for (Local l : typing.getLocals()) { if (map.containsKey(l.getName())) { From cd42555bf8287ba2a89a98e4d7622cdf2aa8c5ec Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 6 Oct 2023 12:31:46 +0200 Subject: [PATCH 29/54] fix TypeAssigner: don't fail completely if type is not augmentable --- .../typeresolving/AugEvalFunction.java | 13 +- .../typeresolving/TypePromotionVisitor.java | 2 +- .../typeresolving/AugEvalFunctionTest.java | 219 ++++++++++-------- .../typeresolving/CastCounterTest.java | 42 ++-- .../typeresolving/TypeAssignerTest.java | 33 +-- .../typeresolving/TypeAssignerTestSuite.java | 17 +- .../typeresolving/TypeResolverTest.java | 34 +-- 7 files changed, 189 insertions(+), 171 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java index 7cefd49af12..b749047583b 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunction.java @@ -89,8 +89,7 @@ public Type evaluate( if (value instanceof Immediate) { if (value instanceof Local) { return typing.getType((Local) value); - // if value instanceof Constant - } else { + } else if (value instanceof Constant) { if (value.getClass() == IntConstant.class) { int val = ((IntConstant) value).getValue(); if (val >= 0 && val < 2) { @@ -123,9 +122,7 @@ public Type evaluate( } else if (value.getClass() == MethodType.class) { return methodTypeClassType; } else { - // return null; - throw new RuntimeException( - "can't evaluatable constant in AugEvalFunction '" + value + "'."); + throw new IllegalStateException("can't evaluate this type of Constant '" + value + "'."); } } } else if (value instanceof Expr) { @@ -203,12 +200,6 @@ public Type evaluate( type = getLeastCommonExceptionType(type, exceptionType); } } - - if (type == null) { - return null; - // throw new RuntimeException("can't evaluatable reference in AugEvalFunction '" + value + - // "'."); - } return type; } else if (value instanceof JArrayRef) { Type type = typing.getType(((JArrayRef) value).getBase()); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java index 8f296bc603f..83b88e2d2c1 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/TypePromotionVisitor.java @@ -73,7 +73,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm BytecodeHierarchy hierarchy = getHierarchy(); Typing typing = getTyping(); Type evaType = evalFunction.evaluate(typing, value, stmt, graph); - if (evaType.equals(stdType)) { + if (evaType == null || evaType.equals(stdType)) { return; } if (!hierarchy.isAncestor(stdType, evaType)) { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java index 47f15fbb2f1..12e0cf0eca2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/AugEvalFunctionTest.java @@ -23,7 +23,6 @@ @Category(Java8Test.class) public class AugEvalFunctionTest extends TypeAssignerTestSuite { - Typing typing = new Typing(new ArrayList<>()); AugEvalFunction evalFunction; @Before @@ -42,42 +41,44 @@ public void testImmediate() { final Body.BodyBuilder builder = createMethodsBuilder("constant", "void"); StmtGraph graph = builder.getStmtGraph(); + Typing specTyping = new Typing(new ArrayList<>()); + for (Stmt s : graph.getStmts()) { - String sn = s.toString(); - switch (sn) { - case "l1 = 127": + String stmtStr = s.toString(); + switch (stmtStr) { + case "$l1 = 127": value = s.getUses().get(0); stmt = s; expected = AugmentIntegerTypes.getInteger127(); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); break; - case "l1 = 32111": + case "$l1 = 32111": value = s.getUses().get(0); stmt = s; expected = AugmentIntegerTypes.getInteger32767(); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); break; - case "l1 = -129": + case "$l1 = -129": value = s.getUses().get(0); stmt = s; expected = PrimitiveType.getShort(); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); break; - case "l2 = 1.0": + case "$l2 = 1.0": value = s.getUses().get(0); stmt = s; expected = PrimitiveType.getDouble(); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); break; - case "l4 = \"example\"": + case "$l4 = \"example\"": value = s.getUses().get(0); stmt = s; expected = identifierFactory.getClassType("java.lang.String"); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); break; default: @@ -87,28 +88,31 @@ public void testImmediate() { StmtGraph graph2 = builder2.getStmtGraph(); for (Stmt s : graph2.getStmts()) { - if (s.toString().equals("l1 = class \"LA;\"")) { + if (s.toString().equals("$l1 = class \"LA;\"")) { value = s.getUses().get(0); stmt = s; break; } } expected = identifierFactory.getClassType("java.lang.Class"); - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(specTyping, value, stmt, graph); Assert.assertEquals(expected, actual); } @Test - public void testExpr() { - Type actual; - Type expected = PrimitiveType.getBoolean(); + public void testIfExpr() { Stmt stmt = null; Value value = null; final Body.BodyBuilder builder = createMethodsBuilder("condition", "void"); StmtGraph graph = builder.getStmtGraph(); + Map map = new HashMap<>(); + map.put("$l1", PrimitiveType.getBoolean()); + map.put("$l2", PrimitiveType.getBoolean()); + Typing specTyping1 = createTyping(builder.getLocals(), map); + for (Stmt s : graph.getStmts()) { - if (s.toString().equals("if l1 >= l2")) { + if (s.toString().equals("if $l1 >= $l2")) { for (Value use : s.getUses()) { if (use instanceof AbstractConditionExpr) { value = use; @@ -117,127 +121,151 @@ public void testExpr() { } } } - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertNotNull(stmt); + Assert.assertEquals( + PrimitiveType.getBoolean(), evalFunction.evaluate(specTyping1, value, stmt, graph)); + } - final Body.BodyBuilder builder2 = createMethodsBuilder("shift", "void"); - StmtGraph graph2 = builder2.getStmtGraph(); + @Test + public void testShiftExpr() { + Stmt stmt = null; + Value value = null; + + final Body.BodyBuilder builder = createMethodsBuilder("shift", "void"); + StmtGraph graph = builder.getStmtGraph(); Map map = new HashMap<>(); - map.put("l1", PrimitiveType.getByte()); - map.put("l2", PrimitiveType.getLong()); - Typing specTyping = createTyping(builder2.getLocals(), map); + map.put("$l1", PrimitiveType.getByte()); + map.put("$l2", PrimitiveType.getLong()); + Typing specTyping = createTyping(builder.getLocals(), map); - for (Stmt s : graph2.getStmts()) { - if (s.toString().equals("l4 = l2 << l1")) { + for (Stmt s : graph.getStmts()) { + if (s.toString().equals("$l4 = $l2 << $l1")) { for (Value use : s.getUses()) { if (use instanceof AbstractIntLongBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getLong(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getLong(), evalFunction.evaluate(specTyping, value, stmt, graph)); } - if (s.toString().equals("l6 = l1 << $stack7")) { + if (s.toString().equals("$l6 = $l1 << $stack7")) { for (Value use : s.getUses()) { if (use instanceof AbstractIntLongBinopExpr) { value = use; stmt = s; + + Assert.assertEquals( + PrimitiveType.getInt(), evalFunction.evaluate(specTyping, value, stmt, graph)); + return; } } - expected = PrimitiveType.getInt(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); } } + } - final Body.BodyBuilder builder3 = createMethodsBuilder("xor", "void"); - StmtGraph graph3 = builder3.getStmtGraph(); + @Test + public void testXorExpr1() { + Stmt stmt = null; + Value value = null; - map.clear(); - map.put("l1", PrimitiveType.getBoolean()); - map.put("l2", PrimitiveType.getBoolean()); - map.put("l4", PrimitiveType.getLong()); - specTyping = createTyping(builder3.getLocals(), map); - for (Stmt s : graph3.getStmts()) { - if (s.toString().equals("l3 = l2 ^ l1")) { + final Body.BodyBuilder builder = createMethodsBuilder("xor", "void"); + StmtGraph graph = builder.getStmtGraph(); + + Map map = new HashMap<>(); + map.put("$l1", PrimitiveType.getBoolean()); + map.put("$l2", PrimitiveType.getBoolean()); + map.put("$l4", PrimitiveType.getLong()); + Typing specTyping = createTyping(builder.getLocals(), map); + + for (Stmt s : graph.getStmts()) { + if (s.toString().equals("$l3 = $l2 ^ $l1")) { for (Value use : s.getUses()) { if (use instanceof AbstractIntLongBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getBoolean(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getBoolean(), evalFunction.evaluate(specTyping, value, stmt, graph)); } - if (s.toString().equals("l6 = $stack8 ^ l4")) { + if (s.toString().equals("$l6 = $stack8 ^ $l4")) { for (Value use : s.getUses()) { if (use instanceof AbstractIntLongBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getLong(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getLong(), evalFunction.evaluate(specTyping, value, stmt, graph)); } } + } + + @Test + public void testXorExpr2() { + Stmt stmt = null; + Value value = null; + + final Body.BodyBuilder builder = createMethodsBuilder("xor", "void"); + StmtGraph graph = builder.getStmtGraph(); - map.clear(); - map.put("l1", AugmentIntegerTypes.getInteger1()); - map.put("l2", PrimitiveType.getByte()); - specTyping = createTyping(builder.getLocals(), map); + Map map = new HashMap<>(); + map.put("$l1", AugmentIntegerTypes.getInteger1()); + map.put("$l2", PrimitiveType.getByte()); + Typing specTyping = createTyping(builder.getLocals(), map); for (Stmt s : graph.getStmts()) { - if (s.toString().equals("l3 = l2 ^ l1")) { + if (s.toString().equals("$l3 = $l2 ^ $l1")) { for (Value use : s.getUses()) { if (use instanceof AbstractIntLongBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getByte(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getByte(), evalFunction.evaluate(specTyping, value, stmt, graph)); } } + } - final Body.BodyBuilder builder4 = createMethodsBuilder("add", "void"); - StmtGraph graph4 = builder4.getStmtGraph(); + @Test + public void testAddExpr() { + Stmt stmt = null; + Value value = null; + + final Body.BodyBuilder builder = createMethodsBuilder("add", "void"); + StmtGraph graph = builder.getStmtGraph(); - map.clear(); - map.put("l1", AugmentIntegerTypes.getInteger1()); - map.put("l2", PrimitiveType.getFloat()); - specTyping = createTyping(builder4.getLocals(), map); + Map map = new HashMap<>(); + map.put("$l1", AugmentIntegerTypes.getInteger1()); + map.put("$l2", PrimitiveType.getFloat()); + Typing specTyping = createTyping(builder.getLocals(), map); - for (Stmt s : graph4.getStmts()) { - if (s.toString().equals("l3 = l2 + $stack4")) { + for (Stmt s : graph.getStmts()) { + if (s.toString().equals("$l3 = $l2 + $stack4")) { for (Value use : s.getUses()) { if (use instanceof AbstractFloatBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getFloat(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getFloat(), evalFunction.evaluate(specTyping, value, stmt, graph)); } - if (s.toString().equals("l1 = l1 + 1")) { + if (s.toString().equals("$l1 = $l1 + 1")) { for (Value use : s.getUses()) { if (use instanceof AbstractFloatBinopExpr) { value = use; stmt = s; } } - expected = PrimitiveType.getInt(); - actual = evalFunction.evaluate(specTyping, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getInt(), evalFunction.evaluate(specTyping, value, stmt, graph)); } } @@ -245,7 +273,7 @@ public void testExpr() { StmtGraph graph5 = builder5.getStmtGraph(); for (Stmt s : graph5.getStmts()) { - if (s.toString().equals("l2 = lengthof l1")) { + if (s.toString().equals("$l2 = lengthof $l1")) { for (Value use : s.getUses()) { if (use instanceof JLengthExpr) { value = use; @@ -254,15 +282,14 @@ public void testExpr() { } } } - expected = PrimitiveType.getInt(); - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getInt(), evalFunction.evaluate(specTyping, value, stmt, graph)); final Body.BodyBuilder builder6 = createMethodsBuilder("instanceOf", "boolean"); StmtGraph graph6 = builder6.getStmtGraph(); for (Stmt s : graph6.getStmts()) { - if (s.toString().equals("$stack3 = l1 instanceof A")) { + if (s.toString().equals("$stack3 = $l1 instanceof A")) { for (Value use : s.getUses()) { if (use instanceof JInstanceOfExpr) { value = use; @@ -271,15 +298,14 @@ public void testExpr() { } } } - expected = PrimitiveType.getBoolean(); - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + PrimitiveType.getBoolean(), evalFunction.evaluate(specTyping, value, stmt, graph)); final Body.BodyBuilder builder7 = createMethodsBuilder("newArrayExpr", "void"); StmtGraph graph7 = builder7.getStmtGraph(); for (Stmt s : graph7.getStmts()) { - if (s.toString().equals("l1 = newmultiarray (A)[3][3]")) { + if (s.toString().equals("$l1 = newmultiarray (A)[3][3]")) { for (Value use : s.getUses()) { if (use instanceof JNewMultiArrayExpr) { value = use; @@ -288,9 +314,8 @@ public void testExpr() { } } } - expected = identifierFactory.getArrayType(identifierFactory.getClassType("A"), 2); - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Type expected = identifierFactory.getArrayType(identifierFactory.getClassType("A"), 2); + Assert.assertEquals(expected, evalFunction.evaluate(specTyping, value, stmt, graph)); final Body.BodyBuilder builder8 = createMethodsBuilder("invokeExpr", "void"); StmtGraph graph8 = builder8.getStmtGraph(); @@ -301,19 +326,17 @@ public void testExpr() { if (use instanceof AbstractInvokeExpr) { value = use; stmt = s; - expected = VoidType.getInstance(); - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals( + VoidType.getInstance(), evalFunction.evaluate(specTyping, value, stmt, graph)); } } - } else if (s.toString().equals("$stack3 = virtualinvoke l1.()")) { + } else if (s.toString().equals("$stack3 = virtualinvoke $l1.()")) { for (Value use : s.getUses()) { if (use instanceof AbstractInvokeExpr) { value = use; stmt = s; expected = identifierFactory.getClassType("B"); - actual = evalFunction.evaluate(typing, value, stmt, graph); - Assert.assertEquals(expected, actual); + Assert.assertEquals(expected, evalFunction.evaluate(specTyping, value, stmt, graph)); } } } @@ -340,7 +363,7 @@ public void testRef() { } } } - actual = evalFunction.evaluate(typing, value, stmt, graph); + actual = evalFunction.evaluate(new Typing(new ArrayList<>()), value, stmt, graph); Assert.assertEquals(expected, actual); final Body.BodyBuilder builder2 = createMethodsBuilder("caughtException2", "void"); @@ -357,30 +380,30 @@ public void testRef() { } } expected = identifierFactory.getClassType("java.lang.RuntimeException"); - actual = evalFunction.evaluate(typing, value, stmt, graph2); + actual = evalFunction.evaluate(new Typing(new ArrayList<>()), value, stmt, graph2); Assert.assertEquals(expected, actual); final Body.BodyBuilder builder3 = createMethodsBuilder("fieldRef", "void"); StmtGraph graph3 = builder3.getStmtGraph(); for (Stmt s : graph3.getStmts()) { - if (s.toString().equals("l1 = l0.")) { + if (s.toString().equals("$l1 = $l0.")) { for (Value use : s.getUses()) { if (use instanceof JFieldRef) { value = use; stmt = s; expected = identifierFactory.getClassType("A"); - actual = evalFunction.evaluate(typing, value, stmt, graph3); + actual = evalFunction.evaluate(new Typing(new ArrayList<>()), value, stmt, graph3); Assert.assertEquals(expected, actual); } } - } else if (s.toString().equals("l0 := @this: ByteCodeTypeTest")) { + } else if (s.toString().equals("$l0 := @this: ByteCodeTypeTest")) { for (Value use : s.getUses()) { if (use instanceof JThisRef) { value = use; stmt = s; expected = identifierFactory.getClassType("ByteCodeTypeTest"); - actual = evalFunction.evaluate(typing, value, stmt, graph3); + actual = evalFunction.evaluate(new Typing(new ArrayList<>()), value, stmt, graph3); Assert.assertEquals(expected, actual); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java index 4fa62fb9c11..d80a4da16cd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java @@ -50,17 +50,17 @@ public void testInvokeStmt() { int count = counter.getCastCount(typing); Assert.assertEquals(0, count); - map.replace("l3", super2); + map.replace("$l3", super2); typing = createTyping(builder.getLocals(), map); count = counter.getCastCount(typing); Assert.assertEquals(1, count); - map.replace("l2", PrimitiveType.getLong()); + map.replace("$l2", PrimitiveType.getLong()); typing = createTyping(builder.getLocals(), map); count = counter.getCastCount(typing); Assert.assertEquals(3, count); - map.replace("l2", AugmentIntegerTypes.getInteger127()); + map.replace("$l2", AugmentIntegerTypes.getInteger127()); typing = createTyping(builder.getLocals(), map); count = counter.getCastCount(typing); Assert.assertEquals(1, count); @@ -79,7 +79,7 @@ public void testAssignStmt() { int count = counter.getCastCount(typing); Assert.assertEquals(0, count); - map.replace("l1", object); + map.replace("$l1", object); typing = createTyping(builder.getLocals(), map); count = counter.getCastCount(typing); Assert.assertEquals(5, count); @@ -105,25 +105,25 @@ public void testInvokeStmtWithNewCasts() { List actualStmts = Utils.filterJimple(builder.build().toString()); Assert.assertEquals( Stream.of( - "CastCounterDemos l0", + "CastCounterDemos $l0", "Sub2 #l3", "int #l2", "integer1 #l0", "long #l1", - "unknown $stack4, $stack5, l1, l2, l3", - "l0 := @this: CastCounterDemos", + "unknown $l1, $l2, $l3, $stack4, $stack5", + "$l0 := @this: CastCounterDemos", "$stack4 = new Sub1", "specialinvoke $stack4.()>()", - "l1 = $stack4", + "$l1 = $stack4", "#l0 = 1", "#l1 = (long) #l0", - "l2 = #l1", + "$l2 = #l1", "$stack5 = new Sub2", "specialinvoke $stack5.()>()", - "l3 = $stack5", - "#l2 = (int) l2", - "#l3 = (Sub2) l3", - "virtualinvoke l1.(#l2, #l3)", + "$l3 = $stack5", + "#l2 = (int) $l2", + "#l3 = (Sub2) $l3", + "virtualinvoke $l1.(#l2, #l3)", "return") .collect(Collectors.toList()), actualStmts); @@ -145,19 +145,21 @@ public void testAssignStmtWithNewCasts() { final Body body = builder.build(); List actualStmts = Utils.filterJimple(body.toString()); + System.out.println(Utils.generateJimpleTest(actualStmts)); + Assert.assertEquals( Stream.of( - "CastCounterDemos l0", + "CastCounterDemos $l0", "Super1[] #l0, #l1", - "unknown $stack3, l1, l2", - "l0 := @this: CastCounterDemos", - "l1 = newarray (Super1)[10]", + "unknown $l1, $l2, $stack3", + "$l0 := @this: CastCounterDemos", + "$l1 = newarray (Super1)[10]", "$stack3 = new Sub1", "specialinvoke $stack3.()>()", - "#l0 = (Super1[]) l1", + "#l0 = (Super1[]) $l1", "#l0[0] = $stack3", - "#l1 = (Super1[]) l1", - "l2 = #l1[2]", + "#l1 = (Super1[]) $l1", + "$l2 = #l1[2]", "return") .collect(Collectors.toList()), actualStmts); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTest.java index e03eaef69fc..606633a7811 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTest.java @@ -30,19 +30,19 @@ public void testInvokeStmt() { Assert.assertEquals( Stream.of( - "CastCounterDemos r0", - "Sub1 $r1, r2", - "Sub2 $r3, r4", - "byte b0", - "r0 := @this: CastCounterDemos", + "CastCounterDemos $r0", + "Sub1 $r1, $r2", + "Sub2 $r3, $r4", + "byte $b0", + "$r0 := @this: CastCounterDemos", "$r1 = new Sub1", "specialinvoke $r1.()>()", - "r2 = $r1", - "b0 = 1", + "$r2 = $r1", + "$b0 = 1", "$r3 = new Sub2", "specialinvoke $r3.()>()", - "r4 = $r3", - "virtualinvoke r2.(b0, r4)", + "$r4 = $r3", + "virtualinvoke $r2.($b0, $r4)", "return") .collect(Collectors.toList()), actualStmts); @@ -54,18 +54,19 @@ public void testAssignStmt() { new TypeAssigner().interceptBody(builder, view); List actualStmts = Utils.filterJimple(builder.build().toString()); + Assert.assertEquals( Stream.of( - "CastCounterDemos r0", + "CastCounterDemos $r0", "Sub1 $r1", - "Super1 r2", - "Super1[] r3", - "r0 := @this: CastCounterDemos", - "r3 = newarray (Super1)[10]", + "Super1 $r2", + "Super1[] $r3", + "$r0 := @this: CastCounterDemos", + "$r3 = newarray (Super1)[10]", "$r1 = new Sub1", "specialinvoke $r1.()>()", - "r3[0] = $r1", - "r2 = r3[2]", + "$r3[0] = $r1", + "$r2 = $r3[2]", "return") .collect(Collectors.toList()), actualStmts); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java index 7354e1655bd..c3e78370808 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java @@ -4,7 +4,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import org.junit.Assert; import sootup.core.jimple.basic.Local; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -52,13 +51,15 @@ public Body.BodyBuilder createMethodsBuilder(String methodName, String returnTyp } public Typing createTyping(Set locals, Map map) { - - final Optional foundOpt = - locals.stream() - .filter(local -> !map.containsKey(local.toString())) - .peek(i -> System.out.println("TEST: missing mapping for: " + i)) - .findAny(); - Assert.assertFalse(foundOpt.isPresent()); + /* + // test creation helper to find missing type mappings + final Optional foundOpt = + locals.stream() + .filter(local -> !map.containsKey(local.toString())) + .peek(i -> System.out.println("TEST: missing mapping for: " + i)) + .findAny(); + Assert.assertFalse(foundOpt.isPresent()); + */ Typing typing = new Typing(locals); for (Local l : typing.getLocals()) { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolverTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolverTest.java index 64ebcca9e0c..58709e7d202 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolverTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeResolverTest.java @@ -29,21 +29,22 @@ public void testInvokeStmt() { Body newbody = builder.build(); List actualStmts = Utils.filterJimple(newbody.toString()); + Assert.assertEquals( Stream.of( - "CastCounterDemos l0", - "Sub1 $stack4, l1", - "Sub2 $stack5, l3", - "byte l2", - "l0 := @this: CastCounterDemos", + "CastCounterDemos $l0", + "Sub1 $l1, $stack4", + "Sub2 $l3, $stack5", + "byte $l2", + "$l0 := @this: CastCounterDemos", "$stack4 = new Sub1", "specialinvoke $stack4.()>()", - "l1 = $stack4", - "l2 = 1", + "$l1 = $stack4", + "$l2 = 1", "$stack5 = new Sub2", "specialinvoke $stack5.()>()", - "l3 = $stack5", - "virtualinvoke l1.(l2, l3)", + "$l3 = $stack5", + "virtualinvoke $l1.($l2, $l3)", "return") .collect(Collectors.toList()), actualStmts); @@ -57,19 +58,18 @@ public void testAssignStmt() { resolver.resolve(builder); Body newbody = builder.build(); List actualStmts = Utils.filterJimple(newbody.toString()); - Assert.assertEquals( Stream.of( - "CastCounterDemos l0", + "CastCounterDemos $l0", "Sub1 $stack3", - "Super1 l2", - "Super1[] l1", - "l0 := @this: CastCounterDemos", - "l1 = newarray (Super1)[10]", + "Super1 $l2", + "Super1[] $l1", + "$l0 := @this: CastCounterDemos", + "$l1 = newarray (Super1)[10]", "$stack3 = new Sub1", "specialinvoke $stack3.()>()", - "l1[0] = $stack3", - "l2 = l1[2]", + "$l1[0] = $stack3", + "$l2 = $l1[2]", "return") .collect(Collectors.toList()), actualStmts); From fe673b5b36a8d2ae9f9d5b290491925f450e25c4 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 6 Oct 2023 13:55:39 +0200 Subject: [PATCH 30/54] fix testcases: adapting to fixed local naming.. --- .../core/jimple/basic/JimpleComparator.java | 6 +- .../typeresolving/CastCounterTest.java | 1 - .../java10/TypeInferenceTest.java | 22 +- .../java11/TypeInferenceLambdaTest.java | 8 +- .../java14/SwitchExprWithYieldTest.java | 28 +- .../java15/MultilineStringsTest.java | 4 +- .../java6/AbstractClassTest.java | 6 +- .../java6/AccessArraysTest.java | 180 ++++++------ .../java6/AnonymousClassInsideMethodTest.java | 8 +- .../java6/AssertStatementTest.java | 12 +- .../java6/AutoboxingTest.java | 8 +- .../java6/AutomaticWideningTest.java | 2 +- .../java6/BitwiseOperationsIntTest.java | 26 +- .../java6/BooleanOperatorsTest.java | 220 +++++++------- .../java6/BreakInWhileLoopTest.java | 12 +- .../java6/CastingInNumTypesTest.java | 42 +-- .../java6/CharLiteralsTest.java | 19 +- .../java6/ContinueInWhileLoopTest.java | 12 +- .../java6/CreateNewInstanceTest.java | 4 +- .../java6/DeclareConstructorTest.java | 22 +- .../java6/DeclareEnumTest.java | 18 +- .../java6/DeclareEnumWithConstructorTest.java | 14 +- .../java6/DeclareFieldTest.java | 4 +- .../java6/DeclareFloatTest.java | 6 +- .../java6/DeclareInnerClassTest.java | 4 +- .../java6/DeclareIntTest.java | 8 +- .../java6/DeclareLongTest.java | 6 +- .../java6/DoWhileLoopTest.java | 12 +- .../java6/EmptyStatementTest.java | 2 +- .../java6/EscapeSequencesInStringTest.java | 32 +- .../EvaluationOrderWithParenthesesTest.java | 2 +- .../java6/FinalMethodTest.java | 2 +- .../java6/FinalVariableTest.java | 4 +- .../java6/ForEachLoopTest.java | 22 +- .../minimaltestsuite/java6/ForLoopTest.java | 16 +- .../java6/GenTypeParamTest.java | 12 +- .../java6/GenericTypeParamOnClassTest.java | 12 +- .../java6/GenericTypeParamOnMethodTest.java | 4 +- .../java6/IfElseStatementTest.java | 124 ++++---- .../java6/InfiniteLoopTest.java | 2 +- .../Initialize3DimensionalArraysTest.java | 36 +-- .../InitializeArraysWhileDeclarationTest.java | 36 +-- .../java6/InitializeArraysWithIndexTest.java | 86 +++--- .../InitializeMultidimensionalArraysTest.java | 36 +-- .../java6/InstanceOfCheckTest.java | 6 +- .../java6/InterfaceImplClassTest.java | 2 +- .../java6/LabelStatementTest.java | 14 +- .../java6/LabelledLoopBreakTest.java | 16 +- .../java6/MethodAcceptingVarTest.java | 40 +-- .../java6/MethodOverloadingTest.java | 14 +- .../java6/MethodOverridingTest.java | 2 +- .../java6/MethodReturningVarTest.java | 14 +- .../java6/MultiInterfaceImplClassTest.java | 2 +- .../java6/NamedClassInsideMethodTest.java | 8 +- .../java6/NewCodeBlockInMethodTest.java | 2 +- .../java6/NullVariableTest.java | 2 +- .../java6/PublicClassTest.java | 2 +- .../java6/ReferenceVarDeclarationTest.java | 2 +- .../java6/ReferencingThisTest.java | 10 +- .../java6/ReflectionTest.java | 14 +- .../java6/StatementEvalTest.java | 10 +- .../java6/StaticImportTest.java | 2 +- .../java6/StaticMethodInvocationTest.java | 2 +- .../java6/StringConcatenationTest.java | 2 +- .../java6/StringWithUnicodeCharTest.java | 2 +- .../minimaltestsuite/java6/SubClassTest.java | 20 +- .../java6/SuperClassTest.java | 10 +- .../java6/SwitchCaseStatementTest.java | 108 +++---- .../java6/SynchronizedBlockTest.java | 16 +- .../java6/SynchronizedMethodTest.java | 2 +- .../java6/TernaryOperatorTest.java | 4 +- .../java6/ThrowExceptionMethodTest.java | 4 +- .../java6/TransientVariableTest.java | 4 +- .../java6/TryCatchFinallyTest.java | 276 +++++++++--------- .../java6/UnaryOpIntTest.java | 13 +- .../java6/UncheckedCastTest.java | 8 +- .../java6/VariableDeclarationTest.java | 14 +- .../java6/VariableShadowingTest.java | 6 +- .../java6/VirtualMethodTest.java | 10 +- .../java6/VolatileVariableTest.java | 6 +- .../minimaltestsuite/java6/WhileLoopTest.java | 12 +- ...\317\201\316\265\317\204\316\267Test.java" | 2 +- .../java7/BinaryLiteralInIntTest.java | 2 +- .../java7/MultiTryCatchTest.java | 40 +-- .../SwitchCaseStatementWithStringTest.java | 60 ++-- .../java7/TryWithResourcesTest.java | 48 +-- .../java7/UnderscoreInIntTest.java | 2 +- .../java8/DefaultMethodInterfaceImplTest.java | 6 +- .../java8/MethodAcceptingLamExprTest.java | 6 +- .../java8/MethodReferenceTest.java | 2 +- .../java9/AnonymousDiamondOperatorTest.java | 12 +- .../java9/DynamicInvokeTest.java | 4 +- .../java9/PrivateMethodInterfaceImplTest.java | 4 +- .../java9/TryWithResourcesConciseTest.java | 30 +- 94 files changed, 1026 insertions(+), 1015 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java b/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java index 350f4e234ad..5da7ae723de 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java +++ b/sootup.core/src/main/java/sootup/core/jimple/basic/JimpleComparator.java @@ -168,14 +168,14 @@ public boolean caseEnterMonitorStmt(JEnterMonitorStmt stmt, Object o) { if (!(o instanceof JEnterMonitorStmt)) { return false; } - return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JEnterMonitorStmt) o).getOp(), this); } public boolean caseExitMonitorStmt(JExitMonitorStmt stmt, Object o) { if (!(o instanceof JExitMonitorStmt)) { return false; } - return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JExitMonitorStmt) o).getOp(), this); } public boolean caseGotoStmt(JGotoStmt stmt, Object o) { @@ -244,7 +244,7 @@ public boolean caseThrowStmt(JThrowStmt stmt, Object o) { if (!(o instanceof JThrowStmt)) { return false; } - return stmt.getOp().equivTo(((JReturnStmt) o).getOp(), this); + return stmt.getOp().equivTo(((JThrowStmt) o).getOp(), this); } public boolean caseAbstractBinopExpr(AbstractBinopExpr obj, Object o) { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java index d80a4da16cd..0ade1555c77 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/CastCounterTest.java @@ -145,7 +145,6 @@ public void testAssignStmtWithNewCasts() { final Body body = builder.build(); List actualStmts = Utils.filterJimple(body.toString()); - System.out.println(Utils.generateJimpleTest(actualStmts)); Assert.assertEquals( Stream.of( diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java10/TypeInferenceTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java10/TypeInferenceTest.java index a2f648e146c..a826432df6d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java10/TypeInferenceTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java10/TypeInferenceTest.java @@ -47,25 +47,25 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TypeInference", - "l1 = \"file.txt\"", - "l2 = \"\"", + "$l0 := @this: TypeInference", + "$l1 = \"file.txt\"", + "$l2 = \"\"", "$stack5 = new java.io.FileReader", - "specialinvoke $stack5.(java.lang.String)>(l1)", - "l3 = $stack5", + "specialinvoke $stack5.(java.lang.String)>($l1)", + "$l3 = $stack5", "$stack6 = new java.io.BufferedReader", - "specialinvoke $stack6.(java.io.Reader)>(l3)", - "l4 = $stack6", + "specialinvoke $stack6.(java.io.Reader)>($l3)", + "$l4 = $stack6", "label1:", - "$stack9 = l4", + "$stack9 = $l4", "$stack7 = virtualinvoke $stack9.()", - "l2 = $stack7", + "$l2 = $stack7", "if $stack7 == null goto label2", "$stack8 = ", - "virtualinvoke $stack8.(l2)", + "virtualinvoke $stack8.($l2)", "goto label1", "label2:", - "virtualinvoke l4.()", + "virtualinvoke $l4.()", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java11/TypeInferenceLambdaTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java11/TypeInferenceLambdaTest.java index f640bfc4714..e42629d65e6 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java11/TypeInferenceLambdaTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java11/TypeInferenceLambdaTest.java @@ -32,13 +32,13 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TypeInferenceLambda", - "l1 = dynamicinvoke \"apply\" () (methodtype: java.lang.Object __METHODTYPE__(java.lang.Object,java.lang.Object), handle: , methodtype: java.lang.Integer __METHODTYPE__(java.lang.Integer,java.lang.Integer))", + "$l0 := @this: TypeInferenceLambda", + "$l1 = dynamicinvoke \"apply\" () (methodtype: java.lang.Object __METHODTYPE__(java.lang.Object,java.lang.Object), handle: , methodtype: java.lang.Integer __METHODTYPE__(java.lang.Integer,java.lang.Integer))", "$stack4 = staticinvoke (2)", "$stack3 = staticinvoke (3)", - "$stack5 = interfaceinvoke l1.($stack4, $stack3)", + "$stack5 = interfaceinvoke $l1.($stack4, $stack3)", "$stack6 = (java.lang.Integer) $stack5", - "l2 = virtualinvoke $stack6.()", + "$l2 = virtualinvoke $stack6.()", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/SwitchExprWithYieldTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/SwitchExprWithYieldTest.java index a0c7f2a48fe..5c3480fd580 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/SwitchExprWithYieldTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/SwitchExprWithYieldTest.java @@ -72,10 +72,10 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SwitchExprWithYieldTest", - "l1 = 5", - "l2 = \"\"", - "switch(l1)", + "$l0 := @this: SwitchExprWithYieldTest", + "$l1 = 5", + "$l2 = \"\"", + "switch($l1)", "case 1: goto label01", "case 2: goto label02", "case 3: goto label02", @@ -89,8 +89,8 @@ public List expectedBodyStmts() { "label03:", "$stack11 = \"somethingElse\"", "label04:", - "l2 = $stack11", - "$stack12 = l1", + "$l2 = $stack11", + "$stack12 = $l1", "switch($stack12)", "case 1: goto label05", "case 2: goto label06", @@ -105,8 +105,8 @@ public List expectedBodyStmts() { "label07:", "$stack9 = \"somethingElse\"", "label08:", - "l2 = $stack9", - "$stack10 = l1", + "$l2 = $stack9", + "$stack10 = $l1", "switch($stack10)", "case 1: goto label09", "case 2: goto label10", @@ -121,18 +121,18 @@ public List expectedBodyStmts() { "label11:", "$stack7 = \"we will not fall through\"", "label12:", - "l2 = $stack7", - "$stack8 = l1", + "$l2 = $stack7", + "$stack8 = $l1", "switch($stack8)", "case 1: goto label13", "default: goto label14", "label13:", - "l2 = dynamicinvoke \"makeConcatWithConstants\" (l2) (\"\\u0001single\")", + "$l2 = dynamicinvoke \"makeConcatWithConstants\" ($l2) (\"\\u0001single\")", "label14:", - "$stack4 = l2", - "$stack5 = dynamicinvoke \"makeConcatWithConstants\" (l2) (\"\\u0001somethingElse\")", + "$stack4 = $l2", + "$stack5 = dynamicinvoke \"makeConcatWithConstants\" ($l2) (\"\\u0001somethingElse\")", "$stack3 = ", - "$stack6 = l2", + "$stack6 = $l2", "virtualinvoke $stack3.($stack6)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java15/MultilineStringsTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java15/MultilineStringsTest.java index 01e2330bca8..8f2b782222a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java15/MultilineStringsTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java15/MultilineStringsTest.java @@ -37,8 +37,8 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MultilineStrings", - "l1 = \"\\n \\n

Hello, world

\\n \\n\\n\"", + "$l0 := @this: MultilineStrings", + "$l1 = \"\\n \\n

Hello, world

\\n \\n\\n\"", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AbstractClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AbstractClassTest.java index 2067f5bc1e5..66b523abcc4 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AbstractClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AbstractClassTest.java @@ -47,11 +47,11 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: AbstractClass", + "$l0 := @this: AbstractClass", "$stack2 = new AbstractClass", "specialinvoke $stack2.()>()", - "l1 = $stack2", - "virtualinvoke l1.()", + "$l1 = $stack2", + "virtualinvoke $l1.()", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AccessArraysTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AccessArraysTest.java index 6c5d40ca547..90711f384ea 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AccessArraysTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AccessArraysTest.java @@ -66,22 +66,22 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIntArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (int)[3]", "$stack7[0] = 1", "$stack7[1] = 2", "$stack7[2] = 3", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -104,22 +104,22 @@ public List expectedBodyStmtsIntArrays() { */ public List expectedBodyStmtsByteArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (byte)[3]", "$stack7[0] = 4", "$stack7[1] = 5", "$stack7[2] = 6", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -142,22 +142,22 @@ public List expectedBodyStmtsByteArrays() { */ public List expectedBodyStmtsShortArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (short)[3]", "$stack7[0] = 10", "$stack7[1] = 20", "$stack7[2] = 30", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -180,22 +180,22 @@ public List expectedBodyStmtsShortArrays() { */ public List expectedBodyStmtsLongArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack9 = newarray (long)[3]", "$stack9[0] = 547087L", "$stack9[1] = 564645L", "$stack9[2] = 654786L", - "l1 = $stack9", - "l4 = l1", - "l5 = lengthof l4", - "l6 = 0", + "$l1 = $stack9", + "$l4 = $l1", + "$l5 = lengthof $l4", + "$l6 = 0", "label1:", - "$stack11 = l6", - "$stack10 = l5", + "$stack11 = $l6", + "$stack10 = $l5", "if $stack11 >= $stack10 goto label2", - "l7 = l4[l6]", - "l2 = l7", - "l6 = l6 + 1", + "$l7 = $l4[$l6]", + "$l2 = $l7", + "$l6 = $l6 + 1", "goto label1", "label2:", "return") @@ -218,23 +218,23 @@ public List expectedBodyStmtsLongArrays() { */ public List expectedBodyStmtsFloatArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (float)[4]", "$stack7[0] = 3.14F", "$stack7[1] = 5.46F", "$stack7[2] = 2.987F", "$stack7[3] = 4.87F", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -258,21 +258,21 @@ public List expectedBodyStmtsFloatArrays() { */ public List expectedBodyStmtsDoubleArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack9 = newarray (double)[2]", "$stack9[0] = 6.765414", "$stack9[1] = 9.676565646", - "l1 = $stack9", - "l4 = l1", - "l5 = lengthof l4", - "l6 = 0", + "$l1 = $stack9", + "$l4 = $l1", + "$l5 = lengthof $l4", + "$l6 = 0", "label1:", - "$stack11 = l6", - "$stack10 = l5", + "$stack11 = $l6", + "$stack10 = $l5", "if $stack11 >= $stack10 goto label2", - "l7 = l4[l6]", - "l2 = l7", - "l6 = l6 + 1", + "$l7 = $l4[$l6]", + "$l2 = $l7", + "$l6 = $l6 + 1", "goto label1", "label2:", "return") @@ -295,21 +295,21 @@ public List expectedBodyStmtsDoubleArrays() { */ public List expectedBodyStmtsBooleanArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (boolean)[2]", "$stack7[0] = 1", "$stack7[1] = 0", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -332,22 +332,22 @@ public List expectedBodyStmtsBooleanArrays() { */ public List expectedBodyStmtsCharArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (char)[3]", "$stack7[0] = 65", "$stack7[1] = 98", "$stack7[2] = 38", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") @@ -369,21 +369,21 @@ public List expectedBodyStmtsCharArrays() { */ public List expectedBodyStmtsStringArrays() { return Stream.of( - "l0 := @this: AccessArrays", + "$l0 := @this: AccessArrays", "$stack7 = newarray (java.lang.String)[2]", "$stack7[0] = \"Hello World\"", "$stack7[1] = \"Greetings\"", - "l1 = $stack7", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l6", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l6", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AnonymousClassInsideMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AnonymousClassInsideMethodTest.java index 78896bfbc51..c29b3a10e5e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AnonymousClassInsideMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AnonymousClassInsideMethodTest.java @@ -45,11 +45,11 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: AnonymousClassInsideMethod", + "$l0 := @this: AnonymousClassInsideMethod", "$stack2 = new AnonymousClassInsideMethod$1", - "specialinvoke $stack2.(AnonymousClassInsideMethod)>(l0)", - "l1 = $stack2", - "interfaceinvoke l1.()", + "specialinvoke $stack2.(AnonymousClassInsideMethod)>($l0)", + "$l1 = $stack2", + "interfaceinvoke $l1.()", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AssertStatementTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AssertStatementTest.java index c806aa4fca3..283726bc4ec 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AssertStatementTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AssertStatementTest.java @@ -42,7 +42,7 @@ public MethodSignature getMethodSignatureExtend2() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: AssertStatement", + "$l0 := @this: AssertStatement", "$stack1 = ", "if $stack1 != 0 goto label1", "if \"\" != null goto label1", @@ -66,7 +66,7 @@ public List expectedBodyStmts() { */ public List expectedBodyStmtsExtend() { return Stream.of( - "l0 := @this: AssertStatement", + "$l0 := @this: AssertStatement", "$stack2 = ", "if $stack2 != 0 goto label1", "if \"\" != null goto label1", @@ -75,7 +75,7 @@ public List expectedBodyStmtsExtend() { "throw $stack4", "label1:", "$stack3 = 4", - "l1 = $stack3", + "$l1 = $stack3", "return") .collect(Collectors.toList()); } @@ -94,7 +94,7 @@ public List expectedBodyStmtsExtend() { */ public List expectedBodyStmtsExtend2() { return Stream.of( - "l0 := @this: AssertStatement", + "$l0 := @this: AssertStatement", "$stack2 = ", "if $stack2 != 0 goto label1", "if \"first\" != null goto label1", @@ -103,7 +103,7 @@ public List expectedBodyStmtsExtend2() { "throw $stack7", "label1:", "$stack6 = 1", - "l1 = $stack6", + "$l1 = $stack6", "$stack3 = ", "if $stack3 != 0 goto label2", "if \"second\" != null goto label2", @@ -112,7 +112,7 @@ public List expectedBodyStmtsExtend2() { "throw $stack5", "label2:", "$stack4 = 2", - "l1 = $stack4", + "$l1 = $stack4", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutoboxingTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutoboxingTest.java index 9a866da3f26..ad5c05229b3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutoboxingTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutoboxingTest.java @@ -35,10 +35,10 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: Autoboxing", - "l1 = 5", - "l1 = l1 + 1", - "l2 = staticinvoke (l1)", + "$l0 := @this: Autoboxing", + "$l1 = 5", + "$l1 = $l1 + 1", + "$l2 = staticinvoke ($l1)", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutomaticWideningTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutomaticWideningTest.java index d117f3c0184..08cbac0ccf4 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutomaticWideningTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/AutomaticWideningTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: AutomaticWidening", "l1 = 10", "l2 = (long) l1", "return") + return Stream.of("$l0 := @this: AutomaticWidening", "$l1 = 10", "$l2 = (long) $l1", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BitwiseOperationsIntTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BitwiseOperationsIntTest.java index 3f8d2a822ae..0be2416ea11 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BitwiseOperationsIntTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BitwiseOperationsIntTest.java @@ -53,7 +53,11 @@ public void test() { */ public List expectedBodyStmtsBitwiseOpAnd() { return Stream.of( - "l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = 20", "l3 = l1 & l2", "return") + "$l0 := @this: BitwiseOperationsInt", + "$l1 = 70", + "$l2 = 20", + "$l3 = $l1 & $l2", + "return") .collect(Collectors.toList()); } @@ -70,7 +74,11 @@ public List expectedBodyStmtsBitwiseOpAnd() { */ public List expectedBodyStmtsBitwiseOpOr() { return Stream.of( - "l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = 20", "l3 = l1 | l2", "return") + "$l0 := @this: BitwiseOperationsInt", + "$l1 = 70", + "$l2 = 20", + "$l3 = $l1 | $l2", + "return") .collect(Collectors.toList()); } @@ -87,7 +95,11 @@ public List expectedBodyStmtsBitwiseOpOr() { */ public List expectedBodyStmtsBitwiseOpXor() { return Stream.of( - "l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = 20", "l3 = l1 ^ l2", "return") + "$l0 := @this: BitwiseOperationsInt", + "$l1 = 70", + "$l2 = 20", + "$l3 = $l1 ^ $l2", + "return") .collect(Collectors.toList()); } @@ -102,7 +114,7 @@ public List expectedBodyStmtsBitwiseOpXor() { * */ public List expectedBodyStmtsBitwiseOpComplement() { - return Stream.of("l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = l1 ^ -1", "return") + return Stream.of("$l0 := @this: BitwiseOperationsInt", "$l1 = 70", "$l2 = $l1 ^ -1", "return") .collect(Collectors.toList()); } @@ -117,7 +129,7 @@ public List expectedBodyStmtsBitwiseOpComplement() { * */ public List expectedBodyStmtsBitwiseOpSignedRightShift() { - return Stream.of("l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = l1 >> 5", "return") + return Stream.of("$l0 := @this: BitwiseOperationsInt", "$l1 = 70", "$l2 = $l1 >> 5", "return") .collect(Collectors.toList()); } @@ -132,7 +144,7 @@ public List expectedBodyStmtsBitwiseOpSignedRightShift() { * */ public List expectedBodyStmtsBitwiseOpLeftShift() { - return Stream.of("l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = l1 << 5", "return") + return Stream.of("$l0 := @this: BitwiseOperationsInt", "$l1 = 70", "$l2 = $l1 << 5", "return") .collect(Collectors.toList()); } @@ -147,7 +159,7 @@ public List expectedBodyStmtsBitwiseOpLeftShift() { * */ public List expectedBodyStmtsBitwiseOpUnsignedRightShift() { - return Stream.of("l0 := @this: BitwiseOperationsInt", "l1 = 70", "l2 = l1 >>> 5", "return") + return Stream.of("$l0 := @this: BitwiseOperationsInt", "$l1 = 70", "$l2 = $l1 >>> 5", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BooleanOperatorsTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BooleanOperatorsTest.java index 7759dea39a2..870446f17bc 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BooleanOperatorsTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BooleanOperatorsTest.java @@ -48,7 +48,7 @@ public void testLogicalOr() { } @Test - public void testLocgicalOpXor() { + public void testlocgicalopXor() { SootMethod method = loadMethod(getMethodSignature("logicalOpXor")); assertJimpleStmts(method, expectedBodyStmtsLogicalOpXor()); } @@ -93,14 +93,14 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsRelationalOpEqual() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 0", + "$l0 := @this: BooleanOperators", + "$l1 = 0", "label1:", - "$stack3 = l1", + "$stack3 = $l1", "$stack2 = 10", "if $stack3 > $stack2 goto label2", - "l1 = l1 + 1", - "if l1 != 5 goto label1", + "$l1 = $l1 + 1", + "if $l1 != 5 goto label1", "goto label2", "label2:", "return") @@ -125,16 +125,16 @@ public List expectedBodyStmtsRelationalOpEqual() { */ public List expectedBodyStmtsRelationalOpNotEqual() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 0", - "l2 = \"\"", + "$l0 := @this: BooleanOperators", + "$l1 = 0", + "$l2 = \"\"", "label1:", - "$stack4 = l1", + "$stack4 = $l1", "$stack3 = 10", "if $stack4 >= $stack3 goto label2", - "l1 = l1 + 1", - "if l1 == 5 goto label1", - "l2 = \"i != 5\"", + "$l1 = $l1 + 1", + "if $l1 == 5 goto label1", + "$l2 = \"i != 5\"", "goto label1", "label2:", "return") @@ -155,16 +155,16 @@ public List expectedBodyStmtsRelationalOpNotEqual() { */ public List expectedBodyStmtsComplementOp() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "if l1 == 0 goto label3", - "if l1 != 0 goto label1", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "if $l1 == 0 goto label3", + "if $l1 != 0 goto label1", "$stack2 = 1", "goto label2", "label1:", "$stack2 = 0", "label2:", - "l1 = $stack2", + "$l1 = $stack2", "label3:", "return") .collect(Collectors.toList()); @@ -201,33 +201,33 @@ public List expectedBodyStmtsComplementOp() { */ public List expectedBodyStmtsLogicalOpAnd() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "l2 = 1", - "l3 = 0", - "l4 = 0", - "l5 = \"\"", - "$stack6 = l1 & l2", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "$l2 = 1", + "$l3 = 0", + "$l4 = 0", + "$l5 = \"\"", + "$stack6 = $l1 & $l2", "if $stack6 == 0 goto label1", - "l5 = \"A\"", + "$l5 = \"A\"", "label1:", - "$stack15 = l3", - "$stack14 = l4", + "$stack15 = $l3", + "$stack14 = $l4", "$stack7 = $stack15 & $stack14", "if $stack7 == 0 goto label2", - "l5 = \"B\"", + "$l5 = \"B\"", "label2:", - "$stack13 = l1", - "$stack12 = l3", + "$stack13 = $l1", + "$stack12 = $l3", "$stack8 = $stack13 & $stack12", "if $stack8 == 0 goto label3", - "l5 = \"C\"", + "$l5 = \"C\"", "label3:", - "$stack11 = l4", - "$stack10 = l2", + "$stack11 = $l4", + "$stack10 = $l2", "$stack9 = $stack11 & $stack10", "if $stack9 == 0 goto label4", - "l5 = \"D\"", + "$l5 = \"D\"", "label4:", "return") .collect(Collectors.toList()); @@ -264,33 +264,33 @@ public List expectedBodyStmtsLogicalOpAnd() { */ public List expectedBodyStmtsLogicalOpOr() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "l2 = 1", - "l3 = 0", - "l4 = 0", - "l5 = \"\"", - "$stack6 = l1 | l2", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "$l2 = 1", + "$l3 = 0", + "$l4 = 0", + "$l5 = \"\"", + "$stack6 = $l1 | $l2", "if $stack6 == 0 goto label1", - "l5 = \"A\"", + "$l5 = \"A\"", "label1:", - "$stack15 = l3", - "$stack14 = l4", + "$stack15 = $l3", + "$stack14 = $l4", "$stack7 = $stack15 | $stack14", "if $stack7 == 0 goto label2", - "l5 = \"B\"", + "$l5 = \"B\"", "label2:", - "$stack13 = l1", - "$stack12 = l3", + "$stack13 = $l1", + "$stack12 = $l3", "$stack8 = $stack13 | $stack12", "if $stack8 == 0 goto label3", - "l5 = \"C\"", + "$l5 = \"C\"", "label3:", - "$stack11 = l4", - "$stack10 = l2", + "$stack11 = $l4", + "$stack10 = $l2", "$stack9 = $stack11 | $stack10", "if $stack9 == 0 goto label4", - "l5 = \"D\"", + "$l5 = \"D\"", "label4:", "return") .collect(Collectors.toList()); @@ -327,33 +327,33 @@ public List expectedBodyStmtsLogicalOpOr() { */ public List expectedBodyStmtsLogicalOpXor() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "l2 = 1", - "l3 = 0", - "l4 = 0", - "l5 = \"\"", - "$stack6 = l1 ^ l2", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "$l2 = 1", + "$l3 = 0", + "$l4 = 0", + "$l5 = \"\"", + "$stack6 = $l1 ^ $l2", "if $stack6 == 0 goto label1", - "l5 = \"A\"", + "$l5 = \"A\"", "label1:", - "$stack15 = l3", - "$stack14 = l4", + "$stack15 = $l3", + "$stack14 = $l4", "$stack7 = $stack15 ^ $stack14", "if $stack7 == 0 goto label2", - "l5 = \"B\"", + "$l5 = \"B\"", "label2:", - "$stack13 = l1", - "$stack12 = l3", + "$stack13 = $l1", + "$stack12 = $l3", "$stack8 = $stack13 ^ $stack12", "if $stack8 == 0 goto label3", - "l5 = \"C\"", + "$l5 = \"C\"", "label3:", - "$stack11 = l4", - "$stack10 = l2", + "$stack11 = $l4", + "$stack10 = $l2", "$stack9 = $stack11 ^ $stack10", "if $stack9 == 0 goto label4", - "l5 = \"D\"", + "$l5 = \"D\"", "label4:", "return") .collect(Collectors.toList()); @@ -390,30 +390,30 @@ public List expectedBodyStmtsLogicalOpXor() { */ public List expectedBodyStmtsConditionalOpAnd() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "l2 = 1", - "l3 = 0", - "l4 = 0", - "l5 = \"\"", - "if l1 == 0 goto label1", - "if l2 == 0 goto label1", - "l5 = \"A\"", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "$l2 = 1", + "$l3 = 0", + "$l4 = 0", + "$l5 = \"\"", + "if $l1 == 0 goto label1", + "if $l2 == 0 goto label1", + "$l5 = \"A\"", "label1:", - "$stack8 = l3", + "$stack8 = $l3", "if $stack8 == 0 goto label2", - "if l4 == 0 goto label2", - "l5 = \"B\"", + "if $l4 == 0 goto label2", + "$l5 = \"B\"", "label2:", - "$stack7 = l1", + "$stack7 = $l1", "if $stack7 == 0 goto label3", - "if l3 == 0 goto label3", - "l5 = \"C\"", + "if $l3 == 0 goto label3", + "$l5 = \"C\"", "label3:", - "$stack6 = l4", + "$stack6 = $l4", "if $stack6 == 0 goto label4", - "if l2 == 0 goto label4", - "l5 = \"D\"", + "if $l2 == 0 goto label4", + "$l5 = \"D\"", "label4:", "return") .collect(Collectors.toList()); @@ -450,38 +450,38 @@ public List expectedBodyStmtsConditionalOpAnd() { */ public List expectedBodyStmtsConditionalOpOr() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 1", - "l2 = 1", - "l3 = 0", - "l4 = 0", - "l5 = \"\"", - "if l1 != 0 goto label1", - "if l2 == 0 goto label2", + "$l0 := @this: BooleanOperators", + "$l1 = 1", + "$l2 = 1", + "$l3 = 0", + "$l4 = 0", + "$l5 = \"\"", + "if $l1 != 0 goto label1", + "if $l2 == 0 goto label2", "label1:", "$stack12 = \"A\"", - "l5 = $stack12", + "$l5 = $stack12", "label2:", - "$stack11 = l3", + "$stack11 = $l3", "if $stack11 != 0 goto label3", - "if l4 == 0 goto label4", + "if $l4 == 0 goto label4", "label3:", "$stack10 = \"B\"", - "l5 = $stack10", + "$l5 = $stack10", "label4:", - "$stack9 = l1", + "$stack9 = $l1", "if $stack9 != 0 goto label5", - "if l3 == 0 goto label6", + "if $l3 == 0 goto label6", "label5:", "$stack8 = \"C\"", - "l5 = $stack8", + "$l5 = $stack8", "label6:", - "$stack7 = l4", + "$stack7 = $l4", "if $stack7 != 0 goto label7", - "if l2 == 0 goto label8", + "if $l2 == 0 goto label8", "label7:", "$stack6 = \"D\"", - "l5 = $stack6", + "$l5 = $stack6", "label8:", "return") .collect(Collectors.toList()); @@ -500,16 +500,16 @@ public List expectedBodyStmtsConditionalOpOr() { */ public List expectedBodyStmtsConditionalOp() { return Stream.of( - "l0 := @this: BooleanOperators", - "l1 = 5", - "l2 = \"\"", - "if l1 >= 10 goto label1", + "$l0 := @this: BooleanOperators", + "$l1 = 5", + "$l2 = \"\"", + "if $l1 >= 10 goto label1", "$stack3 = \"i less than 10\"", "goto label2", "label1:", "$stack3 = \"i greater than 10\"", "label2:", - "l2 = $stack3", + "$l2 = $stack3", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BreakInWhileLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BreakInWhileLoopTest.java index ce08586307f..13f8090bb5b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BreakInWhileLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/BreakInWhileLoopTest.java @@ -37,14 +37,14 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: BreakInWhileLoop", - "l1 = 10", - "l2 = 5", + "$l0 := @this: BreakInWhileLoop", + "$l1 = 10", + "$l2 = 5", "label1:", - "$stack3 = l1", + "$stack3 = $l1", "if $stack3 <= 0 goto label2", - "l1 = l1 + -1", - "if l1 != l2 goto label1", + "$l1 = $l1 + -1", + "if $l1 != $l2 goto label1", "goto label2", "label2:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CastingInNumTypesTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CastingInNumTypesTest.java index e52a98bca3b..88d3ae8eaaa 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CastingInNumTypesTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CastingInNumTypesTest.java @@ -56,52 +56,52 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: CastingInNumTypes", - "l1 = 1", - "l2 = 2", - "l3 = 3", - "l4 = 4551598461L", - "l6 = 5.4F", - "l7 = 4.5515954846546467E39", + "$l0 := @this: CastingInNumTypes", + "$l1 = 1", + "$l2 = 2", + "$l3 = 3", + "$l4 = 4551598461L", + "$l6 = 5.4F", + "$l7 = 4.5515954846546467E39", "$stack11 = ", - "virtualinvoke $stack11.(l1)", + "virtualinvoke $stack11.($l1)", "$stack13 = ", - "$stack12 = (byte) l3", + "$stack12 = (byte) $l3", "virtualinvoke $stack13.($stack12)", "$stack15 = ", - "$stack14 = (double) l2", + "$stack14 = (double) $l2", "virtualinvoke $stack15.($stack14)", "$stack18 = ", - "$stack16 = (int) l4", + "$stack16 = (int) $l4", "$stack17 = (short) $stack16", "virtualinvoke $stack18.($stack17)", "$stack20 = ", - "$stack19 = (double) l6", + "$stack19 = (double) $l6", "virtualinvoke $stack20.($stack19)", "$stack22 = ", - "$stack21 = (int) l4", + "$stack21 = (int) $l4", "virtualinvoke $stack22.($stack21)", "$stack24 = ", - "$stack23 = (float) l7", + "$stack23 = (float) $l7", "virtualinvoke $stack24.($stack23)", "$stack25 = ", - "virtualinvoke $stack25.(l7)", - "l9 = 4.7867778678678685E51", + "virtualinvoke $stack25.($l7)", + "$l9 = 4.7867778678678685E51", "$stack27 = ", - "$stack26 = (float) l9", + "$stack26 = (float) $l9", "virtualinvoke $stack27.($stack26)", "$stack29 = ", - "$stack28 = (long) l9", + "$stack28 = (long) $l9", "virtualinvoke $stack29.($stack28)", "$stack31 = ", - "$stack30 = (int) l9", + "$stack30 = (int) $l9", "virtualinvoke $stack31.($stack30)", "$stack34 = ", - "$stack32 = (int) l9", + "$stack32 = (int) $l9", "$stack33 = (short) $stack32", "virtualinvoke $stack34.($stack33)", "$stack37 = ", - "$stack35 = (int) l9", + "$stack35 = (int) $l9", "$stack36 = (byte) $stack35", "virtualinvoke $stack37.($stack36)", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CharLiteralsTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CharLiteralsTest.java index 6b116ce2f28..9043e3441d3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CharLiteralsTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CharLiteralsTest.java @@ -1,10 +1,9 @@ package sootup.java.bytecode.minimaltestsuite.java6; import categories.Java8Test; +import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; import org.junit.Test; import org.junit.experimental.categories.Category; import sootup.core.model.SootMethod; @@ -54,7 +53,7 @@ public MethodSignature getMethodSignature(String methodName) { * */ public List expectedBodyStmtsCharCharacter() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 97", "return").collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 97", "return"); } /** @@ -67,7 +66,7 @@ public List expectedBodyStmtsCharCharacter() { * */ public List expectedBodyStmtsCharSymbol() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 37", "return").collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 37", "return"); } /** @@ -78,7 +77,7 @@ public List expectedBodyStmtsCharSymbol() { * */ public List expectedBodyStmtsCharBackslashT() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 9", "return").collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 9", "return"); } /** @@ -91,7 +90,7 @@ public List expectedBodyStmtsCharBackslashT() { * */ public List expectedBodyStmtsCharBackslash() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 92", "return").collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 92", "return"); } /** @@ -104,7 +103,7 @@ public List expectedBodyStmtsCharBackslash() { * */ public List expectedBodyStmtsCharSingleQuote() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 39", "return").collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 39", "return"); } /** @@ -117,8 +116,7 @@ public List expectedBodyStmtsCharSingleQuote() { * */ public List expectedBodyStmtsCharUnicode() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 937", "return") - .collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 937", "return"); } /** @@ -131,7 +129,6 @@ public List expectedBodyStmtsCharUnicode() { * */ public List expectedBodyStmtsSpecialChar() { - return Stream.of("l0 := @this: CharLiterals", "l1 = 8482", "return") - .collect(Collectors.toList()); + return Arrays.asList("$l0 := @this: CharLiterals", "$l1 = 8482", "return"); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ContinueInWhileLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ContinueInWhileLoopTest.java index 47e12cba1f1..d7857098ed9 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ContinueInWhileLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ContinueInWhileLoopTest.java @@ -37,17 +37,17 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: ContinueInWhileLoop", - "l1 = 0", + "$l0 := @this: ContinueInWhileLoop", + "$l1 = 0", "label1:", - "$stack3 = l1", + "$stack3 = $l1", "$stack2 = 10", "if $stack3 >= $stack2 goto label3", - "if l1 != 5 goto label2", - "l1 = l1 + 1", + "if $l1 != 5 goto label2", + "$l1 = $l1 + 1", "goto label1", "label2:", - "l1 = l1 + 1", + "$l1 = $l1 + 1", "goto label1", "label3:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CreateNewInstanceTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CreateNewInstanceTest.java index 01893c8dafa..c6930d9abbc 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CreateNewInstanceTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/CreateNewInstanceTest.java @@ -34,10 +34,10 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: CreateNewInstance", + "$l0 := @this: CreateNewInstance", "$stack2 = new Person", "specialinvoke $stack2.(int)>(20)", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareConstructorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareConstructorTest.java index 92bec1708de..63cd4eca971 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareConstructorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareConstructorTest.java @@ -48,11 +48,11 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareConstructor", - "l1 := @parameter0: int", - "specialinvoke l0.()>()", - "l0. = l1", - "l0. = 0", + "$l0 := @this: DeclareConstructor", + "$l1 := @parameter0: int", + "specialinvoke $l0.()>()", + "$l0. = $l1", + "$l0. = 0", "return") .collect(Collectors.toCollection(ArrayList::new)); } @@ -69,12 +69,12 @@ public List expectedBodyStmts() { */ public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: DeclareConstructor", - "l1 := @parameter0: int", - "l2 := @parameter1: int", - "specialinvoke l0.()>()", - "l0. = l1", - "l0. = l2", + "$l0 := @this: DeclareConstructor", + "$l1 := @parameter0: int", + "$l2 := @parameter1: int", + "specialinvoke $l0.()>()", + "$l0. = $l1", + "$l0. = $l2", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumTest.java index 9c544299605..3808d68147c 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumTest.java @@ -45,18 +45,18 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return super.expectedBodyStmts( - "l0 := @this: DeclareEnum", - "l1 = staticinvoke ()", - "l2 = lengthof l1", - "l3 = 0", + "$l0 := @this: DeclareEnum", + "$l1 = staticinvoke ()", + "$l2 = lengthof $l1", + "$l3 = 0", "label1:", - "$stack7 = l3", - "$stack6 = l2", + "$stack7 = $l3", + "$stack6 = $l2", "if $stack7 >= $stack6 goto label2", - "l4 = l1[l3]", + "$l4 = $l1[$l3]", "$stack5 = ", - "virtualinvoke $stack5.(l4)", - "l3 = l3 + 1", + "virtualinvoke $stack5.($l4)", + "$l3 = $l3 + 1", "goto label1", "label2:", "return"); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumWithConstructorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumWithConstructorTest.java index dbe14438e0e..ac43bb021b2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumWithConstructorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareEnumWithConstructorTest.java @@ -114,8 +114,8 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareEnumWithConstructor", - "specialinvoke l0.()>()", + "$l0 := @this: DeclareEnumWithConstructor", + "specialinvoke $l0.()>()", "return") .collect(Collectors.toList()); } @@ -132,10 +132,10 @@ public List expectedBodyStmts() { */ public List expectedMainBodyStmts() { return Stream.of( - "l0 := @parameter0: java.lang.String[]", - "l1 = ", + "$l0 := @parameter0: java.lang.String[]", + "$l1 = ", "$stack2 = ", - "$stack3 = staticinvoke (l1)", + "$stack3 = staticinvoke ($l1)", "virtualinvoke $stack2.($stack3)", "return") .collect(Collectors.toList()); @@ -203,8 +203,8 @@ public List expectedEnumConstructorStmts() { */ public List expectedGetValueStmts() { return Stream.of( - "l0 := @this: DeclareEnumWithConstructor$Number", - "$stack1 = l0.", + "$l0 := @this: DeclareEnumWithConstructor$Number", + "$stack1 = $l0.", "return $stack1") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFieldTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFieldTest.java index c73f0abd160..263bf2034a7 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFieldTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFieldTest.java @@ -67,7 +67,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareField", + "$l0 := @this: DeclareField", "$stack1 = ", "virtualinvoke $stack1.(\"Java\")", "return") @@ -85,7 +85,7 @@ public List expectedBodyStmts() { */ public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: DeclareField", + "$l0 := @this: DeclareField", "$stack2 = ", "$stack1 = ", "virtualinvoke $stack2.($stack1)", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFloatTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFloatTest.java index e63a7dde5e6..0991df4873b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFloatTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareFloatTest.java @@ -35,12 +35,12 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareFloat", + "$l0 := @this: DeclareFloat", "$stack2 = ", - "$stack1 = l0.", + "$stack1 = $l0.", "virtualinvoke $stack2.($stack1)", "$stack4 = ", - "$stack3 = l0.", + "$stack3 = $l0.", "virtualinvoke $stack4.($stack3)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareInnerClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareInnerClassTest.java index 290372f507f..caea0c3e09f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareInnerClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareInnerClassTest.java @@ -53,7 +53,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareInnerClass", + "$l0 := @this: DeclareInnerClass", "$stack1 = ", "virtualinvoke $stack1.(\"methodDisplayOuter\")", "return") @@ -71,7 +71,7 @@ public List expectedBodyStmts() { */ public List expectedInnerClassBodyStmts() { return Stream.of( - "l0 := @this: DeclareInnerClass$InnerClass", + "$l0 := @this: DeclareInnerClass$InnerClass", "$stack1 = ", "virtualinvoke $stack1.(\"methodDisplayInner\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareIntTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareIntTest.java index 7347b299c8e..0b57095cdfd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareIntTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareIntTest.java @@ -35,15 +35,15 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareInt", + "$l0 := @this: DeclareInt", "$stack2 = ", - "$stack1 = l0.", + "$stack1 = $l0.", "virtualinvoke $stack2.($stack1)", "$stack4 = ", - "$stack3 = l0.", + "$stack3 = $l0.", "virtualinvoke $stack4.($stack3)", "$stack6 = ", - "$stack5 = l0.", + "$stack5 = $l0.", "virtualinvoke $stack6.($stack5)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareLongTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareLongTest.java index 0348925d6a8..d9a97917401 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareLongTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DeclareLongTest.java @@ -35,12 +35,12 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DeclareLong", + "$l0 := @this: DeclareLong", "$stack2 = ", - "$stack1 = l0.", + "$stack1 = $l0.", "virtualinvoke $stack2.($stack1)", "$stack4 = ", - "$stack3 = l0.", + "$stack3 = $l0.", "virtualinvoke $stack4.($stack3)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DoWhileLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DoWhileLoopTest.java index 334da8fb824..1017b47f52f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DoWhileLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/DoWhileLoopTest.java @@ -33,13 +33,13 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DoWhileLoop", - "l1 = 10", - "l2 = 0", + "$l0 := @this: DoWhileLoop", + "$l1 = 10", + "$l2 = 0", "label1:", - "l2 = l2 + 1", - "$stack4 = l1", - "$stack3 = l2", + "$l2 = $l2 + 1", + "$stack4 = $l1", + "$stack3 = $l2", "if $stack4 > $stack3 goto label1", "return") .collect(Collectors.toList()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EmptyStatementTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EmptyStatementTest.java index 2b8bbbc5b27..d54dceb6c43 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EmptyStatementTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EmptyStatementTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: EmptyStatement", "l1 = 5", "return") + return Stream.of("$l0 := @this: EmptyStatement", "$l1 = 5", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EscapeSequencesInStringTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EscapeSequencesInStringTest.java index 2c22394bea7..01c9b5ca636 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EscapeSequencesInStringTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EscapeSequencesInStringTest.java @@ -58,8 +58,8 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsEscapeBackslashB() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash b \\u0008\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash b \\u0008\"", "return") .collect(Collectors.toList()); } @@ -74,8 +74,8 @@ public List expectedBodyStmtsEscapeBackslashB() { */ public List expectedBodyStmtsEscapeBackslashT() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash t \\t\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash t \\t\"", "return") .collect(Collectors.toList()); } @@ -90,8 +90,8 @@ public List expectedBodyStmtsEscapeBackslashT() { */ public List expectedBodyStmtsEscapeBackslashN() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash n \\n\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash n \\n\"", "return") .collect(Collectors.toList()); } @@ -107,8 +107,8 @@ public List expectedBodyStmtsEscapeBackslashN() { */ public List expectedBodyStmtsEscapeBackslashF() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash f \\f\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash f \\f\"", "return") .collect(Collectors.toList()); } @@ -124,8 +124,8 @@ public List expectedBodyStmtsEscapeBackslashF() { */ public List expectedBodyStmtsEscapeBackslashR() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash r \\r\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash r \\r\"", "return") .collect(Collectors.toList()); } @@ -141,8 +141,8 @@ public List expectedBodyStmtsEscapeBackslashR() { */ public List expectedBodyStmtsEscapeDoubleQuotes() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes double quotes \\\"\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes double quotes \\\"\"", "return") .collect(Collectors.toList()); } @@ -158,8 +158,8 @@ public List expectedBodyStmtsEscapeDoubleQuotes() { */ public List expectedBodyStmtsEscapeSingleQuote() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes single quote \\'\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes single quote \\'\"", "return") .collect(Collectors.toList()); } @@ -175,8 +175,8 @@ public List expectedBodyStmtsEscapeSingleQuote() { */ public List expectedBodyStmtsEscapeBackslash() { return Stream.of( - "l0 := @this: EscapeSequencesInString", - "l1 = \"This escapes backslash \\\\\"", + "$l0 := @this: EscapeSequencesInString", + "$l1 = \"This escapes backslash \\\\\"", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EvaluationOrderWithParenthesesTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EvaluationOrderWithParenthesesTest.java index 0c15f7a3d51..f991c5c0460 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EvaluationOrderWithParenthesesTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/EvaluationOrderWithParenthesesTest.java @@ -36,7 +36,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: EvaluationOrderWithParentheses", "l1 = 9", "return") + return Stream.of("$l0 := @this: EvaluationOrderWithParentheses", "$l1 = 9", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalMethodTest.java index ea271dd1770..07417f56c52 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalMethodTest.java @@ -43,7 +43,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: FinalMethod", + "$l0 := @this: FinalMethod", "$stack1 = ", "virtualinvoke $stack1.(\"final method\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalVariableTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalVariableTest.java index c1e5a7b2142..e446b7d6626 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalVariableTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/FinalVariableTest.java @@ -31,9 +31,11 @@ public MethodSignature getMethodSignature() { * * */ + // FIXME: test does not test what is should as the class does not contain a variable at all.. it + // needs a use like in sout @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: FinalVariable", "return").collect(Collectors.toList()); + return Stream.of("$l0 := @this: FinalVariable", "return").collect(Collectors.toList()); } @Test diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForEachLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForEachLoopTest.java index 442d53da1c1..e8ebbfcbc48 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForEachLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForEachLoopTest.java @@ -34,7 +34,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: ForEachLoop", + "$l0 := @this: ForEachLoop", "$stack7 = newarray (int)[9]", "$stack7[0] = 10", "$stack7[1] = 20", @@ -45,18 +45,18 @@ public List expectedBodyStmts() { "$stack7[6] = 71", "$stack7[7] = 80", "$stack7[8] = 90", - "l1 = $stack7", - "l2 = 0", - "l3 = l1", - "l4 = lengthof l3", - "l5 = 0", + "$l1 = $stack7", + "$l2 = 0", + "$l3 = $l1", + "$l4 = lengthof $l3", + "$l5 = 0", "label1:", - "$stack9 = l5", - "$stack8 = l4", + "$stack9 = $l5", + "$stack8 = $l4", "if $stack9 >= $stack8 goto label2", - "l6 = l3[l5]", - "l2 = l2 + 1", - "l5 = l5 + 1", + "$l6 = $l3[$l5]", + "$l2 = $l2 + 1", + "$l5 = $l5 + 1", "goto label1", "label2:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForLoopTest.java index 7d70eb24248..ea8ceca6ccb 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ForLoopTest.java @@ -34,16 +34,16 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: ForLoop", - "l1 = 10", - "l2 = 0", - "l3 = 0", + "$l0 := @this: ForLoop", + "$l1 = 10", + "$l2 = 0", + "$l3 = 0", "label1:", - "$stack5 = l3", - "$stack4 = l1", + "$stack5 = $l3", + "$stack4 = $l1", "if $stack5 >= $stack4 goto label2", - "l2 = l2 + 1", - "l3 = l3 + 1", + "$l2 = $l2 + 1", + "$l3 = $l3 + 1", "goto label1", "label2:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenTypeParamTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenTypeParamTest.java index ac80cc6cf35..77ca63a7185 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenTypeParamTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenTypeParamTest.java @@ -38,10 +38,10 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: GenTypeParam", + "$l0 := @this: GenTypeParam", "$stack4 = new java.util.ArrayList", "specialinvoke $stack4.(int)>(3)", - "l1 = $stack4", + "$l1 = $stack4", "$stack5 = newarray (java.lang.Integer)[3]", "$stack6 = staticinvoke (1)", "$stack5[0] = $stack6", @@ -49,16 +49,16 @@ public List expectedBodyStmts() { "$stack5[1] = $stack7", "$stack8 = staticinvoke (3)", "$stack5[2] = $stack8", - "l2 = staticinvoke ($stack5)", + "$l2 = staticinvoke ($stack5)", "$stack9 = new GenTypeParam", "specialinvoke $stack9.()>()", - "l3 = $stack9", - "virtualinvoke l3.(l1, l2)", + "$l3 = $stack9", + "virtualinvoke $l3.($l1, $l2)", "$stack10 = ", "$stack13 = staticinvoke (2)", "$stack12 = staticinvoke (8)", "$stack11 = staticinvoke (3)", - "$stack14 = virtualinvoke l3.($stack13, $stack12, $stack11)", + "$stack14 = virtualinvoke $l3.($stack13, $stack12, $stack11)", "virtualinvoke $stack10.($stack14)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnClassTest.java index 537776adf4e..b43e4738a0b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnClassTest.java @@ -35,15 +35,15 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: GenericTypeParamOnClass", + "$l0 := @this: GenericTypeParamOnClass", "$stack3 = new GenericTypeParamOnClass$A", - "specialinvoke $stack3.(GenericTypeParamOnClass)>(l0)", - "l1 = $stack3", + "specialinvoke $stack3.(GenericTypeParamOnClass)>($l0)", + "$l1 = $stack3", "$stack4 = staticinvoke (5)", - "staticinvoke (l1, $stack4)", - "$stack5 = virtualinvoke l1.()", + "staticinvoke ($l1, $stack4)", + "$stack5 = virtualinvoke $l1.()", "$stack6 = (java.lang.Integer) $stack5", - "l2 = virtualinvoke $stack6.()", + "$l2 = virtualinvoke $stack6.()", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnMethodTest.java index a91b23a261d..e6b55439906 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/GenericTypeParamOnMethodTest.java @@ -33,8 +33,8 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: GenericTypeParamOnMethod", - "virtualinvoke l0.(\"Hello World\")", + "$l0 := @this: GenericTypeParamOnMethod", + "virtualinvoke $l0.(\"Hello World\")", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/IfElseStatementTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/IfElseStatementTest.java index c0e56ddf8e9..96bcd00bf53 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/IfElseStatementTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/IfElseStatementTest.java @@ -77,13 +77,13 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIfStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label1", - "l2 = 1", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label1", + "$l2 = 1", "label1:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -105,16 +105,16 @@ public List expectedBodyStmtsIfStatement() { */ public List expectedBodyStmtsIfElseStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label1", - "l2 = 1", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label1", + "$l2 = 1", "goto label2", "label1:", - "l2 = 2", + "$l2 = 2", "label2:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -138,20 +138,20 @@ public List expectedBodyStmtsIfElseStatement() { */ public List expectedBodyStmtsIfElseIfStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label1", - "l2 = 1", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label1", + "$l2 = 1", "goto label3", "label1:", - "if l1 <= 123 goto label2", - "l2 = 2", + "if $l1 <= 123 goto label2", + "$l2 = 2", "goto label3", "label2:", - "l2 = 3", + "$l2 = 3", "label3:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -177,20 +177,20 @@ public List expectedBodyStmtsIfElseIfStatement() { */ public List expectedBodyStmtsIfElseCascadingStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label2", - "if l1 >= 42 goto label1", - "l2 = 11", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label2", + "if $l1 >= 42 goto label1", + "$l2 = 11", "goto label3", "label1:", - "l2 = 12", + "$l2 = 12", "goto label3", "label2:", - "l2 = 3", + "$l2 = 3", "label3:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -216,20 +216,20 @@ public List expectedBodyStmtsIfElseCascadingStatement() { */ public List expectedBodyStmtsIfElseCascadingInElseStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label1", - "l2 = 1", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label1", + "$l2 = 1", "goto label3", "label1:", - "if l1 >= 42 goto label2", - "l2 = 21", + "if $l1 >= 42 goto label2", + "$l2 = 21", "goto label3", "label2:", - "l2 = 22", + "$l2 = 22", "label3:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -257,24 +257,24 @@ public List expectedBodyStmtsIfElseCascadingInElseStatement() { */ public List expectedBodyStmtsIfElseCascadingElseIfStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label3", - "if l1 >= 42 goto label1", - "l2 = 11", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label3", + "if $l1 >= 42 goto label1", + "$l2 = 11", "goto label4", "label1:", - "if l1 <= 123 goto label2", - "l2 = 12", + "if $l1 <= 123 goto label2", + "$l2 = 12", "goto label4", "label2:", - "l2 = 13", + "$l2 = 13", "goto label4", "label3:", - "l2 = 2", + "$l2 = 2", "label4:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } @@ -302,24 +302,24 @@ public List expectedBodyStmtsIfElseCascadingElseIfStatement() { */ public List expectedBodyStmtsIfElseCascadingElseIfInElseStatement() { return Stream.of( - "l0 := @this: IfElseStatement", - "l1 := @parameter0: int", - "l2 = 0", - "if l1 >= 42 goto label1", - "l2 = 1", + "$l0 := @this: IfElseStatement", + "$l1 := @parameter0: int", + "$l2 = 0", + "if $l1 >= 42 goto label1", + "$l2 = 1", "goto label4", "label1:", - "if l1 >= 42 goto label2", - "l2 = 21", + "if $l1 >= 42 goto label2", + "$l2 = 21", "goto label4", "label2:", - "if l1 <= 123 goto label3", - "l2 = 22", + "if $l1 <= 123 goto label3", + "$l2 = 22", "goto label4", "label3:", - "l2 = 23", + "$l2 = 23", "label4:", - "$stack3 = l2", + "$stack3 = $l2", "return $stack3") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InfiniteLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InfiniteLoopTest.java index 8d083de1640..e3203917518 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InfiniteLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InfiniteLoopTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: InfiniteLoop", "return").collect(Collectors.toList()); + return Stream.of("$l0 := @this: InfiniteLoop", "return").collect(Collectors.toList()); } @Test diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/Initialize3DimensionalArraysTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/Initialize3DimensionalArraysTest.java index 8a1c1ed4bbe..7d25ab2b337 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/Initialize3DimensionalArraysTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/Initialize3DimensionalArraysTest.java @@ -62,7 +62,7 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIntArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (int[][])[2]", "$stack3 = newarray (int[])[2]", "$stack4 = newarray (int)[3]", @@ -86,7 +86,7 @@ public List expectedBodyStmtsIntArrays() { "$stack8[1] = 11", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -103,7 +103,7 @@ public List expectedBodyStmtsIntArrays() { */ public List expectedBodyStmtsByteArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (byte[][])[2]", "$stack3 = newarray (byte[])[2]", "$stack4 = newarray (byte)[3]", @@ -127,7 +127,7 @@ public List expectedBodyStmtsByteArrays() { "$stack8[1] = 6", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -144,7 +144,7 @@ public List expectedBodyStmtsByteArrays() { */ public List expectedBodyStmtsShortArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (short[][])[2]", "$stack3 = newarray (short[])[2]", "$stack4 = newarray (short)[2]", @@ -166,7 +166,7 @@ public List expectedBodyStmtsShortArrays() { "$stack8[1] = 35", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -183,7 +183,7 @@ public List expectedBodyStmtsShortArrays() { */ public List expectedBodyStmtsLongArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (long[][])[2]", "$stack3 = newarray (long[])[2]", "$stack4 = newarray (long)[2]", @@ -206,7 +206,7 @@ public List expectedBodyStmtsLongArrays() { "$stack8[1] = 63543L", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -223,7 +223,7 @@ public List expectedBodyStmtsLongArrays() { */ public List expectedBodyStmtsFloatArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (float[][])[2]", "$stack3 = newarray (float[])[2]", "$stack4 = newarray (float)[2]", @@ -245,7 +245,7 @@ public List expectedBodyStmtsFloatArrays() { "$stack8[1] = 58.14F", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -262,7 +262,7 @@ public List expectedBodyStmtsFloatArrays() { */ public List expectedBodyStmtsDoubleArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (double[][])[2]", "$stack3 = newarray (double[])[2]", "$stack4 = newarray (double)[2]", @@ -283,7 +283,7 @@ public List expectedBodyStmtsDoubleArrays() { "$stack8[1] = 65416.5", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -300,7 +300,7 @@ public List expectedBodyStmtsDoubleArrays() { */ public List expectedBodyStmtsBooleanArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (boolean[][])[2]", "$stack3 = newarray (boolean[])[2]", "$stack4 = newarray (boolean)[2]", @@ -320,7 +320,7 @@ public List expectedBodyStmtsBooleanArrays() { "$stack8[0] = 1", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -337,7 +337,7 @@ public List expectedBodyStmtsBooleanArrays() { */ public List expectedBodyStmtsCharArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (char[][])[2]", "$stack3 = newarray (char[])[2]", "$stack4 = newarray (char)[3]", @@ -360,7 +360,7 @@ public List expectedBodyStmtsCharArrays() { "$stack8[1] = 37", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -376,7 +376,7 @@ public List expectedBodyStmtsCharArrays() { */ public List expectedBodyStmtsStringArrays() { return Stream.of( - "l0 := @this: Initialize3DimensionalArrays", + "$l0 := @this: Initialize3DimensionalArrays", "$stack2 = newarray (java.lang.String[][])[2]", "$stack3 = newarray (java.lang.String[])[2]", "$stack4 = newarray (java.lang.String)[1]", @@ -397,7 +397,7 @@ public List expectedBodyStmtsStringArrays() { "$stack8[1] = \"HNI\"", "$stack6[1] = $stack8", "$stack2[1] = $stack6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWhileDeclarationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWhileDeclarationTest.java index 77662733d59..7abec3bb89f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWhileDeclarationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWhileDeclarationTest.java @@ -61,12 +61,12 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIntArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (int)[3]", "$stack2[0] = 1", "$stack2[1] = 2", "$stack2[2] = 3", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -83,12 +83,12 @@ public List expectedBodyStmtsIntArrays() { */ public List expectedBodyStmtsByteArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (byte)[3]", "$stack2[0] = 4", "$stack2[1] = 5", "$stack2[2] = 6", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -105,12 +105,12 @@ public List expectedBodyStmtsByteArrays() { */ public List expectedBodyStmtsShortArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (short)[3]", "$stack2[0] = 10", "$stack2[1] = 20", "$stack2[2] = 30", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -127,12 +127,12 @@ public List expectedBodyStmtsShortArrays() { */ public List expectedBodyStmtsLongArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (long)[3]", "$stack2[0] = 547087L", "$stack2[1] = 564645L", "$stack2[2] = 654786L", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -149,13 +149,13 @@ public List expectedBodyStmtsLongArrays() { */ public List expectedBodyStmtsFloatArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (float)[4]", "$stack2[0] = 3.14F", "$stack2[1] = 5.46F", "$stack2[2] = 2.987F", "$stack2[3] = 4.87F", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -173,11 +173,11 @@ public List expectedBodyStmtsFloatArrays() { */ public List expectedBodyStmtsDoubleArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (double)[2]", "$stack2[0] = 6.765414", "$stack2[1] = 9.676565646", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -194,11 +194,11 @@ public List expectedBodyStmtsDoubleArrays() { */ public List expectedBodyStmtsBooleanArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (boolean)[2]", "$stack2[0] = 1", "$stack2[1] = 0", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -210,12 +210,12 @@ public List expectedBodyStmtsBooleanArrays() { */ public List expectedBodyStmtsCharArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (char)[3]", "$stack2[0] = 65", "$stack2[1] = 98", "$stack2[2] = 38", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -231,11 +231,11 @@ public List expectedBodyStmtsCharArrays() { */ public List expectedBodyStmtsStringArrays() { return Stream.of( - "l0 := @this: InitializeArraysWhileDeclaration", + "$l0 := @this: InitializeArraysWhileDeclaration", "$stack2 = newarray (java.lang.String)[2]", "$stack2[0] = \"Hello World\"", "$stack2[1] = \"Greetings\"", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWithIndexTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWithIndexTest.java index bdeda0c343f..aff2641c841 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWithIndexTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeArraysWithIndexTest.java @@ -66,11 +66,11 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIntArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (int)[3]", - "l1[0] = 1", - "l1[1] = 2", - "l1[2] = 3", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (int)[3]", + "$l1[0] = 1", + "$l1[1] = 2", + "$l1[2] = 3", "return") .collect(Collectors.toList()); } @@ -90,11 +90,11 @@ public List expectedBodyStmtsIntArrays() { */ public List expectedBodyStmtsByteArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (byte)[3]", - "l1[0] = 4", - "l1[1] = 5", - "l1[2] = 6", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (byte)[3]", + "$l1[0] = 4", + "$l1[1] = 5", + "$l1[2] = 6", "return") .collect(Collectors.toList()); } @@ -113,11 +113,11 @@ public List expectedBodyStmtsByteArrays() { */ public List expectedBodyStmtsShortArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (short)[3]", - "l1[0] = 10", - "l1[1] = 20", - "l1[2] = 30", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (short)[3]", + "$l1[0] = 10", + "$l1[1] = 20", + "$l1[2] = 30", "return") .collect(Collectors.toList()); } @@ -137,11 +137,11 @@ public List expectedBodyStmtsShortArrays() { */ public List expectedBodyStmtsLongArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (long)[3]", - "l1[0] = 547087L", - "l1[1] = 564645L", - "l1[2] = 654786L", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (long)[3]", + "$l1[0] = 547087L", + "$l1[1] = 564645L", + "$l1[2] = 654786L", "return") .collect(Collectors.toList()); } @@ -162,12 +162,12 @@ public List expectedBodyStmtsLongArrays() { */ public List expectedBodyStmtsFloatArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (float)[4]", - "l1[0] = 3.14F", - "l1[1] = 5.46F", - "l1[2] = 2.987F", - "l1[3] = 4.87F", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (float)[4]", + "$l1[0] = 3.14F", + "$l1[1] = 5.46F", + "$l1[2] = 2.987F", + "$l1[3] = 4.87F", "return") .collect(Collectors.toList()); } @@ -187,10 +187,10 @@ public List expectedBodyStmtsFloatArrays() { */ public List expectedBodyStmtsDoubleArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (double)[2]", - "l1[0] = 6.765414", - "l1[1] = 9.676565646", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (double)[2]", + "$l1[0] = 6.765414", + "$l1[1] = 9.676565646", "return") .collect(Collectors.toList()); } @@ -209,10 +209,10 @@ public List expectedBodyStmtsDoubleArrays() { */ public List expectedBodyStmtsBooleanArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (boolean)[2]", - "l1[0] = 1", - "l1[1] = 0", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (boolean)[2]", + "$l1[0] = 1", + "$l1[1] = 0", "return") .collect(Collectors.toList()); } @@ -232,11 +232,11 @@ public List expectedBodyStmtsBooleanArrays() { */ public List expectedBodyStmtsCharArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (char)[3]", - "l1[0] = 65", - "l1[1] = 98", - "l1[2] = 38", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (char)[3]", + "$l1[0] = 65", + "$l1[1] = 98", + "$l1[2] = 38", "return") .collect(Collectors.toList()); } @@ -254,10 +254,10 @@ public List expectedBodyStmtsCharArrays() { */ public List expectedBodyStmtsStringArrays() { return Stream.of( - "l0 := @this: InitializeArraysWithIndex", - "l1 = newarray (java.lang.String)[2]", - "l1[0] = \"Hello World\"", - "l1[1] = \"Greetings\"", + "$l0 := @this: InitializeArraysWithIndex", + "$l1 = newarray (java.lang.String)[2]", + "$l1[0] = \"Hello World\"", + "$l1[1] = \"Greetings\"", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeMultidimensionalArraysTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeMultidimensionalArraysTest.java index b7fb44f92db..b194ca63c78 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeMultidimensionalArraysTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InitializeMultidimensionalArraysTest.java @@ -64,7 +64,7 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsIntArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (int[])[3]", "$stack3 = newarray (int)[3]", "$stack3[0] = 1", @@ -80,7 +80,7 @@ public List expectedBodyStmtsIntArrays() { "$stack5[1] = 8", "$stack5[2] = 9", "$stack2[2] = $stack5", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -96,7 +96,7 @@ public List expectedBodyStmtsIntArrays() { */ public List expectedBodyStmtsByteArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (byte[])[2]", "$stack3 = newarray (byte)[2]", "$stack3[0] = 4", @@ -105,7 +105,7 @@ public List expectedBodyStmtsByteArrays() { "$stack4 = newarray (byte)[1]", "$stack4[0] = 2", "$stack2[1] = $stack4", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -120,7 +120,7 @@ public List expectedBodyStmtsByteArrays() { */ public List expectedBodyStmtsShortArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (short[])[2]", "$stack3 = newarray (short)[3]", "$stack3[0] = 10", @@ -130,7 +130,7 @@ public List expectedBodyStmtsShortArrays() { "$stack4 = newarray (short)[1]", "$stack4[0] = 40", "$stack2[1] = $stack4", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -145,7 +145,7 @@ public List expectedBodyStmtsShortArrays() { */ public List expectedBodyStmtsLongArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (long[])[3]", "$stack3 = newarray (long)[2]", "$stack3[0] = 547087L", @@ -160,7 +160,7 @@ public List expectedBodyStmtsLongArrays() { "$stack5[0] = 34565L", "$stack5[1] = 234L", "$stack2[2] = $stack5", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -177,7 +177,7 @@ public List expectedBodyStmtsLongArrays() { */ public List expectedBodyStmtsFloatArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (float[])[2]", "$stack3 = newarray (float)[2]", "$stack3[0] = 3.14F", @@ -187,7 +187,7 @@ public List expectedBodyStmtsFloatArrays() { "$stack4[0] = 2.987F", "$stack4[1] = 4.87F", "$stack2[1] = $stack4", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -205,7 +205,7 @@ public List expectedBodyStmtsFloatArrays() { */ public List expectedBodyStmtsDoubleArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (double[])[3]", "$stack3 = newarray (double)[2]", "$stack3[0] = 6.765414", @@ -218,7 +218,7 @@ public List expectedBodyStmtsDoubleArrays() { "$stack5[0] = 3.5656", "$stack5[1] = 68.234234", "$stack2[2] = $stack5", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -234,7 +234,7 @@ public List expectedBodyStmtsDoubleArrays() { */ public List expectedBodyStmtsBooleanArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (boolean[])[2]", "$stack3 = newarray (boolean)[2]", "$stack3[0] = 1", @@ -243,7 +243,7 @@ public List expectedBodyStmtsBooleanArrays() { "$stack4 = newarray (boolean)[1]", "$stack4[0] = 1", "$stack2[1] = $stack4", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -260,7 +260,7 @@ public List expectedBodyStmtsBooleanArrays() { */ public List expectedBodyStmtsCharArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (char[])[3]", "$stack3 = newarray (char)[3]", "$stack3[0] = 65", @@ -275,7 +275,7 @@ public List expectedBodyStmtsCharArrays() { "$stack5[0] = 50", "$stack5[1] = 71", "$stack2[2] = $stack5", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } @@ -291,7 +291,7 @@ public List expectedBodyStmtsCharArrays() { */ public List expectedBodyStmtsStringArrays() { return Stream.of( - "l0 := @this: InitializeMultidimensionalArrays", + "$l0 := @this: InitializeMultidimensionalArrays", "$stack2 = newarray (java.lang.String[])[2]", "$stack3 = newarray (java.lang.String)[1]", "$stack3[0] = \"Hello World\"", @@ -300,7 +300,7 @@ public List expectedBodyStmtsStringArrays() { "$stack4[0] = \"Greetings\"", "$stack4[1] = \"Welcome\"", "$stack2[1] = $stack4", - "l1 = $stack2", + "$l1 = $stack2", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InstanceOfCheckTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InstanceOfCheckTest.java index 21fe5fb085b..aeda781ba4e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InstanceOfCheckTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InstanceOfCheckTest.java @@ -47,12 +47,12 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: InstanceOfCheck", + "$l0 := @this: InstanceOfCheck", "$stack2 = new InstanceOfCheck", "specialinvoke $stack2.()>()", - "l1 = $stack2", + "$l1 = $stack2", "$stack4 = ", - "$stack3 = l1 instanceof InstanceOfCheckSuper", + "$stack3 = $l1 instanceof InstanceOfCheckSuper", "virtualinvoke $stack4.($stack3)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InterfaceImplClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InterfaceImplClassTest.java index 2068d7673cd..c6fccb54cbd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InterfaceImplClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/InterfaceImplClassTest.java @@ -57,7 +57,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: InterfaceImplClass", + "$l0 := @this: InterfaceImplClass", "$stack1 = ", "virtualinvoke $stack1.(\"Method from InterfaceImpl is implemented\")", "$stack2 = ", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelStatementTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelStatementTest.java index 11074df2459..6743f3fa4ff 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelStatementTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelStatementTest.java @@ -38,18 +38,18 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: LabelStatement", - "l1 = 20", - "l2 = 1", + "$l0 := @this: LabelStatement", + "$l1 = 20", + "$l2 = 1", "label1:", - "$stack5 = l2", - "$stack4 = l1", + "$stack5 = $l2", + "$stack4 = $l1", "if $stack5 >= $stack4 goto label3", - "$stack3 = l2 % 10", + "$stack3 = $l2 % 10", "if $stack3 != 0 goto label2", "goto label3", "label2:", - "l2 = l2 + 1", + "$l2 = $l2 + 1", "goto label1", "label3:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelledLoopBreakTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelledLoopBreakTest.java index 0d7df0f84ba..c121ab06dbb 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelledLoopBreakTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/LabelledLoopBreakTest.java @@ -37,24 +37,24 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: LabelledLoopBreak", - "l1 = 0", + "$l0 := @this: LabelledLoopBreak", + "$l1 = 0", "label1:", - "$stack4 = l1", + "$stack4 = $l1", "$stack3 = 5", "if $stack4 >= $stack3 goto label5", - "l2 = 0", + "$l2 = 0", "label2:", - "$stack6 = l2", + "$stack6 = $l2", "$stack5 = 5", "if $stack6 >= $stack5 goto label4", - "if l1 != 1 goto label3", + "if $l1 != 1 goto label3", "goto label5", "label3:", - "l2 = l2 + 1", + "$l2 = $l2 + 1", "goto label2", "label4:", - "l1 = l1 + 1", + "$l1 = $l1 + 1", "goto label1", "label5:", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodAcceptingVarTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodAcceptingVarTest.java index 150caff36d2..ea221e6a04d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodAcceptingVarTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodAcceptingVarTest.java @@ -58,10 +58,10 @@ public MethodSignature getMethodSignature(String datatype) { */ public List expectedBodyStmtsShort() { return Stream.of( - "l0 := @this: MethodAcceptingVar", - "l1 := @parameter0: short", - "$stack2 = l1 + 1", - "l1 = (short) $stack2", + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: short", + "$stack2 = $l1 + 1", + "$l1 = (short) $stack2", "return") .collect(Collectors.toList()); } @@ -77,10 +77,10 @@ public List expectedBodyStmtsShort() { */ public List expectedBodyStmtsByte() { return Stream.of( - "l0 := @this: MethodAcceptingVar", - "l1 := @parameter0: byte", - "$stack2 = l1 + 1", - "l1 = (byte) $stack2", + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: byte", + "$stack2 = $l1 + 1", + "$l1 = (byte) $stack2", "return") .collect(Collectors.toList()); } @@ -96,7 +96,7 @@ public List expectedBodyStmtsByte() { */ public List expectedBodyStmtsChar() { return Stream.of( - "l0 := @this: MethodAcceptingVar", "l1 := @parameter0: char", "l1 = 97", "return") + "$l0 := @this: MethodAcceptingVar", "$l1 := @parameter0: char", "$l1 = 97", "return") .collect(Collectors.toList()); } @@ -111,7 +111,10 @@ public List expectedBodyStmtsChar() { */ public List expectedBodyStmtsInt() { return Stream.of( - "l0 := @this: MethodAcceptingVar", "l1 := @parameter0: int", "l1 = l1 + 1", "return") + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: int", + "$l1 = $l1 + 1", + "return") .collect(Collectors.toList()); } @@ -126,9 +129,9 @@ public List expectedBodyStmtsInt() { */ public List expectedBodyStmtsLong() { return Stream.of( - "l0 := @this: MethodAcceptingVar", - "l1 := @parameter0: long", - "l1 = 123456777L", + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: long", + "$l1 = 123456777L", "return") .collect(Collectors.toList()); } @@ -144,7 +147,10 @@ public List expectedBodyStmtsLong() { */ public List expectedBodyStmtsFloat() { return Stream.of( - "l0 := @this: MethodAcceptingVar", "l1 := @parameter0: float", "l1 = 7.77F", "return") + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: float", + "$l1 = 7.77F", + "return") .collect(Collectors.toList()); } @@ -159,9 +165,9 @@ public List expectedBodyStmtsFloat() { */ public List expectedBodyStmtsDouble() { return Stream.of( - "l0 := @this: MethodAcceptingVar", - "l1 := @parameter0: double", - "l1 = 1.787777777", + "$l0 := @this: MethodAcceptingVar", + "$l1 := @parameter0: double", + "$l1 = 1.787777777", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverloadingTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverloadingTest.java index ff73012e8be..8939ef21974 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverloadingTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverloadingTest.java @@ -63,10 +63,10 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MethodOverloading", - "l1 := @parameter0: int", - "l2 := @parameter1: int", - "$stack3 = l1 + l2", + "$l0 := @this: MethodOverloading", + "$l1 := @parameter0: int", + "$l2 := @parameter1: int", + "$stack3 = $l1 + $l2", "return $stack3") .collect(Collectors.toCollection(ArrayList::new)); } @@ -83,9 +83,9 @@ public List expectedBodyStmts() { */ public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: MethodOverloading", - "l1 := @parameter0: int", - "$stack2 = l1 + l1", + "$l0 := @this: MethodOverloading", + "$l1 := @parameter0: int", + "$stack2 = $l1 + $l1", "return $stack2") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverridingTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverridingTest.java index c79500a194d..1464c8a318c 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverridingTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodOverridingTest.java @@ -38,7 +38,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MethodOverridingSubclass", + "$l0 := @this: MethodOverridingSubclass", "$stack1 = ", "virtualinvoke $stack1.(\"Inside MethodOverridingSubclass-calculateArea()\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodReturningVarTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodReturningVarTest.java index d8721d411fa..87aa01753f7 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodReturningVarTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MethodReturningVarTest.java @@ -55,7 +55,7 @@ public MethodSignature getMethodSignature(String datatype) { * */ public List expectedBodyStmtsShort() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 10", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 10", "return $l1") .collect(Collectors.toList()); } @@ -70,7 +70,7 @@ public List expectedBodyStmtsShort() { * */ public List expectedBodyStmtsByte() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 0", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 0", "return $l1") .collect(Collectors.toList()); } @@ -85,7 +85,7 @@ public List expectedBodyStmtsByte() { * */ public List expectedBodyStmtsChar() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 97", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 97", "return $l1") .collect(Collectors.toList()); } @@ -100,7 +100,7 @@ public List expectedBodyStmtsChar() { * */ public List expectedBodyStmtsInt() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 512", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 512", "return $l1") .collect(Collectors.toList()); } @@ -116,7 +116,7 @@ public List expectedBodyStmtsInt() { * */ public List expectedBodyStmtsLong() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 123456789L", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 123456789L", "return $l1") .collect(Collectors.toList()); } @@ -131,7 +131,7 @@ public List expectedBodyStmtsLong() { * */ public List expectedBodyStmtsFloat() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 3.14F", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 3.14F", "return $l1") .collect(Collectors.toList()); } @@ -146,7 +146,7 @@ public List expectedBodyStmtsFloat() { * */ public List expectedBodyStmtsDouble() { - return Stream.of("l0 := @this: MethodReturningVar", "l1 = 1.96969654", "return l1") + return Stream.of("$l0 := @this: MethodReturningVar", "$l1 = 1.96969654", "return $l1") .collect(Collectors.toList()); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MultiInterfaceImplClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MultiInterfaceImplClassTest.java index d10b3342279..4bd448f8f03 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MultiInterfaceImplClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/MultiInterfaceImplClassTest.java @@ -50,7 +50,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MultiInterfaceImplClass", + "$l0 := @this: MultiInterfaceImplClass", "$stack1 = ", "virtualinvoke $stack1.(\"Method from InterfaceImpl is implemented\")", "$stack2 = ", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NamedClassInsideMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NamedClassInsideMethodTest.java index ef23e672f89..04a28a44315 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NamedClassInsideMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NamedClassInsideMethodTest.java @@ -39,11 +39,11 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: NamedClassInsideMethod", + "$l0 := @this: NamedClassInsideMethod", "$stack2 = new NamedClassInsideMethod$1MyMathOperation", - "specialinvoke $stack2.(NamedClassInsideMethod)>(l0)", - "l1 = $stack2", - "interfaceinvoke l1.()", + "specialinvoke $stack2.(NamedClassInsideMethod)>($l0)", + "$l1 = $stack2", + "interfaceinvoke $l1.()", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NewCodeBlockInMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NewCodeBlockInMethodTest.java index 66b58ac4d35..bd8ce690990 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NewCodeBlockInMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NewCodeBlockInMethodTest.java @@ -34,7 +34,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: NewCodeBlockInMethod", "l1 = 5", "return") + return Stream.of("$l0 := @this: NewCodeBlockInMethod", "$l1 = 5", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NullVariableTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NullVariableTest.java index 27877de8a3a..65f8ec9ff9b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NullVariableTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/NullVariableTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: NullVariable", "l1 = null", "return") + return Stream.of("$l0 := @this: NullVariable", "$l1 = null", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/PublicClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/PublicClassTest.java index 0d8b45a6061..5c99da415bf 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/PublicClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/PublicClassTest.java @@ -70,6 +70,6 @@ public MethodSignature getMethodSignature(String modifier) { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: PublicClass", "return").collect(Collectors.toList()); + return Stream.of("$l0 := @this: PublicClass", "return").collect(Collectors.toList()); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferenceVarDeclarationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferenceVarDeclarationTest.java index 246098455a0..d9f4adf9232 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferenceVarDeclarationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferenceVarDeclarationTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: ReferenceVarDeclaration", "l1 = \"Hello World\"", "return") + return Stream.of("$l0 := @this: ReferenceVarDeclaration", "$l1 = \"Hello World\"", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferencingThisTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferencingThisTest.java index 7b779f1dd55..cbfa91c7475 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferencingThisTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReferencingThisTest.java @@ -48,15 +48,15 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: ReferencingThis", + "$l0 := @this: ReferencingThis", "$stack2 = ", "virtualinvoke $stack2.(\" this keyword as an argument in the constructor call\")", "$stack3 = new ReferencingThis", - "$stack5 = l0.", - "$stack4 = l0.", + "$stack5 = $l0.", + "$stack4 = $l0.", "specialinvoke $stack3.(int,int)>($stack5, $stack4)", - "l1 = $stack3", - "virtualinvoke l1.()", + "$l1 = $stack3", + "virtualinvoke $l1.()", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReflectionTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReflectionTest.java index fe508ff268d..045699f4f46 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReflectionTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ReflectionTest.java @@ -44,20 +44,20 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: Reflection", + "$l0 := @this: Reflection", "$stack4 = new Reflection", "specialinvoke $stack4.()>()", - "l1 = $stack4", - "l2 = class \"LReflection;\"", + "$l1 = $stack4", + "$l2 = class \"LReflection;\"", "$stack5 = ", - "virtualinvoke $stack5.(l2)", + "virtualinvoke $stack5.($l2)", "$stack6 = newarray (java.lang.Class)[0]", - "l3 = virtualinvoke l2.($stack6)", + "$l3 = virtualinvoke $l2.($stack6)", "$stack7 = ", - "$stack8 = virtualinvoke l3.()", + "$stack8 = virtualinvoke $l3.()", "virtualinvoke $stack7.($stack8)", "$stack9 = ", - "$stack10 = virtualinvoke l2.()", + "$stack10 = virtualinvoke $l2.()", "$stack11 = lengthof $stack10", "virtualinvoke $stack9.($stack11)", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StatementEvalTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StatementEvalTest.java index 1f399c73106..b723d5b1d22 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StatementEvalTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StatementEvalTest.java @@ -34,11 +34,11 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: StatementEval", - "l1 = 1", - "$stack2 = l1", - "l1 = 3", - "l1 = $stack2 + 3", + "$l0 := @this: StatementEval", + "$l1 = 1", + "$stack2 = $l1", + "$l1 = 3", + "$l1 = $stack2 + 3", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticImportTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticImportTest.java index 1a1c59f1be6..809816b672a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticImportTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticImportTest.java @@ -44,7 +44,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: StaticImport", + "$l0 := @this: StaticImport", "$stack1 = ", "$stack2 = staticinvoke (4.0)", "virtualinvoke $stack1.($stack2)", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticMethodInvocationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticMethodInvocationTest.java index 13133e4ea61..31d9fdd0e7a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticMethodInvocationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StaticMethodInvocationTest.java @@ -29,7 +29,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: StaticMethodInvocation", + "$l0 := @this: StaticMethodInvocation", "staticinvoke ()", "return") .collect(Collectors.toList()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringConcatenationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringConcatenationTest.java index 6686da32e32..1d5e3c765b2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringConcatenationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringConcatenationTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: StringConcatenation", "l1 = \"thestring\"", "return") + return Stream.of("$l0 := @this: StringConcatenation", "$l1 = \"thestring\"", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringWithUnicodeCharTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringWithUnicodeCharTest.java index 37cc9bb42d9..d7054a13c68 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringWithUnicodeCharTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/StringWithUnicodeCharTest.java @@ -33,7 +33,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: StringWithUnicodeChar", "l1 = \"$123\"", "return") + return Stream.of("$l0 := @this: StringWithUnicodeChar", "$l1 = \"$123\"", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SubClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SubClassTest.java index e70dc7d73a2..ad1a20bc70e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SubClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SubClassTest.java @@ -59,22 +59,22 @@ public void testSuperClassStmts() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SubClass", - "l0. = 10", - "l0. = 20", - "l0. = 30", - "l0. = 40", + "$l0 := @this: SubClass", + "$l0. = 10", + "$l0. = 20", + "$l0. = 30", + "$l0. = 40", "return") .collect(Collectors.toCollection(ArrayList::new)); } public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: SubClass", - "specialinvoke l0.()", - "l0. = 100", - "l0. = 200", - "l0. = 300", + "$l0 := @this: SubClass", + "specialinvoke $l0.()", + "$l0. = 100", + "$l0. = 200", + "$l0. = 300", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SuperClassTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SuperClassTest.java index 3f4d30d7992..f524e4c597e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SuperClassTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SuperClassTest.java @@ -36,11 +36,11 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SuperClass", - "l0. = 10", - "l0. = 20", - "l0. = 30", - "l0. = 40", + "$l0 := @this: SuperClass", + "$l0. = 10", + "$l0. = 20", + "$l0. = 30", + "$l0. = 40", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SwitchCaseStatementTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SwitchCaseStatementTest.java index c85adfaf2f4..55f1eafb993 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SwitchCaseStatementTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SwitchCaseStatementTest.java @@ -18,24 +18,24 @@ public void switchCaseStatementEnum() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = ", - "l2 = \"\"", + "$l0 := @this: SwitchCaseStatement", + "$l1 = ", + "$l2 = \"\"", "$stack3 = ", - "$stack4 = virtualinvoke l1.()", + "$stack4 = virtualinvoke $l1.()", "$stack5 = $stack3[$stack4]", "switch($stack5)", "case 1: goto label1", "case 2: goto label2", "default: goto label3", "label1:", - "l2 = \"red\"", + "$l2 = \"red\"", "goto label4", "label2:", - "l2 = \"green\"", + "$l2 = \"green\"", "goto label4", "label3:", - "l2 = \"invalid\"", + "$l2 = \"invalid\"", "label4:", "return")); } @@ -46,24 +46,24 @@ public void testSwitchInt() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 5", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 5", + "switch($l1)", "case 1: goto label1", "case 2: goto label2", "case 3: goto label3", "default: goto label4", "label1:", - "l2 = \"one\"", + "$l2 = \"one\"", "goto label5", "label2:", - "l2 = \"two\"", + "$l2 = \"two\"", "goto label5", "label3:", - "l2 = \"three\"", + "$l2 = \"three\"", "goto label5", "label4:", - "l2 = \"invalid\"", + "$l2 = \"invalid\"", "label5:", "return")); } @@ -74,21 +74,21 @@ public void testSwitchCaseWithoutDefault() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 6", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 6", + "switch($l1)", "case 1: goto label1", "case 2: goto label2", "case 3: goto label3", "default: goto label4", "label1:", - "l2 = \"one\"", + "$l2 = \"one\"", "goto label4", "label2:", - "l2 = \"two\"", + "$l2 = \"two\"", "goto label4", "label3:", - "l2 = \"three\"", + "$l2 = \"three\"", "label4:", "return")); } @@ -99,18 +99,18 @@ public void testSwitchCaseGroupedTargets() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 7", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 7", + "switch($l1)", "case 1: goto label1", "case 2: goto label1", "case 3: goto label2", "default: goto label3", "label1:", - "l2 = \"first\"", + "$l2 = \"first\"", "goto label3", "label2:", - "l2 = \"second\"", + "$l2 = \"second\"", "label3:", "return")); } @@ -121,21 +121,21 @@ public void testSwitchCaseGroupedTargetsDefault() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 8", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 8", + "switch($l1)", "case 1: goto label1", "case 2: goto label1", "case 3: goto label2", "default: goto label3", "label1:", - "l2 = \"first\"", + "$l2 = \"first\"", "goto label4", "label2:", - "l2 = \"second\"", + "$l2 = \"second\"", "goto label4", "label3:", - "l2 = \"other\"", + "$l2 = \"other\"", "label4:", "return")); } @@ -146,29 +146,29 @@ public void switchCaseStatementCaseIncludingIf() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 2", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 2", + "switch($l1)", "case 1: goto label1", "case 2: goto label3", "case 3: goto label4", "default: goto label5", "label1:", - "l2 = 1", - "if l1 != 666 goto label2", - "l2 = 11", + "$l2 = 1", + "if $l1 != 666 goto label2", + "$l2 = 11", "goto label6", "label2:", - "l2 = 12", + "$l2 = 12", "goto label6", "label3:", - "l2 = 2", + "$l2 = 2", "goto label6", "label4:", - "l2 = 3", + "$l2 = 3", "goto label6", "label5:", - "l2 = -1", + "$l2 = -1", "label6:", "return")); } @@ -179,47 +179,47 @@ public void switchCaseStatementCaseIncludingSwitch() { assertJimpleStmts( method, expectedBodyStmts( - "l0 := @this: SwitchCaseStatement", - "l1 = 2", - "switch(l1)", + "$l0 := @this: SwitchCaseStatement", + "$l1 = 2", + "switch($l1)", "case 1: goto label01", "case 2: goto label05", "case 3: goto label10", "default: goto label11", "label01:", - "switch(l1)", + "switch($l1)", "case 10: goto label02", "case 20: goto label03", "default: goto label04", "label02:", - "l2 = 11", + "$l2 = 11", "goto label04", "label03:", - "l2 = 12", + "$l2 = 12", "label04:", "goto label12", "label05:", - "l2 = 2", - "switch(l1)", + "$l2 = 2", + "switch($l1)", "case 20: goto label06", "case 30: goto label07", "case 40: goto label08", "default: goto label09", "label06:", - "l2 = 220", + "$l2 = 220", "goto label09", "label07:", - "l2 = 230", + "$l2 = 230", "goto label09", "label08:", - "l2 = 240", + "$l2 = 240", "label09:", "goto label12", "label10:", - "l2 = 3", + "$l2 = 3", "goto label12", "label11:", - "l2 = -1", + "$l2 = -1", "label12:", "return")); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedBlockTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedBlockTest.java index 3ba8894d087..a6dcc79ed37 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedBlockTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedBlockTest.java @@ -40,25 +40,25 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SynchronizedBlock", - "$stack3 = l0.", - "l1 = $stack3", + "$l0 := @this: SynchronizedBlock", + "$stack3 = $l0.", + "$l1 = $stack3", "entermonitor $stack3", "label1:", "$stack5 = ", - "$stack4 = l0.", + "$stack4 = $l0.", "virtualinvoke $stack5.($stack4)", - "$stack6 = l1", + "$stack6 = $l1", "exitmonitor $stack6", "label2:", "goto label5", "label3:", "$stack7 := @caughtexception", - "l2 = $stack7", - "$stack8 = l1", + "$l2 = $stack7", + "$stack8 = $l1", "exitmonitor $stack8", "label4:", - "throw l2", + "throw $l2", "label5:", "return", "catch java.lang.Throwable from label1 to label2 with label3", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedMethodTest.java index f279cd5c58a..e2a9c6640b0 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/SynchronizedMethodTest.java @@ -40,7 +40,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SynchronizedMethod", + "$l0 := @this: SynchronizedMethod", "$stack1 = ", "virtualinvoke $stack1.(\"test\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TernaryOperatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TernaryOperatorTest.java index b65697a7a96..7d46c3b1706 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TernaryOperatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TernaryOperatorTest.java @@ -34,8 +34,8 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TernaryOperator", - "$stack1 = l0.", + "$l0 := @this: TernaryOperator", + "$stack1 = $l0.", "if $stack1 >= 0 goto label1", "$stack2 = 0", "goto label2", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ThrowExceptionMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ThrowExceptionMethodTest.java index 2c42956f547..68c3b718671 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ThrowExceptionMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/ThrowExceptionMethodTest.java @@ -39,7 +39,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: ThrowExceptionMethod", "l1 = 8 / 0", "return") + return Stream.of("$l0 := @this: ThrowExceptionMethod", "$l1 = 8 / 0", "return") .collect(Collectors.toCollection(ArrayList::new)); } @@ -50,7 +50,7 @@ public MethodSignature getMethodSignature1() { public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: ThrowExceptionMethod", + "$l0 := @this: ThrowExceptionMethod", "$stack1 = new CustomException", "specialinvoke $stack1.(java.lang.String)>(\"Custom Exception\")", "throw $stack1") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TransientVariableTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TransientVariableTest.java index 051a20073cb..e533fc5cb27 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TransientVariableTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TransientVariableTest.java @@ -50,9 +50,9 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TransientVariable", + "$l0 := @this: TransientVariable", "$stack2 = ", - "$stack1 = l0.", + "$stack1 = $l0.", "virtualinvoke $stack2.($stack1)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TryCatchFinallyTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TryCatchFinallyTest.java index dbd227cc1d3..267a068fa17 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TryCatchFinallyTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/TryCatchFinallyTest.java @@ -93,20 +93,20 @@ public MethodSignature getMethodSignature(String methodName) { */ public List expectedBodyStmtsTryCatch() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"try\"", + "$l1 = \"try\"", "$stack3 = ", - "virtualinvoke $stack3.(l1)", + "virtualinvoke $stack3.($l1)", "label2:", "goto label4", "label3:", "$stack4 := @caughtexception", - "l2 = $stack4", - "l1 = \"catch\"", + "$l2 = $stack4", + "$l1 = \"catch\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "label4:", "return", "catch java.lang.Exception from label1 to label2 with label3") @@ -134,35 +134,35 @@ public List expectedBodyStmtsTryCatch() { */ public List expectedBodyStmtsTryCatchFinally() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"try\"", + "$l1 = \"try\"", "$stack4 = ", - "virtualinvoke $stack4.(l1)", + "virtualinvoke $stack4.($l1)", "label2:", - "l1 = \"finally\"", + "$l1 = \"finally\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "goto label6", "label3:", "$stack8 := @caughtexception", - "l2 = $stack8", - "l1 = \"catch\"", + "$l2 = $stack8", + "$l1 = \"catch\"", "$stack9 = ", - "virtualinvoke $stack9.(l1)", + "virtualinvoke $stack9.($l1)", "label4:", - "l1 = \"finally\"", + "$l1 = \"finally\"", "$stack10 = ", - "virtualinvoke $stack10.(l1)", + "virtualinvoke $stack10.($l1)", "goto label6", "label5:", "$stack6 := @caughtexception", - "l3 = $stack6", - "l1 = \"finally\"", + "$l3 = $stack6", + "$l1 = \"finally\"", "$stack7 = ", - "virtualinvoke $stack7.(l1)", - "throw l3", + "virtualinvoke $stack7.($l1)", + "throw $l3", "label6:", "return", "catch java.lang.Exception from label1 to label2 with label3", @@ -189,20 +189,20 @@ public List expectedBodyStmtsTryCatchFinally() { */ public List expectedBodyStmtsTryCatchCombined() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"try\"", + "$l1 = \"try\"", "$stack3 = ", - "virtualinvoke $stack3.(l1)", + "virtualinvoke $stack3.($l1)", "label2:", "goto label4", "label3:", "$stack4 := @caughtexception", - "l2 = $stack4", - "l1 = \"catch\"", + "$l2 = $stack4", + "$l1 = \"catch\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "label4:", "return", "catch java.lang.RuntimeException from label1 to label2 with label3", @@ -231,35 +231,35 @@ public List expectedBodyStmtsTryCatchCombined() { */ public List expectedBodyStmtsTryCatchFinallyCombined() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"try\"", + "$l1 = \"try\"", "$stack4 = ", - "virtualinvoke $stack4.(l1)", + "virtualinvoke $stack4.($l1)", "label2:", - "l1 = \"finally\"", + "$l1 = \"finally\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "goto label6", "label3:", "$stack8 := @caughtexception", - "l2 = $stack8", - "l1 = \"catch\"", + "$l2 = $stack8", + "$l1 = \"catch\"", "$stack9 = ", - "virtualinvoke $stack9.(l1)", + "virtualinvoke $stack9.($l1)", "label4:", - "l1 = \"finally\"", + "$l1 = \"finally\"", "$stack10 = ", - "virtualinvoke $stack10.(l1)", + "virtualinvoke $stack10.($l1)", "goto label6", "label5:", "$stack6 := @caughtexception", - "l3 = $stack6", - "l1 = \"finally\"", + "$l3 = $stack6", + "$l1 = \"finally\"", "$stack7 = ", - "virtualinvoke $stack7.(l1)", - "throw l3", + "virtualinvoke $stack7.($l1)", + "throw $l3", "label6:", "return", "catch java.lang.RuntimeException from label1 to label2 with label3", @@ -294,32 +294,32 @@ public List expectedBodyStmtsTryCatchFinallyCombined() { */ public List expectedBodyStmtsTryCatchNested() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"1try\"", + "$l1 = \"1try\"", "$stack3 = ", - "virtualinvoke $stack3.(l1)", + "virtualinvoke $stack3.($l1)", "label2:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack4 = ", - "virtualinvoke $stack4.(l1)", + "virtualinvoke $stack4.($l1)", "label3:", "goto label7", "label4:", "$stack5 := @caughtexception", - "l2 = $stack5", - "l1 = \"1catch\"", + "$l2 = $stack5", + "$l1 = \"1catch\"", "$stack6 = ", - "virtualinvoke $stack6.(l1)", + "virtualinvoke $stack6.($l1)", "label5:", "return", "label6:", "$stack7 := @caughtexception", - "l2 = $stack7", - "l1 = \"2catch\"", + "$l2 = $stack7", + "$l1 = \"2catch\"", "$stack8 = ", - "virtualinvoke $stack8.(l1)", + "virtualinvoke $stack8.($l1)", "label7:", "goto label5", "catch java.lang.Exception from label1 to label2 with label4", @@ -357,47 +357,47 @@ public List expectedBodyStmtsTryCatchNested() { */ public List expectedBodyStmtsTryCatchFinallyNested() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"1try\"", + "$l1 = \"1try\"", "$stack4 = ", - "virtualinvoke $stack4.(l1)", + "virtualinvoke $stack4.($l1)", "label2:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "label3:", "goto label8", "label4:", "$stack9 := @caughtexception", - "l2 = $stack9", - "l1 = \"1catch\"", + "$l2 = $stack9", + "$l1 = \"1catch\"", "$stack10 = ", - "virtualinvoke $stack10.(l1)", + "virtualinvoke $stack10.($l1)", "label5:", - "l1 = \"1finally\"", + "$l1 = \"1finally\"", "$stack11 = ", - "virtualinvoke $stack11.(l1)", + "virtualinvoke $stack11.($l1)", "goto label9", "label6:", "$stack7 := @caughtexception", - "l3 = $stack7", - "l1 = \"1finally\"", + "$l3 = $stack7", + "$l1 = \"1finally\"", "$stack8 = ", - "virtualinvoke $stack8.(l1)", - "throw l3", + "virtualinvoke $stack8.($l1)", + "throw $l3", "label7:", "$stack12 := @caughtexception", - "l2 = $stack12", - "l1 = \"2catch\"", + "$l2 = $stack12", + "$l1 = \"2catch\"", "$stack13 = ", - "virtualinvoke $stack13.(l1)", + "virtualinvoke $stack13.($l1)", "label8:", "$stack14 = \"1finally\"", - "l1 = $stack14", + "$l1 = $stack14", "$stack6 = ", - "$stack15 = l1", + "$stack15 = $l1", "virtualinvoke $stack6.($stack15)", "goto label9", "label9:", @@ -436,32 +436,32 @@ public List expectedBodyStmtsTryCatchFinallyNested() { */ public List expectedBodyStmtsTryCatchNestedInCatch() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label1:", - "l1 = \"1try\"", + "$l1 = \"1try\"", "$stack4 = ", - "virtualinvoke $stack4.(l1)", + "virtualinvoke $stack4.($l1)", "label2:", "goto label7", "label3:", "$stack7 := @caughtexception", - "l2 = $stack7", - "l1 = \"1catch\"", + "$l2 = $stack7", + "$l1 = \"1catch\"", "$stack8 = ", - "virtualinvoke $stack8.(l1)", + "virtualinvoke $stack8.($l1)", "label4:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack9 = ", - "virtualinvoke $stack9.(l1)", + "virtualinvoke $stack9.($l1)", "label5:", "goto label7", "label6:", "$stack5 := @caughtexception", - "l3 = $stack5", - "l1 = \"2catch\"", + "$l3 = $stack5", + "$l1 = \"2catch\"", "$stack6 = ", - "virtualinvoke $stack6.(l1)", + "virtualinvoke $stack6.($l1)", "label7:", "return", "catch java.lang.Exception from label1 to label2 with label3", @@ -497,48 +497,48 @@ public List expectedBodyStmtsTryCatchNestedInCatch() { */ public List expectedBodyStmtsTryCatchFinallyNestedInCatch() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label01:", - "l1 = \"1try\"", + "$l1 = \"1try\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "label02:", - "l1 = \"1finally\"", + "$l1 = \"1finally\"", "$stack6 = ", - "virtualinvoke $stack6.(l1)", + "virtualinvoke $stack6.($l1)", "goto label10", "label03:", "$stack12 := @caughtexception", - "l2 = $stack12", - "l1 = \"1catch\"", + "$l2 = $stack12", + "$l1 = \"1catch\"", "$stack13 = ", - "virtualinvoke $stack13.(l1)", + "virtualinvoke $stack13.($l1)", "label04:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack14 = ", - "virtualinvoke $stack14.(l1)", + "virtualinvoke $stack14.($l1)", "label05:", "goto label09", "label06:", "$stack7 := @caughtexception", - "l4 = $stack7", + "$l4 = $stack7", "label07:", - "l1 = \"1finally\"", + "$l1 = \"1finally\"", "$stack8 = ", - "virtualinvoke $stack8.(l1)", - "throw l4", + "virtualinvoke $stack8.($l1)", + "throw $l4", "label08:", "$stack9 := @caughtexception", - "l3 = $stack9", - "l1 = \"2catch\"", + "$l3 = $stack9", + "$l1 = \"2catch\"", "$stack10 = ", - "virtualinvoke $stack10.(l1)", + "virtualinvoke $stack10.($l1)", "label09:", "$stack15 = \"1finally\"", - "l1 = $stack15", + "$l1 = $stack15", "$stack11 = ", - "$stack16 = l1", + "$stack16 = $l1", "virtualinvoke $stack11.($stack16)", "goto label10", "label10:", @@ -579,72 +579,72 @@ public List expectedBodyStmtsTryCatchFinallyNestedInCatch() { */ public List expectedBodyStmtsTryCatchFinallyNestedInFinally() { return Stream.of( - "l0 := @this: TryCatchFinally", - "l1 = \"\"", + "$l0 := @this: TryCatchFinally", + "$l1 = \"\"", "label01:", - "l1 = \"1try\"", + "$l1 = \"1try\"", "$stack5 = ", - "virtualinvoke $stack5.(l1)", + "virtualinvoke $stack5.($l1)", "label02:", - "l1 = \"1finally\"", + "$l1 = \"1finally\"", "$stack6 = ", - "virtualinvoke $stack6.(l1)", + "virtualinvoke $stack6.($l1)", "label03:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack7 = ", - "virtualinvoke $stack7.(l1)", + "virtualinvoke $stack7.($l1)", "label04:", "goto label16", "label05:", "$stack20 := @caughtexception", - "l2 = $stack20", - "l1 = \"2catch\"", + "$l2 = $stack20", + "$l1 = \"2catch\"", "$stack21 = ", - "virtualinvoke $stack21.(l1)", + "virtualinvoke $stack21.($l1)", "goto label16", "label06:", "$stack16 := @caughtexception", - "l2 = $stack16", - "l1 = \"1catch\"", + "$l2 = $stack16", + "$l1 = \"1catch\"", "$stack17 = ", - "virtualinvoke $stack17.(l1)", + "virtualinvoke $stack17.($l1)", "label07:", - "l1 = \"1finally\"", + "$l1 = \"1finally\"", "$stack18 = ", - "virtualinvoke $stack18.(l1)", + "virtualinvoke $stack18.($l1)", "label08:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack19 = ", - "virtualinvoke $stack19.(l1)", + "virtualinvoke $stack19.($l1)", "label09:", "goto label16", "label10:", "$stack14 := @caughtexception", - "l2 = $stack14", - "l1 = \"2catch\"", + "$l2 = $stack14", + "$l1 = \"2catch\"", "$stack15 = ", - "virtualinvoke $stack15.(l1)", + "virtualinvoke $stack15.($l1)", "goto label16", "label11:", "$stack10 := @caughtexception", - "l3 = $stack10", - "l1 = \"1finally\"", + "$l3 = $stack10", + "$l1 = \"1finally\"", "$stack11 = ", - "virtualinvoke $stack11.(l1)", + "virtualinvoke $stack11.($l1)", "label12:", - "l1 = \"2try\"", + "$l1 = \"2try\"", "$stack12 = ", - "virtualinvoke $stack12.(l1)", + "virtualinvoke $stack12.($l1)", "label13:", "goto label15", "label14:", "$stack8 := @caughtexception", - "l4 = $stack8", - "l1 = \"2catch\"", + "$l4 = $stack8", + "$l1 = \"2catch\"", "$stack9 = ", - "virtualinvoke $stack9.(l1)", + "virtualinvoke $stack9.($l1)", "label15:", - "$stack13 = l3", + "$stack13 = $l3", "throw $stack13", "label16:", "return", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UnaryOpIntTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UnaryOpIntTest.java index 7c727ff4978..018bc87823d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UnaryOpIntTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UnaryOpIntTest.java @@ -25,11 +25,6 @@ public MethodSignature getMethodSignature() { public void test() { SootMethod method = loadMethod(getMethodSignature()); assertJimpleStmts(method, expectedBodyStmts()); - /** - * TODO Do we need to check the type of variable as int? - * assertTrue(getFields().stream().anyMatch(sootField -> {return - * sootField.getType().equals("int");})); - */ } /** @@ -45,10 +40,10 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: UnaryOpInt", - "$stack3 = l0.", - "$stack2 = l0.", - "l1 = $stack3 + $stack2", + "$l0 := @this: UnaryOpInt", + "$stack3 = $l0.", + "$stack2 = $l0.", + "$l1 = $stack3 + $stack2", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UncheckedCastTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UncheckedCastTest.java index b6355fb5340..adf0084bf6a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UncheckedCastTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/UncheckedCastTest.java @@ -36,7 +36,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: UncheckedCast", + "$l0 := @this: UncheckedCast", "$stack3 = newarray (java.lang.Integer)[4]", "$stack4 = staticinvoke (5)", "$stack3[0] = $stack4", @@ -46,10 +46,10 @@ public List expectedBodyStmts() { "$stack3[2] = $stack6", "$stack7 = staticinvoke (6)", "$stack3[3] = $stack7", - "l1 = staticinvoke ($stack3)", - "l2 = l1", + "$l1 = staticinvoke ($stack3)", + "$l2 = $l1", "$stack8 = ", - "virtualinvoke $stack8.(l2)", + "virtualinvoke $stack8.($l2)", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableDeclarationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableDeclarationTest.java index d52ada95378..aad9ff942f8 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableDeclarationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableDeclarationTest.java @@ -55,7 +55,7 @@ public MethodSignature getMethodSignature(String methodName) { * */ public List expectedBodyStmtsShortVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 10", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 10", "return") .collect(Collectors.toList()); } @@ -69,7 +69,7 @@ public List expectedBodyStmtsShortVariable() { * */ public List expectedBodyStmtsByteVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 0", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 0", "return") .collect(Collectors.toList()); } @@ -83,7 +83,7 @@ public List expectedBodyStmtsByteVariable() { * */ public List expectedBodyStmtsCharVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 97", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 97", "return") .collect(Collectors.toList()); } @@ -97,7 +97,7 @@ public List expectedBodyStmtsCharVariable() { * */ public List expectedBodyStmtsIntVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 512", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 512", "return") .collect(Collectors.toList()); } @@ -111,7 +111,7 @@ public List expectedBodyStmtsIntVariable() { * */ public List expectedBodyStmtsLongVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 123456789L", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 123456789L", "return") .collect(Collectors.toList()); } @@ -125,7 +125,7 @@ public List expectedBodyStmtsLongVariable() { * */ public List expectedBodyStmtsFloatVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 3.14F", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 3.14F", "return") .collect(Collectors.toList()); } @@ -139,7 +139,7 @@ public List expectedBodyStmtsFloatVariable() { * */ public List expectedBodyStmtsDoubleVariable() { - return Stream.of("l0 := @this: VariableDeclaration", "l1 = 1.96969654", "return") + return Stream.of("$l0 := @this: VariableDeclaration", "$l1 = 1.96969654", "return") .collect(Collectors.toList()); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableShadowingTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableShadowingTest.java index 0274c74d09a..779379b6f5d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableShadowingTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VariableShadowingTest.java @@ -34,9 +34,9 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: VariableShadowing", - "l1 = l0.", - "l2 = 10", + "$l0 := @this: VariableShadowing", + "$l1 = $l0.", + "$l2 = 10", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VirtualMethodTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VirtualMethodTest.java index 370ca5d9b55..daf73ace895 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VirtualMethodTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VirtualMethodTest.java @@ -45,18 +45,18 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: VirtualMethod", + "$l0 := @this: VirtualMethod", "$stack3 = new TempEmployee", "specialinvoke $stack3.(int,int)>(1500, 150)", - "l1 = $stack3", + "$l1 = $stack3", "$stack4 = new RegEmployee", "specialinvoke $stack4.(int,int)>(1500, 500)", - "l2 = $stack4", + "$l2 = $stack4", "$stack5 = ", - "$stack6 = virtualinvoke l1.()", + "$stack6 = virtualinvoke $l1.()", "virtualinvoke $stack5.($stack6)", "$stack7 = ", - "$stack8 = virtualinvoke l2.()", + "$stack8 = virtualinvoke $l2.()", "virtualinvoke $stack7.($stack8)", "return") .collect(Collectors.toCollection(ArrayList::new)); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VolatileVariableTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VolatileVariableTest.java index a4d8b332d92..b75e6b0548a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VolatileVariableTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/VolatileVariableTest.java @@ -50,10 +50,10 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: VolatileVariable", - "$stack1 = l0.", + "$l0 := @this: VolatileVariable", + "$stack1 = $l0.", "$stack2 = $stack1 + 1", - "l0. = $stack2", + "$l0. = $stack2", "return $stack1") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/WhileLoopTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/WhileLoopTest.java index 8823b7d6c3c..f02f0a7979f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/WhileLoopTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/WhileLoopTest.java @@ -34,14 +34,14 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: WhileLoop", - "l1 = 10", - "l2 = 0", + "$l0 := @this: WhileLoop", + "$l1 = 10", + "$l2 = 0", "label1:", - "$stack4 = l1", - "$stack3 = l2", + "$stack4 = $l1", + "$stack3 = $l2", "if $stack4 <= $stack3 goto label2", - "l1 = l1 + -1", + "$l1 = $l1 + -1", "goto label1", "label2:", "return") diff --git "a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/\316\261\317\201\316\265\317\204\316\267Test.java" "b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/\316\261\317\201\316\265\317\204\316\267Test.java" index 277e5222619..1bb67d23d64 100644 --- "a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/\316\261\317\201\316\265\317\204\316\267Test.java" +++ "b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java6/\316\261\317\201\316\265\317\204\316\267Test.java" @@ -34,7 +34,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: \\u03b1\\u03c1\\u03b5\\u03c4\\u03b7", + "$l0 := @this: \\u03b1\\u03c1\\u03b5\\u03c4\\u03b7", "$stack1 = ", "virtualinvoke $stack1.(\"this is \\u03b1\\u03c1\\u03b5\\u03c4\\u03b7 class\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/BinaryLiteralInIntTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/BinaryLiteralInIntTest.java index 2dae1068f76..c88285559bc 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/BinaryLiteralInIntTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/BinaryLiteralInIntTest.java @@ -35,7 +35,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: BinaryLiteralInInt", "l1 = -1589272251", "l2 = 5", "l3 = 5", "return") + "$l0 := @this: BinaryLiteralInInt", "$l1 = -1589272251", "$l2 = 5", "$l3 = 5", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/MultiTryCatchTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/MultiTryCatchTest.java index 3e8699c3aee..87ce538510a 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/MultiTryCatchTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/MultiTryCatchTest.java @@ -59,67 +59,67 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MultiTryCatch", + "$l0 := @this: MultiTryCatch", "$stack6 = new java.io.BufferedReader", "$stack7 = new java.io.FileReader", "specialinvoke $stack7.(java.lang.String)>(\"file.txt\")", "specialinvoke $stack6.(java.io.Reader)>($stack7)", - "l1 = $stack6", + "$l1 = $stack6", "label01:", - "l2 = \"\"", - "l3 = 2", + "$l2 = \"\"", + "$l3 = 2", "$stack8 = ", - "virtualinvoke $stack8.(l3)", + "virtualinvoke $stack8.($l3)", "label02:", - "$stack11 = l1", + "$stack11 = $l1", "$stack9 = virtualinvoke $stack11.()", - "l2 = $stack9", + "$l2 = $stack9", "if $stack9 == null goto label16", "$stack10 = ", - "virtualinvoke $stack10.(l2)", + "virtualinvoke $stack10.($l2)", "goto label02", "label03:", "$stack18 := @caughtexception", - "l2 = $stack18", + "$l2 = $stack18", "label04:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label05:", "goto label19", "label06:", "$stack17 := @caughtexception", - "l2 = $stack17", + "$l2 = $stack17", "goto label19", "label07:", "$stack16 := @caughtexception", - "l2 = $stack16", + "$l2 = $stack16", "label08:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label09:", "goto label19", "label10:", "$stack15 := @caughtexception", - "l2 = $stack15", + "$l2 = $stack15", "goto label19", "label11:", "$stack13 := @caughtexception", - "l4 = $stack13", + "$l4 = $stack13", "label12:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label13:", "goto label15", "label14:", "$stack12 := @caughtexception", - "l5 = $stack12", + "$l5 = $stack12", "label15:", - "$stack14 = l4", + "$stack14 = $l4", "throw $stack14", "label16:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label17:", "goto label19", "label18:", "$stack19 := @caughtexception", - "l2 = $stack19", + "$l2 = $stack19", "goto label19", "label19:", "return", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/SwitchCaseStatementWithStringTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/SwitchCaseStatementWithStringTest.java index 91dd2585d20..d8dc068d1bf 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/SwitchCaseStatementWithStringTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/SwitchCaseStatementWithStringTest.java @@ -57,48 +57,48 @@ public MethodSignature getMethodSignature3() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: SwitchCaseStatementWithString", - "l1 = \"something\"", - "l3 = l1", - "l4 = -1", - "$stack5 = virtualinvoke l3.()", + "$l0 := @this: SwitchCaseStatementWithString", + "$l1 = \"something\"", + "$l3 = $l1", + "$l4 = -1", + "$stack5 = virtualinvoke $l3.()", "switch($stack5)", "case 110182: goto label1", "case 115276: goto label2", "case 110339486: goto label3", "default: goto label4", "label1:", - "$stack9 = virtualinvoke l3.(\"one\")", + "$stack9 = virtualinvoke $l3.(\"one\")", "if $stack9 == 0 goto label4", - "l4 = 0", + "$l4 = 0", "goto label4", "label2:", - "$stack8 = virtualinvoke l3.(\"two\")", + "$stack8 = virtualinvoke $l3.(\"two\")", "if $stack8 == 0 goto label4", - "l4 = 1", + "$l4 = 1", "goto label4", "label3:", - "$stack6 = virtualinvoke l3.(\"three\")", + "$stack6 = virtualinvoke $l3.(\"three\")", "if $stack6 == 0 goto label4", - "l4 = 2", + "$l4 = 2", "label4:", - "$stack7 = l4", + "$stack7 = $l4", "switch($stack7)", "case 0: goto label5", "case 1: goto label6", "case 2: goto label7", "default: goto label8", "label5:", - "l2 = 1", + "$l2 = 1", "goto label9", "label6:", - "l2 = 2", + "$l2 = 2", "goto label9", "label7:", - "l2 = 3", + "$l2 = 3", "goto label9", "label8:", - "l2 = -1", + "$l2 = -1", "label9:", "return") .collect(Collectors.toList()); @@ -130,24 +130,24 @@ public List expectedBodyStmts() { */ public List expectedBodyStmts2() { return Stream.of( - "l0 := @this: SwitchCaseStatementWithString", - "l1 = 2", - "switch(l1)", + "$l0 := @this: SwitchCaseStatementWithString", + "$l1 = 2", + "switch($l1)", "case 1: goto label1", "case 2: goto label2", "case 3: goto label3", "default: goto label4", "label1:", - "l2 = \"number 1 detected\"", + "$l2 = \"number 1 detected\"", "goto label5", "label2:", - "l2 = \"number 2 detected\"", + "$l2 = \"number 2 detected\"", "goto label5", "label3:", - "l2 = \"number 3 detected\"", + "$l2 = \"number 3 detected\"", "goto label5", "label4:", - "l2 = \"invalid number\"", + "$l2 = \"invalid number\"", "label5:", "return") .collect(Collectors.toList()); @@ -176,11 +176,11 @@ public List expectedBodyStmts2() { */ public List expectedBodyStmts3() { return Stream.of( - "l0 := @this: SwitchCaseStatementWithString", - "l1 = \"RED\"", - "l2 = \"\"", + "$l0 := @this: SwitchCaseStatementWithString", + "$l1 = \"RED\"", + "$l2 = \"\"", "$stack3 = ", - "$stack4 = staticinvoke (l1)", + "$stack4 = staticinvoke ($l1)", "$stack5 = virtualinvoke $stack4.()", "$stack6 = $stack3[$stack5]", "switch($stack6)", @@ -188,13 +188,13 @@ public List expectedBodyStmts3() { "case 2: goto label2", "default: goto label3", "label1:", - "l2 = \"color red detected\"", + "$l2 = \"color red detected\"", "goto label4", "label2:", - "l2 = \"color green detected\"", + "$l2 = \"color green detected\"", "goto label4", "label3:", - "l2 = \"invalid color\"", + "$l2 = \"invalid color\"", "label4:", "return") .collect(Collectors.toList()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/TryWithResourcesTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/TryWithResourcesTest.java index d34bddbbeb3..1c65fcba61b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/TryWithResourcesTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/TryWithResourcesTest.java @@ -46,62 +46,62 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TryWithResources", + "$l0 := @this: TryWithResources", "$stack6 = new java.io.BufferedReader", "$stack7 = new java.io.FileReader", "specialinvoke $stack7.(java.lang.String)>(\"file.txt\")", "specialinvoke $stack6.(java.io.Reader)>($stack7)", - "l1 = $stack6", - "l2 = null", + "$l1 = $stack6", + "$l2 = null", "label01:", - "l3 = \"\"", + "$l3 = \"\"", "label02:", - "$stack10 = l1", + "$stack10 = $l1", "$stack8 = virtualinvoke $stack10.()", - "l3 = $stack8", + "$l3 = $stack8", "if $stack8 == null goto label11", "$stack9 = ", - "virtualinvoke $stack9.(l3)", + "virtualinvoke $stack9.($l3)", "goto label02", "label03:", "$stack14 := @caughtexception", - "l3 = $stack14", - "l2 = l3", - "throw l3", + "$l3 = $stack14", + "$l2 = $l3", + "throw $l3", "label04:", "$stack12 := @caughtexception", - "l4 = $stack12", + "$l4 = $stack12", "label05:", - "if l1 == null goto label10", - "if l2 == null goto label09", + "if $l1 == null goto label10", + "if $l2 == null goto label09", "label06:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label07:", "goto label10", "label08:", "$stack11 := @caughtexception", - "l5 = $stack11", - "virtualinvoke l2.(l5)", + "$l5 = $stack11", + "virtualinvoke $l2.($l5)", "goto label10", "label09:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label10:", - "$stack13 = l4", + "$stack13 = $l4", "throw $stack13", "label11:", - "if l1 == null goto label16", - "if l2 == null goto label15", + "if $l1 == null goto label16", + "if $l2 == null goto label15", "label12:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "label13:", "goto label16", "label14:", "$stack15 := @caughtexception", - "l3 = $stack15", - "virtualinvoke l2.(l3)", + "$l3 = $stack15", + "virtualinvoke $l2.($l3)", "goto label16", "label15:", - "virtualinvoke l1.()", + "virtualinvoke $l1.()", "goto label16", "label16:", "return", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/UnderscoreInIntTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/UnderscoreInIntTest.java index 8315a6a35cd..93fa375dbff 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/UnderscoreInIntTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java7/UnderscoreInIntTest.java @@ -32,7 +32,7 @@ public MethodSignature getMethodSignature() { */ @Override public List expectedBodyStmts() { - return Stream.of("l0 := @this: UnderscoreInInt", "l1 = 2147483647", "return") + return Stream.of("$l0 := @this: UnderscoreInInt", "$l1 = 2147483647", "return") .collect(Collectors.toList()); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/DefaultMethodInterfaceImplTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/DefaultMethodInterfaceImplTest.java index 5a7b0145f83..85a5b31498b 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/DefaultMethodInterfaceImplTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/DefaultMethodInterfaceImplTest.java @@ -68,7 +68,7 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: DefaultMethodInterfaceImpl", + "$l0 := @this: DefaultMethodInterfaceImpl", "$stack1 = ", "virtualinvoke $stack1.(\"Method interfaceMethod() is implemented\")", "return") @@ -77,8 +77,8 @@ public List expectedBodyStmts() { public List expectedBodyStmts1() { return Stream.of( - "l0 := @this: DefaultMethodInterfaceImpl", - "specialinvoke l0.()", + "$l0 := @this: DefaultMethodInterfaceImpl", + "specialinvoke $l0.()", "$stack1 = ", "virtualinvoke $stack1.(\"Method defaultInterfaceMethod() is implemented\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodAcceptingLamExprTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodAcceptingLamExprTest.java index 9042cd14b7d..4e46134cbc7 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodAcceptingLamExprTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodAcceptingLamExprTest.java @@ -42,13 +42,13 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MethodAcceptingLamExpr", - "l1 = dynamicinvoke \"calcPercentage\" () (methodtype: double __METHODTYPE__(double), handle: , methodtype: double __METHODTYPE__(double))", + "$l0 := @this: MethodAcceptingLamExpr", + "$l1 = dynamicinvoke \"calcPercentage\" () (methodtype: double __METHODTYPE__(double), handle: , methodtype: double __METHODTYPE__(double))", "$stack3 = ", "$stack2 = new java.lang.StringBuilder", "specialinvoke $stack2.()>()", "$stack5 = virtualinvoke $stack2.(\"Percentage : \")", - "$stack4 = interfaceinvoke l1.(45.0)", + "$stack4 = interfaceinvoke $l1.(45.0)", "$stack6 = virtualinvoke $stack5.($stack4)", "$stack7 = virtualinvoke $stack6.()", "virtualinvoke $stack3.($stack7)", diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodReferenceTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodReferenceTest.java index 6823a1cd197..2fb50d0eee2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodReferenceTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java8/MethodReferenceTest.java @@ -41,7 +41,7 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: MethodReference", + "$l0 := @this: MethodReference", "$stack1 = ", "virtualinvoke $stack1.(\"Instance Method\")", "return") diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/AnonymousDiamondOperatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/AnonymousDiamondOperatorTest.java index ed40e17014f..3370e4d5634 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/AnonymousDiamondOperatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/AnonymousDiamondOperatorTest.java @@ -48,15 +48,15 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: AnonymousDiamondOperator", + "$l0 := @this: AnonymousDiamondOperator", "$stack3 = new AnonymousDiamondOperator$1", - "specialinvoke $stack3.(AnonymousDiamondOperator)>(l0)", - "l1 = $stack3", + "specialinvoke $stack3.(AnonymousDiamondOperator)>($l0)", + "$l1 = $stack3", "$stack5 = staticinvoke (22)", "$stack4 = staticinvoke (23)", - "$stack6 = virtualinvoke l1.($stack5, $stack4)", - "l2 = (java.lang.Integer) $stack6", - "$stack7 = virtualinvoke l2.()", + "$stack6 = virtualinvoke $l1.($stack5, $stack4)", + "$l2 = (java.lang.Integer) $stack6", + "$stack7 = virtualinvoke $l2.()", "return $stack7") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java index a12c8732dba..b1c6360e527 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/DynamicInvokeTest.java @@ -28,9 +28,9 @@ public void test() { public List expectedBodyStmts() { return Stream.of( "$l0 = \"This test\"", - "l0 = dynamicinvoke \"makeConcatWithConstants\" (l0) (\"\\u0001 is cool\")", + "$l0 = dynamicinvoke \"makeConcatWithConstants\" ($l0) (\"\\u0001 is cool\")", "$stack1 = ", - "virtualinvoke $stack1.(l0)", + "virtualinvoke $stack1.($l0)", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/PrivateMethodInterfaceImplTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/PrivateMethodInterfaceImplTest.java index 0a9d9465ecc..5b8f2da22a0 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/PrivateMethodInterfaceImplTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/PrivateMethodInterfaceImplTest.java @@ -75,8 +75,8 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: PrivateMethodInterfaceImpl", - "virtualinvoke l0.(4, 2)", + "$l0 := @this: PrivateMethodInterfaceImpl", + "virtualinvoke $l0.(4, 2)", "return") .collect(Collectors.toCollection(ArrayList::new)); } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/TryWithResourcesConciseTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/TryWithResourcesConciseTest.java index dbab57ba4ee..6a861a22c78 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/TryWithResourcesConciseTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java9/TryWithResourcesConciseTest.java @@ -67,41 +67,41 @@ public void test() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: TryWithResourcesConcise", + "$l0 := @this: TryWithResourcesConcise", "$stack5 = new java.io.BufferedReader", "$stack6 = new java.io.FileReader", "specialinvoke $stack6.(java.lang.String)>(\"file.txt\")", "specialinvoke $stack5.(java.io.Reader)>($stack6)", - "l1 = $stack5", - "l2 = l1", + "$l1 = $stack5", + "$l2 = $l1", "label1:", - "l3 = \"\"", + "$l3 = \"\"", "label2:", - "$stack9 = l1", + "$stack9 = $l1", "$stack7 = virtualinvoke $stack9.()", - "l3 = $stack7", + "$l3 = $stack7", "if $stack7 == null goto label8", "$stack8 = ", - "virtualinvoke $stack8.(l3)", + "virtualinvoke $stack8.($l3)", "goto label2", "label3:", "$stack11 := @caughtexception", - "l3 = $stack11", - "if l2 == null goto label7", + "$l3 = $stack11", + "if $l2 == null goto label7", "label4:", - "virtualinvoke l2.()", + "virtualinvoke $l2.()", "label5:", "goto label7", "label6:", "$stack10 := @caughtexception", - "l4 = $stack10", - "virtualinvoke l3.(l4)", + "$l4 = $stack10", + "virtualinvoke $l3.($l4)", "label7:", - "$stack12 = l3", + "$stack12 = $l3", "throw $stack12", "label8:", - "if l2 == null goto label9", - "virtualinvoke l2.()", + "if $l2 == null goto label9", + "virtualinvoke $l2.()", "goto label9", "label9:", "return", From 637fbd9767f9d3baa1d7602fc5ef57e502d249c3 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Fri, 6 Oct 2023 15:50:52 +0200 Subject: [PATCH 31/54] fix aggregator combining variables even when conflicting uses exist --- .../bytecode/interceptors/Aggregator.java | 12 +++++-- .../bytecode/interceptors/AggregatorTest.java | 34 +++++++++++++++++++ 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java index e5ca51382e8..528e4b7d9cb 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/Aggregator.java @@ -21,14 +21,16 @@ * #L% */ -import com.google.common.collect.Lists; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import javax.annotation.Nonnull; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Immediate; +import sootup.core.jimple.basic.LValue; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.AbstractBinopExpr; @@ -69,11 +71,11 @@ public Aggregator(boolean dontAggregateFieldLocals) { */ @Override public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View view) { - StmtGraph graph = builder.getStmtGraph(); List stmts = builder.getStmts(); + Map> usesMap = Body.collectUses(stmts); - for (Stmt stmt : Lists.newArrayList(stmts)) { + for (Stmt stmt : stmts) { if (!(stmt instanceof JAssignStmt)) { continue; } @@ -90,6 +92,10 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi if (!(val instanceof Local)) { continue; } + if (usesMap.get(val).size() > 1) { + // there are other uses, so it can't be aggregated + continue; + } List defs = ((Local) val).getDefs(stmts); if (defs.size() != 1) { continue; diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 3d87a04cfa4..19ffcedfcd8 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -10,6 +10,7 @@ import org.junit.Test; import sootup.core.inputlocation.AnalysisInputLocation; import sootup.core.inputlocation.ClassLoadingOptions; +import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; @@ -27,6 +28,7 @@ import sootup.java.core.JavaSootClass; import sootup.java.core.language.JavaJimple; import sootup.java.core.language.JavaLanguage; +import sootup.java.core.types.JavaClassType; import sootup.java.core.views.JavaView; public class AggregatorTest { @@ -77,6 +79,38 @@ public void testNoAggregation() { } } + @Test + public void noAggregationWithUse() { + Body.BodyBuilder builder = Body.builder(); + + StmtPositionInfo noPositionInfo = StmtPositionInfo.createNoStmtPositionInfo(); + + JavaClassType fileType = JavaIdentifierFactory.getInstance().getClassType("File"); + + Local a = JavaJimple.newLocal("a", fileType); + Local b = JavaJimple.newLocal("b", fileType); + + Stmt assignA = JavaJimple.newAssignStmt(a, JavaJimple.newNewExpr(fileType), noPositionInfo); + // this use of `a` should prevent the aggregator from changing anything + Stmt useA = JavaJimple.newInvokeStmt(Jimple.newSpecialInvokeExpr(a, JavaIdentifierFactory.getInstance().parseMethodSignature("()>")), noPositionInfo); + Stmt assignB = JavaJimple.newAssignStmt(b, a, noPositionInfo); + Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); + + builder.setStartingStmt(assignA); + builder.addFlow(assignA, useA); + builder.addFlow(useA, assignB); + builder.addFlow(assignB, ret); + + builder.setMethodSignature( + JavaIdentifierFactory.getInstance() + .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + + new Aggregator().interceptBody(builder, null); + + // ensure that the assigner doesn't remove any statements + assertEquals(4, builder.getStmts().size()); + } + private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { StmtPositionInfo noPositionInfo = StmtPositionInfo.createNoStmtPositionInfo(); From 6f25a8197d31197ccb90e68980c4f27ee7bc71d3 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 6 Oct 2023 13:59:21 +0200 Subject: [PATCH 32/54] fix tests; add Category --- .../ifds/IFDSTaintAnalysisTest.java | 18 +++++++++--------- .../java/sootup/java/bytecode/Soot1577.java | 7 +++++++ .../java/sootup/java/bytecode/Soot1580.java | 3 +++ 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sootup.analysis/src/test/java/sootup/analysis/interprocedural/ifds/IFDSTaintAnalysisTest.java b/sootup.analysis/src/test/java/sootup/analysis/interprocedural/ifds/IFDSTaintAnalysisTest.java index f43e33e2296..5eeb1cae67d 100644 --- a/sootup.analysis/src/test/java/sootup/analysis/interprocedural/ifds/IFDSTaintAnalysisTest.java +++ b/sootup.analysis/src/test/java/sootup/analysis/interprocedural/ifds/IFDSTaintAnalysisTest.java @@ -73,8 +73,8 @@ public void SimpleTaint() { JimpleIFDSSolver> analysis = executeStaticAnalysis("SimpleTaint"); Set result = getResultsAtLastStatement(analysis); - assertTrue(result.contains("l1")); - assertTrue(result.contains("l2")); + assertTrue(result.contains("$l1")); + assertTrue(result.contains("$l2")); assertTrue(result.contains("SimpleTaint.k")); } @@ -83,7 +83,7 @@ public void SimpleTaintSanitized() { JimpleIFDSSolver> analysis = executeStaticAnalysis("SimpleTaintSanitized"); Set result = getResultsAtLastStatement(analysis); - assertTrue(result.contains("l1")); + assertTrue(result.contains("$l1")); } @Test @@ -91,8 +91,8 @@ public void FunctionTaint() { JimpleIFDSSolver> analysis = executeStaticAnalysis("FunctionTaint"); Set result = getResultsAtLastStatement(analysis); - assertTrue(result.contains("l1")); - assertTrue(result.contains("l2")); + assertTrue(result.contains("$l1")); + assertTrue(result.contains("$l2")); } @Test @@ -100,8 +100,8 @@ public void FunctionTaintPropagated() { JimpleIFDSSolver> analysis = executeStaticAnalysis("FunctionTaintPropagated"); Set result = getResultsAtLastStatement(analysis); - assertTrue(result + " is missing an element.", result.contains("l1")); - assertTrue(result + " is missing an element.", result.contains("l2")); + assertTrue(result + " is missing an element.", result.contains("$l1")); + assertTrue(result + " is missing an element.", result.contains("$l2")); } @Test @@ -109,7 +109,7 @@ public void FunctionTaintSanitized() { JimpleIFDSSolver> analysis = executeStaticAnalysis("FunctionTaintSanitized"); Set result = getResultsAtLastStatement(analysis); - assertTrue(result.contains("l1")); - assertFalse(result.contains("l2")); + assertTrue(result.contains("$l1")); + assertFalse(result.contains("$l2")); } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1577.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1577.java index 5e76875a2fa..a9b04faa65c 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1577.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1577.java @@ -1,8 +1,12 @@ package sootup.java.bytecode; +import categories.Java8Test; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; +import org.junit.experimental.categories.Category; import sootup.core.inputlocation.AnalysisInputLocation; +import sootup.core.inputlocation.EmptyClassLoadingOptions; import sootup.core.model.SootMethod; import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; import sootup.java.core.JavaProject; @@ -10,10 +14,12 @@ import sootup.java.core.language.JavaLanguage; import sootup.java.core.views.JavaView; +@Category(Java8Test.class) public class Soot1577 { final String directory = "../shared-test-resources/soot-1577/"; @Test + @Ignore("FIXME") public void test() { AnalysisInputLocation inputLocation = new JavaClassPathAnalysisInputLocation(directory); @@ -22,6 +28,7 @@ public void test() { JavaProject.builder(new JavaLanguage(7)).addInputLocation(inputLocation).build(); JavaView view = project.createView(); + view.configBodyInterceptors((ail) -> EmptyClassLoadingOptions.Default); Assert.assertEquals(1, view.getClasses().size()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1580.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1580.java index 9cecd0aa910..0d5555d72ec 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1580.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/Soot1580.java @@ -1,9 +1,11 @@ package sootup.java.bytecode; +import categories.Java8Test; import java.nio.file.Path; import java.nio.file.Paths; import org.junit.Assert; import org.junit.Test; +import org.junit.experimental.categories.Category; import sootup.core.inputlocation.AnalysisInputLocation; import sootup.core.model.SootMethod; import sootup.core.types.ClassType; @@ -15,6 +17,7 @@ import sootup.java.core.language.JavaLanguage; import sootup.java.core.views.JavaView; +@Category(Java8Test.class) public class Soot1580 { final Path jar = Paths.get("../shared-test-resources/soot-1580/jpush-android_v3.0.5.jar"); From cf724dbabd2fef6e13fec8bbddbe25e9a5e9d163 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 13:02:18 +0200 Subject: [PATCH 33/54] improve `removeEdge` - doesn't always remove all edges when removing at the tail - returns a boolean indicating if an edge was removed --- .../sootup/core/graph/MutableBasicBlock.java | 8 +- .../core/graph/MutableBlockStmtGraph.java | 81 ++++++------------- .../sootup/core/graph/MutableStmtGraph.java | 8 +- .../core/graph/MutableBlockStmtGraphTest.java | 18 ++--- 4 files changed, 44 insertions(+), 71 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java index 3a06ccdab49..5169dc306f7 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java @@ -82,12 +82,12 @@ public void addSuccessorBlock(@Nonnull MutableBasicBlock block) { successorBlocks.add(block); } - public void removePredecessorBlock(@Nonnull MutableBasicBlock b) { - predecessorBlocks.remove(b); + public boolean removePredecessorBlock(@Nonnull MutableBasicBlock b) { + return predecessorBlocks.remove(b); } - public void removeSuccessorBlock(@Nonnull MutableBasicBlock b) { - successorBlocks.remove(b); + public boolean removeSuccessorBlock(@Nonnull MutableBasicBlock b) { + return successorBlocks.remove(b); } public void addExceptionalSuccessorBlock(@Nonnull ClassType exception, MutableBasicBlock b) { diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 45dbcdebdeb..1468422970c 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -965,20 +965,15 @@ private void linkBlocks(@Nonnull MutableBasicBlock blockA, @Nonnull MutableBasic } @Override - public void removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { - // FIXME: how to handle "partial" removals of targets of flows starting from a Branching Stmt.. - // e.g. because one of the targets are removed.. that changes the whole logic there.. - + public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { MutableBasicBlock blockOfFrom = stmtToBlock.get(from); MutableBasicBlock blockOfTo = stmtToBlock.get(to); if (blockOfFrom == null || blockOfTo == null) { // one of the Stmts is not existing anymore in this graph - so neither a connection. - return; + return false; } - removeBlockBorderEdgesInternal(from, blockOfFrom); - // divide block if from and to are from the same block if (blockOfFrom == blockOfTo) { // divide block and don't link them @@ -992,61 +987,35 @@ public void removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { blockOfFrom.clearSuccessorBlocks(); blocks.add(newBlock); newBlock.getStmts().forEach(s -> stmtToBlock.put(s, newBlock)); + return true; } else { - // throw new IllegalArgumentException("Can't seperate the flow from '"+from+"' to '"+to+"'. - // The Stmts are not connected in this graph!"); + // `from` and `to` are not successive statements in the block + return false; } - } - } + } else { + // `from` and `to` are part of different blocks - protected void removeBlockBorderEdgesInternal( - @Nonnull Stmt from, @Nonnull MutableBasicBlock blockOfFrom) { - // TODO: is it intuitive to remove connections to the BasicBlock in the case we cant merge the - // blocks? - // TODO: reuse tryMerge*Block? - - // add BlockB to BlockA if blockA has no branchingstmt as tail && same traps - if (!blockOfFrom.getStmts().isEmpty() && from == blockOfFrom.getTail()) { - if (blockOfFrom.getPredecessors().size() == 1) { - MutableBasicBlock singlePreviousBlock = blockOfFrom.getPredecessors().get(0); - if (!singlePreviousBlock.getTail().branches() && singlePreviousBlock != blockOfFrom) { - if (singlePreviousBlock - .getExceptionalSuccessors() - .equals(blockOfFrom.getExceptionalSuccessors())) { - blockOfFrom - .getStmts() - .forEach( - k -> { - addNodeToBlock(blockOfFrom, k); - }); - return; - } - } + if (blockOfFrom.getTail() != from || blockOfTo.getHead() != to) { + // `from` and `to` aren't the tail and head of their respective blocks, + // which means they aren't connected + return false; } - // remove outgoing connections from blockA if from stmt is the tail - if (!from.branches()) { - if (!blockOfFrom.getStmts().isEmpty() && blockOfFrom.getSuccessors().size() == 1) { - // merge previous block if possible i.e. no branchingstmt as tail && same traps && no - // other predesccorblocks - MutableBasicBlock singleSuccessorBlock = blockOfFrom.getSuccessors().get(0); - if (singleSuccessorBlock.getPredecessors().size() == 1 - && singleSuccessorBlock.getPredecessors().get(0) == blockOfFrom) { - if (singleSuccessorBlock - .getExceptionalSuccessors() - .equals(blockOfFrom.getExceptionalSuccessors())) { - singleSuccessorBlock - .getStmts() - .forEach( - k -> { - addNodeToBlock(blockOfFrom, k); - }); - } - } - } - } else { - blockOfFrom.clearSuccessorBlocks(); + // remove the connection between the two blocks + boolean predecessorRemoved = blockOfTo.removePredecessorBlock(blockOfFrom); + boolean successorRemoved = blockOfFrom.removeSuccessorBlock(blockOfTo); + assert predecessorRemoved == successorRemoved; + + if (!predecessorRemoved) { + // the blocks weren't connected + return false; } + + // the removal of the edge between `from` and `to` might have created blocks that can be merged + tryMergeWithPredecessorBlock(blockOfTo); + tryMergeWithSuccessorBlock(blockOfFrom); + + return true; } } diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java index 278b27a6143..72263177378 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java @@ -96,8 +96,12 @@ public void setEdges(@Nonnull Stmt from, @Nonnull Stmt... targets) { setEdges(from, Arrays.asList(targets)); } - /** removes the current outgoing flows of "from" to "targets" */ - public abstract void removeEdge(@Nonnull Stmt from, @Nonnull Stmt to); + /** + * removes the current outgoing flows of "from" to "to" + * + * @return true if the edge existed and was removed; false if the edge didn't exist + */ + public abstract boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to); /** Modifications of exceptional flows removes all exceptional flows from "stmt" */ public abstract void clearExceptionalEdges(@Nonnull Stmt stmt); diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index b74b27102cb..07f40952e80 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -256,11 +256,11 @@ public void modifyStmtToBlockAtTail() { assertEquals(1, graph.getBlocksSorted().get(1).getSuccessors().size()); // remove non-existing edge - graph.removeEdge(firstNop, conditionalStmt); + assertFalse(graph.removeEdge(firstNop, conditionalStmt)); assertEquals(2, graph.getBlocksSorted().size()); // remove branchingstmt at end -> edge across blocks - graph.removeEdge(conditionalStmt, firstNop); + assertTrue(graph.removeEdge(conditionalStmt, firstNop)); assertEquals(2, graph.getBlocksSorted().size()); assertEquals(4, graph.getNodes().size()); @@ -268,7 +268,7 @@ public void modifyStmtToBlockAtTail() { assertEquals(3, graph.getNodes().size()); // remove branchingstmt at head - graph.removeEdge(conditionalStmt, secondNop); + assertTrue(graph.removeEdge(conditionalStmt, secondNop)); assertEquals(1, graph.getBlocksSorted().size()); assertEquals(3, graph.getNodes().size()); @@ -402,22 +402,22 @@ public void linkDirectlyAddedBlocks() { assertEquals(1, graph.successors(firstNop).size()); assertEquals(1, graph.successors(secondNop).size()); - graph.removeEdge(secondNop, thirdNop); + assertTrue(graph.removeEdge(secondNop, thirdNop)); assertEquals(2, graph.getBlocksSorted().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); - graph.removeEdge(secondNop, thirdNop); // empty operation + assertFalse(graph.removeEdge(secondNop, thirdNop)); // empty operation assertEquals(2, graph.getBlocksSorted().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); - graph.removeEdge(firstNop, thirdNop); // empty operation + assertFalse(graph.removeEdge(firstNop, thirdNop)); // empty operation assertEquals(2, graph.getBlocksSorted().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); - graph.removeEdge(firstNop, secondNop); + assertTrue(graph.removeEdge(firstNop, secondNop)); assertEquals(3, graph.getBlocksSorted().size()); } @@ -929,7 +929,7 @@ public void removeEdge() { assertEquals(1, graph.successors(stmt1).size()); assertTrue(graph.hasEdgeConnecting(stmt1, stmt2)); - graph.removeEdge(stmt1, stmt2); + assertTrue(graph.removeEdge(stmt1, stmt2)); assertEquals(0, graph.successors(stmt1).size()); assertFalse(graph.hasEdgeConnecting(stmt1, stmt2)); } @@ -960,7 +960,7 @@ public void removeImpossibleEdge() { Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); // nodes are not in the graph! - graph.removeEdge(stmt1, stmt2); + assertFalse(graph.removeEdge(stmt1, stmt2)); } @Test From bec430e49bf90508faa75245c48db3fb9fa06e69 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 13:41:08 +0200 Subject: [PATCH 34/54] re-enable `modifyStmtToBlockAtTail` test and fix removing edges of loops --- .../core/graph/MutableBlockStmtGraph.java | 53 ++++++++++--------- .../core/graph/MutableBlockStmtGraphTest.java | 13 ++--- 2 files changed, 35 insertions(+), 31 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 1468422970c..77dc3437b23 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -974,8 +974,30 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { return false; } - // divide block if from and to are from the same block - if (blockOfFrom == blockOfTo) { + if (blockOfFrom.getTail() == from && blockOfTo.getHead() == to) { + // `from` and `to` are the tail and head of their respective blocks, + // meaning they either connect different blocks, + // or are a loop of the same block + + // remove the connection between the blocks + boolean predecessorRemoved = blockOfTo.removePredecessorBlock(blockOfFrom); + boolean successorRemoved = blockOfFrom.removeSuccessorBlock(blockOfTo); + assert predecessorRemoved == successorRemoved; + + if (!predecessorRemoved) { + // the blocks weren't connected + return false; + } + + // the removal of the edge between `from` and `to` might have created blocks that can be merged + tryMergeWithPredecessorBlock(blockOfTo); + tryMergeWithSuccessorBlock(blockOfFrom); + + return true; + } else if (blockOfFrom == blockOfTo) { + // `from` and `to` are part of the same block but aren't the tail and head, + // which means they are "inner" statements in the block and the block needs to be divided + // divide block and don't link them final List stmtsOfBlock = blockOfFrom.getStmts(); int toIdx = stmtsOfBlock.indexOf(from) + 1; @@ -993,29 +1015,10 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { return false; } } else { - // `from` and `to` are part of different blocks - - if (blockOfFrom.getTail() != from || blockOfTo.getHead() != to) { - // `from` and `to` aren't the tail and head of their respective blocks, - // which means they aren't connected - return false; - } - - // remove the connection between the two blocks - boolean predecessorRemoved = blockOfTo.removePredecessorBlock(blockOfFrom); - boolean successorRemoved = blockOfFrom.removeSuccessorBlock(blockOfTo); - assert predecessorRemoved == successorRemoved; - - if (!predecessorRemoved) { - // the blocks weren't connected - return false; - } - - // the removal of the edge between `from` and `to` might have created blocks that can be merged - tryMergeWithPredecessorBlock(blockOfTo); - tryMergeWithSuccessorBlock(blockOfFrom); - - return true; + // `from` and `to` are part of different blocks, + // and aren't tail and head of their respective block, + // which means they aren't connected + return false; } } diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 07f40952e80..d0874c49137 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -211,6 +211,7 @@ public void removeStmtConditionalTailBetweenBlocks() { Collections.singletonList(thirdNop).toString(), blocksSorted.get(2).getStmts().toString()); } + @Test public void modifyStmtToBlockAtTail() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); assertEquals(0, graph.getBlocksSorted().size()); @@ -247,13 +248,13 @@ public void modifyStmtToBlockAtTail() { graph.putEdge(conditionalStmt, secondNop); assertEquals(2, graph.getBlocksSorted().size()); - assertEquals(3, graph.getBlocksSorted().get(0).getStmts().size()); - assertEquals(2, graph.getBlocksSorted().get(0).getPredecessors().size()); - assertEquals(2, graph.getBlocksSorted().get(0).getSuccessors().size()); + assertEquals(3, graph.getBlockOf(conditionalStmt).getStmts().size()); + assertEquals(2, graph.getBlockOf(conditionalStmt).getPredecessors().size()); + assertEquals(2, graph.getBlockOf(conditionalStmt).getSuccessors().size()); - assertEquals(1, graph.getBlocksSorted().get(1).getStmts().size()); - assertEquals(1, graph.getBlocksSorted().get(1).getPredecessors().size()); - assertEquals(1, graph.getBlocksSorted().get(1).getSuccessors().size()); + assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); + assertEquals(1, graph.getBlockOf(firstNop).getPredecessors().size()); + assertEquals(1, graph.getBlockOf(firstNop).getSuccessors().size()); // remove non-existing edge assertFalse(graph.removeEdge(firstNop, conditionalStmt)); From 3c3675308402fe2185363ccd39e2160efc08d3f0 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 13:52:33 +0200 Subject: [PATCH 35/54] add a test for block merging in `removeEdge` --- .../core/graph/MutableBlockStmtGraphTest.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index d0874c49137..0e8c7824f75 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -278,6 +278,25 @@ public void modifyStmtToBlockAtTail() { assertEquals(1, graph.getBlocksSorted().size()); } + @Test + public void removeEdgeMerge() { + MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); + + graph.addNode(firstNop); + graph.setStartingStmt(firstNop); + graph.putEdge(firstNop, secondNop); + graph.putEdge(secondNop, conditionalStmt); + + assertEquals(1, graph.getBlocks().size()); + // this edge splits the block between the first and second Nop + graph.putEdge(conditionalStmt, secondNop); + assertEquals(2, graph.getBlocks().size()); + + // this edge removal should merge both blocks together again + graph.removeEdge(conditionalStmt, secondNop); + assertEquals(1, graph.getBlocks().size()); + } + @Test public void removeStmtInBetweenBlock() { From 9154b7a6e1d9683e4eb4643c231a010764fb4844 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 14:42:44 +0200 Subject: [PATCH 36/54] fix not setting the predecessor correctly when dividing blocks --- .../main/java/sootup/core/graph/MutableBlockStmtGraph.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 77dc3437b23..c2d87799977 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -1005,7 +1005,11 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { if (toIdx < stmtsOfBlock.size() && stmtsOfBlock.get(toIdx) == to) { MutableBasicBlock newBlock = blockOfFrom.splitBlockUnlinked(from, to); newBlock.copyExceptionalFlowFrom(blockOfFrom); - blockOfFrom.getSuccessors().forEach(newBlock::addSuccessorBlock); + blockOfFrom.getSuccessors().forEach(successor -> { + successor.removePredecessorBlock(blockOfFrom); + newBlock.addSuccessorBlock(successor); + successor.addPredecessorBlock(newBlock); + }); blockOfFrom.clearSuccessorBlocks(); blocks.add(newBlock); newBlock.getStmts().forEach(s -> stmtToBlock.put(s, newBlock)); From 94bfeb381b2ff0f4c83a551d9842639d2e796ce9 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 15:23:45 +0200 Subject: [PATCH 37/54] improve `removeNode` --- .../core/graph/MutableBlockStmtGraph.java | 78 +++++++++++-------- .../core/graph/MutableBlockStmtGraphTest.java | 2 +- 2 files changed, 48 insertions(+), 32 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index c2d87799977..994e4747e4c 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -721,54 +721,70 @@ public void removeNode(@Nonnull Stmt stmt) { removeNode(stmt, true); } + /** + * Removes a Stmt from the StmtGraph. + *

+ * It can optionally keep the flow (edges) of the statement + * by connecting the predecessors of the statement with successors of the statement. + * Keeping the flow does not work when the statement has multiple successors. + * + * @param stmt the Stmt to be removed + * @param keepFlow flag indicating whether to keep the flow or not + * @throws IllegalArgumentException if keepFlow is true but the stmt has multiple successors + */ public void removeNode(@Nonnull Stmt stmt, boolean keepFlow) { - - MutableBasicBlock blockOfRemovedStmt = stmtToBlock.remove(stmt); - if (blockOfRemovedStmt == null) { - throw new IllegalArgumentException("Stmt is not in the StmtGraph!"); + if (keepFlow && successors(stmt).size() > 1) { + // Branching statements can have multiple targets/successors, + // and there is no obvious way to connect the predecessor and successors of the statement. + throw new IllegalArgumentException("can't remove a statement with multiple successors while keeping the flow"); } if (stmt == startingStmt) { startingStmt = null; } - final boolean isHead = blockOfRemovedStmt.getHead() == stmt; - final boolean isTail = blockOfRemovedStmt.getTail() == stmt; - - // do edges from or to this node exist -> remove them? - if (isHead && !keepFlow) { - final MutableBasicBlock finalBlockOfRemovedStmt = blockOfRemovedStmt; - blockOfRemovedStmt - .getPredecessors() - .forEach( - b -> { - b.removeSuccessorBlock(finalBlockOfRemovedStmt); - finalBlockOfRemovedStmt.removePredecessorBlock(b); - }); - blockOfRemovedStmt.clearPredecessorBlocks(); + if (!keepFlow) { + for (Stmt predecessor : predecessors(stmt)) { + removeEdge(predecessor, stmt); + } + for (Stmt successor : successors(stmt)) { + removeEdge(stmt, successor); + } } - if (isTail) { - if (stmt.branches() && !keepFlow) { - blockOfRemovedStmt.clearSuccessorBlocks(); - } + MutableBasicBlock blockOfRemovedStmt = stmtToBlock.remove(stmt); + if (blockOfRemovedStmt == null) { + throw new IllegalArgumentException("Stmt is not in the StmtGraph!"); } - // cleanup or merge blocks if necesssary (stmt itself is not removed from the block yet) if (blockOfRemovedStmt.getStmtCount() > 1) { + // Removing the statement from the block will keep the flow automatically, + // because the flow inside a block is implicit (from one statement to the next) + // and connections between blocks are kept. blockOfRemovedStmt.removeStmt(stmt); - - if (isHead) { - blockOfRemovedStmt = tryMergeWithPredecessorBlock(blockOfRemovedStmt); - } - if (isTail) { - tryMergeWithSuccessorBlock(blockOfRemovedStmt); - } - } else { // cleanup block (i.e. remove!) as its not needed in the graph anymore if it only contains // stmt - which is // now deleted + + if (keepFlow) { + // this is always true because of the check at the start of the method + assert blockOfRemovedStmt.getSuccessors().size() <= 1; + + // connect predecessors to the successor of the statement to keep the flow + if (blockOfRemovedStmt.getSuccessors().size() == 1) { + MutableBasicBlock successor = blockOfRemovedStmt.getSuccessors().get(0); + + for (MutableBasicBlock predecessor : blockOfRemovedStmt.getPredecessors()) { + predecessor.removeSuccessorBlock(blockOfRemovedStmt); + predecessor.addSuccessorBlock(successor); + + successor.removePredecessorBlock(blockOfRemovedStmt); + successor.addPredecessorBlock(predecessor); + } + } + } + blocks.remove(blockOfRemovedStmt); blockOfRemovedStmt.clearPredecessorBlocks(); blockOfRemovedStmt.clearSuccessorBlocks(); diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 0e8c7824f75..1ef370c6560 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -201,7 +201,7 @@ public void removeStmtConditionalTailBetweenBlocks() { graph.setEdges(conditionalStmt, Arrays.asList(secondNop, thirdNop)); assertEquals(3, graph.getBlocksSorted().size()); - graph.removeNode(conditionalStmt); + graph.removeNode(conditionalStmt, false); final List> blocksSorted = graph.getBlocksSorted(); assertEquals( Collections.singletonList(firstNop).toString(), blocksSorted.get(0).getStmts().toString()); From 94ce4059ba1a341e3b6cf10e914965a95a1976ab Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 10 Oct 2023 16:21:35 +0200 Subject: [PATCH 38/54] test for a conditional followed by a block that gets removed by the `DeadAssignmentEliminator` --- .../DeadAssignmentEliminatorTest.java | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java index 51125d4d4de..349052b6237 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java @@ -19,6 +19,60 @@ public class DeadAssignmentEliminatorTest { + /** + *

+   *     void test() {
+   *       if (10 < 20) {
+   *         int a = 42;
+   *       }
+   *       return;
+   *     }
+   * 
+ * + * gets simplified to + * + *
+   *     void test() {
+   *       if (10 < 20) {
+   *       }
+   *       return;
+   *     }
+   * 
+ * + * There used to be a bug that would result in an invalid statement graph because the whole block containing + * `int a = 42;` gets deleted. + */ + @Test + public void conditionalToRemovedBlock() { + StmtPositionInfo noPositionInfo = StmtPositionInfo.createNoStmtPositionInfo(); + + Local a = JavaJimple.newLocal("a", PrimitiveType.getInt()); + Set locals = ImmutableUtils.immutableSet(a); + + Stmt conditional = JavaJimple.newIfStmt(JavaJimple.newLtExpr(IntConstant.getInstance(10), IntConstant.getInstance(20)), noPositionInfo); + Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); + Stmt intToA = JavaJimple.newAssignStmt(a, IntConstant.getInstance(42), noPositionInfo); + + Body.BodyBuilder builder = Body.builder(); + builder.setStartingStmt(conditional); + builder.setMethodSignature( + JavaIdentifierFactory.getInstance() + .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + + builder.setLocals(locals); + builder.setPosition(NoPositionInformation.getInstance()); + + builder.addFlow(conditional, intToA); + builder.addFlow(conditional, ret); + builder.addFlow(intToA, ret); + + Body beforeBody = builder.build(); + new DeadAssignmentEliminator().interceptBody(builder, null); + Body afterBody = builder.build(); + + assertEquals(beforeBody.getStmtGraph().getNodes().size() - 1, afterBody.getStmtGraph().getNodes().size()); + } + @Test public void testRemoveDeadAssignment() { Body.BodyBuilder testBuilder = createBody(false); From 7e8a121dad76be99d78f204a3e56f4c0848acd75 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 17 Oct 2023 14:28:20 +0200 Subject: [PATCH 39/54] fix flaky tests by removing usages of `getBlocksSorted` --- .../core/graph/MutableBlockStmtGraph.java | 3 +- .../core/graph/MutableBlockStmtGraphTest.java | 106 +++++++++--------- 2 files changed, 54 insertions(+), 55 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 994e4747e4c..efc9892d76c 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -391,11 +391,12 @@ public void clearExceptionalEdges(@Nonnull Stmt node) { @Override @Nonnull public Set> getBlocks() { - return blocks.stream().map(ForwardingBasicBlock::new).collect(Collectors.toSet()); + return blocks; } @Nonnull public List> getBlocksSorted() { + // TODO this implementation is incorrect; it doesn't return a consistent order return StreamSupport.stream( Spliterators.spliteratorUnknownSize(blocks.iterator(), Spliterator.ORDERED), false) .map(ForwardingBasicBlock::new) diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 1ef370c6560..a3f505c7919 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -94,22 +94,22 @@ public PackageName getPackageName() { public void addNodeTest() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - assertEquals(0, graph.getBlocksSorted().size()); + assertEquals(0, graph.getBlocks().size()); graph.addNode(firstNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); // test duplicate insertion of the same node graph.addNode(firstNop); - assertEquals(1, graph.getBlocksSorted().size()); - assertEquals(1, graph.getBlocksSorted().get(0).getStmts().size()); + assertEquals(1, graph.getBlocks().size()); + assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); graph.addNode(secondNop); - assertEquals(2, graph.getBlocksSorted().size()); - assertEquals(1, graph.getBlocksSorted().get(1).getStmts().size()); + assertEquals(2, graph.getBlocks().size()); + assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); graph.removeNode(firstNop); - assertEquals(1, graph.getBlocksSorted().size()); - assertEquals(1, graph.getBlocksSorted().get(0).getStmts().size()); + assertEquals(1, graph.getBlocks().size()); + assertEquals(1, graph.getBlockOf(secondNop).getStmts().size()); // removal of not existing try { @@ -117,10 +117,10 @@ public void addNodeTest() { fail("should not be reachable due to exception"); } catch (Exception ignored) { } - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); graph.removeNode(secondNop); - assertEquals(0, graph.getBlocksSorted().size()); + assertEquals(0, graph.getBlocks().size()); } @Test @@ -129,10 +129,10 @@ public void removeStmtBetweenEdges() { graph.setStartingStmt(firstNop); graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); - assertEquals(3, graph.getBlocksSorted().get(0).getStmts().size()); + assertEquals(3, graph.getBlockOf(firstNop).getStmts().size()); graph.removeNode(secondNop); - assertEquals(Arrays.asList(firstNop, thirdNop), graph.getBlocksSorted().get(0).getStmts()); + assertEquals(Arrays.asList(firstNop, thirdNop), graph.getBlockOf(firstNop).getStmts()); } @Test @@ -143,7 +143,7 @@ public void removeStmtTail() { graph.putEdge(secondNop, thirdNop); graph.removeNode(thirdNop); - assertEquals(Arrays.asList(firstNop, secondNop), graph.getBlocksSorted().get(0).getStmts()); + assertEquals(Arrays.asList(firstNop, secondNop), graph.getBlockOf(firstNop).getStmts()); } @Test @@ -156,7 +156,7 @@ public void removeStmtHead() { graph.putEdge(secondNop, thirdNop); graph.removeNode(firstNop); - assertEquals(Arrays.asList(secondNop, thirdNop), graph.getBlocksSorted().get(0).getStmts()); + assertEquals(Arrays.asList(secondNop, thirdNop), graph.getBlockOf(secondNop).getStmts()); } @Test @@ -167,7 +167,7 @@ public void removeStmtConditionalTail() { graph.putEdge(secondNop, conditionalStmt); graph.removeNode(conditionalStmt); - assertEquals(Arrays.asList(firstNop, secondNop), graph.getBlocksSorted().get(0).getStmts()); + assertEquals(Arrays.asList(firstNop, secondNop), graph.getBlockOf(firstNop).getStmts()); } @Test @@ -176,10 +176,10 @@ public void testSetEdges() { graph.setStartingStmt(firstNop); graph.setEdges(firstNop, Collections.singletonList(conditionalStmt)); assertEquals( - Arrays.asList(firstNop, conditionalStmt), graph.getBlocksSorted().get(0).getStmts()); + Arrays.asList(firstNop, conditionalStmt), graph.getBlockOf(firstNop).getStmts()); graph.setEdges(conditionalStmt, Arrays.asList(secondNop, thirdNop)); - assertEquals(3, graph.getBlocksSorted().size()); + assertEquals(3, graph.getBlocks().size()); assertEquals( Arrays.asList(firstNop, conditionalStmt).toString(), @@ -199,54 +199,53 @@ public void removeStmtConditionalTailBetweenBlocks() { graph.putEdge(firstNop, conditionalStmt); graph.setEdges(conditionalStmt, Arrays.asList(secondNop, thirdNop)); - assertEquals(3, graph.getBlocksSorted().size()); + assertEquals(3, graph.getBlocks().size()); graph.removeNode(conditionalStmt, false); - final List> blocksSorted = graph.getBlocksSorted(); assertEquals( - Collections.singletonList(firstNop).toString(), blocksSorted.get(0).getStmts().toString()); + Collections.singletonList(firstNop).toString(), graph.getBlockOf(firstNop).getStmts().toString()); assertEquals( - Collections.singletonList(secondNop).toString(), blocksSorted.get(1).getStmts().toString()); + Collections.singletonList(secondNop).toString(), graph.getBlockOf(secondNop).getStmts().toString()); assertEquals( - Collections.singletonList(thirdNop).toString(), blocksSorted.get(2).getStmts().toString()); + Collections.singletonList(thirdNop).toString(), graph.getBlockOf(thirdNop).getStmts().toString()); } @Test public void modifyStmtToBlockAtTail() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - assertEquals(0, graph.getBlocksSorted().size()); + assertEquals(0, graph.getBlocks().size()); assertEquals(0, graph.getNodes().size()); graph.addNode(firstNop); graph.setStartingStmt(firstNop); assertEquals(1, graph.getNodes().size()); - assertEquals(1, graph.getBlocksSorted().size()); - assertEquals(1, graph.getBlocksSorted().get(0).getStmts().size()); + assertEquals(1, graph.getBlocks().size()); + assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); graph.putEdge(firstNop, secondNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(2, graph.getNodes().size()); graph.putEdge(secondNop, thirdNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(3, graph.getNodes().size()); // insert branchingstmt at end graph.putEdge(thirdNop, conditionalStmt); assertEquals(4, graph.getNodes().size()); - assertEquals(1, graph.getBlocksSorted().size()); - assertEquals(0, graph.getBlocksSorted().get(0).getPredecessors().size()); - assertEquals(0, graph.getBlocksSorted().get(0).getSuccessors().size()); + assertEquals(1, graph.getBlocks().size()); + assertEquals(0, graph.getBlockOf(firstNop).getPredecessors().size()); + assertEquals(0, graph.getBlockOf(firstNop).getSuccessors().size()); // add connection between branchingstmt and first stmt graph.putEdge(conditionalStmt, firstNop); - assertEquals(1, graph.getBlocksSorted().size()); - assertEquals(1, graph.getBlocksSorted().get(0).getPredecessors().size()); - assertEquals(1, graph.getBlocksSorted().get(0).getSuccessors().size()); + assertEquals(1, graph.getBlocks().size()); + assertEquals(1, graph.getBlockOf(firstNop).getPredecessors().size()); + assertEquals(1, graph.getBlockOf(firstNop).getSuccessors().size()); // add connection between branchingstmt and second stmt graph.putEdge(conditionalStmt, secondNop); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(3, graph.getBlockOf(conditionalStmt).getStmts().size()); assertEquals(2, graph.getBlockOf(conditionalStmt).getPredecessors().size()); @@ -258,11 +257,11 @@ public void modifyStmtToBlockAtTail() { // remove non-existing edge assertFalse(graph.removeEdge(firstNop, conditionalStmt)); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); // remove branchingstmt at end -> edge across blocks assertTrue(graph.removeEdge(conditionalStmt, firstNop)); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(4, graph.getNodes().size()); graph.removeNode(firstNop); @@ -270,12 +269,12 @@ public void modifyStmtToBlockAtTail() { // remove branchingstmt at head assertTrue(graph.removeEdge(conditionalStmt, secondNop)); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(3, graph.getNodes().size()); graph.removeNode(secondNop); assertEquals(2, graph.getNodes().size()); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); } @Test @@ -299,13 +298,12 @@ public void removeEdgeMerge() { @Test public void removeStmtInBetweenBlock() { - MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); graph.removeNode(secondNop); - assertEquals(graph.getBlocksSorted().get(0).getStmts(), Arrays.asList(firstNop, thirdNop)); + assertEquals(graph.getBlockOf(firstNop).getStmts(), Arrays.asList(firstNop, thirdNop)); } @Test @@ -360,7 +358,7 @@ public void addSameSuccessorMultipleTimes() { graph.putEdge(conditionalStmt, secondNop); graph.putEdge(conditionalStmt, secondNop); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(0, graph.outDegree(secondNop)); assertEquals(2, graph.inDegree(secondNop)); @@ -378,13 +376,13 @@ public void addBlocks() { graph.putEdge(conditionalStmt, secondNop); graph.putEdge(conditionalStmt, thirdNop); - assertEquals(3, graph.getBlocksSorted().size()); + assertEquals(3, graph.getBlocks().size()); } @Test public void addBlockDirectly() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - assertEquals(0, graph.getBlocksSorted().size()); + assertEquals(0, graph.getBlocks().size()); MutableBasicBlock blockA = new MutableBasicBlock(); blockA.addStmt(firstNop); @@ -394,10 +392,10 @@ public void addBlockDirectly() { blockC.addStmt(thirdNop); graph.addBlock(blockA.getStmts(), Collections.emptyMap()); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); graph.addBlock(blockB.getStmts(), Collections.emptyMap()); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); } @Test @@ -418,27 +416,27 @@ public void linkDirectlyAddedBlocks() { graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(1, graph.successors(secondNop).size()); assertTrue(graph.removeEdge(secondNop, thirdNop)); - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); assertFalse(graph.removeEdge(secondNop, thirdNop)); // empty operation - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); assertFalse(graph.removeEdge(firstNop, thirdNop)); // empty operation - assertEquals(2, graph.getBlocksSorted().size()); + assertEquals(2, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); assertTrue(graph.removeEdge(firstNop, secondNop)); - assertEquals(3, graph.getBlocksSorted().size()); + assertEquals(3, graph.getBlocks().size()); } @Test @@ -447,7 +445,7 @@ public void testRemoveNodeAtBeginning() { graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); graph.removeNode(firstNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.successors(secondNop).size()); assertEquals(0, graph.successors(thirdNop).size()); assertEquals(0, graph.predecessors(secondNop).size()); @@ -460,7 +458,7 @@ public void testRemoveNodeAtBeginning() { graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); graph.removeNode(secondNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(thirdNop).size()); assertEquals(0, graph.predecessors(firstNop).size()); @@ -473,7 +471,7 @@ public void testRemoveNodeAtEnd() { graph.putEdge(firstNop, secondNop); graph.putEdge(secondNop, thirdNop); graph.removeNode(thirdNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.successors(firstNop).size()); assertEquals(0, graph.successors(secondNop).size()); assertEquals(0, graph.predecessors(firstNop).size()); @@ -533,7 +531,7 @@ public void modifyTrapToCompleteBlock() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(firstNop, secondNop); - assertEquals(1, graph.getBlocksSorted().size()); + assertEquals(1, graph.getBlocks().size()); // graph.addTrap(throwableSig, secondNop, secondNop, firstHandlerStmt); } From 2aa4b2bea8b53c01cb83378b012de485767a6a09 Mon Sep 17 00:00:00 2001 From: Tim Balsfulland Date: Tue, 17 Oct 2023 14:30:51 +0200 Subject: [PATCH 40/54] apply formatting --- .../core/graph/MutableBlockStmtGraph.java | 25 +++++++++++-------- .../core/graph/MutableBlockStmtGraphTest.java | 12 +++++---- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index efc9892d76c..a35763b2cb1 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -724,9 +724,9 @@ public void removeNode(@Nonnull Stmt stmt) { /** * Removes a Stmt from the StmtGraph. - *

- * It can optionally keep the flow (edges) of the statement - * by connecting the predecessors of the statement with successors of the statement. + * + *

It can optionally keep the flow (edges) of the statement by connecting the predecessors of + * the statement with successors of the statement. * Keeping the flow does not work when the statement has multiple successors. * * @param stmt the Stmt to be removed @@ -737,7 +737,8 @@ public void removeNode(@Nonnull Stmt stmt, boolean keepFlow) { if (keepFlow && successors(stmt).size() > 1) { // Branching statements can have multiple targets/successors, // and there is no obvious way to connect the predecessor and successors of the statement. - throw new IllegalArgumentException("can't remove a statement with multiple successors while keeping the flow"); + throw new IllegalArgumentException( + "can't remove a statement with multiple successors while keeping the flow"); } if (stmt == startingStmt) { @@ -1006,7 +1007,8 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { return false; } - // the removal of the edge between `from` and `to` might have created blocks that can be merged + // the removal of the edge between `from` and `to` might have created blocks that can be + // merged tryMergeWithPredecessorBlock(blockOfTo); tryMergeWithSuccessorBlock(blockOfFrom); @@ -1022,11 +1024,14 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { if (toIdx < stmtsOfBlock.size() && stmtsOfBlock.get(toIdx) == to) { MutableBasicBlock newBlock = blockOfFrom.splitBlockUnlinked(from, to); newBlock.copyExceptionalFlowFrom(blockOfFrom); - blockOfFrom.getSuccessors().forEach(successor -> { - successor.removePredecessorBlock(blockOfFrom); - newBlock.addSuccessorBlock(successor); - successor.addPredecessorBlock(newBlock); - }); + blockOfFrom + .getSuccessors() + .forEach( + successor -> { + successor.removePredecessorBlock(blockOfFrom); + newBlock.addSuccessorBlock(successor); + successor.addPredecessorBlock(newBlock); + }); blockOfFrom.clearSuccessorBlocks(); blocks.add(newBlock); newBlock.getStmts().forEach(s -> stmtToBlock.put(s, newBlock)); diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index a3f505c7919..1764c6147be 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -175,8 +175,7 @@ public void testSetEdges() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.setStartingStmt(firstNop); graph.setEdges(firstNop, Collections.singletonList(conditionalStmt)); - assertEquals( - Arrays.asList(firstNop, conditionalStmt), graph.getBlockOf(firstNop).getStmts()); + assertEquals(Arrays.asList(firstNop, conditionalStmt), graph.getBlockOf(firstNop).getStmts()); graph.setEdges(conditionalStmt, Arrays.asList(secondNop, thirdNop)); assertEquals(3, graph.getBlocks().size()); @@ -203,11 +202,14 @@ public void removeStmtConditionalTailBetweenBlocks() { graph.removeNode(conditionalStmt, false); assertEquals( - Collections.singletonList(firstNop).toString(), graph.getBlockOf(firstNop).getStmts().toString()); + Collections.singletonList(firstNop).toString(), + graph.getBlockOf(firstNop).getStmts().toString()); assertEquals( - Collections.singletonList(secondNop).toString(), graph.getBlockOf(secondNop).getStmts().toString()); + Collections.singletonList(secondNop).toString(), + graph.getBlockOf(secondNop).getStmts().toString()); assertEquals( - Collections.singletonList(thirdNop).toString(), graph.getBlockOf(thirdNop).getStmts().toString()); + Collections.singletonList(thirdNop).toString(), + graph.getBlockOf(thirdNop).getStmts().toString()); } @Test From f95383705f32513d719a74abb59ee0e0ad70540a Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 18 Oct 2023 11:10:40 +0200 Subject: [PATCH 41/54] style --- .../core/graph/MutableBlockStmtGraph.java | 4 ++-- .../bytecode/interceptors/AggregatorTest.java | 11 ++++++++--- .../DeadAssignmentEliminatorTest.java | 19 +++++++++++++------ 3 files changed, 23 insertions(+), 11 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index a35763b2cb1..24736edee64 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -726,8 +726,8 @@ public void removeNode(@Nonnull Stmt stmt) { * Removes a Stmt from the StmtGraph. * *

It can optionally keep the flow (edges) of the statement by connecting the predecessors of - * the statement with successors of the statement. - * Keeping the flow does not work when the statement has multiple successors. + * the statement with successors of the statement. Keeping the flow does not work when the + * statement has multiple successors. * * @param stmt the Stmt to be removed * @param keepFlow flag indicating whether to keep the flow or not diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 19ffcedfcd8..3605c3c457d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -92,7 +92,12 @@ public void noAggregationWithUse() { Stmt assignA = JavaJimple.newAssignStmt(a, JavaJimple.newNewExpr(fileType), noPositionInfo); // this use of `a` should prevent the aggregator from changing anything - Stmt useA = JavaJimple.newInvokeStmt(Jimple.newSpecialInvokeExpr(a, JavaIdentifierFactory.getInstance().parseMethodSignature("()>")), noPositionInfo); + Stmt useA = + JavaJimple.newInvokeStmt( + Jimple.newSpecialInvokeExpr( + a, + JavaIdentifierFactory.getInstance().parseMethodSignature("()>")), + noPositionInfo); Stmt assignB = JavaJimple.newAssignStmt(b, a, noPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); @@ -102,8 +107,8 @@ public void noAggregationWithUse() { builder.addFlow(assignB, ret); builder.setMethodSignature( - JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + JavaIdentifierFactory.getInstance() + .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); new Aggregator().interceptBody(builder, null); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java index 349052b6237..9b9f864492d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java @@ -20,6 +20,8 @@ public class DeadAssignmentEliminatorTest { /** + * + * *

    *     void test() {
    *       if (10 < 20) {
@@ -39,8 +41,8 @@ public class DeadAssignmentEliminatorTest {
    *     }
    * 
* - * There used to be a bug that would result in an invalid statement graph because the whole block containing - * `int a = 42;` gets deleted. + * There used to be a bug that would result in an invalid statement graph because the whole block + * containing `int a = 42;` gets deleted. */ @Test public void conditionalToRemovedBlock() { @@ -49,15 +51,18 @@ public void conditionalToRemovedBlock() { Local a = JavaJimple.newLocal("a", PrimitiveType.getInt()); Set locals = ImmutableUtils.immutableSet(a); - Stmt conditional = JavaJimple.newIfStmt(JavaJimple.newLtExpr(IntConstant.getInstance(10), IntConstant.getInstance(20)), noPositionInfo); + Stmt conditional = + JavaJimple.newIfStmt( + JavaJimple.newLtExpr(IntConstant.getInstance(10), IntConstant.getInstance(20)), + noPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); Stmt intToA = JavaJimple.newAssignStmt(a, IntConstant.getInstance(42), noPositionInfo); Body.BodyBuilder builder = Body.builder(); builder.setStartingStmt(conditional); builder.setMethodSignature( - JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + JavaIdentifierFactory.getInstance() + .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); builder.setLocals(locals); builder.setPosition(NoPositionInformation.getInstance()); @@ -70,7 +75,9 @@ public void conditionalToRemovedBlock() { new DeadAssignmentEliminator().interceptBody(builder, null); Body afterBody = builder.build(); - assertEquals(beforeBody.getStmtGraph().getNodes().size() - 1, afterBody.getStmtGraph().getNodes().size()); + assertEquals( + beforeBody.getStmtGraph().getNodes().size() - 1, + afterBody.getStmtGraph().getNodes().size()); } @Test From 596b6dd1247b9433706dbc23ef95eeaabfa6d25b Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 18 Oct 2023 11:52:46 +0200 Subject: [PATCH 42/54] fix Iterator in getBlocksSorted() --- .../main/java/sootup/core/graph/MutableBlockStmtGraph.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 24736edee64..0e86c2d5e53 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -396,10 +396,8 @@ public Set> getBlocks() { @Nonnull public List> getBlocksSorted() { - // TODO this implementation is incorrect; it doesn't return a consistent order return StreamSupport.stream( - Spliterators.spliteratorUnknownSize(blocks.iterator(), Spliterator.ORDERED), false) - .map(ForwardingBasicBlock::new) + Spliterators.spliteratorUnknownSize(getBlockIterator(), Spliterator.ORDERED), false) .collect(Collectors.toList()); } From 3795d358696fe29a0e6bfe7147f130574dd4142a Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 20 Oct 2023 11:12:15 +0200 Subject: [PATCH 43/54] wip: fixing Stmt edge connection modification.. theres a problem with non-FallsThroughStmt.. now: giving the modifier more power over branching successors --- .../sootup/core/graph/MutableBasicBlock.java | 124 ++++++++++++++---- .../core/graph/MutableBlockStmtGraph.java | 83 +++++++----- .../sootup/core/graph/MutableStmtGraph.java | 9 +- .../core/jimple/common/stmt/JIfStmt.java | 3 + .../src/main/java/sootup/core/model/Body.java | 11 +- .../core/graph/MutableBlockStmtGraphTest.java | 106 +++++++-------- .../bytecode/frontend/AsmMethodSource.java | 2 +- .../interceptors/ConditionalBranchFolder.java | 11 +- .../StaticSingleAssignmentFormer.java | 13 +- .../typeresolving/CastCounter.java | 6 +- .../interceptors/NopEliminatorTest.java | 12 +- 11 files changed, 229 insertions(+), 151 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java index 5169dc306f7..245a33dbd47 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java @@ -1,7 +1,10 @@ package sootup.core.graph; import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.types.ClassType; @@ -29,8 +32,8 @@ */ public class MutableBasicBlock implements BasicBlock { - @Nonnull private final List predecessorBlocks = new ArrayList<>(); - @Nonnull private final List successorBlocks = new ArrayList<>(); + @Nonnull private final ArrayList predecessorBlocks = new ArrayList<>(); + private MutableBasicBlock[] successorBlocks = null; @Nonnull private final Map exceptionalSuccessorBlocks; @@ -54,16 +57,22 @@ public boolean equals(Object o) { return super.equals(o); } - public void addStmt(@Nonnull Stmt stmt) { - if (getStmtCount() > 0 && getTail() instanceof BranchingStmt) { + public void addStmt(@Nonnull Stmt newStmt) { + final Stmt tail = getTail(); + if (getStmtCount() > 0 && tail instanceof BranchingStmt) { throw new IllegalArgumentException( "Can't add another Stmt to a Block after a BranchingStmt."); } - stmts.add(stmt); + stmts.add(newStmt); + updateSuccessorContainer(newStmt); } public void removeStmt(@Nonnull Stmt stmt) { - stmts.remove(stmt); + final int idx = stmts.indexOf(stmt); + stmts.remove(idx); + if (idx == stmts.size() - 1) { + updateSuccessorContainer(getTail()); + } } public void replaceStmt(Stmt oldStmt, Stmt newStmt) { @@ -72,22 +81,56 @@ public void replaceStmt(Stmt oldStmt, Stmt newStmt) { throw new IllegalArgumentException("oldStmt does not exist in this Block!"); } stmts.set(idx, newStmt); + // did the last stmt change? + if (idx == stmts.size() - 1) { + updateSuccessorContainer(newStmt); + } + } + + protected void updateSuccessorContainer(@Nonnull Stmt newStmt) { + // we are not keeping/copying the currently stored flows as they are associated with a specific + // stmt + final int expectedSuccessorCount = newStmt.getExpectedSuccessorCount(); + if (expectedSuccessorCount != successorBlocks.length) { + // will not happen that often as only the last item can have more (or less) than one successor + // - n-1 items must have 1 successor as they are FallsThrough + successorBlocks = new MutableBasicBlock[expectedSuccessorCount]; + } } public void addPredecessorBlock(@Nonnull MutableBasicBlock block) { predecessorBlocks.add(block); } - public void addSuccessorBlock(@Nonnull MutableBasicBlock block) { - successorBlocks.add(block); + public boolean setSuccessorBlock(int successorIdx, @Nullable MutableBasicBlock block) { + if (successorBlocks == null) { + successorBlocks = new MutableBasicBlock[block.getTail().getExpectedSuccessorCount()]; + } + if (successorIdx >= successorBlocks.length) { + throw new IndexOutOfBoundsException( + "successorIdx '" + + successorIdx + + "' is out of bounds ('" + + successorBlocks.length + + "')"); + } + successorBlocks[successorIdx] = block; + return true; } public boolean removePredecessorBlock(@Nonnull MutableBasicBlock b) { + if (successorBlocks == null) { + return false; + } return predecessorBlocks.remove(b); } - public boolean removeSuccessorBlock(@Nonnull MutableBasicBlock b) { - return successorBlocks.remove(b); + private void removeAllFromSuccessorBlock(@Nonnull MutableBasicBlock b) { + for (int i = 0; i < successorBlocks.length; i++) { + if (successorBlocks[i] == b) { + successorBlocks[i] = null; + } + } } public void addExceptionalSuccessorBlock(@Nonnull ClassType exception, MutableBasicBlock b) { @@ -124,7 +167,7 @@ public List getPredecessors() { @Nonnull @Override public List getSuccessors() { - return Collections.unmodifiableList(successorBlocks); + return Arrays.stream(successorBlocks).filter(Objects::isNull).collect(Collectors.toList()); } @Override @@ -163,7 +206,7 @@ public int getStmtCount() { @Nonnull @Override public Stmt getHead() { - if (stmts.size() < 1) { + if (stmts.isEmpty()) { throw new IndexOutOfBoundsException("Cant get the head - this Block has no assigned Stmts."); } return stmts.get(0); @@ -220,7 +263,7 @@ protected MutableBasicBlock splitBlockUnlinked(int splitIdx) { } /** - * splits a BasicBlock into first|second + * splits a BasicBlock into first|second we know splitStmt must be a FallsThroughStmt * * @param shouldBeNewHead if true: splitStmt is the Head of the second BasicBlock. if * shouldBeNewHead is false splitStmt is the tail of the first BasicBlock @@ -240,18 +283,20 @@ public MutableBasicBlock splitBlockLinked(@Nonnull Stmt splitStmt, boolean shoul } MutableBasicBlock newBlock = splitBlockUnlinked(splitIdx); - successorBlocks.forEach( - succBlock -> { - // copy successors to the newBlock - newBlock.addSuccessorBlock(succBlock); - // and relink predecessors of the successors to newblock as well - succBlock.removePredecessorBlock(this); - succBlock.addPredecessorBlock(newBlock); - }); - successorBlocks.clear(); + for (int i = 0; i < successorBlocks.length; i++) { + MutableBasicBlock succBlock = successorBlocks[i]; // copy successors to the newBlock + newBlock.setSuccessorBlock(i, succBlock); + // and relink predecessors of the successors to newblock as well + succBlock.removePredecessorBlock(this); + succBlock.addPredecessorBlock(newBlock); + } + successorBlocks = null; newBlock.addPredecessorBlock(this); - addSuccessorBlock(newBlock); + setSuccessorBlock( + 0, + newBlock); // 0 as this can only be a block if the Stmts before the last Stmt are + // FallsThroughStmt return newBlock; } @@ -263,8 +308,8 @@ public void copyExceptionalFlowFrom(MutableBasicBlock sourceBlock) { } public void clearSuccessorBlocks() { - successorBlocks.forEach(b -> b.removePredecessorBlock(this)); - successorBlocks.clear(); + Stream.of(successorBlocks).forEach(b -> b.removePredecessorBlock(this)); + successorBlocks = null; } public void clearExceptionalSuccessorBlocks() { @@ -276,7 +321,7 @@ public void clearPredecessorBlocks() { Map> toRemove = new HashMap<>(); predecessorBlocks.forEach( pb -> { - pb.removeSuccessorBlock(this); + pb.removeAllFromSuccessorBlock(this); toRemove.put(pb, pb.collectExceptionalSuccessorBlocks(this)); }); @@ -294,6 +339,33 @@ public void clearPredecessorBlocks() { public String toString() { return "Block " + getStmts(); } + + /** set newBlock to null to unset.. */ + public boolean replaceSuccessorBlock( + @Nonnull MutableBasicBlock oldBlock, @Nullable MutableBasicBlock newBlock) { + boolean found = false; + for (int i = 0; i < successorBlocks.length; i++) { + if (successorBlocks[i] == oldBlock) { + successorBlocks[i] = newBlock; + found = true; + } + } + return found; + } + + public boolean replacePredecessorBlock(MutableBasicBlock oldBlock, MutableBasicBlock newBlock) { + boolean found = false; + + for (ListIterator iterator = predecessorBlocks.listIterator(); + iterator.hasNext(); ) { + MutableBasicBlock predecessorBlock = iterator.next(); + if (predecessorBlock == oldBlock) { + iterator.set(newBlock); + found = true; + } + } + return found; + } } /* diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 0e86c2d5e53..6b2004a13f9 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -14,6 +14,7 @@ import sootup.core.jimple.basic.Trap; import sootup.core.jimple.common.ref.JCaughtExceptionRef; import sootup.core.jimple.common.stmt.BranchingStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JIdentityStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.signatures.MethodSignature; @@ -217,7 +218,7 @@ public void initializeWith( if (stmt.fallsThrough()) { // hint: possible bad performance if stmts is not instanceof RandomAccess - putEdge(stmt, stmts.get(i + 1)); + putEdge(stmt, 0, stmts.get(i + 1)); } if (stmt instanceof BranchingStmt) { @@ -243,9 +244,10 @@ public void initializeWith( + "."); } - for (Stmt target : targets) { + for (int j = 0; j < targets.size(); j++) { + Stmt target = targets.get(j); // a possible fallsthrough (i.e. from IfStmt) is not in branchingMap - putEdge(stmt, target); + putEdge(stmt, j, target); } } } @@ -418,8 +420,8 @@ public void addBlock(@Nonnull List stmts, @Nonnull Map tr * @param trapMap */ private MutableBasicBlock addBlockInternal( - @Nonnull List stmts, Map trapMap) { - final Iterator iterator = stmts.iterator(); + @Nonnull List stmts, Map trapMap) { + final Iterator iterator = stmts.iterator(); final Stmt node = iterator.next(); MutableBasicBlock block = getOrCreateBlock(node); if (block.getHead() != node || !block.getSuccessors().isEmpty()) { @@ -776,11 +778,8 @@ public void removeNode(@Nonnull Stmt stmt, boolean keepFlow) { MutableBasicBlock successor = blockOfRemovedStmt.getSuccessors().get(0); for (MutableBasicBlock predecessor : blockOfRemovedStmt.getPredecessors()) { - predecessor.removeSuccessorBlock(blockOfRemovedStmt); - predecessor.addSuccessorBlock(successor); - - successor.removePredecessorBlock(blockOfRemovedStmt); - successor.addPredecessorBlock(predecessor); + predecessor.replaceSuccessorBlock(blockOfRemovedStmt, successor); + successor.replacePredecessorBlock(blockOfRemovedStmt, predecessor); } } } @@ -828,15 +827,17 @@ public void validateBlocks() { } } - /** + /* + * Note: if there is a stmt branching to successor this is not updated to the new stmt + * * @param beforeStmt the Stmt which succeeds the inserted Stmts (its NOT preceeding as this - * simplifies the handling of BranchingStmts) - * @param stmts can only allow fallsthrough Stmts except for the last Stmt in the List there is a - * single BranchingStmt allowed! + * simplifies the handling of BranchingStmts) + * @param stmts can only allow fallsthrough Stmts except for the last Stmt in the List there is a + * single BranchingStmt allowed! */ public void insertBefore( @Nonnull Stmt beforeStmt, - @Nonnull List stmts, + @Nonnull List stmts, @Nonnull Map exceptionMap) { if (stmts.isEmpty()) { return; @@ -852,18 +853,18 @@ public void insertBefore( // the stmts have only fallsthrough Stmts there could be some allocation/deallocation be saved final MutableBasicBlock predecessorBlock = addBlockInternal(stmts, exceptionMap); for (MutableBasicBlock predecessor : Lists.newArrayList(block.getPredecessors())) { - // cleanup old - predecessor.removeSuccessorBlock(block); + // cleanup old & add new link + predecessor.replaceSuccessorBlock(block, predecessorBlock); block.removePredecessorBlock(predecessor); - // add new link - linkBlocks(predecessor, predecessorBlock); + predecessorBlock.addPredecessorBlock(predecessor); } if (!tryMergeBlocks(predecessorBlock, block)) { - // hint: ms: this could be bad/unintuitive behaviour for a branching stmt for branching - predecessorBlock.addSuccessorBlock(block); + // all inserted Stmts are FallingThrough: so successorIdx = 0 + predecessorBlock.setSuccessorBlock(0, block); block.addPredecessorBlock(predecessorBlock); } + } else { final MutableBasicBlock successorBlock = block.splitBlockLinked(beforeStmt, true); exceptionMap.forEach( @@ -882,8 +883,12 @@ public void insertBefore( } } - @Override - public void putEdge(@Nonnull Stmt stmtA, @Nonnull Stmt stmtB) { + public void putEdge(@Nonnull FallsThroughStmt stmtA, @Nonnull Stmt stmtB) { + putEdge(stmtA, 0, stmtB); + } + + public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { + // FIXME: implement succesorIdx handling MutableBasicBlock blockA = stmtToBlock.get(stmtA); MutableBasicBlock blockB = stmtToBlock.get(stmtB); @@ -976,7 +981,15 @@ public void putEdge(@Nonnull Stmt stmtA, @Nonnull Stmt stmtB) { * makes blockA the predecessor of BlockB and BlockB the Successor of BlockA in a combined Method */ private void linkBlocks(@Nonnull MutableBasicBlock blockA, @Nonnull MutableBasicBlock blockB) { - blockA.addSuccessorBlock(blockB); + final int idx; + if (blockA.getTail() instanceof FallsThroughStmt) { + idx = 0; + } else { + throw new IllegalStateException( + "its not clear which successorIdx should be chosen to link these."); + } + + blockA.setSuccessorBlock(idx, blockB); blockB.addPredecessorBlock(blockA); } @@ -997,7 +1010,7 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { // remove the connection between the blocks boolean predecessorRemoved = blockOfTo.removePredecessorBlock(blockOfFrom); - boolean successorRemoved = blockOfFrom.removeSuccessorBlock(blockOfTo); + boolean successorRemoved = blockOfFrom.replaceSuccessorBlock(blockOfTo, null); assert predecessorRemoved == successorRemoved; if (!predecessorRemoved) { @@ -1022,14 +1035,13 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { if (toIdx < stmtsOfBlock.size() && stmtsOfBlock.get(toIdx) == to) { MutableBasicBlock newBlock = blockOfFrom.splitBlockUnlinked(from, to); newBlock.copyExceptionalFlowFrom(blockOfFrom); - blockOfFrom - .getSuccessors() - .forEach( - successor -> { - successor.removePredecessorBlock(blockOfFrom); - newBlock.addSuccessorBlock(successor); - successor.addPredecessorBlock(newBlock); - }); + List successors = blockOfFrom.getSuccessors(); + for (int i = 0; i < successors.size(); i++) { + MutableBasicBlock successor = successors.get(i); + successor.removePredecessorBlock(blockOfFrom); + newBlock.setSuccessorBlock(i, successor); + successor.addPredecessorBlock(newBlock); + } blockOfFrom.clearSuccessorBlocks(); blocks.add(newBlock); newBlock.getStmts().forEach(s -> stmtToBlock.put(s, newBlock)); @@ -1057,7 +1069,10 @@ public void setEdges(@Nonnull Stmt fromStmt, @Nonnull List targets) { // cleanup existing edges fromBlock.clearSuccessorBlocks(); } - targets.forEach(target -> putEdge(fromStmt, target)); + for (int i = 0; i < targets.size(); i++) { + Stmt target = targets.get(i); + putEdge(fromStmt, i, target); + } } @Nullable diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java index 72263177378..36faeb1bddb 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java @@ -23,6 +23,7 @@ import java.util.*; import javax.annotation.Nonnull; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.types.ClassType; @@ -65,14 +66,14 @@ public void addBlock(@Nonnull List stmts) { public abstract void insertBefore( @Nonnull Stmt beforeStmt, - @Nonnull List stmts, + @Nonnull List stmts, @Nonnull Map exceptionMap); /** * inserts the "newStmt" before the position of "beforeStmt" i.e. * newStmt.successors().contains(beforeStmt) will be true */ - public void insertBefore(@Nonnull Stmt beforeStmt, @Nonnull Stmt newStmt) { + public void insertBefore(@Nonnull Stmt beforeStmt, @Nonnull FallsThroughStmt newStmt) { insertBefore(beforeStmt, Collections.singletonList(newStmt), Collections.emptyMap()); } @@ -86,7 +87,9 @@ public void insertBefore(@Nonnull Stmt beforeStmt, @Nonnull Stmt newStmt) { * StmtGraph it will be added. if "to" needs to be added to the StmtGraph i.e. "to" is not already * in the StmtGraph the method assumes "to" has the same exceptional flows as "from". */ - public abstract void putEdge(@Nonnull Stmt from, @Nonnull Stmt to); + public abstract void putEdge(@Nonnull FallsThroughStmt from, @Nonnull Stmt to); + + public abstract void putEdge(@Nonnull Stmt from, int successorIdx, @Nonnull Stmt to); /** replaces the current outgoing flows of "from" to "targets" */ public abstract void setEdges(@Nonnull Stmt from, @Nonnull List targets); diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java index be6baa406e1..8fad16e990c 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIfStmt.java @@ -41,6 +41,9 @@ */ public final class JIfStmt extends AbstractStmt implements BranchingStmt, FallsThroughStmt { + public static final int FALSE_BRANCH_IDX = 0; + public static final int TRUE_BRANCH_IDX = 1; + @Nonnull private final AbstractConditionExpr condition; public JIfStmt(@Nonnull AbstractConditionExpr condition, @Nonnull StmtPositionInfo positionInfo) { diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index c85b4f129bb..547dfd215f8 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -469,15 +469,6 @@ public BodyBuilder clearExceptionEdgesOf(@Nonnull Stmt stmt) { return this; } - /* - * Note: if there is a stmt branching to successor this is not updated to the new stmt - * */ - @Nonnull - public BodyBuilder insertBefore(@Nonnull Stmt beforeStmt, Stmt newstmt) { - graph.insertBefore(beforeStmt, newstmt); - return this; - } - @Nonnull @Deprecated public List getTraps() { @@ -486,7 +477,7 @@ public List getTraps() { @Nonnull public BodyBuilder addFlow(@Nonnull Stmt fromStmt, @Nonnull Stmt toStmt) { - graph.putEdge(fromStmt, toStmt); + graph.putEdge(fromStmt, 0, toStmt); cachedLinearizedStmts = null; return this; } diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 1764c6147be..2fbce4c089b 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -24,9 +24,9 @@ @Category(Java8Test.class) public class MutableBlockStmtGraphTest { - Stmt firstNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt secondNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt thirdNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + JNopStmt firstNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + JNopStmt secondNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + JNopStmt thirdNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); BranchingStmt conditionalStmt = new JIfStmt( @@ -240,13 +240,13 @@ public void modifyStmtToBlockAtTail() { assertEquals(0, graph.getBlockOf(firstNop).getSuccessors().size()); // add connection between branchingstmt and first stmt - graph.putEdge(conditionalStmt, firstNop); + graph.putEdge(conditionalStmt, JIfStmt.FALSE_BRANCH_IDX, firstNop); assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.getBlockOf(firstNop).getPredecessors().size()); assertEquals(1, graph.getBlockOf(firstNop).getSuccessors().size()); // add connection between branchingstmt and second stmt - graph.putEdge(conditionalStmt, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.TRUE_BRANCH_IDX, secondNop); assertEquals(2, graph.getBlocks().size()); assertEquals(3, graph.getBlockOf(conditionalStmt).getStmts().size()); @@ -290,7 +290,7 @@ public void removeEdgeMerge() { assertEquals(1, graph.getBlocks().size()); // this edge splits the block between the first and second Nop - graph.putEdge(conditionalStmt, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.TRUE_BRANCH_IDX, secondNop); assertEquals(2, graph.getBlocks().size()); // this edge removal should merge both blocks together again @@ -331,6 +331,11 @@ public void addBadSuccessorCount() { graph.putEdge(firstNop, thirdNop); } + public void setBadSuccessorIdx() { + MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); + graph.putEdge(firstNop, 1, secondNop); + } + @Test(expected = IllegalArgumentException.class) public void addDuplicateBadSuccessorCount() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); @@ -341,24 +346,16 @@ public void addDuplicateBadSuccessorCount() { @Test public void addMultipleBranchingEdgesToSameTarget() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(conditionalStmt, secondNop); - graph.putEdge(conditionalStmt, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.FALSE_BRANCH_IDX, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.TRUE_BRANCH_IDX, secondNop); assertEquals(2, graph.successors(conditionalStmt).size()); } - @Test(expected = IllegalArgumentException.class) - public void addMultipleBranchingEdgesToSameTargetBAdCount() { - MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(conditionalStmt, secondNop); - graph.putEdge(conditionalStmt, secondNop); - graph.putEdge(conditionalStmt, secondNop); - } - @Test public void addSameSuccessorMultipleTimes() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(conditionalStmt, secondNop); - graph.putEdge(conditionalStmt, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.FALSE_BRANCH_IDX, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.TRUE_BRANCH_IDX, secondNop); assertEquals(2, graph.getBlocks().size()); @@ -375,8 +372,8 @@ public void addSameSuccessorMultipleTimes() { public void addBlocks() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(firstNop, conditionalStmt); - graph.putEdge(conditionalStmt, secondNop); - graph.putEdge(conditionalStmt, thirdNop); + graph.putEdge(conditionalStmt, JIfStmt.FALSE_BRANCH_IDX, secondNop); + graph.putEdge(conditionalStmt, JIfStmt.TRUE_BRANCH_IDX, thirdNop); assertEquals(3, graph.getBlocks().size()); } @@ -610,8 +607,8 @@ public PackageName getPackageName() { graph0.addNode(stmt1, Collections.singletonMap(exception1, catchStmt1)); graph0.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); - graph0.putEdge(stmt1, stmt2); - graph0.putEdge(stmt2, returnStmt); + graph0.putEdge(stmt1, JIfStmt.FALSE_BRANCH_IDX, stmt2); + graph0.putEdge(stmt2, JIfStmt.TRUE_BRANCH_IDX, returnStmt); { final List traps = graph0.getTraps(); @@ -624,13 +621,13 @@ public PackageName getPackageName() { // test merging traps from sequential blocks with the same trap MutableBlockStmtGraph graph1 = new MutableBlockStmtGraph(); graph1.setStartingStmt(stmt1); - graph1.putEdge(stmt1, stmt2); + graph1.putEdge(stmt1, JIfStmt.FALSE_BRANCH_IDX, stmt2); graph1.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph1.addNode(stmt3, Collections.singletonMap(exception1, catchStmt1)); - graph1.putEdge(stmt2, returnStmt); - graph1.putEdge(stmt3, returnStmt); - graph1.putEdge(catchStmt1, stmt3); + graph1.putEdge(stmt2, JIfStmt.FALSE_BRANCH_IDX, returnStmt); + graph1.putEdge(stmt3, JIfStmt.FALSE_BRANCH_IDX, returnStmt); + graph1.putEdge(catchStmt1, JIfStmt.FALSE_BRANCH_IDX, stmt3); { final List traps = graph1.getTraps(); @@ -645,10 +642,10 @@ public PackageName getPackageName() { graph2.addNode(stmt1, Collections.singletonMap(exception1, catchStmt1)); graph2.addNode(stmt2, Collections.singletonMap(exception1, catchStmt2)); - graph2.putEdge(stmt1, stmt2); + graph2.putEdge(stmt1, 0, stmt2); assertEquals(4, graph2.getBlocks().size()); - graph2.putEdge(stmt2, returnStmt); + graph2.putEdge(stmt2, 0, returnStmt); { assertEquals(5, graph2.getBlocks().size()); final List traps = graph2.getTraps(); @@ -662,9 +659,9 @@ public PackageName getPackageName() { graph3.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph3.addNode(stmt3, Collections.emptyMap()); - graph3.putEdge(stmt1, stmt2); - graph3.putEdge(stmt2, stmt3); - graph3.putEdge(stmt3, returnStmt); + graph3.putEdge(stmt1, 0, stmt2); + graph3.putEdge(stmt2, 0, stmt3); + graph3.putEdge(stmt3, 0, returnStmt); { final List traps = graph3.getTraps(); @@ -687,9 +684,9 @@ public PackageName getPackageName() { graph4.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph4.addNode(stmt3, Collections.emptyMap()); - graph4.putEdge(stmt1, stmt2); - graph4.putEdge(stmt2, stmt3); - graph4.putEdge(stmt3, returnStmt); + graph4.putEdge(stmt1, 0, stmt2); + graph4.putEdge(stmt2, 0, stmt3); + graph4.putEdge(stmt3, 0, returnStmt); assertEquals(3, graph4.getTraps().size()); @@ -721,9 +718,9 @@ public PackageName getPackageName() { } }); - graph5.putEdge(stmt1, stmt2); - graph5.putEdge(stmt2, stmt3); - graph5.putEdge(stmt3, returnStmt); + graph5.putEdge(stmt1, 0, stmt2); + graph5.putEdge(stmt2, 0, stmt3); + graph5.putEdge(stmt3, 0, returnStmt); { final List traps = graph5.getTraps(); @@ -758,9 +755,9 @@ public PackageName getPackageName() { } }); - graph6.putEdge(stmt1, stmt2); - graph6.putEdge(stmt2, stmt3); - graph6.putEdge(stmt3, returnStmt); + graph6.putEdge(stmt1, 0, stmt2); + graph6.putEdge(stmt2, 0, stmt3); + graph6.putEdge(stmt3, 0, returnStmt); { final List traps = graph6.getTraps(); assertEquals(5, traps.size()); @@ -813,8 +810,8 @@ public void copyOfImmutable() { @Test public void copyOf() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + JNopStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + JNopStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(stmt1, stmt2); graph.setStartingStmt(stmt1); @@ -866,17 +863,6 @@ public void setEdgesSimple() { assertEquals(stmt3, graph.successors(stmt1).get(1)); } - @Test(expected = IllegalArgumentException.class) - public void setEdgesReplacing() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt3 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - - MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, stmt2); - graph.putEdge(stmt1, stmt3); - } - @Test public void removeNodeWOEdges() { Stmt stmt = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); @@ -892,7 +878,7 @@ public void removeNodeWOPredecessors() { Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, stmt2); + graph.putEdge(stmt1, 0, stmt2); assertTrue(graph.getNodes().contains(stmt1)); assertEquals(Collections.singletonList(stmt2), graph.successors(stmt1)); @@ -916,7 +902,7 @@ public void removeNodeWOSuccessors() { Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, stmt2); + graph.putEdge(stmt1, 0, stmt2); assertTrue(graph.getNodes().contains(stmt2)); assertEquals(Collections.singletonList(stmt2), graph.successors(stmt1)); @@ -944,7 +930,7 @@ public void removeEdge() { Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, stmt2); + graph.putEdge(stmt1, 0, stmt2); assertEquals(1, graph.successors(stmt1).size()); assertTrue(graph.hasEdgeConnecting(stmt1, stmt2)); @@ -989,15 +975,15 @@ public void putEdge() { Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); // stmt2 is not in the graph! - graph.putEdge(stmt1, stmt2); + graph.putEdge(stmt1, 0, stmt2); } @Test public void simpleInsertion() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt3 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt3 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(stmt1, stmt2); diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java index c701a30c82a..5bbd31675a8 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/frontend/AsmMethodSource.java @@ -2033,7 +2033,7 @@ private void arrangeStmts( // connect tail of stmtList with its target Stmt targetStmt = insnToStmt.get(entry.getKey()); - graph.putEdge(gotoStmt, targetStmt); + graph.putEdge(gotoStmt, 0, targetStmt); } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java index d790445d6ad..c69df1171b5 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java @@ -83,15 +83,16 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi // link previous stmt with always-reached successor of the if-Stmt for (Stmt predecessor : stmtGraph.predecessors(ifStmt)) { - builder.removeFlow(predecessor, ifStmt); - builder.addFlow(predecessor, tautologicSuccessor); + stmtGraph.removeEdge(predecessor, ifStmt); + // FIXME: [ms] fix successorIdx + stmtGraph.putEdge(predecessor, 0, tautologicSuccessor); } // removeFlow calls should be obsolete as of following removeStmt - builder.removeFlow(ifStmt, tautologicSuccessor); - builder.removeFlow(ifStmt, neverReachedSucessor); + stmtGraph.removeEdge(ifStmt, tautologicSuccessor); + stmtGraph.removeEdge(ifStmt, neverReachedSucessor); - builder.removeStmt(ifStmt); + stmtGraph.removeNode(ifStmt); pruneExclusivelyReachableStmts(stmtGraph, neverReachedSucessor); } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java index c85f6962b5d..9e6ed6091ab 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java @@ -34,6 +34,7 @@ import sootup.core.jimple.basic.Value; import sootup.core.jimple.common.expr.JPhiExpr; import sootup.core.jimple.common.stmt.AbstractDefinitionStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JAssignStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; @@ -268,12 +269,12 @@ private Map, Set> decideBlockToPhiStmts( * @param blockToDefs maps each block to the set of defs' local in itself */ private void addPhiStmts( - Map, Set> blockToPhiStmts, + Map, Set> blockToPhiStmts, MutableStmtGraph blockGraph, Map, Set> blockToDefs) { // key: phiStmt value: size of phiStmt's arguments - Map phiToNum = new HashMap(); + Map phiToNum = new HashMap<>(); // determine the arguments' size of each phiStmt for (BasicBlock block : blockGraph.getBlocks()) { @@ -300,14 +301,14 @@ private void addPhiStmts( // if the arguments' size of a phiStmt is less than 2, delete it from blockToPhiStmts map // add other phiStmts into corresponding block for (BasicBlock block : blockToPhiStmts.keySet()) { - Set phis = blockToPhiStmts.get(block); - Set checkedPhis = new HashSet<>(blockToPhiStmts.get(block)); - for (Stmt cphi : checkedPhis) { + Set phis = blockToPhiStmts.get(block); + Set checkedPhis = new HashSet<>(blockToPhiStmts.get(block)); + for (FallsThroughStmt cphi : checkedPhis) { if (phiToNum.get(cphi) < 2) { phis.remove(cphi); } } - for (Stmt phi : phis) { + for (FallsThroughStmt phi : phis) { blockGraph.insertBefore(block.getHead(), phi); } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java index 5ed1a1a024f..658c07e1408 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/typeresolving/CastCounter.java @@ -26,6 +26,7 @@ import java.util.HashMap; import java.util.Map; import javax.annotation.Nonnull; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.Value; @@ -114,6 +115,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm this.castCount++; // TODO: modifiers later must be added + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); Local old_local; if (value instanceof Local) { old_local = (Local) value; @@ -122,7 +124,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm builder.addLocal(old_local); typing.set(old_local, evaType); JAssignStmt newAssign = Jimple.newAssignStmt(old_local, value, stmt.getPositionInfo()); - builder.insertBefore(stmt, newAssign); + stmtGraph.insertBefore(stmt, newAssign); } Local new_local = generateTempLocal(stdType); @@ -132,7 +134,7 @@ public void visit(@Nonnull Value value, @Nonnull Type stdType, @Nonnull Stmt stm JAssignStmt newCast = Jimple.newAssignStmt( new_local, Jimple.newCastExpr(old_local, stdType), stmt.getPositionInfo()); - builder.insertBefore(stmt, newCast); + stmtGraph.insertBefore(stmt, newCast); Stmt newStmt; if (stmt.getUses().contains(value)) { diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java index 06ba6e2aa86..5bc08601679 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java @@ -6,10 +6,12 @@ import java.util.*; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JNopStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; @@ -81,7 +83,8 @@ private static Body.BodyBuilder createBody(boolean withNop) { Local b = JavaJimple.newLocal("b", stringType); Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); - Stmt bToA = JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); + FallsThroughStmt bToA = + JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); Stmt ret = JavaJimple.newReturnStmt(b, noPositionInfo); Stmt jump = JavaJimple.newGotoStmt(noPositionInfo); @@ -96,12 +99,13 @@ private static Body.BodyBuilder createBody(boolean withNop) { builder.addFlow(strToA, jump); builder.addFlow(jump, bToA); builder.addFlow(bToA, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); if (withNop) { // strToA, jump, bToA, nop, ret; JNopStmt nop = new JNopStmt(noPositionInfo); - builder.removeFlow(bToA, ret); - builder.addFlow(bToA, nop); - builder.addFlow(nop, ret); + stmtGraph.removeEdge(bToA, ret); + stmtGraph.putEdge(bToA, nop); + stmtGraph.putEdge(nop, ret); } builder.setLocals(locals); builder.setPosition(NoPositionInformation.getInstance()); From 8957a1786a0fb0877031605c37e0ab63253d653f Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 20 Oct 2023 13:15:37 +0200 Subject: [PATCH 44/54] fix core; multiple NPEs --- .../sootup/core/graph/MutableBasicBlock.java | 48 +++++++++---------- .../core/graph/MutableBlockStmtGraph.java | 25 +++++++--- .../core/jimple/common/stmt/JGotoStmt.java | 1 + .../core/graph/MutableBlockStmtGraphTest.java | 41 ++++++++-------- 4 files changed, 64 insertions(+), 51 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java index 245a33dbd47..817975dc0c4 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java @@ -33,7 +33,8 @@ public class MutableBasicBlock implements BasicBlock { @Nonnull private final ArrayList predecessorBlocks = new ArrayList<>(); - private MutableBasicBlock[] successorBlocks = null; + private MutableBasicBlock[] successorBlocks = + new MutableBasicBlock[1]; // 1 := most propable amount of successors/elements @Nonnull private final Map exceptionalSuccessorBlocks; @@ -58,21 +59,16 @@ public boolean equals(Object o) { } public void addStmt(@Nonnull Stmt newStmt) { - final Stmt tail = getTail(); - if (getStmtCount() > 0 && tail instanceof BranchingStmt) { + if (getStmtCount() > 0 && getTail() instanceof BranchingStmt) { throw new IllegalArgumentException( "Can't add another Stmt to a Block after a BranchingStmt."); } stmts.add(newStmt); - updateSuccessorContainer(newStmt); } public void removeStmt(@Nonnull Stmt stmt) { final int idx = stmts.indexOf(stmt); stmts.remove(idx); - if (idx == stmts.size() - 1) { - updateSuccessorContainer(getTail()); - } } public void replaceStmt(Stmt oldStmt, Stmt newStmt) { @@ -81,10 +77,6 @@ public void replaceStmt(Stmt oldStmt, Stmt newStmt) { throw new IllegalArgumentException("oldStmt does not exist in this Block!"); } stmts.set(idx, newStmt); - // did the last stmt change? - if (idx == stmts.size() - 1) { - updateSuccessorContainer(newStmt); - } } protected void updateSuccessorContainer(@Nonnull Stmt newStmt) { @@ -103,15 +95,15 @@ public void addPredecessorBlock(@Nonnull MutableBasicBlock block) { } public boolean setSuccessorBlock(int successorIdx, @Nullable MutableBasicBlock block) { - if (successorBlocks == null) { - successorBlocks = new MutableBasicBlock[block.getTail().getExpectedSuccessorCount()]; - } + updateSuccessorContainer(getTail()); if (successorIdx >= successorBlocks.length) { throw new IndexOutOfBoundsException( "successorIdx '" + successorIdx + "' is out of bounds ('" + successorBlocks.length + + " for " + + getTail() + "')"); } successorBlocks[successorIdx] = block; @@ -119,9 +111,6 @@ public boolean setSuccessorBlock(int successorIdx, @Nullable MutableBasicBlock b } public boolean removePredecessorBlock(@Nonnull MutableBasicBlock b) { - if (successorBlocks == null) { - return false; - } return predecessorBlocks.remove(b); } @@ -167,7 +156,13 @@ public List getPredecessors() { @Nonnull @Override public List getSuccessors() { - return Arrays.stream(successorBlocks).filter(Objects::isNull).collect(Collectors.toList()); + if (stmts.isEmpty()) { + return Collections.emptyList(); + } + if (getTail().getExpectedSuccessorCount() != successorBlocks.length) { + return Collections.emptyList(); + } + return Arrays.stream(successorBlocks).filter(Objects::nonNull).collect(Collectors.toList()); } @Override @@ -258,7 +253,6 @@ protected MutableBasicBlock splitBlockUnlinked(int splitIdx) { if (splitIdx < stmts.size()) { stmts.subList(splitIdx, stmts.size()).clear(); } - return secondBlock; } @@ -285,18 +279,20 @@ public MutableBasicBlock splitBlockLinked(@Nonnull Stmt splitStmt, boolean shoul MutableBasicBlock newBlock = splitBlockUnlinked(splitIdx); for (int i = 0; i < successorBlocks.length; i++) { MutableBasicBlock succBlock = successorBlocks[i]; // copy successors to the newBlock + if (succBlock == null) { + continue; + } newBlock.setSuccessorBlock(i, succBlock); // and relink predecessors of the successors to newblock as well succBlock.removePredecessorBlock(this); succBlock.addPredecessorBlock(newBlock); } - successorBlocks = null; + successorBlocks = new MutableBasicBlock[1]; newBlock.addPredecessorBlock(this); setSuccessorBlock( - 0, - newBlock); // 0 as this can only be a block if the Stmts before the last Stmt are - // FallsThroughStmt + 0, newBlock); // 0 as this can only be a block if the Stmts before the last Stmt are + // FallsThroughStmt return newBlock; } @@ -308,8 +304,10 @@ public void copyExceptionalFlowFrom(MutableBasicBlock sourceBlock) { } public void clearSuccessorBlocks() { - Stream.of(successorBlocks).forEach(b -> b.removePredecessorBlock(this)); - successorBlocks = null; + Stream.of(successorBlocks) + .filter(Objects::nonNull) + .forEach(b -> b.removePredecessorBlock(this)); + successorBlocks = new MutableBasicBlock[1]; } public void clearExceptionalSuccessorBlocks() { diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 6b2004a13f9..65e15a36bab 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -63,7 +63,10 @@ public MutableBlockStmtGraph(boolean isStatic, MethodSignature sig, LocalGenerat /** copies a StmtGraph into this Mutable instance */ public MutableBlockStmtGraph(@Nonnull StmtGraph> graph) { - setStartingStmt(graph.getStartingStmt()); + final Stmt startStmt = graph.getStartingStmt(); + if (startStmt != null) { + setStartingStmt(startStmt); + } // copy blocks into this graph graph .getBlocks() @@ -923,11 +926,14 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { // one if (blockB == null) { blockB = createStmtsBlock(stmtB); - linkBlocks(blockA, blockB); + blockA.setSuccessorBlock(succesorIdx, blockB); + blockB.addPredecessorBlock(blockA); } else { if (blockB.getHead() == stmtB) { // stmtB is at the beginning of the second Block -> connect blockA and blockB - linkBlocks(blockA, blockB); + + blockA.setSuccessorBlock(succesorIdx, blockB); + blockB.addPredecessorBlock(blockA); } else { MutableBasicBlock newBlock = blockB.splitBlockLinked(stmtB, true); @@ -939,9 +945,12 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { // successor of block is the origin: end of block flows to beginning of new splitted // block (i.e. // the same block) - linkBlocks(newBlock, newBlock); + newBlock.setSuccessorBlock(succesorIdx, newBlock); + newBlock.addPredecessorBlock(newBlock); + } else { - linkBlocks(blockA, newBlock); + blockA.setSuccessorBlock(succesorIdx, newBlock); + newBlock.addPredecessorBlock(blockA); } } } @@ -965,7 +974,8 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { } else { // stmtA does not branch but stmtB is already a branch target or has different traps => // link blocks - linkBlocks(blockA, blockB); + blockA.setSuccessorBlock(succesorIdx, blockB); + blockB.addPredecessorBlock(blockA); } } else { throw new IllegalArgumentException( @@ -979,7 +989,10 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { /** * makes blockA the predecessor of BlockB and BlockB the Successor of BlockA in a combined Method + * Deprecated: can only assume (bad - it could be otherwise) which successorIdx shall be chosen to + * link the block in case of branching stmts */ + @Deprecated private void linkBlocks(@Nonnull MutableBasicBlock blockA, @Nonnull MutableBasicBlock blockB) { final int idx; if (blockA.getTail() instanceof FallsThroughStmt) { diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java index 6e7bdf798bc..335e938b732 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JGotoStmt.java @@ -33,6 +33,7 @@ /** Unconditionally jumps to a target Stmt */ public class JGotoStmt extends AbstractStmt implements BranchingStmt { + public static final int BRANCH_IDX = 0; public JGotoStmt(StmtPositionInfo positionInfo) { super(positionInfo); diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 2fbce4c089b..79b710c64ab 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -363,6 +363,7 @@ public void addSameSuccessorMultipleTimes() { assertEquals(2, graph.inDegree(secondNop)); assertEquals(Arrays.asList(conditionalStmt, conditionalStmt), graph.predecessors(secondNop)); assertEquals(2, graph.outDegree(conditionalStmt)); + assertEquals(Arrays.asList(secondNop, secondNop), graph.successors(conditionalStmt)); assertTrue(graph.hasEdgeConnecting(conditionalStmt, secondNop)); assertFalse(graph.hasEdgeConnecting(secondNop, conditionalStmt)); @@ -607,8 +608,8 @@ public PackageName getPackageName() { graph0.addNode(stmt1, Collections.singletonMap(exception1, catchStmt1)); graph0.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); - graph0.putEdge(stmt1, JIfStmt.FALSE_BRANCH_IDX, stmt2); - graph0.putEdge(stmt2, JIfStmt.TRUE_BRANCH_IDX, returnStmt); + graph0.putEdge(stmt1, 0, stmt2); + graph0.putEdge(stmt2, 0, returnStmt); { final List traps = graph0.getTraps(); @@ -621,13 +622,13 @@ public PackageName getPackageName() { // test merging traps from sequential blocks with the same trap MutableBlockStmtGraph graph1 = new MutableBlockStmtGraph(); graph1.setStartingStmt(stmt1); - graph1.putEdge(stmt1, JIfStmt.FALSE_BRANCH_IDX, stmt2); + graph1.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); graph1.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph1.addNode(stmt3, Collections.singletonMap(exception1, catchStmt1)); - graph1.putEdge(stmt2, JIfStmt.FALSE_BRANCH_IDX, returnStmt); - graph1.putEdge(stmt3, JIfStmt.FALSE_BRANCH_IDX, returnStmt); - graph1.putEdge(catchStmt1, JIfStmt.FALSE_BRANCH_IDX, stmt3); + graph1.putEdge(stmt2, JGotoStmt.BRANCH_IDX, returnStmt); + graph1.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); + graph1.putEdge(catchStmt1, JGotoStmt.BRANCH_IDX, stmt3); { final List traps = graph1.getTraps(); @@ -642,10 +643,10 @@ public PackageName getPackageName() { graph2.addNode(stmt1, Collections.singletonMap(exception1, catchStmt1)); graph2.addNode(stmt2, Collections.singletonMap(exception1, catchStmt2)); - graph2.putEdge(stmt1, 0, stmt2); + graph2.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); assertEquals(4, graph2.getBlocks().size()); - graph2.putEdge(stmt2, 0, returnStmt); + graph2.putEdge(stmt2, JGotoStmt.BRANCH_IDX, returnStmt); { assertEquals(5, graph2.getBlocks().size()); final List traps = graph2.getTraps(); @@ -659,9 +660,9 @@ public PackageName getPackageName() { graph3.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph3.addNode(stmt3, Collections.emptyMap()); - graph3.putEdge(stmt1, 0, stmt2); - graph3.putEdge(stmt2, 0, stmt3); - graph3.putEdge(stmt3, 0, returnStmt); + graph3.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); + graph3.putEdge(stmt2, JGotoStmt.BRANCH_IDX, stmt3); + graph3.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); { final List traps = graph3.getTraps(); @@ -684,9 +685,9 @@ public PackageName getPackageName() { graph4.addNode(stmt2, Collections.singletonMap(exception1, catchStmt1)); graph4.addNode(stmt3, Collections.emptyMap()); - graph4.putEdge(stmt1, 0, stmt2); - graph4.putEdge(stmt2, 0, stmt3); - graph4.putEdge(stmt3, 0, returnStmt); + graph4.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); + graph4.putEdge(stmt2, JGotoStmt.BRANCH_IDX, stmt3); + graph4.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); assertEquals(3, graph4.getTraps().size()); @@ -718,9 +719,9 @@ public PackageName getPackageName() { } }); - graph5.putEdge(stmt1, 0, stmt2); - graph5.putEdge(stmt2, 0, stmt3); - graph5.putEdge(stmt3, 0, returnStmt); + graph5.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); + graph5.putEdge(stmt2, JGotoStmt.BRANCH_IDX, stmt3); + graph5.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); { final List traps = graph5.getTraps(); @@ -755,9 +756,9 @@ public PackageName getPackageName() { } }); - graph6.putEdge(stmt1, 0, stmt2); - graph6.putEdge(stmt2, 0, stmt3); - graph6.putEdge(stmt3, 0, returnStmt); + graph6.putEdge(stmt1, JGotoStmt.BRANCH_IDX, stmt2); + graph6.putEdge(stmt2, JGotoStmt.BRANCH_IDX, stmt3); + graph6.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); { final List traps = graph6.getTraps(); assertEquals(5, traps.size()); From fa04b0510339f3b25d42434481d2ffc25143c057 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 20 Oct 2023 15:08:07 +0200 Subject: [PATCH 45/54] adapting tests --- .../sootup/core/graph/MutableBasicBlock.java | 5 +- .../core/graph/MutableBlockStmtGraph.java | 85 ++++++------ .../common/stmt/AbstractDefinitionStmt.java | 2 +- .../jimple/common/stmt/JIdentityStmt.java | 2 +- .../StaticSingleAssignmentFormer.java | 20 +-- .../ConditionalBranchFolderTest.java | 16 ++- .../interceptors/CopyPropagatorTest.java | 131 +++++++++--------- .../DeadAssignmentEliminatorTest.java | 34 +++-- .../LocalLivenessAnalyserTest.java | 26 ++-- .../interceptors/LocalPackerTest.java | 68 +++++---- .../interceptors/LocalSplitterTest.java | 38 ++--- .../interceptors/TrapTightenerTest.java | 30 ++-- .../UnreachableCodeEliminatorTest.java | 25 ++-- .../typeresolving/BytecodeHierarchyTest.java | 2 +- .../typeresolving/TypeAssignerTestSuite.java | 2 +- 15 files changed, 271 insertions(+), 215 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java index 817975dc0c4..e3c30624afd 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBasicBlock.java @@ -159,9 +159,7 @@ public List getSuccessors() { if (stmts.isEmpty()) { return Collections.emptyList(); } - if (getTail().getExpectedSuccessorCount() != successorBlocks.length) { - return Collections.emptyList(); - } + final int expectedSuccessorCount = getTail().getExpectedSuccessorCount(); return Arrays.stream(successorBlocks).filter(Objects::nonNull).collect(Collectors.toList()); } @@ -277,6 +275,7 @@ public MutableBasicBlock splitBlockLinked(@Nonnull Stmt splitStmt, boolean shoul } MutableBasicBlock newBlock = splitBlockUnlinked(splitIdx); + for (int i = 0; i < successorBlocks.length; i++) { MutableBasicBlock succBlock = successorBlocks[i]; // copy successors to the newBlock if (succBlock == null) { diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index 65e15a36bab..da053775e98 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -87,8 +87,11 @@ public MutableBlockStmtGraph(@Nonnull StmtGraph> graph) // getBlockOf is necessary to find the new existing/copied block which are refering to // the same a immutable Stmt final MutableBasicBlock blockOf = stmtToBlock.get(b.getTail()); - b.getSuccessors() - .forEach(succ -> linkBlocks(blockOf, stmtToBlock.get(succ.getHead()))); + List> successors = b.getSuccessors(); + for (int i = 0; i < successors.size(); i++) { + BasicBlock succ = successors.get(i); + linkBlocks(blockOf, i, stmtToBlock.get(succ.getHead())); + } }); } @@ -105,11 +108,11 @@ public void initializeWith( } final Stmt lastStmt = stmts.get(stmts.size() - 1); - if (lastStmt.fallsThrough()) { + if (lastStmt instanceof FallsThroughStmt) { throw new IllegalArgumentException( - "Theres a fallsthrough Stmt at the end of the list of stmts ('" + "Theres FallsthroughStmt at the end of the list of stmts ('" + lastStmt - + "') which has no sucessor - which means it currently falls into the abyss i.e. it can't fall through to another Stmt."); + + "') which has no successor - which means it currently falls into the abyss i.e. it can't fall through to another Stmt."); } HashMap trapstmtToIdx = new HashMap<>(); @@ -227,8 +230,8 @@ public void initializeWith( if (stmt instanceof BranchingStmt) { // => end of Block final List targets = branchingMap.get(stmt); - int expectedBranchEntries = - stmt.getExpectedSuccessorCount() - (stmt.fallsThrough() ? 1 : 0); + int idxOffset = (stmt instanceof FallsThroughStmt) ? 1 : 0; + int expectedBranchEntries = stmt.getExpectedSuccessorCount() - idxOffset; if (targets == null || targets.size() != expectedBranchEntries) { int targetCount; if (targets == null) { @@ -246,11 +249,10 @@ public void initializeWith( + targetCount + "."); } - for (int j = 0; j < targets.size(); j++) { Stmt target = targets.get(j); // a possible fallsthrough (i.e. from IfStmt) is not in branchingMap - putEdge(stmt, j, target); + putEdge(stmt, j + idxOffset, target); } } } @@ -459,7 +461,7 @@ private MutableBasicBlock addBlockInternal( // try to merge if (!tryMergeBlocks(block, overwrittenBlock)) { // otherwise link them - linkBlocks(block, overwrittenBlock); + linkBlocks(block, 0, overwrittenBlock); } } else { throw new IllegalArgumentException( @@ -552,7 +554,7 @@ private MutableBasicBlock excludeStmtFromBlock(@Nonnull Stmt splitStmt, MutableB blocks.add(excludedFromOrigBlock); } - if (stmtIdx + 1 < blockStmts.size()) { + if (stmtIdx + 1 < blockStmts.size()) { // ms: equivalent to: block.getTail() != splitStmt // "third"/after/leftover block is necessary as there are stmts after the splitElement final MutableBasicBlock restOfOrigBlock = new MutableBasicBlock(); for (int i = stmtIdx + 1; i < blockStmts.size(); i++) { @@ -561,16 +563,15 @@ private MutableBasicBlock excludeStmtFromBlock(@Nonnull Stmt splitStmt, MutableB } // copy successors of block which are now the successors of the "third"/leftover block - block - .getSuccessors() - .forEach( - successor -> { - linkBlocks(restOfOrigBlock, successor); - }); + List successors = block.getSuccessors(); + for (int i = 0; i < successors.size(); i++) { + MutableBasicBlock successor = successors.get(i); + linkBlocks(restOfOrigBlock, i, successor); + } block.clearSuccessorBlocks(); // link third/leftover block with previous stmts from the separated block - linkBlocks(excludedFromOrigBlock, restOfOrigBlock); + linkBlocks(excludedFromOrigBlock, 0, restOfOrigBlock); block.clearSuccessorBlocks(); // add blocks exceptional flows @@ -584,22 +585,25 @@ private MutableBasicBlock excludeStmtFromBlock(@Nonnull Stmt splitStmt, MutableB blocks.add(restOfOrigBlock); + // cleanup original block -> "beforeBlock" -> remove now copied Stmts + for (int i = blockStmts.size() - 1; i >= stmtIdx; i--) { + block.removeStmt(blockStmts.get(i)); + } + } else { // there are no more stmts after stmtIdx -> less than 3 blocks are necessary // copy origin successors to second block as its now the last part of the origin block - block - .getSuccessors() - .forEach( - successorBlock -> { - linkBlocks(excludedFromOrigBlock, successorBlock); - }); + List successors = block.getSuccessors(); + for (int i = 0; i < successors.size(); i++) { + MutableBasicBlock successorBlock = successors.get(i); + linkBlocks(excludedFromOrigBlock, i, successorBlock); + } block.clearSuccessorBlocks(); - linkBlocks(block, excludedFromOrigBlock); - } - - // cleanup original block -> "beforeBlock" -> remove now copied Stmts - for (int i = blockStmts.size() - 1; i >= stmtIdx; i--) { - block.removeStmt(blockStmts.get(i)); + // cleanup original block -> "beforeBlock" -> remove now copied Stmts + for (int i = blockStmts.size() - 1; i >= stmtIdx; i--) { + block.removeStmt(blockStmts.get(i)); + } + linkBlocks(block, 0, excludedFromOrigBlock); } return excludedFromOrigBlock; @@ -685,7 +689,11 @@ protected boolean tryMergeBlocks( // update linking info into firstBlock // done in clearPredecessorBlock firstBlock.removeSuccessorBlock(followingBlock); - followingBlock.getSuccessors().forEach(succ -> linkBlocks(firstBlock, succ)); + List successors = followingBlock.getSuccessors(); + for (int i = 0; i < successors.size(); i++) { + MutableBasicBlock succ = successors.get(i); + linkBlocks(firstBlock, i, succ); + } followingBlock.clearSuccessorBlocks(); blocks.remove(followingBlock); @@ -891,7 +899,7 @@ public void putEdge(@Nonnull FallsThroughStmt stmtA, @Nonnull Stmt stmtB) { } public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { - // FIXME: implement succesorIdx handling + MutableBasicBlock blockA = stmtToBlock.get(stmtA); MutableBasicBlock blockB = stmtToBlock.get(stmtB); @@ -993,16 +1001,9 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { * link the block in case of branching stmts */ @Deprecated - private void linkBlocks(@Nonnull MutableBasicBlock blockA, @Nonnull MutableBasicBlock blockB) { - final int idx; - if (blockA.getTail() instanceof FallsThroughStmt) { - idx = 0; - } else { - throw new IllegalStateException( - "its not clear which successorIdx should be chosen to link these."); - } - - blockA.setSuccessorBlock(idx, blockB); + private void linkBlocks( + @Nonnull MutableBasicBlock blockA, int successorIdx, @Nonnull MutableBasicBlock blockB) { + blockA.setSuccessorBlock(successorIdx, blockB); blockB.addPredecessorBlock(blockA); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java index e9236b3169f..e900acb6d51 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/AbstractDefinitionStmt.java @@ -80,5 +80,5 @@ public boolean branches() { } @Nonnull - public abstract Stmt withNewDef(@Nonnull Local newLocal); + public abstract FallsThroughStmt withNewDef(@Nonnull Local newLocal); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java index d0e57427df5..440b2cf27be 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/JIdentityStmt.java @@ -109,7 +109,7 @@ public JIdentityStmt withPositionInfo(@Nonnull StmtPositionInfo positionInfo) { @Nonnull @Override - public Stmt withNewDef(@Nonnull Local newLocal) { + public FallsThroughStmt withNewDef(@Nonnull Local newLocal) { return withLocal(newLocal); } } diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java index 9e6ed6091ab..9fef83abefe 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormer.java @@ -91,7 +91,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi // decide which block should be add a phi assignStmt, and store such info in a map // key: Block which contains phiStmts. Values : a set of phiStmts which contained by // corresponding Block - Map, Set> blockToPhiStmts = + Map, Set> blockToPhiStmts = decideBlockToPhiStmts(builder, dominanceFinder, blockToDefs, localToBlocks); // delete meaningless phiStmts and add other phiStmts into stmtGraph @@ -111,7 +111,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi // rename each def-local and its corresponding name and add args and blocks into phiStmts for (BasicBlock block : treeNodes) { // replace use and def in each stmts in the current block - Set newPhiStmts = new HashSet<>(); + Set newPhiStmts = new HashSet<>(); for (Stmt stmt : block.getStmts()) { // replace use final List uses = stmt.getUses(); @@ -133,7 +133,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi newLocals.add(newDef); nextFreeIdx++; localToNameStack.get(def).push(newDef); - Stmt newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(newDef); + FallsThroughStmt newStmt = ((AbstractDefinitionStmt) stmt).withNewDef(newDef); stmtGraph.replaceNode(stmt, newStmt); if (constainsPhiExpr(newStmt)) { newPhiStmts.add(newStmt); @@ -151,14 +151,14 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi succs.addAll(block.getExceptionalSuccessors().values()); for (BasicBlock succ : succs) { if (blockToPhiStmts.containsKey(succ)) { - Set phiStmts = blockToPhiStmts.get(succ); + Set phiStmts = blockToPhiStmts.get(succ); newPhiStmts = new HashSet<>(phiStmts); for (Stmt phiStmt : phiStmts) { Local def = (Local) phiStmt.getDefs().get(0); Local oriDef = getOriginalLocal(def, localToNameStack.keySet()); if (!localToNameStack.get(oriDef).isEmpty()) { Local arg = localToNameStack.get(oriDef).peek(); - Stmt newPhiStmt = addNewArgToPhi(phiStmt, arg, block); + FallsThroughStmt newPhiStmt = addNewArgToPhi(phiStmt, arg, block); newPhiStmts.remove(phiStmt); newPhiStmts.add(newPhiStmt); stmtGraph.replaceNode(phiStmt, newPhiStmt); @@ -205,12 +205,12 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi * @return a map, key: block, value: a set of phiStmts that are added in front of the * corresponding block */ - private Map, Set> decideBlockToPhiStmts( + private Map, Set> decideBlockToPhiStmts( Body.BodyBuilder builder, DominanceFinder dominanceFinder, Map, Set> blockToDefs, Map>> localToBlocks) { - Map, Set> blockToPhiStmts = new HashMap<>(); + Map, Set> blockToPhiStmts = new HashMap<>(); Map, Set> blockToPhiLocals = new HashMap<>(); Map>> localToPhiBlocks = new HashMap<>(); @@ -234,7 +234,7 @@ private Map, Set> decideBlockToPhiStmts( blockToPhiStmts.get(df).add(phiStmt); blockToPhiLocals.get(df).add(local); } else { - Set phiStmts = new LinkedHashSet<>(); + Set phiStmts = new LinkedHashSet<>(); phiStmts.add(phiStmt); blockToPhiStmts.put(df, phiStmts); Set phiLocals = new HashSet<>(); @@ -353,9 +353,9 @@ private Local getOriginalLocal(Local local, Set oriLocals) { throw new RuntimeException(local + " has no original local!"); } - private Stmt addNewArgToPhi(Stmt phiStmt, Local arg, BasicBlock block) { + private FallsThroughStmt addNewArgToPhi(Stmt phiStmt, Local arg, BasicBlock block) { - Stmt newPhiStmt = null; + FallsThroughStmt newPhiStmt = null; for (Value use : phiStmt.getUses()) { if (use instanceof JPhiExpr) { JPhiExpr newPhiExpr = (JPhiExpr) use; diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java index 8666144261e..96cbedac390 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java @@ -8,11 +8,14 @@ import java.util.Set; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.StringConstant; import sootup.core.jimple.common.expr.JEqExpr; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -89,9 +92,9 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { Local b = JavaJimple.newLocal("b", stringType); StringConstant stringConstant = javaJimple.newStringConstant("str"); - Stmt strToA = JavaJimple.newAssignStmt(a, stringConstant, noPositionInfo); + FallsThroughStmt strToA = JavaJimple.newAssignStmt(a, stringConstant, noPositionInfo); - Stmt strToB; + FallsThroughStmt strToB; StringConstant anotherStringConstant; JEqExpr jEqExpr; switch (constantCondition) { @@ -132,12 +135,13 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { Set locals = ImmutableUtils.immutableSet(a, b); Body.BodyBuilder bodyBuilder = Body.builder(); + final MutableStmtGraph stmtGraph = bodyBuilder.getStmtGraph(); bodyBuilder.setLocals(locals); bodyBuilder.setStartingStmt(strToA); - bodyBuilder.addFlow(strToA, strToB); - bodyBuilder.addFlow(strToB, ifStmt); - bodyBuilder.addFlow(ifStmt, reta); - bodyBuilder.addFlow(ifStmt, retb); + stmtGraph.putEdge(strToA, strToB); + stmtGraph.putEdge(strToB, ifStmt); + stmtGraph.putEdge(ifStmt, JIfStmt.FALSE_BRANCH_IDX, reta); + stmtGraph.putEdge(ifStmt, JIfStmt.TRUE_BRANCH_IDX, retb); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java index 7f6746222b1..804018076a3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java @@ -7,6 +7,7 @@ import java.util.Set; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; @@ -17,8 +18,7 @@ import sootup.core.jimple.common.expr.Expr; import sootup.core.jimple.common.expr.JCastExpr; import sootup.core.jimple.common.ref.IdentityRef; -import sootup.core.jimple.common.stmt.JAssignStmt; -import sootup.core.jimple.common.stmt.Stmt; +import sootup.core.jimple.common.stmt.*; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; import sootup.core.types.VoidType; @@ -58,60 +58,62 @@ public class CopyPropagatorTest { // build Stmts // r0 := @this Test - Stmt startingStmt = JavaJimple.newIdentityStmt(r0, identityRef, noStmtPositionInfo); + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(r0, identityRef, noStmtPositionInfo); // r1 = new ref Expr expr = JavaJimple.newNewExpr(refType); - Stmt stmt1 = JavaJimple.newAssignStmt(r1, expr, noStmtPositionInfo); + FallsThroughStmt stmt1 = JavaJimple.newAssignStmt(r1, expr, noStmtPositionInfo); // r2 = r1 - Stmt stmt2 = JavaJimple.newAssignStmt(r2, r1, noStmtPositionInfo); + FallsThroughStmt stmt2 = JavaJimple.newAssignStmt(r2, r1, noStmtPositionInfo); // r3 = r2 - Stmt stmt3 = JavaJimple.newAssignStmt(r3, r2, noStmtPositionInfo); + FallsThroughStmt stmt3 = JavaJimple.newAssignStmt(r3, r2, noStmtPositionInfo); // r4 = r3 - Stmt stmt4 = JavaJimple.newAssignStmt(r4, r3, noStmtPositionInfo); + FallsThroughStmt stmt4 = JavaJimple.newAssignStmt(r4, r3, noStmtPositionInfo); // return Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); // r3 = r1; - Stmt estmt3 = JavaJimple.newAssignStmt(r3, r1, noStmtPositionInfo); + FallsThroughStmt estmt3 = JavaJimple.newAssignStmt(r3, r1, noStmtPositionInfo); // r4 = r1 - Stmt estmt4 = JavaJimple.newAssignStmt(r4, r1, noStmtPositionInfo); + FallsThroughStmt estmt4 = JavaJimple.newAssignStmt(r4, r1, noStmtPositionInfo); // i1 = 5 - Stmt stmt5 = JavaJimple.newAssignStmt(i1, IntConstant.getInstance(5), noStmtPositionInfo); + FallsThroughStmt stmt5 = + JavaJimple.newAssignStmt(i1, IntConstant.getInstance(5), noStmtPositionInfo); // i2 = 0 - Stmt stmt6 = JavaJimple.newAssignStmt(i2, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt stmt6 = + JavaJimple.newAssignStmt(i2, IntConstant.getInstance(0), noStmtPositionInfo); // if i2 > i1 goto AbstractConditionExpr condition = JavaJimple.newGtExpr(i2, i1); - Stmt stmt7 = JavaJimple.newIfStmt(condition, noStmtPositionInfo); + Stmt ifStmt7 = JavaJimple.newIfStmt(condition, noStmtPositionInfo); // i3 = i1 + 1 Expr add1 = JavaJimple.newAddExpr(i1, IntConstant.getInstance(1)); - Stmt stmt8 = JavaJimple.newAssignStmt(i3, add1, noStmtPositionInfo); + FallsThroughStmt stmt8 = JavaJimple.newAssignStmt(i3, add1, noStmtPositionInfo); // i2 = i2 + 1 Expr add2 = JavaJimple.newAddExpr(i2, IntConstant.getInstance(1)); - Stmt stmt9 = JavaJimple.newAssignStmt(i2, add2, noStmtPositionInfo); + FallsThroughStmt stmt9 = JavaJimple.newAssignStmt(i2, add2, noStmtPositionInfo); Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); // if i2 > 5 goto AbstractConditionExpr econdition = JavaJimple.newGtExpr(i2, IntConstant.getInstance(5)); - Stmt estmt7 = JavaJimple.newIfStmt(econdition, noStmtPositionInfo); + Stmt eifstmt7 = JavaJimple.newIfStmt(econdition, noStmtPositionInfo); // i3 = 5 + 1 Expr eadd1 = JavaJimple.newAddExpr(IntConstant.getInstance(5), IntConstant.getInstance(1)); - Stmt estmt8 = JavaJimple.newAssignStmt(i3, eadd1, noStmtPositionInfo); + FallsThroughStmt estmt8 = JavaJimple.newAssignStmt(i3, eadd1, noStmtPositionInfo); // r0 := @this Test; r1 = (ref) 0; r2 = (ref) 0L; r3 = (ref) 1; r4 = r1, r5 = r2 // r1 = (ref) 0 JCastExpr intCast = JavaJimple.newCastExpr(IntConstant.getInstance(0), refType); - Stmt stmt10 = JavaJimple.newAssignStmt(r1, intCast, noStmtPositionInfo); + FallsThroughStmt stmt10 = JavaJimple.newAssignStmt(r1, intCast, noStmtPositionInfo); // r2 = (ref) 0L JCastExpr longCast = JavaJimple.newCastExpr(LongConstant.getInstance(0), refType); - Stmt stmt11 = JavaJimple.newAssignStmt(r2, longCast, noStmtPositionInfo); + FallsThroughStmt stmt11 = JavaJimple.newAssignStmt(r2, longCast, noStmtPositionInfo); // r3 = (ref) 1 JCastExpr intCast1 = JavaJimple.newCastExpr(IntConstant.getInstance(1), refType); - Stmt stmt12 = JavaJimple.newAssignStmt(r3, intCast1, noStmtPositionInfo); + FallsThroughStmt stmt12 = JavaJimple.newAssignStmt(r3, intCast1, noStmtPositionInfo); // r5 = r2 - Stmt stmt13 = JavaJimple.newAssignStmt(r5, r2, noStmtPositionInfo); + FallsThroughStmt stmt13 = JavaJimple.newAssignStmt(r5, r2, noStmtPositionInfo); // r6 = r3 - Stmt stmt14 = JavaJimple.newAssignStmt(r6, r3, noStmtPositionInfo); + FallsThroughStmt stmt14 = JavaJimple.newAssignStmt(r6, r3, noStmtPositionInfo); JAssignStmt eestmt4 = JavaJimple.newAssignStmt(r4, NullConstant.getInstance(), noStmtPositionInfo); @@ -173,13 +175,14 @@ private Body createChainBody() { Set locals = ImmutableUtils.immutableSet(r0, r1, r2, r3, r4); builder.setLocals(locals); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, stmt4); - builder.addFlow(stmt4, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, stmt4); + stmtGraph.putEdge(stmt4, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -201,13 +204,14 @@ private Body createExpectedChainBody() { Set locals = ImmutableUtils.immutableSet(r0, r1, r2, r3, r4); builder.setLocals(locals); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, estmt3); - builder.addFlow(estmt3, estmt4); - builder.addFlow(estmt4, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, estmt3); + stmtGraph.putEdge(estmt3, estmt4); + stmtGraph.putEdge(estmt4, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -232,16 +236,17 @@ private Body.BodyBuilder createLoopBody() { Set locals = ImmutableUtils.immutableSet(r0, i1, i2, i3); builder.setLocals(locals); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt5); - builder.addFlow(stmt5, stmt6); - builder.addFlow(stmt6, stmt7); - builder.addFlow(stmt7, stmt8); - builder.addFlow(stmt8, stmt9); - builder.addFlow(stmt9, gotoStmt); - builder.addFlow(gotoStmt, stmt7); - builder.addFlow(stmt7, ret); + stmtGraph.putEdge(startingStmt, stmt5); + stmtGraph.putEdge(stmt5, stmt6); + stmtGraph.putEdge(stmt6, ifStmt7); + stmtGraph.putEdge(ifStmt7, JIfStmt.FALSE_BRANCH_IDX, stmt8); + stmtGraph.putEdge(stmt8, stmt9); + stmtGraph.putEdge(stmt9, gotoStmt); + stmtGraph.putEdge(gotoStmt, JGotoStmt.BRANCH_IDX, ifStmt7); + stmtGraph.putEdge(ifStmt7, JIfStmt.TRUE_BRANCH_IDX, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -262,16 +267,16 @@ private Body createExpectedLoopBody() { Set locals = ImmutableUtils.immutableSet(r0, i1, i2, i3); builder.setLocals(locals); - + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt5); - builder.addFlow(stmt5, stmt6); - builder.addFlow(stmt6, estmt7); - builder.addFlow(estmt7, estmt8); - builder.addFlow(estmt8, stmt9); - builder.addFlow(stmt9, gotoStmt); - builder.addFlow(gotoStmt, estmt7); - builder.addFlow(estmt7, ret); + stmtGraph.putEdge(startingStmt, stmt5); + stmtGraph.putEdge(stmt5, stmt6); + stmtGraph.putEdge(stmt6, eifstmt7); + stmtGraph.putEdge(eifstmt7, JIfStmt.FALSE_BRANCH_IDX, estmt8); + stmtGraph.putEdge(estmt8, stmt9); + stmtGraph.putEdge(stmt9, gotoStmt); + stmtGraph.putEdge(gotoStmt, JGotoStmt.BRANCH_IDX, eifstmt7); + stmtGraph.putEdge(eifstmt7, JIfStmt.TRUE_BRANCH_IDX, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -295,15 +300,16 @@ private Body createCastExprBody() { Set locals = ImmutableUtils.immutableSet(r0, r1, r2, r3, r4, r5); builder.setLocals(locals); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt10); - builder.addFlow(stmt10, stmt11); - builder.addFlow(stmt11, stmt12); - builder.addFlow(stmt12, estmt4); - builder.addFlow(estmt4, stmt13); - builder.addFlow(stmt13, stmt14); - builder.addFlow(stmt14, ret); + stmtGraph.putEdge(startingStmt, stmt10); + stmtGraph.putEdge(stmt10, stmt11); + stmtGraph.putEdge(stmt11, stmt12); + stmtGraph.putEdge(stmt12, estmt4); + stmtGraph.putEdge(estmt4, stmt13); + stmtGraph.putEdge(stmt13, stmt14); + stmtGraph.putEdge(stmt14, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -325,15 +331,16 @@ private Body createExpectedCastExprBody() { Set locals = ImmutableUtils.immutableSet(r0, r1, r2, r3, r4, r5); builder.setLocals(locals); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt10); - builder.addFlow(stmt10, stmt11); - builder.addFlow(stmt11, stmt12); - builder.addFlow(stmt12, eestmt4); - builder.addFlow(eestmt4, estmt13); - builder.addFlow(estmt13, stmt14); - builder.addFlow(stmt14, ret); + stmtGraph.putEdge(startingStmt, stmt10); + stmtGraph.putEdge(stmt10, stmt11); + stmtGraph.putEdge(stmt11, stmt12); + stmtGraph.putEdge(stmt12, eestmt4); + stmtGraph.putEdge(eestmt4, estmt13); + stmtGraph.putEdge(estmt13, stmt14); + stmtGraph.putEdge(stmt14, ret); // set startingStmt builder.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java index 9b9f864492d..23fb44c68ed 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java @@ -4,11 +4,14 @@ import java.util.*; import org.junit.Test; +import sootup.core.graph.MutableStmtGraph; import sootup.core.graph.StmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.types.PrimitiveType; @@ -56,7 +59,8 @@ public void conditionalToRemovedBlock() { JavaJimple.newLtExpr(IntConstant.getInstance(10), IntConstant.getInstance(20)), noPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); - Stmt intToA = JavaJimple.newAssignStmt(a, IntConstant.getInstance(42), noPositionInfo); + FallsThroughStmt intToA = + JavaJimple.newAssignStmt(a, IntConstant.getInstance(42), noPositionInfo); Body.BodyBuilder builder = Body.builder(); builder.setStartingStmt(conditional); @@ -65,11 +69,11 @@ public void conditionalToRemovedBlock() { .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); builder.setLocals(locals); - builder.setPosition(NoPositionInformation.getInstance()); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); - builder.addFlow(conditional, intToA); - builder.addFlow(conditional, ret); - builder.addFlow(intToA, ret); + stmtGraph.putEdge(conditional, JIfStmt.FALSE_BRANCH_IDX, intToA); + stmtGraph.putEdge(conditional, JIfStmt.TRUE_BRANCH_IDX, ret); + stmtGraph.putEdge(intToA, ret); Body beforeBody = builder.build(); new DeadAssignmentEliminator().interceptBody(builder, null); @@ -115,25 +119,29 @@ private static Body.BodyBuilder createBody(boolean essentialOption) { Local b = JavaJimple.newLocal("b", objectType); Local c = JavaJimple.newLocal("c", PrimitiveType.getInt()); - Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); + FallsThroughStmt strToA = + JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); Stmt ret = JavaJimple.newReturnStmt(a, noPositionInfo); Set locals = ImmutableUtils.immutableSet(a, b, c); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(strToA); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(strToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); if (essentialOption) { - Stmt newToB = JavaJimple.newAssignStmt(b, JavaJimple.newNewExpr(objectType), noPositionInfo); - builder.addFlow(strToA, newToB); - builder.addFlow(newToB, ret); + FallsThroughStmt newToB = + JavaJimple.newAssignStmt(b, JavaJimple.newNewExpr(objectType), noPositionInfo); + stmtGraph.putEdge(strToA, newToB); + stmtGraph.putEdge(newToB, ret); } else { - Stmt intToC = JavaJimple.newAssignStmt(c, IntConstant.getInstance(42), noPositionInfo); - builder.addFlow(strToA, intToC); - builder.addFlow(intToC, ret); + FallsThroughStmt intToC = + JavaJimple.newAssignStmt(c, IntConstant.getInstance(42), noPositionInfo); + stmtGraph.putEdge(strToA, intToC); + stmtGraph.putEdge(intToC, ret); } builder.setLocals(locals); builder.setPosition(NoPositionInformation.getInstance()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java index c3c1c4ed964..b58d345143e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java @@ -5,10 +5,13 @@ import java.util.Set; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.Position; @@ -37,12 +40,14 @@ public class LocalLivenessAnalyserTest { Local b = JavaJimple.newLocal("b", intType); Local c = JavaJimple.newLocal("c", intType); - Stmt aeq0 = JavaJimple.newAssignStmt(a, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt beqaplus1 = + FallsThroughStmt aeq0 = + JavaJimple.newAssignStmt(a, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt beqaplus1 = JavaJimple.newAssignStmt( b, JavaJimple.newAddExpr(a, IntConstant.getInstance(0)), noStmtPositionInfo); - Stmt ceqcplusb = JavaJimple.newAssignStmt(c, JavaJimple.newAddExpr(c, b), noStmtPositionInfo); - Stmt aeqbplus2 = + FallsThroughStmt ceqcplusb = + JavaJimple.newAssignStmt(c, JavaJimple.newAddExpr(c, b), noStmtPositionInfo); + FallsThroughStmt aeqbplus2 = JavaJimple.newAssignStmt( a, JavaJimple.newAddExpr(b, IntConstant.getInstance(2)), noStmtPositionInfo); Stmt ifalt9 = @@ -87,6 +92,7 @@ public void testLivenessAnalyser() { private Body createBody() { Body.BodyBuilder builder = Body.builder(); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); builder.setMethodSignature(methodSignature); // build set locals @@ -95,12 +101,12 @@ private Body createBody() { builder.setLocals(locals); // set graph - builder.addFlow(aeq0, beqaplus1); - builder.addFlow(beqaplus1, ceqcplusb); - builder.addFlow(ceqcplusb, aeqbplus2); - builder.addFlow(aeqbplus2, ifalt9); - builder.addFlow(ifalt9, ret); - builder.addFlow(ifalt9, beqaplus1); + stmtGraph.putEdge(aeq0, beqaplus1); + stmtGraph.putEdge(beqaplus1, ceqcplusb); + stmtGraph.putEdge(ceqcplusb, aeqbplus2); + stmtGraph.putEdge(aeqbplus2, ifalt9); + stmtGraph.putEdge(ifalt9, JIfStmt.FALSE_BRANCH_IDX, ret); + stmtGraph.putEdge(ifalt9, JIfStmt.TRUE_BRANCH_IDX, beqaplus1); // set first stmt builder.setStartingStmt(aeq0); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java index 87e1dc44911..8bc99f7b8de 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java @@ -14,6 +14,9 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; +import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.Position; @@ -58,37 +61,48 @@ public class LocalPackerTest { Local el4 = JavaJimple.newLocal("l3", exception); // build stmts - Stmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); - Stmt identityStmt0 = JavaJimple.newIdentityStmt(l1hash1, identityRef0, noStmtPositionInfo); - Stmt identityStmt1 = JavaJimple.newIdentityStmt(l2hash2, identityRef1, noStmtPositionInfo); - Stmt stmt1 = JavaJimple.newAssignStmt(l3, IntConstant.getInstance(10), noStmtPositionInfo); - Stmt stmt2 = JavaJimple.newAssignStmt(l2hash3, l3, noStmtPositionInfo); - Stmt stmt3 = JavaJimple.newAssignStmt(l1hash4, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt stmt4 = + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); + FallsThroughStmt identityStmt0 = + JavaJimple.newIdentityStmt(l1hash1, identityRef0, noStmtPositionInfo); + FallsThroughStmt identityStmt1 = + JavaJimple.newIdentityStmt(l2hash2, identityRef1, noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l3, IntConstant.getInstance(10), noStmtPositionInfo); + FallsThroughStmt stmt2 = JavaJimple.newAssignStmt(l2hash3, l3, noStmtPositionInfo); + FallsThroughStmt stmt3 = + JavaJimple.newAssignStmt(l1hash4, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt stmt4 = JavaJimple.newAssignStmt( l1hash5, JavaJimple.newAddExpr(l1hash4, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt5 = + FallsThroughStmt stmt5 = JavaJimple.newAssignStmt( l1hash5, JavaJimple.newAddExpr(l1hash5, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l1hash5, l3), noStmtPositionInfo); + FallsThroughStmt stmt6 = + JavaJimple.newIfStmt(JavaJimple.newGtExpr(l1hash5, l3), noStmtPositionInfo); Stmt gt = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); - Stmt trapHandler = JavaJimple.newIdentityStmt(l4, caughtExceptionRef, noStmtPositionInfo); + FallsThroughStmt trapHandler = + JavaJimple.newIdentityStmt(l4, caughtExceptionRef, noStmtPositionInfo); Stmt throwStmt = JavaJimple.newThrowStmt(l4, noStmtPositionInfo); - Stmt eidentityStmt0 = JavaJimple.newIdentityStmt(l1, identityRef0, noStmtPositionInfo); - Stmt eidentityStmt1 = JavaJimple.newIdentityStmt(l2, identityRef1, noStmtPositionInfo); - Stmt estmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(10), noStmtPositionInfo); - Stmt estmt2 = JavaJimple.newAssignStmt(l2, l1, noStmtPositionInfo); - Stmt estmt3 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt estmt4 = + FallsThroughStmt eidentityStmt0 = + JavaJimple.newIdentityStmt(l1, identityRef0, noStmtPositionInfo); + FallsThroughStmt eidentityStmt1 = + JavaJimple.newIdentityStmt(l2, identityRef1, noStmtPositionInfo); + FallsThroughStmt estmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(10), noStmtPositionInfo); + FallsThroughStmt estmt2 = JavaJimple.newAssignStmt(l2, l1, noStmtPositionInfo); + FallsThroughStmt estmt3 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt estmt4 = JavaJimple.newAssignStmt( l2, JavaJimple.newAddExpr(l2, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt estmt5 = + FallsThroughStmt estmt5 = JavaJimple.newAssignStmt( l2, JavaJimple.newAddExpr(l2, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt estmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l2, l1), noStmtPositionInfo); - Stmt etrapHandler = JavaJimple.newIdentityStmt(el4, caughtExceptionRef, noStmtPositionInfo); + FallsThroughStmt estmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l2, l1), noStmtPositionInfo); + FallsThroughStmt etrapHandler = + JavaJimple.newIdentityStmt(el4, caughtExceptionRef, noStmtPositionInfo); Stmt ethrowStmt = JavaJimple.newThrowStmt(el4, noStmtPositionInfo); /** @@ -247,9 +261,9 @@ private Body.BodyBuilder createBodyBuilder() { graph.putEdge(stmt3, stmt4); graph.putEdge(stmt4, stmt5); graph.putEdge(stmt5, stmt6); - graph.putEdge(stmt6, gt); - graph.putEdge(gt, stmt5); - graph.putEdge(stmt6, ret); + graph.putEdge(stmt6, JIfStmt.FALSE_BRANCH_IDX, gt); + graph.putEdge(gt, JGotoStmt.BRANCH_IDX, stmt5); + graph.putEdge(stmt6, JIfStmt.TRUE_BRANCH_IDX, ret); builder.setStartingStmt(startingStmt); @@ -281,9 +295,9 @@ private Body createExpectedBody() { graph.putEdge(estmt3, estmt4); graph.putEdge(estmt4, estmt5); graph.putEdge(estmt5, estmt6); - graph.putEdge(estmt6, gt); - graph.putEdge(gt, estmt5); - graph.putEdge(estmt6, ret); + graph.putEdge(estmt6, JIfStmt.FALSE_BRANCH_IDX, gt); + graph.putEdge(gt, JGotoStmt.BRANCH_IDX, estmt5); + graph.putEdge(estmt6, JIfStmt.TRUE_BRANCH_IDX, ret); builder.setStartingStmt(startingStmt); @@ -324,7 +338,7 @@ private Body.BodyBuilder createTrapBody() { graph.putEdge(stmt4, stmt5); graph.putEdge(stmt5, stmt6); graph.putEdge(stmt6, gt); - graph.putEdge(gt, stmt5); + graph.putEdge(gt, JGotoStmt.BRANCH_IDX, stmt5); graph.putEdge(stmt6, ret); graph.putEdge(trapHandler, throwStmt); @@ -360,7 +374,7 @@ private Body.BodyBuilder createExpectedTrapBody() { graph.putEdge(estmt4, estmt5); graph.putEdge(estmt6, gt); - graph.putEdge(gt, estmt5); + graph.putEdge(gt, JGotoStmt.BRANCH_IDX, estmt5); graph.putEdge(estmt6, ret); graph.putEdge(etrapHandler, ethrowStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java index 7b3f0bc0705..2aecffbe021 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java @@ -11,6 +11,8 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.Position; @@ -255,7 +257,7 @@ private Body createBBBody() { Stmt stmt5 = JavaJimple.newAssignStmt( l1, JavaJimple.newSubExpr(l1, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt6 = + FallsThroughStmt stmt6 = JavaJimple.newAssignStmt( l1, JavaJimple.newAddExpr(l1, IntConstant.getInstance(2)), noStmtPositionInfo); Stmt ret = JavaJimple.newReturnStmt(l1, noStmtPositionInfo); @@ -264,7 +266,7 @@ private Body createBBBody() { graph.setEdges(stmt2, Arrays.asList(stmt3, stmt5)); graph.addBlock(Arrays.asList(stmt3, stmt4), Collections.emptyMap()); graph.addBlock(Arrays.asList(stmt5, stmt6), Collections.emptyMap()); - graph.putEdge(stmt4, ret); + graph.putEdge(stmt4, JGotoStmt.BRANCH_IDX, ret); graph.putEdge(stmt6, ret); graph.setStartingStmt(startingStmt); @@ -316,7 +318,7 @@ private Body createExpectedBBBody() { l1hash3, JavaJimple.newSubExpr(l1hash1, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt6 = + FallsThroughStmt stmt6 = JavaJimple.newAssignStmt( l1hash2, JavaJimple.newAddExpr(l1hash3, IntConstant.getInstance(2)), @@ -326,7 +328,7 @@ private Body createExpectedBBBody() { graph.addBlock(Arrays.asList(startingStmt, stmt1, stmt2), Collections.emptyMap()); graph.setEdges(stmt2, Arrays.asList(stmt3, stmt5)); graph.addBlock(Arrays.asList(stmt3, stmt4), Collections.emptyMap()); - graph.putEdge(stmt4, ret); + graph.putEdge(stmt4, JGotoStmt.BRANCH_IDX, ret); graph.addBlock(Arrays.asList(stmt5, stmt6), Collections.emptyMap()); graph.putEdge(stmt6, ret); @@ -465,7 +467,7 @@ private Body createLoopBody() { graph.addBlock(Arrays.asList(startingStmt, stmt1, stmt2, stmt3, stmt4), Collections.emptyMap()); graph.setEdges(stmt4, Arrays.asList(stmt5, ret)); graph.addBlock(Arrays.asList(stmt5, stmt6, stmt7, stmt8), Collections.emptyMap()); - graph.putEdge(stmt8, stmt2); + graph.putEdge(stmt8, JGotoStmt.BRANCH_IDX, stmt2); graph.setStartingStmt(startingStmt); @@ -510,7 +512,7 @@ private Body createExpectedLoopBody() { graph.addBlock(Arrays.asList(startingStmt, stmt1, stmt2, stmt3, stmt4), Collections.emptyMap()); graph.setEdges(stmt4, Arrays.asList(stmt5, ret)); graph.addBlock(Arrays.asList(stmt5, stmt6, stmt7, stmt8), Collections.emptyMap()); - graph.putEdge(stmt8, stmt2); + graph.putEdge(stmt8, JGotoStmt.BRANCH_IDX, stmt2); graph.setStartingStmt(startingStmt); @@ -534,11 +536,15 @@ private Body.BodyBuilder createTrapBody() { builder.setLocals(locals); - Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt stmt2 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt stmt3 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt stmt4 = JavaJimple.newIdentityStmt(stack3, caughtExceptionRef, noStmtPositionInfo); - Stmt stmt5 = JavaJimple.newAssignStmt(l3, l1, noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt stmt2 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); + FallsThroughStmt stmt3 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); + FallsThroughStmt stmt4 = + JavaJimple.newIdentityStmt(stack3, caughtExceptionRef, noStmtPositionInfo); + FallsThroughStmt stmt5 = JavaJimple.newAssignStmt(l3, l1, noStmtPositionInfo); Stmt stmt6 = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); @@ -549,7 +555,7 @@ private Body.BodyBuilder createTrapBody() { graph.addNode(stmt3); graph.putEdge(stmt2, stmt3); graph.putEdge(stmt3, ret); - graph.putEdge(stmt6, ret); + graph.putEdge(stmt6, JGotoStmt.BRANCH_IDX, ret); graph.setStartingStmt(startingStmt); @@ -571,11 +577,11 @@ private Body createExpectedTrapBody() { Stmt l1hash1assign0Stmt = JavaJimple.newAssignStmt(l1hash1, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt l1hash2assign1Stmt = + FallsThroughStmt l1hash2assign1Stmt = JavaJimple.newAssignStmt(l1hash2, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt l2assign2Stmt = + FallsThroughStmt l2assign2Stmt = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt exceptionCatchStmt = + FallsThroughStmt exceptionCatchStmt = JavaJimple.newIdentityStmt(stack3, caughtExceptionRef, noStmtPositionInfo); Stmt l3assignl1hash2Stmt = JavaJimple.newAssignStmt(l3, l1hash2, noStmtPositionInfo); Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); @@ -589,7 +595,7 @@ private Body createExpectedTrapBody() { graph.addNode(l2assign2Stmt); graph.putEdge(l1hash2assign1Stmt, l2assign2Stmt); graph.putEdge(l2assign2Stmt, ret); - graph.putEdge(gotoStmt, ret); + graph.putEdge(gotoStmt, JGotoStmt.BRANCH_IDX, ret); graph.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java index 412d12564ef..c109af4935f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java @@ -13,6 +13,8 @@ import sootup.core.jimple.basic.Trap; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.Position; @@ -46,22 +48,26 @@ public class TrapTightenerTest { ClassType exception = factory.getClassType("java.lang.Throwable"); JavaJimple javaJimple = JavaJimple.getInstance(); IdentityRef caughtExceptionRef = javaJimple.newCaughtExceptionRef(); - Stmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); // stmts - Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt stmt2 = JavaJimple.newEnterMonitorStmt(l1, noStmtPositionInfo); - Stmt stmt3 = JavaJimple.newAssignStmt(l2, l1, noStmtPositionInfo); - Stmt stmt4 = JavaJimple.newExitMonitorStmt(l2, noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); + FallsThroughStmt stmt2 = JavaJimple.newEnterMonitorStmt(l1, noStmtPositionInfo); + FallsThroughStmt stmt3 = JavaJimple.newAssignStmt(l2, l1, noStmtPositionInfo); + FallsThroughStmt stmt4 = JavaJimple.newExitMonitorStmt(l2, noStmtPositionInfo); Stmt stmt5 = JavaJimple.newGotoStmt(noStmtPositionInfo); - Stmt stmt6 = JavaJimple.newIdentityStmt(l3, caughtExceptionRef, noStmtPositionInfo); - Stmt stmt7 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt stmt8 = JavaJimple.newExitMonitorStmt(l2, noStmtPositionInfo); + FallsThroughStmt stmt6 = JavaJimple.newIdentityStmt(l3, caughtExceptionRef, noStmtPositionInfo); + FallsThroughStmt stmt7 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); + FallsThroughStmt stmt8 = JavaJimple.newExitMonitorStmt(l2, noStmtPositionInfo); Stmt stmt9 = JavaJimple.newThrowStmt(l3, noStmtPositionInfo); - Stmt stmt10 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(3), noStmtPositionInfo); - Stmt stmt11 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(4), noStmtPositionInfo); + FallsThroughStmt stmt10 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(3), noStmtPositionInfo); + FallsThroughStmt stmt11 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(4), noStmtPositionInfo); // trap Trap trap1 = new Trap(exception, stmt2, stmt5, stmt6); Trap trap2 = new Trap(exception, stmt1, stmt5, stmt6); @@ -180,7 +186,7 @@ private Body createSimpleBody() { graph.putEdge(stmt10, stmt5); graph.putEdge(stmt6, stmt11); graph.putEdge(stmt11, stmt9); - graph.putEdge(stmt5, ret); + graph.putEdge(stmt5, JGotoStmt.BRANCH_IDX, ret); // build startingStmt builder.setStartingStmt(startingStmt); @@ -203,7 +209,7 @@ private Body creatBodyWithMonitor() { graph.addBlock(Arrays.asList(stmt2, stmt3, stmt4), Collections.singletonMap(exception, stmt6)); graph.putEdge(stmt1, stmt2); graph.putEdge(stmt4, stmt5); - graph.putEdge(stmt5, ret); + graph.putEdge(stmt5, JGotoStmt.BRANCH_IDX, ret); // build startingStmt builder.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java index c4bbc037a5d..d0936494a39 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java @@ -15,6 +15,7 @@ import sootup.core.jimple.basic.Trap; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -51,20 +52,24 @@ public class UnreachableCodeEliminatorTest { IdentityRef idRef = javaJimple.newCaughtExceptionRef(); // build stmts - Stmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); - Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt stmt2 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); + FallsThroughStmt stmt2 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt stmt3 = JavaJimple.newAssignStmt(l3, IntConstant.getInstance(3), noStmtPositionInfo); + FallsThroughStmt stmt3 = + JavaJimple.newAssignStmt(l3, IntConstant.getInstance(3), noStmtPositionInfo); - Stmt jump = JavaJimple.newGotoStmt(noStmtPositionInfo); + Stmt jGotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret1 = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); Stmt ret2 = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); - Stmt handlerStmt = JavaJimple.newIdentityStmt(stack0, idRef, noStmtPositionInfo); - Stmt beginStmt = JavaJimple.newAssignStmt(l3, stack0, noStmtPositionInfo); - Stmt endStmt = JavaJimple.newAssignStmt(l4, IntConstant.getInstance(4), noStmtPositionInfo); + FallsThroughStmt handlerStmt = JavaJimple.newIdentityStmt(stack0, idRef, noStmtPositionInfo); + FallsThroughStmt beginStmt = JavaJimple.newAssignStmt(l3, stack0, noStmtPositionInfo); + FallsThroughStmt endStmt = + JavaJimple.newAssignStmt(l4, IntConstant.getInstance(4), noStmtPositionInfo); Trap trap2 = JavaJimple.newTrap(exception, beginStmt, beginStmt, handlerStmt); @@ -193,8 +198,8 @@ public void testTrappedBody3() { // build stmtsGraph for the builder graph.addBlock( Arrays.asList(startingStmt, stmt1, ret1), Collections.singletonMap(exception, handlerStmt)); - graph.addBlock(Arrays.asList(handlerStmt, jump)); - graph.putEdge(jump, ret1); + graph.addBlock(Arrays.asList(handlerStmt, jGotoStmt)); + graph.putEdge(jGotoStmt, 0, ret1); // set startingStmt graph.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchyTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchyTest.java index d9ba4e99aa3..676cfce1ba6 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchyTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/BytecodeHierarchyTest.java @@ -77,7 +77,7 @@ public void setUp() { new JavaClassPathAnalysisInputLocation(jarFile + File.pathSeparator + rtJarClassPath); JavaProject p = JavaProject.builder(new JavaLanguage(8)).addInputLocation(analysisInputLocation).build(); - view = p.createOnDemandView(); + view = p.createView(); ViewTypeHierarchy typeHierarchy = new ViewTypeHierarchy(view); // create types diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java index c3e78370808..f837968b729 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/typeresolving/TypeAssignerTestSuite.java @@ -35,7 +35,7 @@ public void buildView(String baseDir, String className) { .addInputLocation(analysisInputLocation) .addInputLocation(rtJar) .build(); - view = project.createOnDemandView(); + view = project.createView(); classType = identifierFactory.getClassType(className); clazz = view.getClass(classType).get(); } From e528214a54a3683b8f3e1cde63baadee8ecc8e02 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Fri, 20 Oct 2023 17:15:50 +0200 Subject: [PATCH 46/54] add deprecation notices.. --- .../src/main/java/sootup/core/model/Body.java | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index 547dfd215f8..f02194d2e8d 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -399,17 +399,19 @@ public List getStmts() { return cachedLinearizedStmts; } + /** Deprecated: please use methods of getStmtGraph() directly */ @Nonnull - public Set getLocals() { - return Collections.unmodifiableSet(locals); - } - - @Nonnull + @Deprecated public BodyBuilder setStartingStmt(@Nonnull Stmt startingStmt) { graph.setStartingStmt(startingStmt); return this; } + @Nonnull + public Set getLocals() { + return Collections.unmodifiableSet(locals); + } + @Nonnull public BodyBuilder setLocals(@Nonnull Set locals) { this.locals = locals; @@ -448,22 +450,34 @@ public void replaceLocal(@Nonnull Local oldLocal, @Nonnull Local newLocal) { } } - /** replace the oldStmt with newStmt in stmtGraph and branches */ + /** + * replace the oldStmt with newStmt in stmtGraph and branches + * + *

Deprecated: please use methods of getStmtGraph() directly + */ @Nonnull + @Deprecated public BodyBuilder replaceStmt(@Nonnull Stmt oldStmt, @Nonnull Stmt newStmt) { graph.replaceNode(oldStmt, newStmt); return this; } - /** remove the a stmt from the graph and stmt */ + /** + * remove the a stmt from the graph and stmt + * + *

Deprecated: please use methods of getStmtGraph() directly + */ @Nonnull + @Deprecated public BodyBuilder removeStmt(@Nonnull Stmt stmt) { graph.removeNode(stmt); cachedLinearizedStmts = null; return this; } + /** Deprecated: please use methods of getStmtGraph() directly */ @Nonnull + @Deprecated public BodyBuilder clearExceptionEdgesOf(@Nonnull Stmt stmt) { graph.clearExceptionalEdges(stmt); return this; @@ -475,20 +489,15 @@ public List getTraps() { return graph.getTraps(); } + /** Deprecated: please use methods of getStmtGraph() directly */ @Nonnull + @Deprecated public BodyBuilder addFlow(@Nonnull Stmt fromStmt, @Nonnull Stmt toStmt) { graph.putEdge(fromStmt, 0, toStmt); cachedLinearizedStmts = null; return this; } - @Nonnull - public BodyBuilder removeFlow(@Nonnull Stmt fromStmt, @Nonnull Stmt toStmt) { - graph.removeEdge(fromStmt, toStmt); - cachedLinearizedStmts = null; - return this; - } - public BodyBuilder setModifiers(@Nonnull Set modifiers) { this.modifiers = modifiers; return this; From 6717701668ec616f00a409aee017ccdfca627b2b Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Mon, 23 Oct 2023 11:20:05 +0200 Subject: [PATCH 47/54] elaborate api with improved typing: Stmts with zero expected successors can not be added as they are FallsThroughStmt nor BranchingStmt --- .../core/graph/MutableBlockStmtGraph.java | 50 +++++++---- .../sootup/core/graph/MutableStmtGraph.java | 10 ++- .../jimple/javabytecode/stmt/JSwitchStmt.java | 2 +- .../src/main/java/sootup/core/model/Body.java | 8 +- .../core/graph/MutableBlockStmtGraphTest.java | 61 +++++++------ .../interceptors/ConditionalBranchFolder.java | 3 +- .../bytecode/interceptors/AggregatorTest.java | 31 ++++--- .../CastAndReturnInlinerTest.java | 43 ++++++---- .../ConditionalBranchFolderTest.java | 5 +- .../ConstantPropagatorAndFolderTest.java | 19 +++-- .../interceptors/CopyPropagatorTest.java | 6 +- .../DeadAssignmentEliminatorTest.java | 5 +- .../EmptySwitchEliminatorTest.java | 37 ++++---- .../LocalLivenessAnalyserTest.java | 3 +- .../LocalNameStandardizerTest.java | 85 +++++++++++-------- .../interceptors/LocalPackerTest.java | 28 +++--- .../interceptors/LocalSplitterTest.java | 63 +++++++------- .../interceptors/NopEliminatorTest.java | 17 ++-- .../StaticSingleAssignmentFormerTest.java | 17 ++-- .../interceptors/TrapTightenerTest.java | 5 +- .../UnreachableCodeEliminatorTest.java | 17 ++-- .../UnusedLocalEliminatorTest.java | 22 +++-- .../core/printer/LegacyJimplePrinterTest.java | 34 ++++---- 23 files changed, 328 insertions(+), 243 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java index da053775e98..aa961c2f61d 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableBlockStmtGraph.java @@ -222,9 +222,9 @@ public void initializeWith( addNode(stmt, exceptionToHandlerMap); - if (stmt.fallsThrough()) { + if (stmt instanceof FallsThroughStmt) { // hint: possible bad performance if stmts is not instanceof RandomAccess - putEdge(stmt, 0, stmts.get(i + 1)); + putEdge((FallsThroughStmt) stmt, stmts.get(i + 1)); } if (stmt instanceof BranchingStmt) { @@ -249,10 +249,11 @@ public void initializeWith( + targetCount + "."); } + final BranchingStmt bStmt = (BranchingStmt) stmt; for (int j = 0; j < targets.size(); j++) { Stmt target = targets.get(j); // a possible fallsthrough (i.e. from IfStmt) is not in branchingMap - putEdge(stmt, j + idxOffset, target); + putEdge(bStmt, j + idxOffset, target); } } } @@ -894,11 +895,37 @@ public void insertBefore( } } + /** Replaces all SuccessorEdge(s) of from to oldTo by mewTo */ + @Override + public boolean replaceSucessorEdge(@Nonnull Stmt from, @Nonnull Stmt oldTo, @Nonnull Stmt newTo) { + final MutableBasicBlock mutableBasicBlock = stmtToBlock.get(from); + if (mutableBasicBlock == null) { + throw new IllegalArgumentException("stmt '" + from + "' does not exist in this StmtGraph!"); + } + final MutableBasicBlock oldTargetBlock = stmtToBlock.get(oldTo); + + boolean found = false; + for (ListIterator iterator = + mutableBasicBlock.getSuccessors().listIterator(); + iterator.hasNext(); ) { + MutableBasicBlock block = iterator.next(); + if (block == oldTargetBlock) { + iterator.set(getOrCreateBlock(newTo)); + found = true; + } + } + return found; + } + public void putEdge(@Nonnull FallsThroughStmt stmtA, @Nonnull Stmt stmtB) { - putEdge(stmtA, 0, stmtB); + putEdge_internal(stmtA, 0, stmtB); } - public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { + public void putEdge(@Nonnull BranchingStmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { + putEdge_internal(stmtA, succesorIdx, stmtB); + } + + protected void putEdge_internal(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { MutableBasicBlock blockA = stmtToBlock.get(stmtA); MutableBasicBlock blockB = stmtToBlock.get(stmtB); @@ -919,15 +946,8 @@ public void putEdge(@Nonnull Stmt stmtA, int succesorIdx, @Nonnull Stmt stmtB) { } } - if (blockA.getSuccessors().size() >= stmtA.getExpectedSuccessorCount()) { - throw new IllegalArgumentException( - "Can't add another flow - there are already enough flows i.e. " - + stmtA.getExpectedSuccessorCount() - + " outgoing from StmtA '" - + stmtA - + "'"); - } - + // TODO: [ms] check to refactor this directly into putEdge - Attention: JIfStmt is + // FallsThroughStmt AND BranchingStmt if (stmtA.branches()) { // branching Stmt A indicates the end of BlockA and connects to another BlockB: reuse or // create new @@ -1073,7 +1093,7 @@ public boolean removeEdge(@Nonnull Stmt from, @Nonnull Stmt to) { } @Override - public void setEdges(@Nonnull Stmt fromStmt, @Nonnull List targets) { + public void setEdges(@Nonnull BranchingStmt fromStmt, @Nonnull List targets) { if (fromStmt.getExpectedSuccessorCount() != targets.size()) { throw new IllegalArgumentException( "Size of Targets is not the amount of from's expected successors."); diff --git a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java index 36faeb1bddb..408290ffa8f 100644 --- a/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java +++ b/sootup.core/src/main/java/sootup/core/graph/MutableStmtGraph.java @@ -23,6 +23,7 @@ import java.util.*; import javax.annotation.Nonnull; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.types.ClassType; @@ -89,13 +90,16 @@ public void insertBefore(@Nonnull Stmt beforeStmt, @Nonnull FallsThroughStmt new */ public abstract void putEdge(@Nonnull FallsThroughStmt from, @Nonnull Stmt to); - public abstract void putEdge(@Nonnull Stmt from, int successorIdx, @Nonnull Stmt to); + public abstract void putEdge(@Nonnull BranchingStmt from, int successorIdx, @Nonnull Stmt to); + + public abstract boolean replaceSucessorEdge( + @Nonnull Stmt from, @Nonnull Stmt oldTo, @Nonnull Stmt newTo); /** replaces the current outgoing flows of "from" to "targets" */ - public abstract void setEdges(@Nonnull Stmt from, @Nonnull List targets); + public abstract void setEdges(@Nonnull BranchingStmt from, @Nonnull List targets); /** replaces the current outgoing flows of "from" to each target of "targets" */ - public void setEdges(@Nonnull Stmt from, @Nonnull Stmt... targets) { + public void setEdges(@Nonnull BranchingStmt from, @Nonnull Stmt... targets) { setEdges(from, Arrays.asList(targets)); } diff --git a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java index b6911287a72..69ac3649c59 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/javabytecode/stmt/JSwitchStmt.java @@ -82,7 +82,7 @@ public boolean isTableSwitch() { } @Nonnull - public Optional getDefaultTarget(Body body) { + public Optional getDefaultTarget(@Nonnull Body body) { return Optional.ofNullable(body.getBranchTargetsOf(this).get(values.size())); } diff --git a/sootup.core/src/main/java/sootup/core/model/Body.java b/sootup.core/src/main/java/sootup/core/model/Body.java index f02194d2e8d..706d8e7e8d8 100644 --- a/sootup.core/src/main/java/sootup/core/model/Body.java +++ b/sootup.core/src/main/java/sootup/core/model/Body.java @@ -353,8 +353,8 @@ public static BodyBuilder builder(@Nonnull Body body, Set modifi * Body.BodyBuilder builder = Body.builder(); * builder.setMethodSignature( ... ); * builder.setStartingStmt(stmt1); - * builder.addFlow(stmt1,stmt2); - * builder.addFlow(stmt2,stmt3); + * stmtGraph.putEdge(stmt1,stmt2); + * stmtGraph.putEdge(stmt2,stmt3); * ... * Body body = builder.build(); * @@ -492,8 +492,8 @@ public List getTraps() { /** Deprecated: please use methods of getStmtGraph() directly */ @Nonnull @Deprecated - public BodyBuilder addFlow(@Nonnull Stmt fromStmt, @Nonnull Stmt toStmt) { - graph.putEdge(fromStmt, 0, toStmt); + public BodyBuilder addFlow(@Nonnull FallsThroughStmt fromStmt, @Nonnull Stmt toStmt) { + graph.putEdge(fromStmt, toStmt); cachedLinearizedStmts = null; return this; } diff --git a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java index 79b710c64ab..7cd8a6e349d 100644 --- a/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java +++ b/sootup.core/src/test/java/sootup/core/graph/MutableBlockStmtGraphTest.java @@ -24,6 +24,7 @@ @Category(Java8Test.class) public class MutableBlockStmtGraphTest { + BranchingStmt firstGoto = new JGotoStmt(StmtPositionInfo.createNoStmtPositionInfo()); JNopStmt firstNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); JNopStmt secondNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); JNopStmt thirdNop = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); @@ -95,25 +96,25 @@ public void addNodeTest() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); assertEquals(0, graph.getBlocks().size()); - graph.addNode(firstNop); + graph.addNode(firstGoto); assertEquals(1, graph.getBlocks().size()); // test duplicate insertion of the same node - graph.addNode(firstNop); + graph.addNode(firstGoto); assertEquals(1, graph.getBlocks().size()); - assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); + assertEquals(1, graph.getBlockOf(firstGoto).getStmts().size()); graph.addNode(secondNop); assertEquals(2, graph.getBlocks().size()); - assertEquals(1, graph.getBlockOf(firstNop).getStmts().size()); + assertEquals(1, graph.getBlockOf(firstGoto).getStmts().size()); - graph.removeNode(firstNop); + graph.removeNode(firstGoto); assertEquals(1, graph.getBlocks().size()); assertEquals(1, graph.getBlockOf(secondNop).getStmts().size()); // removal of not existing try { - graph.removeNode(firstNop); + graph.removeNode(firstGoto); fail("should not be reachable due to exception"); } catch (Exception ignored) { } @@ -173,16 +174,15 @@ public void removeStmtConditionalTail() { @Test public void testSetEdges() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - graph.setStartingStmt(firstNop); - graph.setEdges(firstNop, Collections.singletonList(conditionalStmt)); - assertEquals(Arrays.asList(firstNop, conditionalStmt), graph.getBlockOf(firstNop).getStmts()); + graph.setStartingStmt(firstGoto); + graph.setEdges(firstGoto, Collections.singletonList(conditionalStmt)); + assertEquals(Arrays.asList(conditionalStmt), graph.successors(firstGoto)); graph.setEdges(conditionalStmt, Arrays.asList(secondNop, thirdNop)); - assertEquals(3, graph.getBlocks().size()); + assertEquals(4, graph.getBlocks().size()); assertEquals( - Arrays.asList(firstNop, conditionalStmt).toString(), - graph.getBlockOf(firstNop).getStmts().toString()); + Arrays.asList(firstGoto).toString(), graph.getBlockOf(firstGoto).getStmts().toString()); assertEquals( Collections.singletonList(secondNop).toString(), graph.getBlockOf(secondNop).getStmts().toString()); @@ -324,16 +324,16 @@ public void checkInfoMethods() { assertFalse(graph.hasEdgeConnecting(secondNop, firstNop)); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = IndexOutOfBoundsException.class) public void addBadSuccessorCount() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(firstNop, secondNop); - graph.putEdge(firstNop, thirdNop); + graph.putEdge(firstGoto, 1, thirdNop); } public void setBadSuccessorIdx() { MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(firstNop, 1, secondNop); + graph.putEdge(firstGoto, 1, secondNop); } @Test(expected = IllegalArgumentException.class) @@ -385,7 +385,7 @@ public void addBlockDirectly() { assertEquals(0, graph.getBlocks().size()); MutableBasicBlock blockA = new MutableBasicBlock(); - blockA.addStmt(firstNop); + blockA.addStmt(firstGoto); MutableBasicBlock blockB = new MutableBasicBlock(); blockB.addStmt(secondNop); MutableBasicBlock blockC = new MutableBasicBlock(); @@ -521,6 +521,7 @@ public void testBlockAddStmtInvalidDuplicateStmtObjectViaGraphDirectManiupaltion @Test(expected = IllegalArgumentException.class) public void testBlockStmtValidity() { + // try adding a stmt after branchingstmt -> definitely the last stmt of a block -> must fail MutableBasicBlock block = new MutableBasicBlock(); block.addStmt(conditionalStmt); block.addStmt(firstNop); @@ -528,7 +529,6 @@ public void testBlockStmtValidity() { @Test public void modifyTrapToCompleteBlock() { - MutableBlockStmtGraph graph = new MutableBlockStmtGraph(); graph.putEdge(firstNop, secondNop); assertEquals(1, graph.getBlocks().size()); @@ -628,7 +628,6 @@ public PackageName getPackageName() { graph1.putEdge(stmt2, JGotoStmt.BRANCH_IDX, returnStmt); graph1.putEdge(stmt3, JGotoStmt.BRANCH_IDX, returnStmt); - graph1.putEdge(catchStmt1, JGotoStmt.BRANCH_IDX, stmt3); { final List traps = graph1.getTraps(); @@ -843,7 +842,7 @@ public void addNode() { @Test public void setEdgesSimple() { - Stmt stmt1 = + BranchingStmt stmt1 = new JIfStmt( new JNeExpr(BooleanConstant.getInstance(1), BooleanConstant.getInstance(0)), StmtPositionInfo.createNoStmtPositionInfo()); @@ -876,10 +875,10 @@ public void removeNodeWOEdges() { @Test public void removeNodeWOPredecessors() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, 0, stmt2); + graph.putEdge(stmt1, stmt2); assertTrue(graph.getNodes().contains(stmt1)); assertEquals(Collections.singletonList(stmt2), graph.successors(stmt1)); @@ -900,10 +899,10 @@ public void removeNodeWOPredecessors() { @Test public void removeNodeWOSuccessors() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, 0, stmt2); + graph.putEdge(stmt1, stmt2); assertTrue(graph.getNodes().contains(stmt2)); assertEquals(Collections.singletonList(stmt2), graph.successors(stmt1)); @@ -928,10 +927,10 @@ public void removeNodeWOSuccessors() { @Test public void removeEdge() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); - graph.putEdge(stmt1, 0, stmt2); + graph.putEdge(stmt1, stmt2); assertEquals(1, graph.successors(stmt1).size()); assertTrue(graph.hasEdgeConnecting(stmt1, stmt2)); @@ -972,11 +971,11 @@ public void removeImpossibleEdge() { @Test public void putEdge() { - Stmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); - Stmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt1 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); + FallsThroughStmt stmt2 = new JNopStmt(StmtPositionInfo.createNoStmtPositionInfo()); MutableStmtGraph graph = new MutableBlockStmtGraph(); // stmt2 is not in the graph! - graph.putEdge(stmt1, 0, stmt2); + graph.putEdge(stmt1, stmt2); } @Test diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java index c69df1171b5..9aa9c3bf293 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java @@ -28,6 +28,7 @@ import sootup.core.graph.StmtGraph; import sootup.core.jimple.common.constant.Constant; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; @@ -85,7 +86,7 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi for (Stmt predecessor : stmtGraph.predecessors(ifStmt)) { stmtGraph.removeEdge(predecessor, ifStmt); // FIXME: [ms] fix successorIdx - stmtGraph.putEdge(predecessor, 0, tautologicSuccessor); + stmtGraph.putEdge((FallsThroughStmt) predecessor, tautologicSuccessor); } // removeFlow calls should be obsolete as of following removeStmt diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index 3605c3c457d..d336a4d8f68 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -8,6 +8,7 @@ import java.util.Set; import javax.annotation.Nonnull; import org.junit.Test; +import sootup.core.graph.MutableStmtGraph; import sootup.core.inputlocation.AnalysisInputLocation; import sootup.core.inputlocation.ClassLoadingOptions; import sootup.core.jimple.Jimple; @@ -16,6 +17,7 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.expr.JAddExpr; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.SootMethod; @@ -90,21 +92,23 @@ public void noAggregationWithUse() { Local a = JavaJimple.newLocal("a", fileType); Local b = JavaJimple.newLocal("b", fileType); - Stmt assignA = JavaJimple.newAssignStmt(a, JavaJimple.newNewExpr(fileType), noPositionInfo); + FallsThroughStmt assignA = + JavaJimple.newAssignStmt(a, JavaJimple.newNewExpr(fileType), noPositionInfo); // this use of `a` should prevent the aggregator from changing anything - Stmt useA = + FallsThroughStmt useA = JavaJimple.newInvokeStmt( Jimple.newSpecialInvokeExpr( a, JavaIdentifierFactory.getInstance().parseMethodSignature("()>")), noPositionInfo); - Stmt assignB = JavaJimple.newAssignStmt(b, a, noPositionInfo); + FallsThroughStmt assignB = JavaJimple.newAssignStmt(b, a, noPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noPositionInfo); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); - builder.setStartingStmt(assignA); - builder.addFlow(assignA, useA); - builder.addFlow(useA, assignB); - builder.addFlow(assignB, ret); + stmtGraph.setStartingStmt(assignA); + stmtGraph.putEdge(assignA, useA); + stmtGraph.putEdge(useA, assignB); + stmtGraph.putEdge(assignB, ret); builder.setMethodSignature( JavaIdentifierFactory.getInstance() @@ -122,8 +126,9 @@ private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { Local a = JavaJimple.newLocal("a", PrimitiveType.getInt()); Local b = JavaJimple.newLocal("b", PrimitiveType.getInt()); - Stmt intToA = JavaJimple.newAssignStmt(a, IntConstant.getInstance(7), noPositionInfo); - Stmt intToB; + FallsThroughStmt intToA = + JavaJimple.newAssignStmt(a, IntConstant.getInstance(7), noPositionInfo); + FallsThroughStmt intToB; if (withAggregation) { intToB = JavaJimple.newAssignStmt(b, new JAddExpr(a, IntConstant.getInstance(4)), noPositionInfo); @@ -135,13 +140,13 @@ private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { Set locals = ImmutableUtils.immutableSet(a, b); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(intToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); - - builder.addFlow(intToA, intToB); - builder.addFlow(intToB, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(intToA); + stmtGraph.putEdge(intToA, intToB); + stmtGraph.putEdge(intToB, ret); builder.setLocals(locals); builder.setPosition(NoPositionInformation.getInstance()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java index cf5541c65cf..a10fdd54e7f 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java @@ -6,8 +6,12 @@ import java.util.*; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.common.stmt.BranchingStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.util.ImmutableUtils; @@ -49,20 +53,23 @@ public void testModification() { Local a = JavaJimple.newLocal("a", objectType); Local b = JavaJimple.newLocal("b", stringType); - Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); - Stmt bToA = JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); + FallsThroughStmt strToA = + JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); + FallsThroughStmt bToA = + JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); Stmt ret = JavaJimple.newReturnStmt(b, noPositionInfo); - Stmt jump = JavaJimple.newGotoStmt(noPositionInfo); + BranchingStmt jump = JavaJimple.newGotoStmt(noPositionInfo); Set locals = ImmutableUtils.immutableSet(a, b); Body.BodyBuilder bodyBuilder = Body.builder(); bodyBuilder.setLocals(locals); - bodyBuilder.setStartingStmt(strToA); - bodyBuilder.addFlow(strToA, jump); - bodyBuilder.addFlow(jump, bToA); - bodyBuilder.addFlow(bToA, ret); + final MutableStmtGraph stmtGraph = bodyBuilder.getStmtGraph(); + stmtGraph.setStartingStmt(strToA); + stmtGraph.putEdge(strToA, jump); + stmtGraph.putEdge(jump, JGotoStmt.BRANCH_IDX, bToA); + stmtGraph.putEdge(bToA, ret); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() @@ -101,22 +108,26 @@ public void testNoModification() { Local b = JavaJimple.newLocal("b", stringType); Local c = JavaJimple.newLocal("c", stringType); - Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); - Stmt strToC = JavaJimple.newAssignStmt(c, javaJimple.newStringConstant("str2"), noPositionInfo); - Stmt bToA = JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); + FallsThroughStmt strToA = + JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); + FallsThroughStmt strToC = + JavaJimple.newAssignStmt(c, javaJimple.newStringConstant("str2"), noPositionInfo); + FallsThroughStmt bToA = + JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); // Note this returns c, not b, hence the cast and return must not be inlined Stmt ret = JavaJimple.newReturnStmt(c, noPositionInfo); - Stmt jump = JavaJimple.newGotoStmt(noPositionInfo); + BranchingStmt jump = JavaJimple.newGotoStmt(noPositionInfo); Set locals = ImmutableUtils.immutableSet(a, b); Body.BodyBuilder bodyBuilder = Body.builder(); bodyBuilder.setLocals(locals); - bodyBuilder.setStartingStmt(strToA); - bodyBuilder.addFlow(strToA, strToC); - bodyBuilder.addFlow(strToC, jump); - bodyBuilder.addFlow(jump, bToA); - bodyBuilder.addFlow(bToA, ret); + final MutableStmtGraph stmtGraph = bodyBuilder.getStmtGraph(); + stmtGraph.setStartingStmt(strToA); + stmtGraph.putEdge(strToA, strToC); + stmtGraph.putEdge(strToC, jump); + stmtGraph.putEdge(jump, JGotoStmt.BRANCH_IDX, bToA); + stmtGraph.putEdge(bToA, ret); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java index 96cbedac390..f6026ab68ba 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java @@ -14,6 +14,7 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.StringConstant; import sootup.core.jimple.common.expr.JEqExpr; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; @@ -128,7 +129,7 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { throw new IllegalArgumentException(); } - Stmt ifStmt = Jimple.newIfStmt(jEqExpr, noPositionInfo); + BranchingStmt ifStmt = Jimple.newIfStmt(jEqExpr, noPositionInfo); Stmt reta = JavaJimple.newReturnStmt(a, noPositionInfo); Stmt retb = JavaJimple.newReturnStmt(b, noPositionInfo); @@ -137,11 +138,11 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { Body.BodyBuilder bodyBuilder = Body.builder(); final MutableStmtGraph stmtGraph = bodyBuilder.getStmtGraph(); bodyBuilder.setLocals(locals); - bodyBuilder.setStartingStmt(strToA); stmtGraph.putEdge(strToA, strToB); stmtGraph.putEdge(strToB, ifStmt); stmtGraph.putEdge(ifStmt, JIfStmt.FALSE_BRANCH_IDX, reta); stmtGraph.putEdge(ifStmt, JIfStmt.TRUE_BRANCH_IDX, retb); + stmtGraph.setStartingStmt(strToA); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java index e6e4ad668af..8b1212a49b2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java @@ -8,11 +8,13 @@ import java.util.Set; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.expr.JAddExpr; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.types.PrimitiveType; @@ -81,9 +83,11 @@ private static Body.BodyBuilder createBody(boolean constantFolding) { Set locals = ImmutableUtils.immutableSet(a, b, c); - Stmt assignA = JavaJimple.newAssignStmt(a, IntConstant.getInstance(3), noPositionInfo); - Stmt assignB = JavaJimple.newAssignStmt(b, IntConstant.getInstance(4), noPositionInfo); - Stmt assignC; + FallsThroughStmt assignA = + JavaJimple.newAssignStmt(a, IntConstant.getInstance(3), noPositionInfo); + FallsThroughStmt assignB = + JavaJimple.newAssignStmt(b, IntConstant.getInstance(4), noPositionInfo); + FallsThroughStmt assignC; if (constantFolding) { assignC = JavaJimple.newAssignStmt( @@ -96,14 +100,15 @@ private static Body.BodyBuilder createBody(boolean constantFolding) { Stmt ret = JavaJimple.newReturnStmt(c, noPositionInfo); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(assignA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); - builder.addFlow(assignA, assignB); - builder.addFlow(assignB, assignC); - builder.addFlow(assignC, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(assignA); + stmtGraph.putEdge(assignA, assignB); + stmtGraph.putEdge(assignB, assignC); + stmtGraph.putEdge(assignC, ret); builder.setLocals(locals); builder.setPosition(NoPositionInformation.getInstance()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java index 804018076a3..58eb579b316 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CopyPropagatorTest.java @@ -84,18 +84,18 @@ public class CopyPropagatorTest { JavaJimple.newAssignStmt(i2, IntConstant.getInstance(0), noStmtPositionInfo); // if i2 > i1 goto AbstractConditionExpr condition = JavaJimple.newGtExpr(i2, i1); - Stmt ifStmt7 = JavaJimple.newIfStmt(condition, noStmtPositionInfo); + BranchingStmt ifStmt7 = JavaJimple.newIfStmt(condition, noStmtPositionInfo); // i3 = i1 + 1 Expr add1 = JavaJimple.newAddExpr(i1, IntConstant.getInstance(1)); FallsThroughStmt stmt8 = JavaJimple.newAssignStmt(i3, add1, noStmtPositionInfo); // i2 = i2 + 1 Expr add2 = JavaJimple.newAddExpr(i2, IntConstant.getInstance(1)); FallsThroughStmt stmt9 = JavaJimple.newAssignStmt(i2, add2, noStmtPositionInfo); - Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); // if i2 > 5 goto AbstractConditionExpr econdition = JavaJimple.newGtExpr(i2, IntConstant.getInstance(5)); - Stmt eifstmt7 = JavaJimple.newIfStmt(econdition, noStmtPositionInfo); + BranchingStmt eifstmt7 = JavaJimple.newIfStmt(econdition, noStmtPositionInfo); // i3 = 5 + 1 Expr eadd1 = JavaJimple.newAddExpr(IntConstant.getInstance(5), IntConstant.getInstance(1)); FallsThroughStmt estmt8 = JavaJimple.newAssignStmt(i3, eadd1, noStmtPositionInfo); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java index 23fb44c68ed..83f7624a179 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java @@ -10,6 +10,7 @@ import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; @@ -54,7 +55,7 @@ public void conditionalToRemovedBlock() { Local a = JavaJimple.newLocal("a", PrimitiveType.getInt()); Set locals = ImmutableUtils.immutableSet(a); - Stmt conditional = + BranchingStmt conditional = JavaJimple.newIfStmt( JavaJimple.newLtExpr(IntConstant.getInstance(10), IntConstant.getInstance(20)), noPositionInfo); @@ -63,7 +64,6 @@ public void conditionalToRemovedBlock() { JavaJimple.newAssignStmt(a, IntConstant.getInstance(42), noPositionInfo); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(conditional); builder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); @@ -71,6 +71,7 @@ public void conditionalToRemovedBlock() { builder.setLocals(locals); final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(conditional); stmtGraph.putEdge(conditional, JIfStmt.FALSE_BRANCH_IDX, intToA); stmtGraph.putEdge(conditional, JIfStmt.TRUE_BRANCH_IDX, ret); stmtGraph.putEdge(intToA, ret); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/EmptySwitchEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/EmptySwitchEliminatorTest.java index aed8dedd836..731ab9d26cd 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/EmptySwitchEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/EmptySwitchEliminatorTest.java @@ -4,11 +4,15 @@ import java.util.*; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.BranchingStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -38,11 +42,13 @@ public class EmptySwitchEliminatorTest { // build Stmts // l0 := @this Test - Stmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); // l1 = 3 - Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(3), noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(3), noStmtPositionInfo); // l2 = 0 - Stmt defaultStmt = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt defaultStmt = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); // return Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); @@ -62,7 +68,7 @@ public void testEmptySwitch() { private Body createEmptySwitchBody() { // build an empty instance of SwitchStmt List values = new ArrayList<>(); - Stmt sw = JavaJimple.newLookupSwitchStmt(l1, values, noStmtPositionInfo); + BranchingStmt sw = JavaJimple.newLookupSwitchStmt(l1, values, noStmtPositionInfo); // build an instance of BodyBuilder Body.BodyBuilder builder = Body.builder(); @@ -72,15 +78,15 @@ private Body createEmptySwitchBody() { Set locals = ImmutableUtils.immutableSet(l0, l1, l2); builder.setLocals(locals); - + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, sw); - builder.addFlow(sw, defaultStmt); - builder.addFlow(defaultStmt, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, sw); + stmtGraph.putEdge(sw, 0, defaultStmt); + stmtGraph.putEdge(defaultStmt, ret); // set startingStmt - builder.setStartingStmt(startingStmt); + stmtGraph.setStartingStmt(startingStmt); // set Position builder.setPosition(NoPositionInformation.getInstance()); @@ -90,7 +96,7 @@ private Body createEmptySwitchBody() { private Body createExpectedEmptySwitchBody() { // build a new instance of JGotoStmt - Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); // build an instance of BodyBuilder Body.BodyBuilder builder = Body.builder(); @@ -102,10 +108,11 @@ private Body createExpectedEmptySwitchBody() { builder.setLocals(locals); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, gotoStmt); - builder.addFlow(gotoStmt, defaultStmt); - builder.addFlow(defaultStmt, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, gotoStmt); + stmtGraph.putEdge(gotoStmt, JGotoStmt.BRANCH_IDX, defaultStmt); + stmtGraph.putEdge(defaultStmt, ret); // set startingStmt builder.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java index b58d345143e..f900d0f4da8 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalLivenessAnalyserTest.java @@ -10,6 +10,7 @@ import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JIfStmt; import sootup.core.jimple.common.stmt.Stmt; @@ -50,7 +51,7 @@ public class LocalLivenessAnalyserTest { FallsThroughStmt aeqbplus2 = JavaJimple.newAssignStmt( a, JavaJimple.newAddExpr(b, IntConstant.getInstance(2)), noStmtPositionInfo); - Stmt ifalt9 = + BranchingStmt ifalt9 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(IntConstant.getInstance(9), a), noStmtPositionInfo); Stmt ret = JavaJimple.newReturnStmt(c, noStmtPositionInfo); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalNameStandardizerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalNameStandardizerTest.java index 76d0475066d..7b7cb7af8f2 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalNameStandardizerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalNameStandardizerTest.java @@ -4,6 +4,7 @@ import java.util.*; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; @@ -12,6 +13,7 @@ import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.expr.Expr; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -59,27 +61,37 @@ public class LocalNameStandardizerTest { Set expectedLocals = ImmutableUtils.immutableSet(z0, d0, i0, i1, r0, r1, r2, e0); // build stmts - Stmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); - Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt stmt2 = JavaJimple.newAssignStmt(l2, BooleanConstant.getInstance(true), noStmtPositionInfo); - Stmt stmt3 = JavaJimple.newAssignStmt(l3, IntConstant.getInstance(0), noStmtPositionInfo); + FallsThroughStmt startingStmt = JavaJimple.newIdentityStmt(l0, identityRef, noStmtPositionInfo); + FallsThroughStmt stmt1 = + JavaJimple.newAssignStmt(l1, IntConstant.getInstance(1), noStmtPositionInfo); + FallsThroughStmt stmt2 = + JavaJimple.newAssignStmt(l2, BooleanConstant.getInstance(true), noStmtPositionInfo); + FallsThroughStmt stmt3 = + JavaJimple.newAssignStmt(l3, IntConstant.getInstance(0), noStmtPositionInfo); Expr expr = JavaJimple.newNewExpr(otherRefType); - Stmt stmt4 = JavaJimple.newAssignStmt(l4, expr, noStmtPositionInfo); - Stmt stmt5 = JavaJimple.newAssignStmt(l5, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt stmt6 = JavaJimple.newAssignStmt(l6, l6, noStmtPositionInfo); - Stmt stmt7 = JavaJimple.newAssignStmt(l7, DoubleConstant.getInstance(1.1), noStmtPositionInfo); + FallsThroughStmt stmt4 = JavaJimple.newAssignStmt(l4, expr, noStmtPositionInfo); + FallsThroughStmt stmt5 = + JavaJimple.newAssignStmt(l5, IntConstant.getInstance(2), noStmtPositionInfo); + FallsThroughStmt stmt6 = JavaJimple.newAssignStmt(l6, l6, noStmtPositionInfo); + FallsThroughStmt stmt7 = + JavaJimple.newAssignStmt(l7, DoubleConstant.getInstance(1.1), noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); - Stmt estartingStmt = JavaJimple.newIdentityStmt(r2, identityRef, noStmtPositionInfo); - Stmt estmt1 = JavaJimple.newAssignStmt(i0, IntConstant.getInstance(1), noStmtPositionInfo); - Stmt estmt2 = JavaJimple.newAssignStmt(z0, BooleanConstant.getInstance(true), noStmtPositionInfo); - Stmt estmt3 = JavaJimple.newAssignStmt(e0, IntConstant.getInstance(0), noStmtPositionInfo); - - Stmt estmt4 = JavaJimple.newAssignStmt(r0, expr, noStmtPositionInfo); - Stmt estmt5 = JavaJimple.newAssignStmt(i1, IntConstant.getInstance(2), noStmtPositionInfo); - Stmt estmt6 = JavaJimple.newAssignStmt(r1, r1, noStmtPositionInfo); - Stmt estmt7 = JavaJimple.newAssignStmt(d0, DoubleConstant.getInstance(1.1), noStmtPositionInfo); + FallsThroughStmt estartingStmt = JavaJimple.newIdentityStmt(r2, identityRef, noStmtPositionInfo); + FallsThroughStmt estmt1 = + JavaJimple.newAssignStmt(i0, IntConstant.getInstance(1), noStmtPositionInfo); + FallsThroughStmt estmt2 = + JavaJimple.newAssignStmt(z0, BooleanConstant.getInstance(true), noStmtPositionInfo); + FallsThroughStmt estmt3 = + JavaJimple.newAssignStmt(e0, IntConstant.getInstance(0), noStmtPositionInfo); + + FallsThroughStmt estmt4 = JavaJimple.newAssignStmt(r0, expr, noStmtPositionInfo); + FallsThroughStmt estmt5 = + JavaJimple.newAssignStmt(i1, IntConstant.getInstance(2), noStmtPositionInfo); + FallsThroughStmt estmt6 = JavaJimple.newAssignStmt(r1, r1, noStmtPositionInfo); + FallsThroughStmt estmt7 = + JavaJimple.newAssignStmt(d0, DoubleConstant.getInstance(1.1), noStmtPositionInfo); @Test public void testBody() { @@ -107,15 +119,17 @@ private Body createBody() { builder.setLocals(locals); - // build stmtGraph for the builder - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, stmt4); - builder.addFlow(stmt4, stmt5); - builder.addFlow(stmt5, stmt6); - builder.addFlow(stmt6, stmt7); - builder.addFlow(stmt7, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + // build stm + // builder.getStmtGraph()tGraph for the builder + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, stmt4); + stmtGraph.putEdge(stmt4, stmt5); + stmtGraph.putEdge(stmt5, stmt6); + stmtGraph.putEdge(stmt6, stmt7); + stmtGraph.putEdge(stmt7, ret); // set startingStmt builder.setStartingStmt(startingStmt); @@ -138,17 +152,18 @@ private Body createExpectedBody() { builder.setLocals(locals); // build stmtGraph for the builder - builder.addFlow(estartingStmt, estmt1); - builder.addFlow(estmt1, estmt2); - builder.addFlow(estmt2, estmt3); - builder.addFlow(estmt3, estmt4); - builder.addFlow(estmt4, estmt5); - builder.addFlow(estmt5, estmt6); - builder.addFlow(estmt6, estmt7); - builder.addFlow(estmt7, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.putEdge(estartingStmt, estmt1); + stmtGraph.putEdge(estmt1, estmt2); + stmtGraph.putEdge(estmt2, estmt3); + stmtGraph.putEdge(estmt3, estmt4); + stmtGraph.putEdge(estmt4, estmt5); + stmtGraph.putEdge(estmt5, estmt6); + stmtGraph.putEdge(estmt6, estmt7); + stmtGraph.putEdge(estmt7, ret); // set startingStmt - builder.setStartingStmt(estartingStmt); + stmtGraph.setStartingStmt(estartingStmt); // set Position builder.setPosition(NoPositionInformation.getInstance()); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java index 8bc99f7b8de..538e9205bae 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalPackerTest.java @@ -14,10 +14,7 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; -import sootup.core.jimple.common.stmt.FallsThroughStmt; -import sootup.core.jimple.common.stmt.JGotoStmt; -import sootup.core.jimple.common.stmt.JIfStmt; -import sootup.core.jimple.common.stmt.Stmt; +import sootup.core.jimple.common.stmt.*; import sootup.core.model.Body; import sootup.core.model.Position; import sootup.core.signatures.MethodSignature; @@ -77,9 +74,8 @@ public class LocalPackerTest { FallsThroughStmt stmt5 = JavaJimple.newAssignStmt( l1hash5, JavaJimple.newAddExpr(l1hash5, IntConstant.getInstance(1)), noStmtPositionInfo); - FallsThroughStmt stmt6 = - JavaJimple.newIfStmt(JavaJimple.newGtExpr(l1hash5, l3), noStmtPositionInfo); - Stmt gt = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt stmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l1hash5, l3), noStmtPositionInfo); + BranchingStmt gt = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); FallsThroughStmt trapHandler = JavaJimple.newIdentityStmt(l4, caughtExceptionRef, noStmtPositionInfo); @@ -100,7 +96,7 @@ public class LocalPackerTest { FallsThroughStmt estmt5 = JavaJimple.newAssignStmt( l2, JavaJimple.newAddExpr(l2, IntConstant.getInstance(1)), noStmtPositionInfo); - FallsThroughStmt estmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l2, l1), noStmtPositionInfo); + BranchingStmt estmt6 = JavaJimple.newIfStmt(JavaJimple.newGtExpr(l2, l1), noStmtPositionInfo); FallsThroughStmt etrapHandler = JavaJimple.newIdentityStmt(el4, caughtExceptionRef, noStmtPositionInfo); Stmt ethrowStmt = JavaJimple.newThrowStmt(el4, noStmtPositionInfo); @@ -265,7 +261,7 @@ private Body.BodyBuilder createBodyBuilder() { graph.putEdge(gt, JGotoStmt.BRANCH_IDX, stmt5); graph.putEdge(stmt6, JIfStmt.TRUE_BRANCH_IDX, ret); - builder.setStartingStmt(startingStmt); + graph.setStartingStmt(startingStmt); return builder; } @@ -299,7 +295,7 @@ private Body createExpectedBody() { graph.putEdge(gt, JGotoStmt.BRANCH_IDX, estmt5); graph.putEdge(estmt6, JIfStmt.TRUE_BRANCH_IDX, ret); - builder.setStartingStmt(startingStmt); + graph.setStartingStmt(startingStmt); // build position Position position = NoPositionInformation.getInstance(); @@ -337,12 +333,12 @@ private Body.BodyBuilder createTrapBody() { graph.putEdge(stmt3, stmt4); graph.putEdge(stmt4, stmt5); graph.putEdge(stmt5, stmt6); - graph.putEdge(stmt6, gt); + graph.putEdge(stmt6, JIfStmt.FALSE_BRANCH_IDX, gt); graph.putEdge(gt, JGotoStmt.BRANCH_IDX, stmt5); - graph.putEdge(stmt6, ret); + graph.putEdge(stmt6, JIfStmt.TRUE_BRANCH_IDX, ret); graph.putEdge(trapHandler, throwStmt); - builder.setStartingStmt(startingStmt); + graph.setStartingStmt(startingStmt); return builder; } @@ -373,12 +369,12 @@ private Body.BodyBuilder createExpectedTrapBody() { graph.addNode(estmt5, Collections.singletonMap(exception, etrapHandler)); graph.putEdge(estmt4, estmt5); - graph.putEdge(estmt6, gt); + graph.putEdge(estmt6, JIfStmt.FALSE_BRANCH_IDX, gt); graph.putEdge(gt, JGotoStmt.BRANCH_IDX, estmt5); - graph.putEdge(estmt6, ret); + graph.putEdge(estmt6, JIfStmt.TRUE_BRANCH_IDX, ret); graph.putEdge(etrapHandler, ethrowStmt); - builder.setStartingStmt(startingStmt); + graph.setStartingStmt(startingStmt); return builder; } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java index 2aecffbe021..baee876f4e3 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/LocalSplitterTest.java @@ -11,6 +11,7 @@ import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; @@ -247,13 +248,13 @@ private Body createBBBody() { builder.setLocals(locals); Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt stmt2 = + BranchingStmt stmt2 = JavaJimple.newIfStmt( JavaJimple.newGeExpr(l1, IntConstant.getInstance(0)), noStmtPositionInfo); Stmt stmt3 = JavaJimple.newAssignStmt( l1, JavaJimple.newAddExpr(l1, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt4 = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt stmt4 = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt stmt5 = JavaJimple.newAssignStmt( l1, JavaJimple.newSubExpr(l1, IntConstant.getInstance(1)), noStmtPositionInfo); @@ -272,14 +273,14 @@ private Body createBBBody() { graph.setStartingStmt(startingStmt); /* set graph - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, stmt4); - builder.addFlow(stmt4, ret); - builder.addFlow(stmt2, stmt5); - builder.addFlow(stmt5, stmt6); - builder.addFlow(stmt6, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, stmt4); + stmtGraph.putEdge(stmt4, ret); + stmtGraph.putEdge(stmt2, stmt5); + stmtGraph.putEdge(stmt5, stmt6); + stmtGraph.putEdge(stmt6, ret); // build startingStmt builder.setStartingStmt(startingStmt); @@ -304,7 +305,7 @@ private Body createExpectedBBBody() { builder.setLocals(locals); Stmt stmt1 = JavaJimple.newAssignStmt(l1hash1, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt stmt2 = + BranchingStmt stmt2 = JavaJimple.newIfStmt( JavaJimple.newGeExpr(l1hash1, IntConstant.getInstance(0)), noStmtPositionInfo); Stmt stmt3 = @@ -312,7 +313,7 @@ private Body createExpectedBBBody() { l1hash2, JavaJimple.newAddExpr(l1hash1, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt4 = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt stmt4 = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt stmt5 = JavaJimple.newAssignStmt( l1hash3, @@ -335,14 +336,14 @@ private Body createExpectedBBBody() { graph.setStartingStmt(startingStmt); /* set graph - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, stmt4); - builder.addFlow(stmt4, ret); - builder.addFlow(stmt2, stmt5); - builder.addFlow(stmt5, stmt6); - builder.addFlow(stmt6, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, stmt4); + stmtGraph.putEdge(stmt4, ret); + stmtGraph.putEdge(stmt2, stmt5); + stmtGraph.putEdge(stmt5, stmt6); + stmtGraph.putEdge(stmt6, ret); // build startingStmt builder.setStartingStmt(startingStmt); @@ -417,11 +418,11 @@ private Body createExpectedMuiltilocalsBody() { graph.setStartingStmt(startingStmt); /* set graph - builder.addFlow(startingStmt, stmt1); - builder.addFlow(stmt1, stmt2); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, stmt4); - builder.addFlow(stmt4, ret); + stmtGraph.putEdge(startingStmt, stmt1); + stmtGraph.putEdge(stmt1, stmt2); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, stmt4); + stmtGraph.putEdge(stmt4, ret); // set first stmt builder.setStartingStmt(startingStmt); @@ -449,7 +450,7 @@ private Body createLoopBody() { Stmt stmt1 = JavaJimple.newAssignStmt(l1, IntConstant.getInstance(0), noStmtPositionInfo); Stmt stmt2 = JavaJimple.newAssignStmt(stack4, l1, noStmtPositionInfo); Stmt stmt3 = JavaJimple.newAssignStmt(stack3, IntConstant.getInstance(10), noStmtPositionInfo); - Stmt stmt4 = + BranchingStmt stmt4 = JavaJimple.newIfStmt( JavaJimple.newGeExpr(stack4, stack3), noStmtPositionInfo); // branch to ret Stmt stmt5 = @@ -461,7 +462,7 @@ private Body createLoopBody() { Stmt stmt7 = JavaJimple.newAssignStmt( l1, JavaJimple.newAddExpr(l1, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt8 = JavaJimple.newGotoStmt(noStmtPositionInfo); // goto stmt2 + BranchingStmt stmt8 = JavaJimple.newGotoStmt(noStmtPositionInfo); // goto stmt2 Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); graph.addBlock(Arrays.asList(startingStmt, stmt1, stmt2, stmt3, stmt4), Collections.emptyMap()); @@ -492,7 +493,7 @@ private Body createExpectedLoopBody() { Stmt stmt1 = JavaJimple.newAssignStmt(l1hash1, IntConstant.getInstance(0), noStmtPositionInfo); Stmt stmt2 = JavaJimple.newAssignStmt(stack4, l1hash1, noStmtPositionInfo); Stmt stmt3 = JavaJimple.newAssignStmt(stack3, IntConstant.getInstance(10), noStmtPositionInfo); - Stmt stmt4 = + BranchingStmt stmt4 = JavaJimple.newIfStmt( JavaJimple.newGeExpr(stack4, stack3), noStmtPositionInfo); // branch to ret Stmt stmt5 = @@ -506,7 +507,7 @@ private Body createExpectedLoopBody() { l1hash1, JavaJimple.newAddExpr(l1hash2, IntConstant.getInstance(1)), noStmtPositionInfo); - Stmt stmt8 = JavaJimple.newGotoStmt(noStmtPositionInfo); // goto stmt2 + BranchingStmt stmt8 = JavaJimple.newGotoStmt(noStmtPositionInfo); // goto stmt2 Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); graph.addBlock(Arrays.asList(startingStmt, stmt1, stmt2, stmt3, stmt4), Collections.emptyMap()); @@ -545,7 +546,7 @@ private Body.BodyBuilder createTrapBody() { FallsThroughStmt stmt4 = JavaJimple.newIdentityStmt(stack3, caughtExceptionRef, noStmtPositionInfo); FallsThroughStmt stmt5 = JavaJimple.newAssignStmt(l3, l1, noStmtPositionInfo); - Stmt stmt6 = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt stmt6 = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); // build graph @@ -584,7 +585,7 @@ private Body createExpectedTrapBody() { FallsThroughStmt exceptionCatchStmt = JavaJimple.newIdentityStmt(stack3, caughtExceptionRef, noStmtPositionInfo); Stmt l3assignl1hash2Stmt = JavaJimple.newAssignStmt(l3, l1hash2, noStmtPositionInfo); - Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); graph.addBlock( diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java index 5bc08601679..591e9695bf0 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java @@ -11,9 +11,7 @@ import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; -import sootup.core.jimple.common.stmt.FallsThroughStmt; -import sootup.core.jimple.common.stmt.JNopStmt; -import sootup.core.jimple.common.stmt.Stmt; +import sootup.core.jimple.common.stmt.*; import sootup.core.model.Body; import sootup.core.util.ImmutableUtils; import sootup.java.core.JavaIdentifierFactory; @@ -82,24 +80,25 @@ private static Body.BodyBuilder createBody(boolean withNop) { Local a = JavaJimple.newLocal("a", objectType); Local b = JavaJimple.newLocal("b", stringType); - Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); + FallsThroughStmt strToA = + JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); FallsThroughStmt bToA = JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); Stmt ret = JavaJimple.newReturnStmt(b, noPositionInfo); - Stmt jump = JavaJimple.newGotoStmt(noPositionInfo); + BranchingStmt jump = JavaJimple.newGotoStmt(noPositionInfo); Set locals = ImmutableUtils.immutableSet(a, b); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(strToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); - builder.addFlow(strToA, jump); - builder.addFlow(jump, bToA); - builder.addFlow(bToA, ret); final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(strToA); + stmtGraph.putEdge(strToA, jump); + stmtGraph.putEdge(jump, JGotoStmt.BRANCH_IDX, bToA); + stmtGraph.putEdge(bToA, ret); if (withNop) { // strToA, jump, bToA, nop, ret; JNopStmt nop = new JNopStmt(noPositionInfo); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormerTest.java index 243716f48b6..3174630aaf6 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/StaticSingleAssignmentFormerTest.java @@ -12,6 +12,8 @@ import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; import sootup.core.jimple.common.stmt.BranchingStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -69,9 +71,11 @@ public class StaticSingleAssignmentFormerTest { l3, JavaJimple.newAddExpr(l3, IntConstant.getInstance(2)), noStmtPositionInfo); Stmt gotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); - Stmt handlerStmt = JavaJimple.newIdentityStmt(stack4, caughtExceptionRef, noStmtPositionInfo); - Stmt l2eq0 = JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); - Stmt goTo = JavaJimple.newGotoStmt(noStmtPositionInfo); + FallsThroughStmt handlerStmt = + JavaJimple.newIdentityStmt(stack4, caughtExceptionRef, noStmtPositionInfo); + FallsThroughStmt l2eq0 = + JavaJimple.newAssignStmt(l2, IntConstant.getInstance(0), noStmtPositionInfo); + BranchingStmt goTo = JavaJimple.newGotoStmt(noStmtPositionInfo); @Test public void testSSA() { @@ -303,9 +307,10 @@ private Body.BodyBuilder createTrapBody() { // add exception graph.addNode(assignl1tol2, Collections.singletonMap(exception, handlerStmt)); - builder.addFlow(handlerStmt, l2eq0); - builder.addFlow(l2eq0, goTo); - builder.addFlow(goTo, assignl3plus1tol3); + + graph.putEdge(handlerStmt, l2eq0); + graph.putEdge(l2eq0, goTo); + graph.putEdge(goTo, JGotoStmt.BRANCH_IDX, assignl3plus1tol3); return builder; } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java index c109af4935f..f5990ed4e06 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/TrapTightenerTest.java @@ -13,6 +13,7 @@ import sootup.core.jimple.basic.Trap; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; @@ -57,7 +58,7 @@ public class TrapTightenerTest { FallsThroughStmt stmt2 = JavaJimple.newEnterMonitorStmt(l1, noStmtPositionInfo); FallsThroughStmt stmt3 = JavaJimple.newAssignStmt(l2, l1, noStmtPositionInfo); FallsThroughStmt stmt4 = JavaJimple.newExitMonitorStmt(l2, noStmtPositionInfo); - Stmt stmt5 = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt stmt5 = JavaJimple.newGotoStmt(noStmtPositionInfo); FallsThroughStmt stmt6 = JavaJimple.newIdentityStmt(l3, caughtExceptionRef, noStmtPositionInfo); FallsThroughStmt stmt7 = @@ -160,7 +161,7 @@ public void testMonitoredBody() { // modify exceptionalStmtGraph builder.clearExceptionEdgesOf(stmt2); builder.clearExceptionEdgesOf(stmt4); - // builder.addFlow(, stmt6); + // stmtGraph.putEdge(, stmt6); TrapTightener trapTightener = new TrapTightener(); trapTightener.interceptBody(builder, null); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java index d0936494a39..7c47b97a0f9 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnreachableCodeEliminatorTest.java @@ -15,7 +15,9 @@ import sootup.core.jimple.basic.Trap; import sootup.core.jimple.common.constant.IntConstant; import sootup.core.jimple.common.ref.IdentityRef; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.signatures.MethodSignature; @@ -61,7 +63,7 @@ public class UnreachableCodeEliminatorTest { FallsThroughStmt stmt3 = JavaJimple.newAssignStmt(l3, IntConstant.getInstance(3), noStmtPositionInfo); - Stmt jGotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); + BranchingStmt jGotoStmt = JavaJimple.newGotoStmt(noStmtPositionInfo); Stmt ret1 = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); Stmt ret2 = JavaJimple.newReturnVoidStmt(noStmtPositionInfo); @@ -90,14 +92,15 @@ public void testSimpleBody() { builder.setLocals(locals); // build stmtsGraph for the builder - builder.addFlow(startingStmt, stmt1); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.putEdge(startingStmt, stmt1); - builder.addFlow(stmt1, ret1); - builder.addFlow(stmt2, stmt3); - builder.addFlow(stmt3, ret2); + stmtGraph.putEdge(stmt1, ret1); + stmtGraph.putEdge(stmt2, stmt3); + stmtGraph.putEdge(stmt3, ret2); // set startingStmt - builder.setStartingStmt(startingStmt); + stmtGraph.setStartingStmt(startingStmt); // set Position builder.setPosition(NoPositionInformation.getInstance()); @@ -199,7 +202,7 @@ public void testTrappedBody3() { graph.addBlock( Arrays.asList(startingStmt, stmt1, ret1), Collections.singletonMap(exception, handlerStmt)); graph.addBlock(Arrays.asList(handlerStmt, jGotoStmt)); - graph.putEdge(jGotoStmt, 0, ret1); + graph.putEdge(jGotoStmt, JGotoStmt.BRANCH_IDX, ret1); // set startingStmt graph.setStartingStmt(startingStmt); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java index 50d82a9c1d9..38a2d167811 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java @@ -7,8 +7,12 @@ import java.util.Set; import org.junit.Test; import org.junit.experimental.categories.Category; +import sootup.core.graph.MutableStmtGraph; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.StmtPositionInfo; +import sootup.core.jimple.common.stmt.BranchingStmt; +import sootup.core.jimple.common.stmt.FallsThroughStmt; +import sootup.core.jimple.common.stmt.JGotoStmt; import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.util.ImmutableUtils; @@ -68,18 +72,20 @@ private static Body.BodyBuilder createBody(boolean unusedLocals) { locals = ImmutableUtils.immutableSet(a, b); } - Stmt strToA = JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); - Stmt bToA = JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); + FallsThroughStmt strToA = + JavaJimple.newAssignStmt(a, javaJimple.newStringConstant("str"), noPositionInfo); + FallsThroughStmt bToA = + JavaJimple.newAssignStmt(b, JavaJimple.newCastExpr(a, stringType), noPositionInfo); Stmt ret = JavaJimple.newReturnStmt(b, noPositionInfo); - Stmt jump = JavaJimple.newGotoStmt(noPositionInfo); + BranchingStmt jump = JavaJimple.newGotoStmt(noPositionInfo); final Body.BodyBuilder builder = Body.builder(); locals.forEach(builder::addLocal); - - builder.setStartingStmt(strToA); - builder.addFlow(strToA, jump); - builder.addFlow(jump, bToA); - builder.addFlow(bToA, ret); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(strToA); + stmtGraph.putEdge(strToA, jump); + stmtGraph.putEdge(jump, JGotoStmt.BRANCH_IDX, bToA); + stmtGraph.putEdge(bToA, ret); builder.setMethodSignature( JavaIdentifierFactory.getInstance() diff --git a/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java b/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java index d3c8c45bc76..4d296a70305 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java @@ -9,13 +9,14 @@ import sootup.core.Project; import sootup.core.frontend.OverridingBodySource; import sootup.core.frontend.OverridingClassSource; +import sootup.core.graph.MutableStmtGraph; import sootup.core.inputlocation.EagerInputLocation; import sootup.core.jimple.basic.NoPositionInformation; import sootup.core.jimple.basic.StmtPositionInfo; import sootup.core.jimple.common.constant.IntConstant; +import sootup.core.jimple.common.stmt.BranchingStmt; import sootup.core.jimple.common.stmt.JNopStmt; import sootup.core.jimple.common.stmt.JReturnVoidStmt; -import sootup.core.jimple.common.stmt.Stmt; import sootup.core.jimple.javabytecode.stmt.JSwitchStmt; import sootup.core.model.*; import sootup.core.signatures.MethodSignature; @@ -80,18 +81,19 @@ public void printSwitchStmt() { final JNopStmt jNop = new JNopStmt(noPosInfo); final JNopStmt jNop2 = new JNopStmt(noPosInfo); - Stmt tableSwitch = new JSwitchStmt(IntConstant.getInstance(42), 4, 5, noPosInfo); + BranchingStmt tableSwitch = new JSwitchStmt(IntConstant.getInstance(42), 4, 5, noPosInfo); { Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(tableSwitch); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(tableSwitch); - builder.addFlow(tableSwitch, jNop); - builder.addFlow(tableSwitch, jNop2); - builder.addFlow(tableSwitch, returnstmt); + stmtGraph.putEdge(tableSwitch, 0, jNop); + stmtGraph.putEdge(tableSwitch, 1, jNop2); + stmtGraph.putEdge(tableSwitch, 2, returnstmt); - builder.addFlow(jNop, jNop2); - builder.addFlow(jNop2, returnstmt); + stmtGraph.putEdge(jNop, jNop2); + stmtGraph.putEdge(jNop2, returnstmt); SootClass tableClass = buildClass(builder); @@ -116,17 +118,19 @@ public void printSwitchStmt() { } { - Stmt lookupSwitch = new JSwitchStmt(IntConstant.getInstance(123), lookupValues, noPosInfo); + BranchingStmt lookupSwitch = + new JSwitchStmt(IntConstant.getInstance(123), lookupValues, noPosInfo); Body.BodyBuilder builder = Body.builder(); - builder.setStartingStmt(lookupSwitch); + final MutableStmtGraph stmtGraph = builder.getStmtGraph(); + stmtGraph.setStartingStmt(lookupSwitch); - builder.addFlow(lookupSwitch, jNop); - builder.addFlow(lookupSwitch, jNop2); - builder.addFlow(lookupSwitch, returnstmt); + stmtGraph.putEdge(lookupSwitch, 0, jNop); + stmtGraph.putEdge(lookupSwitch, 1, jNop2); + stmtGraph.putEdge(lookupSwitch, 2, returnstmt); - builder.addFlow(jNop, jNop2); - builder.addFlow(jNop2, returnstmt); + stmtGraph.putEdge(jNop, jNop2); + stmtGraph.putEdge(jNop2, returnstmt); SootClass lookupClass = buildClass(builder); From 9f5250444474e62ec3d6fde6c1f1c39537b5e3bd Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Mon, 23 Oct 2023 14:00:30 +0200 Subject: [PATCH 48/54] fix merge leftover --- .../java/bytecode/minimaltestsuite/java14/RecordTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/RecordTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/RecordTest.java index c56e28548c9..61fa413bd4e 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/RecordTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/minimaltestsuite/java14/RecordTest.java @@ -46,9 +46,9 @@ public MethodSignature getMethodSignature() { @Override public List expectedBodyStmts() { return Stream.of( - "l0 := @this: RecordTest", - "l1 := @parameter0: java.lang.Object", - "$stack2 = dynamicinvoke \"equals\" (l0, l1) (class \"LRecordTest;\", \"a;b\", methodhandle: \"REF_GET_FIELD\" , methodhandle: \"REF_GET_FIELD\" )", + "$l0 := @this: RecordTest", + "$l1 := @parameter0: java.lang.Object", + "$stack2 = dynamicinvoke \"equals\" ($l0, $l1) (class \"LRecordTest;\", \"a;b\", methodhandle: \"REF_GET_FIELD\" , methodhandle: \"REF_GET_FIELD\" )", "return $stack2") .collect(Collectors.toCollection(ArrayList::new)); } From c0d4a801c11e13c36b28d524a11236b1675eb976 Mon Sep 17 00:00:00 2001 From: Jonas Klauke <97177448+JonasKlauke@users.noreply.github.com> Date: Wed, 25 Oct 2023 09:51:55 +0200 Subject: [PATCH 49/54] remove an unnecessary Fix me comment --- .../java/bytecode/interceptors/ConditionalBranchFolder.java | 1 - 1 file changed, 1 deletion(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java index 9aa9c3bf293..066ddd4b2e1 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/ConditionalBranchFolder.java @@ -85,7 +85,6 @@ public void interceptBody(@Nonnull Body.BodyBuilder builder, @Nonnull View vi // link previous stmt with always-reached successor of the if-Stmt for (Stmt predecessor : stmtGraph.predecessors(ifStmt)) { stmtGraph.removeEdge(predecessor, ifStmt); - // FIXME: [ms] fix successorIdx stmtGraph.putEdge((FallsThroughStmt) predecessor, tautologicSuccessor); } From 3ed4a0673d2230f5d497e226a5ea1748d7622e28 Mon Sep 17 00:00:00 2001 From: Jonas Klauke <97177448+JonasKlauke@users.noreply.github.com> Date: Wed, 25 Oct 2023 09:52:35 +0200 Subject: [PATCH 50/54] added removed licence header --- .../sootup/core/jimple/common/stmt/Stmt.java | 22 ++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index ccbc9f1b331..3087f87c0f2 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -1,5 +1,25 @@ package sootup.core.jimple.common.stmt; - +/*- + * #%L + * Soot - a J*va Optimization Framework + * %% + * Copyright (C) 2018-2023 Linghui Luo, Christian Brüggemann, Markus Schmidt + * %% + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Lesser General Public License as + * published by the Free Software Foundation, either version 2.1 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Lesser Public License for more details. + * + * You should have received a copy of the GNU General Lesser Public + * License along with this program. If not, see + * . + * #L% + */ import java.util.List; import javax.annotation.Nonnull; import sootup.core.jimple.basic.EquivTo; From 85d97c46a26a786204c26d1aa14cc728c63da0bd Mon Sep 17 00:00:00 2001 From: Jonas Klauke Date: Wed, 25 Oct 2023 10:01:05 +0200 Subject: [PATCH 51/54] fmt --- .../src/main/java/sootup/core/jimple/common/stmt/Stmt.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java index 3087f87c0f2..36c4d2cad90 100644 --- a/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java +++ b/sootup.core/src/main/java/sootup/core/jimple/common/stmt/Stmt.java @@ -9,12 +9,12 @@ * it under the terms of the GNU Lesser General Public License as * published by the Free Software Foundation, either version 2.1 of the * License, or (at your option) any later version. - * + * * This program is distributed in the hope that it will be useful, * but WITHOUT ANY WARRANTY; without even the implied warranty of * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the * GNU General Lesser Public License for more details. - * + * * You should have received a copy of the GNU General Lesser Public * License along with this program. If not, see * . From 5f6a1cd377e1a4f38aed5d96253b1f2349743936 Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Wed, 25 Oct 2023 13:51:19 +0200 Subject: [PATCH 52/54] unify the parameter order; remove 'convenvience' method with unnecessary mix of layers Signatures<->SootClass --- .../main/java/sootup/core/IdentifierFactory.java | 16 ++-------------- .../PathBasedAnalysisInputLocationTest.java | 2 +- .../bytecode/interceptors/AggregatorTest.java | 2 +- .../interceptors/CastAndReturnInlinerTest.java | 4 ++-- .../ConditionalBranchFolderTest.java | 4 ++-- .../ConstantPropagatorAndFolderTest.java | 2 +- .../DeadAssignmentEliminatorTest.java | 2 +- .../bytecode/interceptors/NopEliminatorTest.java | 2 +- .../interceptors/UnusedLocalEliminatorTest.java | 2 +- .../sootup/java/core/JavaIdentifierFactory.java | 16 ++++------------ .../java/core/JavaModuleIdentifierFactory.java | 4 ++-- .../core/jimple/common/stmt/JInvokeStmtTest.java | 10 +++++----- .../sootup/java/core/model/SootMethodTest.java | 2 +- .../java/core/printer/JimplePrinterTest.java | 4 ++-- .../core/printer/LegacyJimplePrinterTest.java | 2 +- .../signatures/JavaIdentifierFactoryTest.java | 16 ++++++++-------- .../frontend/InstructionConverter.java | 10 +++++----- .../java/sootup/tests/MutableSootClientTest.java | 2 +- 18 files changed, 41 insertions(+), 61 deletions(-) diff --git a/sootup.core/src/main/java/sootup/core/IdentifierFactory.java b/sootup.core/src/main/java/sootup/core/IdentifierFactory.java index 2c094925636..cd747f44586 100644 --- a/sootup.core/src/main/java/sootup/core/IdentifierFactory.java +++ b/sootup.core/src/main/java/sootup/core/IdentifierFactory.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Optional; import javax.annotation.Nonnull; -import sootup.core.model.SootClass; import sootup.core.signatures.FieldSignature; import sootup.core.signatures.FieldSubSignature; import sootup.core.signatures.MethodSignature; @@ -54,15 +53,15 @@ public interface IdentifierFactory { /** * Gets the method signature. * - * @param methodName the method name * @param fullyQualifiedNameDeclClass the fully qualified name decl class + * @param methodName the method name * @param fqReturnType the fq return type * @param parameters the parameters * @return the method signature */ MethodSignature getMethodSignature( - String methodName, String fullyQualifiedNameDeclClass, + String methodName, String fqReturnType, List parameters); @@ -96,17 +95,6 @@ MethodSignature getMethodSignature( Type fqReturnType, List parameters); - /** - * Gets the method signature. - * - * @param declaringClass the declaring class - * @param subSignature the sub signature - * @return the method signature - */ - @Nonnull - MethodSignature getMethodSignature( - @Nonnull SootClass declaringClass, @Nonnull MethodSubSignature subSignature); - /** * Gets the method signature. * diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/inputlocation/PathBasedAnalysisInputLocationTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/inputlocation/PathBasedAnalysisInputLocationTest.java index b60892e6002..80ff26614ac 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/inputlocation/PathBasedAnalysisInputLocationTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/inputlocation/PathBasedAnalysisInputLocationTest.java @@ -380,7 +380,7 @@ public Object resolveAnnotationsDefaultValue() { public MethodSignature getSignature() { return JavaIdentifierFactory.getInstance() .getMethodSignature( - utilsClass, optionalToStreamMethodSubSignature); + utilsClass.getType(), optionalToStreamMethodSubSignature); } }) .withSignature( diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index ca6666af81c..4d3f8635f0c 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -88,7 +88,7 @@ private static Body.BodyBuilder createBodyBuilder(boolean withAggregation) { builder.setStartingStmt(intToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); builder.addFlow(intToA, intToB); builder.addFlow(intToB, ret); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java index cf5541c65cf..65cf0a8d025 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/CastAndReturnInlinerTest.java @@ -66,7 +66,7 @@ public void testModification() { bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); Body testBody = bodyBuilder.build(); new CastAndReturnInliner().interceptBody(bodyBuilder, null); @@ -119,7 +119,7 @@ public void testNoModification() { bodyBuilder.addFlow(bToA, ret); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); Body testBody = bodyBuilder.build(); new CastAndReturnInliner().interceptBody(bodyBuilder, null); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java index 8666144261e..a512b1d72a7 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConditionalBranchFolderTest.java @@ -111,7 +111,7 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { final MethodSignature methodSignature = JavaIdentifierFactory.getInstance() .getMethodSignature( - "toString", "java.lang.Object", "String", Collections.emptyList()); + "java.lang.Object", "toString", "String", Collections.emptyList()); Local base = new Local( "someObjectThatHasSomethingToString", @@ -140,7 +140,7 @@ private static Body.BodyBuilder createBodyBuilder(int constantCondition) { bodyBuilder.addFlow(ifStmt, retb); bodyBuilder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); return bodyBuilder; } } diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java index e6e4ad668af..ddc7872b716 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/ConstantPropagatorAndFolderTest.java @@ -99,7 +99,7 @@ private static Body.BodyBuilder createBody(boolean constantFolding) { builder.setStartingStmt(assignA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); builder.addFlow(assignA, assignB); builder.addFlow(assignB, assignC); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java index 51125d4d4de..dc1772c3a57 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/DeadAssignmentEliminatorTest.java @@ -63,7 +63,7 @@ private static Body.BodyBuilder createBody(boolean essentialOption) { builder.setStartingStmt(strToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); if (essentialOption) { Stmt newToB = JavaJimple.newAssignStmt(b, JavaJimple.newNewExpr(objectType), noPositionInfo); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java index 06ba6e2aa86..71b633cc385 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/NopEliminatorTest.java @@ -91,7 +91,7 @@ private static Body.BodyBuilder createBody(boolean withNop) { builder.setStartingStmt(strToA); builder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "ab.c", "void", Collections.emptyList())); + .getMethodSignature("ab.c", "test", "void", Collections.emptyList())); builder.addFlow(strToA, jump); builder.addFlow(jump, bToA); diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java index 50d82a9c1d9..a058cd0c069 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/UnusedLocalEliminatorTest.java @@ -83,7 +83,7 @@ private static Body.BodyBuilder createBody(boolean unusedLocals) { builder.setMethodSignature( JavaIdentifierFactory.getInstance() - .getMethodSignature("test", "a.b.c", "void", Collections.emptyList())); + .getMethodSignature("a.b.c", "test", "void", Collections.emptyList())); return builder; } } diff --git a/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java b/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java index 963179dfd0d..6384ef1ff8b 100644 --- a/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java +++ b/sootup.java.core/src/main/java/sootup/java/core/JavaIdentifierFactory.java @@ -34,7 +34,6 @@ import org.apache.commons.io.FilenameUtils; import org.apache.commons.lang3.ClassUtils; import sootup.core.IdentifierFactory; -import sootup.core.model.SootClass; import sootup.core.signatures.FieldSignature; import sootup.core.signatures.FieldSubSignature; import sootup.core.signatures.MethodSignature; @@ -264,16 +263,16 @@ public PackageName getPackageName(@Nonnull final String packageName) { /** * Always creates a new MethodSignature AND a new ClassSignature. * - * @param methodName the method's name * @param fullyQualifiedNameDeclClass the fully-qualified name of the declaring class - * @param parameters the methods parameters fully-qualified name or a primitive's name + * @param methodName the method's name * @param fqReturnType the fully-qualified name of the return type or a primitive's name + * @param parameters the methods parameters fully-qualified name or a primitive's name * @return a MethodSignature */ @Override public MethodSignature getMethodSignature( - final String methodName, final String fullyQualifiedNameDeclClass, + final String methodName, final String fqReturnType, final List parameters) { JavaClassType declaringClass = getClassType(fullyQualifiedNameDeclClass); @@ -321,13 +320,6 @@ public MethodSignature getMethodSignature( return new MethodSignature(declaringClassSignature, methodName, parameters, fqReturnType); } - @Override - @Nonnull - public MethodSignature getMethodSignature( - @Nonnull SootClass declaringClass, @Nonnull MethodSubSignature subSignature) { - return getMethodSignature(declaringClass.getType(), subSignature); - } - @Override @Nonnull public MethodSignature getMethodSignature( @@ -425,7 +417,7 @@ public MethodSignature parseMethodSignature(@Nonnull String methodSignature) { }) .collect(Collectors.toList()); - return getMethodSignature(methodName, className, returnName, argsList); + return getMethodSignature(className, methodName, returnName, argsList); } @Nonnull diff --git a/sootup.java.core/src/main/java/sootup/java/core/JavaModuleIdentifierFactory.java b/sootup.java.core/src/main/java/sootup/java/core/JavaModuleIdentifierFactory.java index 2f504ac914f..0a24d88ac3f 100644 --- a/sootup.java.core/src/main/java/sootup/java/core/JavaModuleIdentifierFactory.java +++ b/sootup.java.core/src/main/java/sootup/java/core/JavaModuleIdentifierFactory.java @@ -206,12 +206,12 @@ public ModuleJavaClassType getClassType(String fullyQualifiedClassName) { @Override public MethodSignature getMethodSignature( - String methodName, String fullyQualifiedNameDeclClass, + String methodName, String fqReturnType, List parameters) { return super.getMethodSignature( - methodName, fullyQualifiedNameDeclClass, fqReturnType, parameters); + fullyQualifiedNameDeclClass, methodName, fqReturnType, parameters); } } } diff --git a/sootup.java.core/src/test/java/sootup/java/core/jimple/common/stmt/JInvokeStmtTest.java b/sootup.java.core/src/test/java/sootup/java/core/jimple/common/stmt/JInvokeStmtTest.java index dba62305329..6bb00884cfb 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/jimple/common/stmt/JInvokeStmtTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/jimple/common/stmt/JInvokeStmtTest.java @@ -93,7 +93,7 @@ public void test() { // JStaticInvokeExpr MethodSignature statMethodSig = dif.getMethodSignature( - "print", "java.system.Out", "void", Collections.singletonList("String")); + "java.system.Out", "print", "void", Collections.singletonList("String")); Stmt staticInvokeStmt = new JInvokeStmt( new JStaticInvokeExpr( @@ -112,7 +112,7 @@ public void test() { // JSpecialInvoke MethodSignature smethodSig = - dif.getMethodSignature("", "java.lang.Object", "void", Collections.emptyList()); + dif.getMethodSignature("java.lang.Object", "", "void", Collections.emptyList()); Stmt specialInvokeStmt = new JInvokeStmt( new JSpecialInvokeExpr( @@ -129,7 +129,7 @@ public void test() { // JInterfaceInvoke MethodSignature imethodSig = - dif.getMethodSignature("remove", "java.util.Iterator", "void", Collections.emptyList()); + dif.getMethodSignature("java.util.Iterator", "remove", "void", Collections.emptyList()); Stmt interfaceInvokeStmt = new JInvokeStmt( new JInterfaceInvokeExpr( @@ -147,12 +147,12 @@ public void test() { // JDynamicInvoke MethodSignature dmethodSig = dif.getMethodSignature( - "mylambda", JDynamicInvokeExpr.INVOKEDYNAMIC_DUMMY_CLASS_NAME, + "mylambda", "void", Collections.emptyList()); MethodSignature bootstrapMethodSig = - dif.getMethodSignature("run", "Runnable", "void", Collections.emptyList()); + dif.getMethodSignature("Runnable", "run", "void", Collections.emptyList()); List bootstrapArgs = Collections.emptyList(); List methodArgs = Collections.emptyList(); diff --git a/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java b/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java index 3d031009c1b..6efbcc3d058 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/model/SootMethodTest.java @@ -50,7 +50,7 @@ public void testCreateMethod() { LocalGenerator generator = new LocalGenerator(new HashSet<>()); MethodSignature methodSignature = view.getIdentifierFactory() - .getMethodSignature("main", "dummyMain", "void", Collections.emptyList()); + .getMethodSignature("dummyMain", "main", "void", Collections.emptyList()); Body.BodyBuilder bodyBuilder = Body.builder(); final JIdentityStmt firstStmt = diff --git a/sootup.java.core/src/test/java/sootup/java/core/printer/JimplePrinterTest.java b/sootup.java.core/src/test/java/sootup/java/core/printer/JimplePrinterTest.java index 6938330d8eb..b2be615cb61 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/printer/JimplePrinterTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/printer/JimplePrinterTest.java @@ -63,7 +63,7 @@ private SootClass buildClass() { String className = "some.package.SomeClass"; MethodSignature methodSignatureOne = view.getIdentifierFactory() - .getMethodSignature("main", className, "void", Collections.emptyList()); + .getMethodSignature(className, "main", "void", Collections.emptyList()); StmtPositionInfo noPosInfo = StmtPositionInfo.createNoStmtPositionInfo(); final JReturnVoidStmt returnVoidStmt = new JReturnVoidStmt(noPosInfo); @@ -87,7 +87,7 @@ private SootClass buildClass() { MethodSignature methodSignatureTwo = view.getIdentifierFactory() - .getMethodSignature("otherMethod", className, "int", Collections.emptyList()); + .getMethodSignature(className, "otherMethod", "int", Collections.emptyList()); bodyBuilder .setMethodSignature(methodSignatureTwo) .setPosition(NoPositionInformation.getInstance()); diff --git a/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java b/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java index d3c8c45bc76..d53c06192af 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/printer/LegacyJimplePrinterTest.java @@ -36,7 +36,7 @@ SootClass buildClass(Body.BodyBuilder builder) { MethodSignature methodSignature = view.getIdentifierFactory() - .getMethodSignature("main", "dummyMain", "void", Collections.emptyList()); + .getMethodSignature("dummyMain", "main", "void", Collections.emptyList()); Body body = builder .setMethodSignature(methodSignature) diff --git a/sootup.java.core/src/test/java/sootup/java/core/signatures/JavaIdentifierFactoryTest.java b/sootup.java.core/src/test/java/sootup/java/core/signatures/JavaIdentifierFactoryTest.java index a5fcf27b61d..c3cc1cf74d4 100644 --- a/sootup.java.core/src/test/java/sootup/java/core/signatures/JavaIdentifierFactoryTest.java +++ b/sootup.java.core/src/test/java/sootup/java/core/signatures/JavaIdentifierFactoryTest.java @@ -189,7 +189,7 @@ public void getMethodSignature() { List parameters = Collections.singletonList("java.lang.Class"); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "java.lang.A", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "java.lang.A", parameters); assertEquals(declClass, methodSignature.getDeclClassType()); assertEquals(returnType, methodSignature.getType()); assertEquals(parameter, methodSignature.getParameterTypes().get(0)); @@ -203,7 +203,7 @@ public void getMethodSignatureString() { List parameters = Collections.singletonList("java.lang.Class"); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "java.lang.A", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "java.lang.A", parameters); assertEquals( "", methodSignature.toString()); } @@ -215,7 +215,7 @@ public void getMethodSignatureString2() { List parameters = Collections.singletonList("java.lang.Class"); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); assertEquals("", methodSignature.toString()); } @@ -226,7 +226,7 @@ public void getMethodSignatureString3() { List parameters = Collections.emptyList(); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); assertEquals("", methodSignature.toString()); } @@ -258,10 +258,10 @@ public void compMethodSignature2() { List parameters = new ArrayList<>(); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); parameters.add("boolean"); MethodSignature methodSignature2 = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); assertNotEquals(methodSignature, methodSignature2); assertNotEquals(methodSignature.hashCode(), methodSignature2.hashCode()); @@ -274,9 +274,9 @@ public void compMethodSignature1() { List parameters = Collections.emptyList(); MethodSignature methodSignature = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); MethodSignature methodSignature2 = - identifierFactory.getMethodSignature("foo", "java.lang.System", "void", parameters); + identifierFactory.getMethodSignature("java.lang.System", "foo", "void", parameters); assertEquals(methodSignature, methodSignature2); assertEquals(methodSignature.hashCode(), methodSignature2.hashCode()); diff --git a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java index 37606aac35d..737a7c92145 100644 --- a/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java +++ b/sootup.java.sourcecode/src/main/java/sootup/java/sourcecode/frontend/InstructionConverter.java @@ -379,7 +379,7 @@ private List convertAssertInstruction( stmts.add(newAssignStmt); MethodSignature methodSig = identifierFactory.getMethodSignature( - "", "java.lang.AssertionError", "void", Collections.emptyList()); + "java.lang.AssertionError", "", "void", Collections.emptyList()); JSpecialInvokeExpr invoke = Jimple.newSpecialInvokeExpr(failureLocal, methodSig); JInvokeStmt invokeStmt = Jimple.newInvokeStmt( @@ -795,7 +795,7 @@ private Stmt convertInvokeInstruction(AstJavaInvokeInstruction invokeInst) { MethodSignature methodSig = identifierFactory.getMethodSignature( - target.getName().toString(), declaringClassSignature, returnType, parameters); + declaringClassSignature, target.getName().toString(), returnType, parameters); if (!callee.isStatic()) { int receiver = invokeInst.getReceiver(); @@ -940,8 +940,8 @@ private List convertStringAddition( MethodSignature initMethod = identifierFactory.getMethodSignature( - "", sbType.getFullyQualifiedName(), + "", VoidType.getInstance().toString(), Collections.singletonList(type.toString())); CAstSourcePositionMap.Position[] pos1 = new CAstSourcePositionMap.Position[2]; @@ -957,8 +957,8 @@ private List convertStringAddition( MethodSignature appendMethod = identifierFactory.getMethodSignature( - "append", sbType.getFullyQualifiedName(), + "append", sbType.toString(), Collections.singletonList(type.toString())); Local strBuilderLocal2 = localGenerator.generateLocal(sbType); @@ -976,7 +976,7 @@ private List convertStringAddition( MethodSignature toStringMethod = identifierFactory.getMethodSignature( - "toString", sbType.getFullyQualifiedName(), sbType.toString(), Collections.emptyList()); + sbType.getFullyQualifiedName(), "toString", sbType.toString(), Collections.emptyList()); Stmt toStringStmt = Jimple.newAssignStmt( diff --git a/sootup.tests/src/test/java/sootup/tests/MutableSootClientTest.java b/sootup.tests/src/test/java/sootup/tests/MutableSootClientTest.java index cccf19b9a7d..68198d20df5 100644 --- a/sootup.tests/src/test/java/sootup/tests/MutableSootClientTest.java +++ b/sootup.tests/src/test/java/sootup/tests/MutableSootClientTest.java @@ -129,7 +129,7 @@ public void methodRemovalTest() { public void methodAdditionTest() { MethodSignature methodSignature = p.getIdentifierFactory() - .getMethodSignature("addedMethod", "utils.Operations", "void", Collections.emptyList()); + .getMethodSignature("utils.Operations", "addedMethod", "void", Collections.emptyList()); Body.BodyBuilder bodyBuilder = Body.builder(); Body body = bodyBuilder.setMethodSignature(methodSignature).build(); JavaSootMethod newMethod = From 4af7a74d11eb9fd945b728bbb44f174f6668b1ee Mon Sep 17 00:00:00 2001 From: Bhargav Shirin Nalamati Date: Wed, 25 Oct 2023 21:51:22 +0530 Subject: [PATCH 53/54] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 5d2b9119d87..051ff3107c3 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ See our [Todo list](https://github.com/soot-oss/SootUp/wiki/TODOs). ## Feel free to improve Soot! ### Feedback and Feature Requests -For feedbacks and feature requests, best create appropriate [issues](../../issues). +For feedback and feature requests, best create appropriate [issues](../../issues). ### Collaboration You want to collaborate? Please read our [coding guidelines and the contributors notice](../../wiki/contribution-to-SootUp). From 36153fee0dfc1c47fb673a5b9b6e39965847873e Mon Sep 17 00:00:00 2001 From: Markus Schmidt Date: Thu, 26 Oct 2023 11:41:56 +0200 Subject: [PATCH 54/54] cleanup --- .../BytecodeBodyInterceptors.java | 29 +++++++++---------- .../bytecode/interceptors/AggregatorTest.java | 29 ------------------- 2 files changed, 14 insertions(+), 44 deletions(-) diff --git a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java index 944a91b084c..19150d7ea6d 100644 --- a/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java +++ b/sootup.java.bytecode/src/main/java/sootup/java/bytecode/interceptors/BytecodeBodyInterceptors.java @@ -20,6 +20,7 @@ * . * #L% */ + import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -29,22 +30,20 @@ /** Built-in sets of {@link BodyInterceptor}s for the bytecode frontend */ public enum BytecodeBodyInterceptors { Default( - /* new CastAndReturnInliner(), - new UnreachableCodeEliminator(), - new LocalSplitter(), - new Aggregator(), - - */ + new NopEliminator(), + new CastAndReturnInliner(), + new UnreachableCodeEliminator(), + new LocalSplitter(), + new Aggregator(), + new CopyPropagator(), + new DeadAssignmentEliminator(), + new UnusedLocalEliminator(), + new ConditionalBranchFolder(), + new EmptySwitchEliminator(), new TypeAssigner() - // ms: is already called from typeassigner? new LocalNameStandardizer(), - /* new CopyPropagator(), - new DeadAssignmentEliminator(), - new ConditionalBranchFolder(), - new EmptySwitchEliminator(), - new NopEliminator(), - new UnusedLocalEliminator(), - new UnreachableCodeEliminator() - */ ); + // ms: is already called from typeassigner? new LocalNameStandardizer() + // new LocalNameStandardizer(), + ); @Nonnull private final List bodyInterceptors; diff --git a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java index d336a4d8f68..6112204829d 100644 --- a/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java +++ b/sootup.java.bytecode/src/test/java/sootup/java/bytecode/interceptors/AggregatorTest.java @@ -2,15 +2,12 @@ import static org.junit.Assert.assertEquals; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Set; -import javax.annotation.Nonnull; import org.junit.Test; import sootup.core.graph.MutableStmtGraph; import sootup.core.inputlocation.AnalysisInputLocation; -import sootup.core.inputlocation.ClassLoadingOptions; import sootup.core.jimple.Jimple; import sootup.core.jimple.basic.Local; import sootup.core.jimple.basic.NoPositionInformation; @@ -21,7 +18,6 @@ import sootup.core.jimple.common.stmt.Stmt; import sootup.core.model.Body; import sootup.core.model.SootMethod; -import sootup.core.transform.BodyInterceptor; import sootup.core.types.PrimitiveType; import sootup.core.util.ImmutableUtils; import sootup.java.bytecode.inputlocation.JavaClassPathAnalysisInputLocation; @@ -165,31 +161,6 @@ public void testResource_Misuse() { JavaProject project = JavaProject.builder(language).addInputLocation(inputLocation).build(); JavaView view = project.createView(); - - view.configBodyInterceptors( - (analysisInputLocation) -> - new ClassLoadingOptions() { - @Nonnull - @Override - public List getBodyInterceptors() { - return Arrays.asList( - /* new CastAndReturnInliner(), - new UnreachableCodeEliminator(), - new LocalSplitter(), // FIXME: - // FIXME new Aggregator() */ - new TypeAssigner() - // ms: is already called from typeassigner? new LocalNameStandardizer(), - /* new CopyPropagator(), - new DeadAssignmentEliminator(), // FIXME: removes a branch wrongfully - new ConditionalBranchFolder(), - new EmptySwitchEliminator(), - new NopEliminator(), - new UnusedLocalEliminator(), - new UnreachableCodeEliminator() - */ ); - } - }); - { final SootMethod sootMethod = view.getMethod(view.getIdentifierFactory().parseMethodSignature(""))