summaryrefslogtreecommitdiff
path: root/src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala
diff options
context:
space:
mode:
Diffstat (limited to 'src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala')
-rw-r--r--src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala492
1 files changed, 324 insertions, 168 deletions
diff --git a/src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala b/src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala
index 96455c0e38..a740ca525c 100644
--- a/src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala
+++ b/src/compiler/scala/tools/nsc/backend/jvm/opt/CallGraph.scala
@@ -7,182 +7,320 @@ package scala.tools.nsc
package backend.jvm
package opt
+import scala.collection.immutable.IntMap
import scala.reflect.internal.util.{NoPosition, Position}
-import scala.tools.asm.tree.analysis.{Value, Analyzer, BasicInterpreter}
-import scala.tools.asm.{Opcodes, Type, Handle}
+import scala.tools.asm.{Handle, Opcodes, Type}
import scala.tools.asm.tree._
-import scala.collection.concurrent
-import scala.collection.convert.decorateAsScala._
-import scala.tools.nsc.backend.jvm.BTypes.InternalName
+import scala.collection.{concurrent, mutable}
+import scala.collection.JavaConverters._
+import scala.tools.nsc.backend.jvm.BTypes.{InternalName, MethodInlineInfo}
import scala.tools.nsc.backend.jvm.BackendReporting._
-import scala.tools.nsc.backend.jvm.analysis.{NotNull, NullnessAnalyzer}
-import ByteCodeRepository.{Source, CompilationUnit}
+import scala.tools.nsc.backend.jvm.analysis._
import BytecodeUtils._
class CallGraph[BT <: BTypes](val btypes: BT) {
import btypes._
+ import backendUtils._
- val callsites: concurrent.Map[MethodInsnNode, Callsite] = recordPerRunCache(concurrent.TrieMap.empty)
+ /**
+ * The call graph contains the callsites in the program being compiled.
+ *
+ * Indexing the call graph by the containing MethodNode and the invocation MethodInsnNode allows
+ * finding callsites efficiently. For example, an inlining heuristic might want to know all
+ * callsites within a callee method.
+ *
+ * Note that the call graph is not guaranteed to be complete: callsites may be missing. In
+ * particular, if a method is very large, all of its callsites might not be in the hash map.
+ * The reason is that adding a method to the call graph requires running an ASM analyzer, which
+ * can be too slow.
+ *
+ * Note that call graph entries (Callsite instances) keep a reference to the invocation
+ * MethodInsnNode, which keeps all AbstractInsnNodes of the method reachable. Adding classes
+ * from the classpath to the call graph (in addition to classes being compiled) may prevent
+ * method instruction nodes from being GCd. The ByteCodeRepository has a fixed size cache for
+ * parsed ClassNodes - keeping all ClassNodes alive consumed too much memory.
+ * The call graph is less problematic because only methods being called are kept alive, not entire
+ * classes. But we should keep an eye on this.
+ */
+ val callsites: mutable.Map[MethodNode, Map[MethodInsnNode, Callsite]] = recordPerRunCache(concurrent.TrieMap.empty withDefaultValue Map.empty)
+
+ /**
+ * Closure instantiations in the program being compiled.
+ *
+ * Indexing closure instantiations by the containing MethodNode is beneficial for the closure
+ * optimizer: finding callsites to re-write requires running a producers-consumers analysis on
+ * the method. Here the closure instantiations are already grouped by method.
+ */
+ val closureInstantiations: mutable.Map[MethodNode, Map[InvokeDynamicInsnNode, ClosureInstantiation]] = recordPerRunCache(concurrent.TrieMap.empty withDefaultValue Map.empty)
+
+ def removeCallsite(invocation: MethodInsnNode, methodNode: MethodNode): Option[Callsite] = {
+ val methodCallsites = callsites(methodNode)
+ val newCallsites = methodCallsites - invocation
+ if (newCallsites.isEmpty) callsites.remove(methodNode)
+ else callsites(methodNode) = newCallsites
+ methodCallsites.get(invocation)
+ }
+
+ def addCallsite(callsite: Callsite): Unit = {
+ val methodCallsites = callsites(callsite.callsiteMethod)
+ callsites(callsite.callsiteMethod) = methodCallsites + (callsite.callsiteInstruction -> callsite)
+ }
- val closureInstantiations: concurrent.Map[InvokeDynamicInsnNode, ClosureInstantiation] = recordPerRunCache(concurrent.TrieMap.empty)
+ def containsCallsite(callsite: Callsite): Boolean = callsites(callsite.callsiteMethod) contains callsite.callsiteInstruction
+ def findCallSite(method: MethodNode, call: MethodInsnNode): Option[Callsite] = callsites.getOrElse(method, Map.empty).get(call)
+
+ def removeClosureInstantiation(indy: InvokeDynamicInsnNode, methodNode: MethodNode): Option[ClosureInstantiation] = {
+ val methodClosureInits = closureInstantiations(methodNode)
+ val newClosureInits = methodClosureInits - indy
+ if (newClosureInits.isEmpty) closureInstantiations.remove(methodNode)
+ else closureInstantiations(methodNode) = newClosureInits
+ methodClosureInits.get(indy)
+ }
+
+ def addClosureInstantiation(closureInit: ClosureInstantiation) = {
+ val methodClosureInits = closureInstantiations(closureInit.ownerMethod)
+ closureInstantiations(closureInit.ownerMethod) = methodClosureInits + (closureInit.lambdaMetaFactoryCall.indy -> closureInit)
+ }
def addClass(classNode: ClassNode): Unit = {
val classType = classBTypeFromClassNode(classNode)
- for {
- m <- classNode.methods.asScala
- (calls, closureInits) = analyzeCallsites(m, classType)
- } {
- calls foreach (callsite => callsites(callsite.callsiteInstruction) = callsite)
- closureInits foreach (lmf => closureInstantiations(lmf.indy) = ClosureInstantiation(lmf, m, classType))
- }
+ classNode.methods.asScala.foreach(addMethod(_, classType))
}
- /**
- * Returns a list of callsites in the method, plus a list of closure instantiation indy instructions.
- */
- def analyzeCallsites(methodNode: MethodNode, definingClass: ClassBType): (List[Callsite], List[LambdaMetaFactoryCall]) = {
+ def addIfMissing(methodNode: MethodNode, definingClass: ClassBType): Unit = {
+ if (!callsites.contains(methodNode)) addMethod(methodNode, definingClass)
+ }
- case class CallsiteInfo(safeToInline: Boolean, safeToRewrite: Boolean,
- annotatedInline: Boolean, annotatedNoInline: Boolean,
- warning: Option[CalleeInfoWarning])
+ def addMethod(methodNode: MethodNode, definingClass: ClassBType): Unit = {
+ if (!BytecodeUtils.isAbstractMethod(methodNode) && !BytecodeUtils.isNativeMethod(methodNode)) {
+ // TODO: run dataflow analyses to make the call graph more precise
+ // - producers to get forwarded parameters (ForwardedParam)
+ // - typeAnalysis for more precise argument types, more precise callee
+
+ // For now we run a NullnessAnalyzer. It is used to determine if the receiver of an instance
+ // call is known to be not-null, in which case we don't have to emit a null check when inlining.
+ // It is also used to get the stack height at the call site.
+
+ val analyzer = {
+ if (compilerSettings.optNullnessTracking && AsmAnalyzer.sizeOKForNullness(methodNode)) {
+ Some(new AsmAnalyzer(methodNode, definingClass.internalName, new NullnessAnalyzer(btypes, methodNode)))
+ } else if (AsmAnalyzer.sizeOKForBasicValue(methodNode)) {
+ Some(new AsmAnalyzer(methodNode, definingClass.internalName))
+ } else None
+ }
- /**
- * Analyze a callsite and gather meta-data that can be used for inlining decisions.
- */
- def analyzeCallsite(calleeMethodNode: MethodNode, calleeDeclarationClassBType: ClassBType, receiverTypeInternalName: InternalName, calleeSource: Source): CallsiteInfo = {
- val methodSignature = calleeMethodNode.name + calleeMethodNode.desc
+ // if the method is too large to run an analyzer, it is not added to the call graph
+ if (analyzer.nonEmpty) {
+ val Some(a) = analyzer
+ def receiverNotNullByAnalysis(call: MethodInsnNode, numArgs: Int) = a.analyzer match {
+ case nullnessAnalyzer: NullnessAnalyzer =>
+ val frame = nullnessAnalyzer.frameAt(call, methodNode)
+ frame.getStack(frame.getStackSize - 1 - numArgs) eq NotNullValue
+ case _ => false
+ }
- try {
- // The inlineInfo.methodInfos of a ClassBType holds an InlineInfo for each method *declared*
- // within a class (not for inherited methods). Since we already have the classBType of the
- // callee, we only check there for the methodInlineInfo, we should find it there.
- calleeDeclarationClassBType.info.orThrow.inlineInfo.methodInfos.get(methodSignature) match {
- case Some(methodInlineInfo) =>
- val canInlineFromSource = compilerSettings.YoptInlineGlobal || calleeSource == CompilationUnit
+ var methodCallsites = Map.empty[MethodInsnNode, Callsite]
+ var methodClosureInstantiations = Map.empty[InvokeDynamicInsnNode, ClosureInstantiation]
+
+ // lazy so it is only computed if actually used by computeArgInfos
+ lazy val prodCons = new ProdConsAnalyzer(methodNode, definingClass.internalName)
+
+ methodNode.instructions.iterator.asScala foreach {
+ case call: MethodInsnNode if a.frameAt(call) != null => // skips over unreachable code
+ val callee: Either[OptimizerWarning, Callee] = for {
+ (method, declarationClass) <- byteCodeRepository.methodNode(call.owner, call.name, call.desc): Either[OptimizerWarning, (MethodNode, InternalName)]
+ (declarationClassNode, calleeSourceFilePath) <- byteCodeRepository.classNodeAndSourceFilePath(declarationClass): Either[OptimizerWarning, (ClassNode, Option[String])]
+ } yield {
+ val declarationClassBType = classBTypeFromClassNode(declarationClassNode)
+ val info = analyzeCallsite(method, declarationClassBType, call, calleeSourceFilePath)
+ import info._
+ Callee(
+ callee = method,
+ calleeDeclarationClass = declarationClassBType,
+ isStaticallyResolved = isStaticallyResolved,
+ sourceFilePath = sourceFilePath,
+ annotatedInline = annotatedInline,
+ annotatedNoInline = annotatedNoInline,
+ samParamTypes = info.samParamTypes,
+ calleeInfoWarning = warning)
+ }
- val isAbstract = BytecodeUtils.isAbstractMethod(calleeMethodNode)
+ val argInfos = computeArgInfos(callee, call, prodCons)
- // (1) A non-final method can be safe to inline if the receiver type is a final subclass. Example:
- // class A { @inline def f = 1 }; object B extends A; B.f // can be inlined
- //
- // TODO: type analysis can render more calls statically resolved. Example:
- // new A.f // can be inlined, the receiver type is known to be exactly A.
- val isStaticallyResolved: Boolean = {
- methodInlineInfo.effectivelyFinal ||
- classBTypeFromParsedClassfile(receiverTypeInternalName).info.orThrow.inlineInfo.isEffectivelyFinal // (1)
+ val receiverNotNull = call.getOpcode == Opcodes.INVOKESTATIC || {
+ val numArgs = Type.getArgumentTypes(call.desc).length
+ receiverNotNullByAnalysis(call, numArgs)
}
- val isRewritableTraitCall = isStaticallyResolved && methodInlineInfo.traitMethodWithStaticImplementation
-
- val warning = calleeDeclarationClassBType.info.orThrow.inlineInfo.warning.map(
- MethodInlineInfoIncomplete(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, _))
-
- // (1) For invocations of final trait methods, the callee isStaticallyResolved but also
- // abstract. Such a callee is not safe to inline - it needs to be re-written to the
- // static impl method first (safeToRewrite).
- // (2) Final trait methods can be rewritten from the interface to the static implementation
- // method to enable inlining.
- CallsiteInfo(
- safeToInline =
- canInlineFromSource &&
- isStaticallyResolved && // (1)
- !isAbstract &&
- !BytecodeUtils.isConstructor(calleeMethodNode) &&
- !BytecodeUtils.isNativeMethod(calleeMethodNode),
- safeToRewrite = canInlineFromSource && isRewritableTraitCall, // (2)
- annotatedInline = methodInlineInfo.annotatedInline,
- annotatedNoInline = methodInlineInfo.annotatedNoInline,
- warning = warning)
-
- case None =>
- val warning = MethodInlineInfoMissing(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, calleeDeclarationClassBType.info.orThrow.inlineInfo.warning)
- CallsiteInfo(false, false, false, false, Some(warning))
+ methodCallsites += call -> Callsite(
+ callsiteInstruction = call,
+ callsiteMethod = methodNode,
+ callsiteClass = definingClass,
+ callee = callee,
+ argInfos = argInfos,
+ callsiteStackHeight = a.frameAt(call).getStackSize,
+ receiverKnownNotNull = receiverNotNull,
+ callsitePosition = callsitePositions.getOrElse(call, NoPosition),
+ annotatedInline = inlineAnnotatedCallsites(call),
+ annotatedNoInline = noInlineAnnotatedCallsites(call)
+ )
+
+ case LambdaMetaFactoryCall(indy, samMethodType, implMethod, instantiatedMethodType) if a.frameAt(indy) != null =>
+ val lmf = LambdaMetaFactoryCall(indy, samMethodType, implMethod, instantiatedMethodType)
+ val capturedArgInfos = computeCapturedArgInfos(lmf, prodCons)
+ methodClosureInstantiations += indy -> ClosureInstantiation(
+ lmf,
+ methodNode,
+ definingClass,
+ capturedArgInfos)
+
+ case _ =>
}
- } catch {
- case Invalid(noInfo: NoClassBTypeInfo) =>
- val warning = MethodInlineInfoError(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, noInfo)
- CallsiteInfo(false, false, false, false, Some(warning))
+
+ callsites(methodNode) = methodCallsites
+ closureInstantiations(methodNode) = methodClosureInstantiations
}
}
+ }
- // TODO: run dataflow analyses to make the call graph more precise
- // - producers to get forwarded parameters (ForwardedParam)
- // - typeAnalysis for more precise argument types, more precise callee
-
- // For now we run a NullnessAnalyzer. It is used to determine if the receiver of an instance
- // call is known to be not-null, in which case we don't have to emit a null check when inlining.
- // It is also used to get the stack height at the call site.
- localOpt.minimalRemoveUnreachableCode(methodNode, definingClass.internalName)
-
- val analyzer: Analyzer[_ <: Value] = {
- if (compilerSettings.YoptNullnessTracking) new NullnessAnalyzer
- else new Analyzer(new BasicInterpreter)
+ def computeArgInfos(callee: Either[OptimizerWarning, Callee], callsiteInsn: MethodInsnNode, prodCons: => ProdConsAnalyzer): IntMap[ArgInfo] = {
+ if (callee.isLeft) IntMap.empty
+ else {
+ lazy val numArgs = Type.getArgumentTypes(callsiteInsn.desc).length + (if (callsiteInsn.getOpcode == Opcodes.INVOKESTATIC) 0 else 1)
+ argInfosForSams(callee.get.samParamTypes, callsiteInsn, numArgs, prodCons)
}
- analyzer.analyze(definingClass.internalName, methodNode)
+ }
- def receiverNotNullByAnalysis(call: MethodInsnNode, numArgs: Int) = analyzer match {
- case nullnessAnalyzer: NullnessAnalyzer =>
- val frame = nullnessAnalyzer.frameAt(call, methodNode)
- frame.getStack(frame.getStackSize - 1 - numArgs).nullness == NotNull
+ def computeCapturedArgInfos(lmf: LambdaMetaFactoryCall, prodCons: => ProdConsAnalyzer): IntMap[ArgInfo] = {
+ val capturedSams = capturedSamTypes(lmf)
+ val numCaptures = Type.getArgumentTypes(lmf.indy.desc).length
+ argInfosForSams(capturedSams, lmf.indy, numCaptures, prodCons)
+ }
- case _ => false
+ private def argInfosForSams(sams: IntMap[ClassBType], consumerInsn: AbstractInsnNode, numConsumed: => Int, prodCons: => ProdConsAnalyzer): IntMap[ArgInfo] = {
+ // TODO: use type analysis instead of ProdCons - should be more efficient
+ // some random thoughts:
+ // - assign special types to parameters and indy-lambda-functions to track them
+ // - upcast should not change type flow analysis: don't lose information.
+ // - can we do something about factory calls? Foo(x) for case class foo gives a Foo.
+ // inline the factory? analysis across method boundary?
+
+ // assign to a lazy val to prevent repeated evaluation of the by-name arg
+ lazy val prodConsI = prodCons
+ lazy val firstConsumedSlot = {
+ val consumerFrame = prodConsI.frameAt(consumerInsn)
+ consumerFrame.stackTop - numConsumed + 1
}
-
- val callsites = new collection.mutable.ListBuffer[Callsite]
- val closureInstantiations = new collection.mutable.ListBuffer[LambdaMetaFactoryCall]
-
- methodNode.instructions.iterator.asScala foreach {
- case call: MethodInsnNode =>
- val callee: Either[OptimizerWarning, Callee] = for {
- (method, declarationClass) <- byteCodeRepository.methodNode(call.owner, call.name, call.desc): Either[OptimizerWarning, (MethodNode, InternalName)]
- (declarationClassNode, source) <- byteCodeRepository.classNodeAndSource(declarationClass): Either[OptimizerWarning, (ClassNode, Source)]
- declarationClassBType = classBTypeFromClassNode(declarationClassNode)
- } yield {
- val CallsiteInfo(safeToInline, safeToRewrite, annotatedInline, annotatedNoInline, warning) = analyzeCallsite(method, declarationClassBType, call.owner, source)
- Callee(
- callee = method,
- calleeDeclarationClass = declarationClassBType,
- safeToInline = safeToInline,
- safeToRewrite = safeToRewrite,
- annotatedInline = annotatedInline,
- annotatedNoInline = annotatedNoInline,
- calleeInfoWarning = warning)
+ sams flatMap {
+ case (index, _) =>
+ val prods = prodConsI.initialProducersForValueAt(consumerInsn, firstConsumedSlot + index)
+ if (prods.size != 1) None
+ else {
+ val argInfo = prods.head match {
+ case LambdaMetaFactoryCall(_, _, _, _) => Some(FunctionLiteral)
+ case ParameterProducer(local) => Some(ForwardedParam(local))
+ case _ => None
+ }
+ argInfo.map((index, _))
}
+ }
+ }
- val argInfos = if (callee.isLeft) Nil else {
- // TODO: for now it's Nil, because we don't run any data flow analysis
- // there's no point in using the parameter types, that doesn't add any information.
- // NOTE: need to run the same analyses after inlining, to re-compute the argInfos for the
- // new duplicated callsites, see Inliner.inline
- Nil
- }
+ def samParamTypes(methodNode: MethodNode, receiverType: ClassBType): IntMap[ClassBType] = {
+ val paramTypes = {
+ val params = Type.getMethodType(methodNode.desc).getArgumentTypes.map(t => bTypeForDescriptorOrInternalNameFromClassfile(t.getDescriptor))
+ val isStatic = BytecodeUtils.isStaticMethod(methodNode)
+ if (isStatic) params else receiverType +: params
+ }
+ samTypes(paramTypes)
+ }
- val receiverNotNull = call.getOpcode == Opcodes.INVOKESTATIC || {
- val numArgs = Type.getArgumentTypes(call.desc).length
- receiverNotNullByAnalysis(call, numArgs)
- }
+ def capturedSamTypes(lmf: LambdaMetaFactoryCall): IntMap[ClassBType] = {
+ val capturedTypes = Type.getArgumentTypes(lmf.indy.desc).map(t => bTypeForDescriptorOrInternalNameFromClassfile(t.getDescriptor))
+ samTypes(capturedTypes)
+ }
- callsites += Callsite(
- callsiteInstruction = call,
- callsiteMethod = methodNode,
- callsiteClass = definingClass,
- callee = callee,
- argInfos = argInfos,
- callsiteStackHeight = analyzer.frameAt(call, methodNode).getStackSize,
- receiverKnownNotNull = receiverNotNull,
- callsitePosition = callsitePositions.getOrElse(call, NoPosition)
- )
-
- case LambdaMetaFactoryCall(indy, samMethodType, implMethod, instantiatedMethodType) =>
- closureInstantiations += LambdaMetaFactoryCall(indy, samMethodType, implMethod, instantiatedMethodType)
-
- case _ =>
- }
+ private def samTypes(types: Array[BType]): IntMap[ClassBType] = {
+ var res = IntMap.empty[ClassBType]
+ for (i <- types.indices) {
+ types(i) match {
+ case c: ClassBType =>
+ if (c.info.get.inlineInfo.sam.isDefined) res = res.updated(i, c)
- (callsites.toList, closureInstantiations.toList)
+ case _ =>
+ }
+ }
+ res
}
/**
+ * Just a named tuple used as return type of `analyzeCallsite`.
+ */
+ private case class CallsiteInfo(isStaticallyResolved: Boolean, sourceFilePath: Option[String],
+ annotatedInline: Boolean, annotatedNoInline: Boolean,
+ samParamTypes: IntMap[ClassBType],
+ warning: Option[CalleeInfoWarning])
+
+ /**
+ * Analyze a callsite and gather meta-data that can be used for inlining decisions.
+ */
+ private def analyzeCallsite(calleeMethodNode: MethodNode, calleeDeclarationClassBType: ClassBType, call: MethodInsnNode, calleeSourceFilePath: Option[String]): CallsiteInfo = {
+ val methodSignature = calleeMethodNode.name + calleeMethodNode.desc
+
+ try {
+ // The inlineInfo.methodInfos of a ClassBType holds an InlineInfo for each method *declared*
+ // within a class (not for inherited methods). Since we already have the classBType of the
+ // callee, we only check there for the methodInlineInfo, we should find it there.
+ calleeDeclarationClassBType.info.orThrow.inlineInfo.methodInfos.get(methodSignature) match {
+ case Some(methodInlineInfo) =>
+ val isAbstract = BytecodeUtils.isAbstractMethod(calleeMethodNode)
+
+ val receiverType = classBTypeFromParsedClassfile(call.owner)
+ // (1) A non-final method can be safe to inline if the receiver type is a final subclass. Example:
+ // class A { @inline def f = 1 }; object B extends A; B.f // can be inlined
+ //
+ // TODO: (1) doesn't cover the following example:
+ // trait TravLike { def map = ... }
+ // sealed trait List extends TravLike { ... } // assume map is not overridden
+ // final case class :: / final case object Nil
+ // (l: List).map // can be inlined
+ // we need to know that
+ // - the receiver is sealed
+ // - what are the children of the receiver
+ // - all children are final
+ // - none of the children overrides map
+ //
+ // TODO: type analysis can render more calls statically resolved. Example:
+ // new A.f // can be inlined, the receiver type is known to be exactly A.
+ val isStaticallyResolved: Boolean = {
+ isNonVirtualCall(call) || // SD-86: super calls (invokespecial) can be inlined -- TODO: check if that's still needed, and if it's correct: scala-dev#143
+ methodInlineInfo.effectivelyFinal ||
+ receiverType.info.orThrow.inlineInfo.isEffectivelyFinal // (1)
+ }
+
+ val warning = calleeDeclarationClassBType.info.orThrow.inlineInfo.warning.map(
+ MethodInlineInfoIncomplete(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, _))
+
+ CallsiteInfo(
+ isStaticallyResolved = isStaticallyResolved,
+ sourceFilePath = calleeSourceFilePath,
+ annotatedInline = methodInlineInfo.annotatedInline,
+ annotatedNoInline = methodInlineInfo.annotatedNoInline,
+ samParamTypes = samParamTypes(calleeMethodNode, receiverType),
+ warning = warning)
+
+ case None =>
+ val warning = MethodInlineInfoMissing(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, calleeDeclarationClassBType.info.orThrow.inlineInfo.warning)
+ CallsiteInfo(false, None, false, false, IntMap.empty, Some(warning))
+ }
+ } catch {
+ case Invalid(noInfo: NoClassBTypeInfo) =>
+ val warning = MethodInlineInfoError(calleeDeclarationClassBType.internalName, calleeMethodNode.name, calleeMethodNode.desc, noInfo)
+ CallsiteInfo(false, None, false, false, IntMap.empty, Some(warning))
+ }
+ }
+
+ /**
* A callsite in the call graph.
*
* @param callsiteInstruction The invocation instruction
@@ -197,21 +335,35 @@ class CallGraph[BT <: BTypes](val btypes: BT) {
* @param callsitePosition The source position of the callsite, used for inliner warnings.
*/
final case class Callsite(callsiteInstruction: MethodInsnNode, callsiteMethod: MethodNode, callsiteClass: ClassBType,
- callee: Either[OptimizerWarning, Callee], argInfos: List[ArgInfo],
- callsiteStackHeight: Int, receiverKnownNotNull: Boolean, callsitePosition: Position) {
+ callee: Either[OptimizerWarning, Callee], argInfos: IntMap[ArgInfo],
+ callsiteStackHeight: Int, receiverKnownNotNull: Boolean, callsitePosition: Position,
+ annotatedInline: Boolean, annotatedNoInline: Boolean) {
+ /**
+ * Contains callsites that were created during inlining by cloning this callsite. Used to find
+ * corresponding callsites when inlining post-inline requests.
+ */
+ val inlinedClones = mutable.Set.empty[ClonedCallsite]
+
+ // an annotation at the callsite takes precedence over an annotation at the definition site
+ def isInlineAnnotated = annotatedInline || (callee.get.annotatedInline && !annotatedNoInline)
+ def isNoInlineAnnotated = annotatedNoInline || (callee.get.annotatedNoInline && !annotatedInline)
+
override def toString =
"Invocation of" +
s" ${callee.map(_.calleeDeclarationClass.internalName).getOrElse("?")}.${callsiteInstruction.name + callsiteInstruction.desc}" +
s"@${callsiteMethod.instructions.indexOf(callsiteInstruction)}" +
- s" in ${callsiteClass.internalName}.${callsiteMethod.name}"
+ s" in ${callsiteClass.internalName}.${callsiteMethod.name}${callsiteMethod.desc}"
}
+ final case class ClonedCallsite(callsite: Callsite, clonedWhenInlining: Callsite)
+
/**
* Information about invocation arguments, obtained through data flow analysis of the callsite method.
*/
sealed trait ArgInfo
- final case class ArgTypeInfo(argType: BType, isPrecise: Boolean, knownNotNull: Boolean) extends ArgInfo
+ case object FunctionLiteral extends ArgInfo
final case class ForwardedParam(index: Int) extends ArgInfo
+ // final case class ArgTypeInfo(argType: BType, isPrecise: Boolean, knownNotNull: Boolean) extends ArgInfo
// can be extended, e.g., with constant types
/**
@@ -221,46 +373,50 @@ class CallGraph[BT <: BTypes](val btypes: BT) {
* virtual calls, an override of the callee might be invoked. Also,
* the callee can be abstract.
* @param calleeDeclarationClass The class in which the callee is declared
- * @param safeToInline True if the callee can be safely inlined: it cannot be overridden,
- * and the inliner settings (project / global) allow inlining it.
- * @param safeToRewrite True if the callee is the interface method of a concrete trait method
- * that can be safely re-written to the static implementation method.
+ * @param isStaticallyResolved True if the callee cannot be overridden
* @param annotatedInline True if the callee is annotated @inline
* @param annotatedNoInline True if the callee is annotated @noinline
+ * @param samParamTypes A map from parameter positions to SAM parameter types
* @param calleeInfoWarning An inliner warning if some information was not available while
* gathering the information about this callee.
*/
- final case class Callee(callee: MethodNode, calleeDeclarationClass: ClassBType,
- safeToInline: Boolean, safeToRewrite: Boolean,
+ final case class Callee(callee: MethodNode, calleeDeclarationClass: btypes.ClassBType,
+ isStaticallyResolved: Boolean, sourceFilePath: Option[String],
annotatedInline: Boolean, annotatedNoInline: Boolean,
+ samParamTypes: IntMap[btypes.ClassBType],
calleeInfoWarning: Option[CalleeInfoWarning]) {
- assert(!(safeToInline && safeToRewrite), s"A callee of ${callee.name} can be either safeToInline or safeToRewrite, but not both.")
+ override def toString = s"Callee($calleeDeclarationClass.${callee.name})"
+
+ def canInlineFromSource = inlinerHeuristics.canInlineFromSource(sourceFilePath)
+ def isAbstract = isAbstractMethod(callee)
+ def isSpecialMethod = isConstructor(callee) || isNativeMethod(callee) || hasCallerSensitiveAnnotation(callee)
+
+ def safeToInline = isStaticallyResolved && canInlineFromSource && !isAbstract && !isSpecialMethod
}
- final case class ClosureInstantiation(lambdaMetaFactoryCall: LambdaMetaFactoryCall, ownerMethod: MethodNode, ownerClass: ClassBType) {
+ /**
+ * Metadata about a closure instantiation, stored in the call graph
+ *
+ * @param lambdaMetaFactoryCall the InvokeDynamic instruction
+ * @param ownerMethod the method where the closure is allocated
+ * @param ownerClass the class containing the above method
+ * @param capturedArgInfos information about captured arguments. Used for updating the call
+ * graph when re-writing a closure invocation to the body method.
+ */
+ final case class ClosureInstantiation(lambdaMetaFactoryCall: LambdaMetaFactoryCall, ownerMethod: MethodNode, ownerClass: ClassBType, capturedArgInfos: IntMap[ArgInfo]) {
+ /**
+ * Contains closure instantiations that were created during inlining by cloning this instantiation.
+ */
+ val inlinedClones = mutable.Set.empty[ClosureInstantiation]
override def toString = s"ClosureInstantiation($lambdaMetaFactoryCall, ${ownerMethod.name + ownerMethod.desc}, $ownerClass)"
}
final case class LambdaMetaFactoryCall(indy: InvokeDynamicInsnNode, samMethodType: Type, implMethod: Handle, instantiatedMethodType: Type)
object LambdaMetaFactoryCall {
- private val lambdaMetaFactoryInternalName: InternalName = "java/lang/invoke/LambdaMetafactory"
-
- private val metafactoryHandle = {
- val metafactoryMethodName: String = "metafactory"
- val metafactoryDesc: String = "(Ljava/lang/invoke/MethodHandles$Lookup;Ljava/lang/String;Ljava/lang/invoke/MethodType;Ljava/lang/invoke/MethodType;Ljava/lang/invoke/MethodHandle;Ljava/lang/invoke/MethodType;)Ljava/lang/invoke/CallSite;"
- new Handle(Opcodes.H_INVOKESTATIC, lambdaMetaFactoryInternalName, metafactoryMethodName, metafactoryDesc)
- }
-
- private val altMetafactoryHandle = {
- val altMetafactoryMethodName: String = "altMetafactory"
- val altMetafactoryDesc: String = "(Ljava/lang/invoke/MethodHandles$Lookup;Ljava/lang/String;Ljava/lang/invoke/MethodType;[Ljava/lang/Object;)Ljava/lang/invoke/CallSite;"
- new Handle(Opcodes.H_INVOKESTATIC, lambdaMetaFactoryInternalName, altMetafactoryMethodName, altMetafactoryDesc)
- }
-
def unapply(insn: AbstractInsnNode): Option[(InvokeDynamicInsnNode, Type, Handle, Type)] = insn match {
- case indy: InvokeDynamicInsnNode if indy.bsm == metafactoryHandle || indy.bsm == altMetafactoryHandle =>
+ case indy: InvokeDynamicInsnNode if indy.bsm == coreBTypes.lambdaMetaFactoryMetafactoryHandle || indy.bsm == coreBTypes.lambdaMetaFactoryAltMetafactoryHandle =>
indy.bsmArgs match {
- case Array(samMethodType: Type, implMethod: Handle, instantiatedMethodType: Type, xs@_*) => // xs binding because IntelliJ gets confused about _@_*
+ case Array(samMethodType: Type, implMethod: Handle, instantiatedMethodType: Type, _@_*) =>
// LambdaMetaFactory performs a number of automatic adaptations when invoking the lambda
// implementation method (casting, boxing, unboxing, and primitive widening, see Javadoc).
//
@@ -284,7 +440,7 @@ class CallGraph[BT <: BTypes](val btypes: BT) {
// When re-writing the closure callsite to the implMethod, we have to insert a cast.
//
// The check below ensures that
- // (1) the implMethod type has the expected singature (captured types plus argument types
+ // (1) the implMethod type has the expected signature (captured types plus argument types
// from instantiatedMethodType)
// (2) the receiver of the implMethod matches the first captured type
// (3) all parameters that are not the same in samMethodType and instantiatedMethodType