package scala.tools.nsc
package typechecker
import scala.collection.{ mutable, immutable }
import scala.tools.nsc.util.BatchSourceFile
import mutable.ListBuffer
import symtab.Flags._
import util.Statistics
import util.Statistics._
import scala.tools.util.StringOps.{ countAsString, countElementsAsString }
trait Typers extends Modes {
self: Analyzer =>
import global._
import definitions._
final def forArgMode(fun: Tree, mode: Int) =
if (treeInfo.isSelfOrSuperConstrCall(fun)) mode | SCCmode
else mode
val transformed = new mutable.HashMap[Tree, Tree]
final val shortenImports = false
def resetTyper() {
resetContexts()
resetNamer()
resetImplicits()
transformed.clear()
resetSynthetics()
}
object UnTyper extends Traverser {
override def traverse(tree: Tree) = {
if (tree != EmptyTree) tree.tpe = null
if (tree.hasSymbol) tree.symbol = NoSymbol
super.traverse(tree)
}
}
def newTyper(context: Context): Typer = new NormalTyper(context)
private class NormalTyper(context : Context) extends Typer(context)
private final val SYNTHETIC_PRIVATE = TRANS_FLAG
abstract class Typer(context0: Context) extends TyperDiagnostics {
import context0.unit
import typeDebug.{ ptTree, ptBlock, ptLine }
val infer = new Inferencer(context0) {
override def isCoercible(tp: Type, pt: Type): Boolean = undoLog undo {
tp.isError || pt.isError ||
context0.implicitsEnabled &&
inferView(EmptyTree, tp, pt, false) != EmptyTree
}}
def applyImplicitArgs(fun: Tree): Tree = fun.tpe match {
case MethodType(params, _) =>
val argResultsBuff = new ListBuffer[SearchResult]()
val argBuff = new ListBuffer[Tree]()
def mkPositionalArg(argTree: Tree, paramName: Name) = argTree
def mkNamedArg(argTree: Tree, paramName: Name) = atPos(argTree.pos)(new AssignOrNamedArg(Ident(paramName), (argTree)))
var mkArg: (Tree, Name) => Tree = mkPositionalArg
def errorMessage(paramName: Name, paramTp: Type) =
paramTp.typeSymbol match {
case ImplicitNotFoundMsg(msg) => msg.format(paramName, paramTp)
case _ =>
"could not find implicit value for "+
(if (paramName startsWith nme.EVIDENCE_PARAM_PREFIX) "evidence parameter of type "
else "parameter "+paramName+": ")+paramTp
}
for(param <- params) {
var paramTp = param.tpe
for(ar <- argResultsBuff)
paramTp = paramTp.subst(ar.subst.from, ar.subst.to)
val res = inferImplicit(fun, paramTp, true, false, context)
argResultsBuff += res
if (res != SearchFailure) {
argBuff += mkArg(res.tree, param.name)
} else {
mkArg = mkNamedArg
if (!param.hasDefault)
context.error(fun.pos, errorMessage(param.name, param.tpe))
}
}
val args = argBuff.toList
for (ar <- argResultsBuff) {
ar.subst traverse fun
for (arg <- args) ar.subst traverse arg
}
new ApplyToImplicitArgs(fun, args) setPos fun.pos
case ErrorType =>
fun
}
def inferView(tree: Tree, from: Type, to: Type, reportAmbiguous: Boolean): Tree = {
if (settings.debug.value) log("infer view from "+from+" to "+to)
if (phase.id > currentRun.typerPhase.id) EmptyTree
else from match {
case MethodType(_, _) => EmptyTree
case OverloadedType(_, _) => EmptyTree
case PolyType(_, _) => EmptyTree
case _ =>
def wrapImplicit(from: Type): Tree = {
val result = inferImplicit(tree, functionType(List(from), to), reportAmbiguous, true, context)
if (result.subst != EmptyTreeTypeSubstituter) result.subst traverse tree
result.tree
}
val result = wrapImplicit(from)
if (result != EmptyTree) result
else wrapImplicit(appliedType(ByNameParamClass.typeConstructor, List(from)))
}
}
import infer._
private var namerCache: Namer = null
def namer = {
if ((namerCache eq null) || namerCache.context != context)
namerCache = newNamer(context)
namerCache
}
var context = context0
def context1 = context
def checkStable(tree: Tree): Tree =
if (treeInfo.isPureExpr(tree)) tree
else errorTree(
tree,
"stable identifier required, but "+tree+" found."+
(if (isStableExceptVolatile(tree)) {
val tpe = tree.symbol.tpe match {
case PolyType(_, rtpe) => rtpe
case t => t
}
"\n Note that "+tree.symbol+" is not stable because its type, "+tree.tpe+", is volatile."
} else ""))
private def isStableExceptVolatile(tree: Tree) = {
tree.hasSymbol && tree.symbol != NoSymbol && tree.tpe.isVolatile &&
{ val savedTpe = tree.symbol.info
val savedSTABLE = tree.symbol getFlag STABLE
tree.symbol setInfo AnyRefClass.tpe
tree.symbol setFlag STABLE
val result = treeInfo.isPureExpr(tree)
tree.symbol setInfo savedTpe
tree.symbol setFlag savedSTABLE
result
}
}
def checkClassType(tpt: Tree, existentialOK: Boolean, stablePrefix: Boolean) {
def check(tpe: Type): Unit = tpe.normalize match {
case TypeRef(pre, sym, _) if sym.isClass && !sym.isRefinementClass =>
if (stablePrefix && phase.id <= currentRun.typerPhase.id && !pre.isStable)
error(tpt.pos, "type "+pre+" is not a stable prefix")
case ErrorType => ;
case PolyType(_, restpe) => check(restpe)
case ExistentialType(_, restpe) if existentialOK => check(restpe)
case AnnotatedType(_, underlying, _) => check(underlying)
case t => error(tpt.pos, "class type required but "+t+" found")
}
check(tpt.tpe)
}
def checkNonCyclic(pos: Position, tp: Type): Boolean = {
def checkNotLocked(sym: Symbol): Boolean = {
sym.initialize
sym.lockOK || {error(pos, "cyclic aliasing or subtyping involving "+sym); false}
}
tp match {
case TypeRef(pre, sym, args) =>
(checkNotLocked(sym)) && (
!sym.isNonClassType ||
checkNonCyclic(pos, appliedType(pre.memberInfo(sym), args), sym)
)
case SingleType(pre, sym) =>
checkNotLocked(sym)
case st: SubType =>
checkNonCyclic(pos, st.supertype)
case ct: CompoundType =>
ct.parents forall (x => checkNonCyclic(pos, x))
case _ =>
true
}
}
def checkNonCyclic(pos: Position, tp: Type, lockedSym: Symbol): Boolean = try {
lockedSym.lock {
throw new TypeError("illegal cyclic reference involving " + lockedSym)
}
checkNonCyclic(pos, tp)
} finally {
lockedSym.unlock()
}
def checkNonCyclic(sym: Symbol) {
if (!checkNonCyclic(sym.pos, sym.tpe)) sym.setInfo(ErrorType)
}
def checkNonCyclic(defn: Tree, tpt: Tree) {
if (!checkNonCyclic(defn.pos, tpt.tpe, defn.symbol)) {
tpt.tpe = ErrorType
defn.symbol.setInfo(ErrorType)
}
}
def checkParamsConvertible(pos: Position, tpe: Type) {
tpe match {
case MethodType(formals, restpe) =>
if (restpe.isDependent)
error(pos, "method with dependent type "+tpe+" cannot be converted to function value")
checkParamsConvertible(pos, restpe)
case _ =>
}
}
def checkStarPatOK(pos: Position, mode: Int) =
if ((mode & STARmode) == 0 && phase.id <= currentRun.typerPhase.id)
error(pos, "star patterns must correspond with varargs parameters")
object checkNoEscaping extends TypeMap {
private var owner: Symbol = _
private var scope: Scope = _
private var hiddenSymbols: List[Symbol] = _
def privates[T <: Tree](owner: Symbol, tree: T): T =
check(owner, EmptyScope, WildcardType, tree)
def locals[T <: Tree](scope: Scope, pt: Type, tree: T): T =
check(NoSymbol, scope, pt, tree)
def check[T <: Tree](owner: Symbol, scope: Scope, pt: Type, tree: T): T = {
this.owner = owner
this.scope = scope
hiddenSymbols = List()
val tp1 = apply(tree.tpe)
if (hiddenSymbols.isEmpty) tree setType tp1
else if (hiddenSymbols exists (_.isErroneous)) setError(tree)
else if (isFullyDefined(pt)) tree setType pt
else if (tp1.typeSymbol.isAnonymousClass)
check(owner, scope, pt, tree setType tp1.typeSymbol.classBound)
else if (owner == NoSymbol)
tree setType packSymbols(hiddenSymbols.reverse, tp1)
else if (!phase.erasedTypes) {
val badSymbol = hiddenSymbols.head
error(tree.pos,
(if (badSymbol.isPrivate) "private " else "") + badSymbol +
" escapes its defining scope as part of type "+tree.tpe)
setError(tree)
} else tree
}
def addHidden(sym: Symbol) =
if (!(hiddenSymbols contains sym)) hiddenSymbols = sym :: hiddenSymbols
override def apply(t: Type): Type = {
def checkNoEscape(sym: Symbol) {
if (sym.isPrivate && !sym.hasFlag(SYNTHETIC_PRIVATE)) {
var o = owner
while (o != NoSymbol && o != sym.owner && o != sym.owner.linkedClassOfClass &&
!o.isLocal && !o.isPrivate &&
!o.privateWithin.hasTransOwner(sym.owner))
o = o.owner
if (o == sym.owner || o == sym.owner.linkedClassOfClass)
addHidden(sym)
} else if (sym.owner.isTerm && !sym.isTypeParameterOrSkolem) {
var e = scope.lookupEntry(sym.name)
var found = false
while (!found && (e ne null) && e.owner == scope) {
if (e.sym == sym) {
found = true
addHidden(sym)
} else {
e = scope.lookupNextEntry(e)
}
}
}
}
mapOver(
t match {
case TypeRef(_, sym, args) =>
checkNoEscape(sym)
if (!hiddenSymbols.isEmpty && hiddenSymbols.head == sym &&
sym.isAliasType && sameLength(sym.typeParams, args)) {
hiddenSymbols = hiddenSymbols.tail
t.normalize
} else t
case SingleType(_, sym) =>
checkNoEscape(sym)
t
case _ =>
t
})
}
}
def reenterValueParams(vparamss: List[List[ValDef]]) {
for (vparams <- vparamss)
for (vparam <- vparams)
vparam.symbol = context.scope enter vparam.symbol
}
def reenterTypeParams(tparams: List[TypeDef]): List[Symbol] =
for (tparam <- tparams) yield {
tparam.symbol = context.scope enter tparam.symbol
tparam.symbol.deSkolemize
}
def qualifyingClass(tree: Tree, qual: Name, packageOK: Boolean): Symbol =
context.enclClass.owner.ownerChain.find(o => qual.isEmpty || o.isClass && o.name == qual) match {
case Some(c) if packageOK || !c.isPackageClass =>
c
case _ =>
error(
tree.pos,
if (qual.isEmpty) tree+" can be used only in a class, object, or template"
else qual+" is not an enclosing class")
NoSymbol
}
def constrTyperIf(inConstr: Boolean): Typer =
if (inConstr) {
assert(context.undetparams.isEmpty)
newTyper(context.makeConstructorContext)
} else this
def labelTyper(ldef: LabelDef): Typer =
if (ldef.symbol == NoSymbol) {
val typer1 = newTyper(context.makeNewScope(ldef, context.owner))
typer1.enterLabelDef(ldef)
typer1
} else this
final val xtypes = false
def reallyExists(sym: Symbol) = {
if (isStale(sym)) sym.setInfo(NoType)
sym.exists
}
def isStale(sym: Symbol): Boolean = {
sym.rawInfo.isInstanceOf[loaders.ClassfileLoader] && {
sym.rawInfo.load(sym)
(sym.sourceFile ne null) &&
(currentRun.compiledFiles contains sym.sourceFile.path)
}
}
private def isStableContext(tree: Tree, mode: Int, pt: Type) =
isNarrowable(tree.tpe) && ((mode & (EXPRmode | LHSmode)) == EXPRmode) &&
(xtypes ||
(pt.isStable ||
(mode & QUALmode) != 0 && !tree.symbol.isConstant ||
pt.typeSymbol.isAbstractType && pt.bounds.lo.isStable && !(tree.tpe <:< pt)) ||
pt.typeSymbol.isRefinementClass && !(tree.tpe <:< pt))
private def makeAccessible(tree: Tree, sym: Symbol, pre: Type, site: Tree): (Tree, Type) =
if (isInPackageObject(sym, pre.typeSymbol)) {
if (pre.typeSymbol == ScalaPackageClass && sym.isTerm) {
def dealias(sym: Symbol) =
({ val t = gen.mkAttributedRef(sym) ; t.setPos(tree.pos) ; t }, sym.owner.thisType)
sym.name match {
case nme.List => return dealias(ListModule)
case nme.Seq => return dealias(SeqModule)
case nme.Nil => return dealias(NilModule)
case _ =>
}
}
val qual = typedQualifier { atPos(tree.pos.focusStart) {
tree match {
case Ident(_) => Ident(nme.PACKAGEkw)
case Select(qual, _) => Select(qual, nme.PACKAGEkw)
case SelectFromTypeTree(qual, _) => Select(qual, nme.PACKAGEkw)
}
}}
val tree1 = atPos(tree.pos) {
tree match {
case Ident(name) => Select(qual, name)
case Select(_, name) => Select(qual, name)
case SelectFromTypeTree(_, name) => SelectFromTypeTree(qual, name)
}
}
(checkAccessible(tree1, sym, qual.tpe, qual), qual.tpe)
} else {
(checkAccessible(tree, sym, pre, site), pre)
}
private def isInPackageObject(sym: Symbol, pkg: Symbol) =
pkg.isPackageClass && {
sym.alternatives forall { sym =>
!sym.owner.isPackage && {
sym.owner.isPackageObjectClass &&
sym.owner.owner == pkg ||
pkg.isInitialized && {
val pkgobj = pkg.info.member(nme.PACKAGEkw)
pkgobj.isInitialized &&
(pkgobj.info.member(sym.name).alternatives contains sym)
}
}
}
}
private def stabilize(tree: Tree, pre: Type, mode: Int, pt: Type): Tree = {
if (tree.symbol.isOverloaded && !inFunMode(mode))
inferExprAlternative(tree, pt)
val sym = tree.symbol
if (tree.tpe.isError) tree
else if ((mode & (PATTERNmode | FUNmode)) == PATTERNmode && tree.isTerm) {
if (sym.isValue) checkStable(tree)
else errorTree(tree, sym+" is not a value")
} else if ((mode & (EXPRmode | QUALmode)) == EXPRmode && !sym.isValue && !phase.erasedTypes) {
errorTree(tree, sym+" is not a value")
} else {
if (sym.isStable && pre.isStable && !isByNameParamType(tree.tpe) &&
(isStableContext(tree, mode, pt) || sym.isModule && !sym.isMethod))
tree.setType(singleType(pre, sym))
else tree
}
}
private def isNarrowable(tpe: Type): Boolean = tpe match {
case TypeRef(_, _, _) | RefinedType(_, _) => true
case ExistentialType(_, tpe1) => isNarrowable(tpe1)
case AnnotatedType(_, tpe1, _) => isNarrowable(tpe1)
case PolyType(_, tpe1) => isNarrowable(tpe1)
case NullaryMethodType(tpe1) => isNarrowable(tpe1)
case _ => !phase.erasedTypes
}
def stabilizeFun(tree: Tree, mode: Int, pt: Type): Tree = {
val sym = tree.symbol
val pre = tree match {
case Select(qual, _) => qual.tpe
case _ => NoPrefix
}
if (tree.tpe.isInstanceOf[MethodType] && pre.isStable && sym.tpe.params.isEmpty &&
(isStableContext(tree, mode, pt) || sym.isModule))
tree.setType(MethodType(List(), singleType(pre, sym)))
else tree
}
def member(qual: Tree, name: Name) = {
def callSiteWithinClass(clazz: Symbol) = context.enclClass.owner hasTransOwner clazz
val includeLocals = qual.tpe match {
case ThisType(clazz) if callSiteWithinClass(clazz) => true
case SuperType(clazz, _) if callSiteWithinClass(clazz.typeSymbol) => true
case _ => phase.next.erasedTypes
}
if (includeLocals) qual.tpe member name
else qual.tpe nonLocalMember name
}
def silent[T](op: Typer => T,
reportAmbiguousErrors: Boolean = context.reportAmbiguousErrors,
newtree: Tree = context.tree): Any = {
val rawTypeStart = startCounter(rawTypeFailed)
val findMemberStart = startCounter(findMemberFailed)
val subtypeStart = startCounter(subtypeFailed)
val failedSilentStart = startTimer(failedSilentNanos)
try {
if (context.reportGeneralErrors ||
reportAmbiguousErrors != context.reportAmbiguousErrors ||
newtree != context.tree) {
val context1 = context.makeSilent(reportAmbiguousErrors, newtree)
context1.undetparams = context.undetparams
context1.savedTypeBounds = context.savedTypeBounds
context1.namedApplyBlockInfo = context.namedApplyBlockInfo
val typer1 = newTyper(context1)
val result = op(typer1)
context.undetparams = context1.undetparams
context.savedTypeBounds = context1.savedTypeBounds
context.namedApplyBlockInfo = context1.namedApplyBlockInfo
result
} else {
op(this)
}
} catch {
case ex: CyclicReference => throw ex
case ex: TypeError =>
stopCounter(rawTypeFailed, rawTypeStart)
stopCounter(findMemberFailed, findMemberStart)
stopCounter(subtypeFailed, subtypeStart)
stopTimer(failedSilentNanos, failedSilentStart)
ex
}
}
def tryBoth(tree: Tree)(op1: (Typer, Tree) => Tree)(op2: (Typer, Tree) => Tree): Tree =
silent(op1(_, tree)) match {
case result1: Tree =>
result1
case ex1: TypeError =>
silent(op2(_, resetAllAttrs(tree))) match {
case result2: Tree =>
result2
case ex2: TypeError =>
reportTypeError(tree.pos, ex1)
setError(tree)
}
}
protected def adapt(tree: Tree, mode: Int, pt: Type, original: Tree = EmptyTree): Tree = tree.tpe match {
case atp @ AnnotatedType(_, _, _) if canAdaptAnnotations(tree, mode, pt) =>
adaptAnnotations(tree, mode, pt)
case ct @ ConstantType(value) if inNoModes(mode, TYPEmode | FUNmode) && (ct <:< pt) && !forScaladoc && !forInteractive =>
val sym = tree.symbol
if (sym != null && sym.isDeprecated) {
val msg = sym.toString + sym.locationString +" is deprecated: "+ sym.deprecationMessage.getOrElse("")
unit.deprecationWarning(tree.pos, msg)
}
treeCopy.Literal(tree, value)
case OverloadedType(pre, alts) if !inFunMode(mode) =>
inferExprAlternative(tree, pt)
adapt(tree, mode, pt, original)
case NullaryMethodType(restpe) =>
adapt(tree setType restpe, mode, pt, original)
case TypeRef(_, ByNameParamClass, List(arg))
if ((mode & EXPRmode) != 0) =>
adapt(tree setType arg, mode, pt, original)
case tr @ TypeRef(_, sym, _)
if sym.isAliasType && tr.normalize.isInstanceOf[ExistentialType] &&
((mode & (EXPRmode | LHSmode)) == EXPRmode) =>
adapt(tree setType tr.normalize.skolemizeExistential(context.owner, tree), mode, pt, original)
case et @ ExistentialType(_, _) if ((mode & (EXPRmode | LHSmode)) == EXPRmode) =>
adapt(tree setType et.skolemizeExistential(context.owner, tree), mode, pt, original)
case PolyType(tparams, restpe) if inNoModes(mode, TAPPmode | PATTERNmode | HKmode) =>
val tparams1 = cloneSymbols(tparams)
val tree1 = if (tree.isType) tree
else TypeApply(tree, tparams1 map (tparam =>
TypeTree(tparam.tpeHK) setPos tree.pos.focus)) setPos tree.pos
context.undetparams ++= tparams1
adapt(tree1 setType restpe.substSym(tparams, tparams1), mode, pt, original)
case mt: MethodType if mt.isImplicit && ((mode & (EXPRmode | FUNmode | LHSmode)) == EXPRmode) =>
if (context.undetparams nonEmpty) {
context.undetparams =
inferExprInstance(tree, context.extractUndetparams(), pt,
if(settings.YdepMethTpes.value) mt.approximate else mt,
keepNothings = false,
useWeaklyCompatible = true)
}
val typer1 = constrTyperIf(treeInfo.isSelfOrSuperConstrCall(tree))
if (original != EmptyTree && pt != WildcardType)
typer1.silent(tpr => tpr.typed(tpr.applyImplicitArgs(tree), mode, pt)) match {
case result: Tree => result
case ex: TypeError =>
if (settings.debug.value) log("fallback on implicits: "+tree+"/"+resetAllAttrs(original))
val tree1 = typed(resetAllAttrs(original), mode, WildcardType)
tree1.tpe = addAnnotations(tree1, tree1.tpe)
if (tree1.isEmpty) tree1 else adapt(tree1, mode, pt, EmptyTree)
}
else
typer1.typed(typer1.applyImplicitArgs(tree), mode, pt)
case mt: MethodType
if (((mode & (EXPRmode | FUNmode | LHSmode)) == EXPRmode) &&
(context.undetparams.isEmpty || inPolyMode(mode))) =>
val meth = tree match {
case Block(_, tree1) => tree1.symbol
case _ => tree.symbol
}
if (!meth.isConstructor && isFunctionType(pt)) {
if (settings.debug.value) log("eta-expanding "+tree+":"+tree.tpe+" to "+pt)
checkParamsConvertible(tree.pos, tree.tpe)
val tree0 = etaExpand(context.unit, tree)
if (meth.typeParams.nonEmpty) {
instantiate(typed(tree0, mode, WildcardType), mode, pt)
} else
typed(tree0, mode, pt)
} else if (!meth.isConstructor && mt.params.isEmpty) {
adapt(typed(Apply(tree, List()) setPos tree.pos), mode, pt, original)
} else if (context.implicitsEnabled) {
errorTree(tree, "missing arguments for "+meth+meth.locationString+
(if (meth.isConstructor) ""
else ";\nfollow this method with `_' if you want to treat it as a partially applied function"))
} else {
setError(tree)
}
case _ =>
def applyPossible = {
def applyMeth = member(adaptToName(tree, nme.apply), nme.apply)
if ((mode & TAPPmode) != 0)
tree.tpe.typeParams.isEmpty && applyMeth.filter(! _.tpe.typeParams.isEmpty) != NoSymbol
else
applyMeth.filter(_.tpe.paramSectionCount > 0) != NoSymbol
}
if (tree.isType) {
if (inFunMode(mode)) {
tree
} else if (tree.hasSymbol && !tree.symbol.typeParams.isEmpty && !inHKMode(mode) &&
!(tree.symbol.isJavaDefined && context.unit.isJava)) {
errorTree(tree, tree.symbol+" takes type parameters")
tree setType tree.tpe
} else if (
(inHKMode(mode)) &&
!sameLength(tree.tpe.typeParams, pt.typeParams) &&
!(tree.tpe.typeSymbol==AnyClass ||
tree.tpe.typeSymbol==NothingClass ||
pt == WildcardType )) {
errorTree(tree, tree.tpe+" takes "+countElementsAsString(tree.tpe.typeParams.length, "type parameter")+
", expected: "+countAsString(pt.typeParams.length))
tree setType tree.tpe
} else tree match {
case TypeTree() => tree
case _ => TypeTree(tree.tpe) setOriginal(tree)
}
} else if ((mode & (PATTERNmode | FUNmode)) == (PATTERNmode | FUNmode)) {
val extractor = tree.symbol.filter(sym => reallyExists(unapplyMember(sym.tpe)))
if (extractor != NoSymbol) {
tree setSymbol extractor
val unapply = unapplyMember(extractor.tpe)
val clazz = unapplyParameterType(unapply)
if (unapply.isCase && clazz.isCase && !(clazz.ancestors exists (_.isCase))) {
val prefix = tree.tpe.prefix
val tree1 = TypeTree(clazz.primaryConstructor.tpe.asSeenFrom(prefix, clazz.owner))
.setOriginal(tree)
inferConstructorInstance(tree1, clazz.typeParams, pt)
tree1
} else {
tree
}
} else {
errorTree(tree, tree.symbol + " is not a case class constructor, nor does it have an unapply/unapplySeq method")
}
} else if (inAllModes(mode, EXPRmode | FUNmode) &&
!tree.tpe.isInstanceOf[MethodType] &&
!tree.tpe.isInstanceOf[OverloadedType] &&
applyPossible) {
assert(!inHKMode(mode))
val qual = adaptToName(tree, nme.apply) match {
case id @ Ident(_) =>
val pre = if (id.symbol.owner.isPackageClass) id.symbol.owner.thisType
else if (id.symbol.owner.isClass)
context.enclosingSubClassContext(id.symbol.owner).prefix
else NoPrefix
stabilize(id, pre, EXPRmode | QUALmode, WildcardType)
case sel @ Select(qualqual, _) =>
stabilize(sel, qualqual.tpe, EXPRmode | QUALmode, WildcardType)
case other =>
other
}
typed(atPos(tree.pos)(Select(qual, nme.apply)), mode, pt)
} else if (!context.undetparams.isEmpty && !inPolyMode(mode)) {
assert(!inHKMode(mode))
instantiate(tree, mode, pt)
} else if (tree.tpe <:< pt) {
tree
} else {
if (inPatternMode(mode)) {
if ((tree.symbol ne null) && tree.symbol.isModule)
inferModulePattern(tree, pt)
if (isPopulated(tree.tpe, approximateAbstracts(pt)))
return tree
}
val tree1 = constfold(tree, pt)
if (tree1.tpe <:< pt) adapt(tree1, mode, pt, original)
else {
if ((mode & (EXPRmode | FUNmode)) == EXPRmode) {
pt.normalize match {
case TypeRef(_, sym, _) =>
if (sym == UnitClass && tree.tpe <:< AnyClass.tpe) {
if (settings.warnValueDiscard.value)
context.unit.warning(tree.pos, "discarded non-Unit value")
return typed(atPos(tree.pos)(Block(List(tree), Literal(()))), mode, pt)
}
else if (isNumericValueClass(sym) && isNumericSubType(tree.tpe, pt)) {
if (settings.warnNumericWiden.value)
context.unit.warning(tree.pos, "implicit numeric widening")
return typed(atPos(tree.pos)(Select(tree, "to"+sym.name)), mode, pt)
}
case AnnotatedType(_, _, _) if canAdaptAnnotations(tree, mode, pt) =>
return typed(adaptAnnotations(tree, mode, pt), mode, pt)
case _ =>
}
if (!context.undetparams.isEmpty) {
return instantiate(tree, mode, pt)
}
if (context.implicitsEnabled && !tree.tpe.isError && !pt.isError) {
if (settings.debug.value) log("inferring view from "+tree.tpe+" to "+pt)
val coercion = inferView(tree, tree.tpe, pt, true)
if (forMSIL && coercion != null && isCorrespondingDelegate(tree.tpe, pt)) {
val meth: Symbol = tree.tpe.member(nme.apply)
if(settings.debug.value)
log("replacing forward delegate view with: " + meth + ":" + meth.tpe)
return typed(Select(tree, meth), mode, pt)
}
if (coercion != EmptyTree) {
if (settings.debug.value) log("inferred view from "+tree.tpe+" to "+pt+" = "+coercion+":"+coercion.tpe)
return newTyper(context.makeImplicit(context.reportAmbiguousErrors)).typed(
new ApplyImplicitView(coercion, List(tree)) setPos tree.pos, mode, pt)
}
}
}
if (settings.debug.value) {
log("error tree = "+tree)
if (settings.explaintypes.value) explainTypes(tree.tpe, pt)
}
try {
typeErrorTree(tree, tree.tpe, pt)
} catch {
case ex: TypeError =>
if (phase.id > currentRun.typerPhase.id &&
pt.existentialSkolems.nonEmpty) {
context.unit.warning(tree.pos, "recovering from existential Skolem type error in tree \n"+tree+"\nwith type "+tree.tpe+"\n expected type = "+pt+"\n context = "+context.tree)
adapt(tree, mode, pt.subst(pt.existentialSkolems, pt.existentialSkolems map (_ => WildcardType)))
} else
throw ex
}
}
}
}
def instantiate(tree: Tree, mode: Int, pt: Type): Tree = {
inferExprInstance(tree, context.extractUndetparams(), pt)
adapt(tree, mode, pt)
}
def adaptToMember(qual: Tree, searchTemplate: Type): Tree = {
var qtpe = qual.tpe.widen
if (qual.isTerm &&
((qual.symbol eq null) || !qual.symbol.isTerm || qual.symbol.isValue) &&
phase.id <= currentRun.typerPhase.id && !qtpe.isError &&
qtpe.typeSymbol != NullClass && qtpe.typeSymbol != NothingClass && qtpe != WildcardType &&
!qual.isInstanceOf[ApplyImplicitView] &&
context.implicitsEnabled) {
if (qtpe.normalize.isInstanceOf[ExistentialType]) {
qtpe = qtpe.normalize.skolemizeExistential(context.owner, qual)
qual setType qtpe
}
val coercion = inferView(qual, qual.tpe, searchTemplate, true)
if (coercion != EmptyTree)
typedQualifier(atPos(qual.pos)(new ApplyImplicitView(coercion, List(qual))))
else
qual
} else {
qual
}
}
def adaptToArguments(qual: Tree, name: Name, args: List[Tree], pt: Type): Tree = {
def doAdapt(restpe: Type) =
adaptToMember(qual, HasMethodMatching(name, args map (_.tpe), restpe))
if (pt != WildcardType) {
silent(_ => doAdapt(pt)) match {
case result: Tree if result != qual =>
result
case _ =>
if (settings.debug.value) log("fallback on implicits in adaptToArguments: "+qual+" . "+name)
doAdapt(WildcardType)
}
} else
doAdapt(pt)
}
def adaptToMemberWithArgs(tree: Tree, qual: Tree, name: Name, mode: Int): Tree = {
try {
adaptToMember(qual, HasMember(name))
} catch {
case ex: TypeError =>
context.tree match {
case Apply(tree1, args) if (tree1 eq tree) && args.nonEmpty =>
silent(_.typedArgs(args, mode)) match {
case args: List[_] =>
adaptToArguments(qual, name, args.asInstanceOf[List[Tree]], WildcardType)
case _ =>
throw ex
}
case _ =>
throw ex
}
}
}
def adaptToName(qual: Tree, name: Name) =
if (member(qual, name) != NoSymbol) qual
else adaptToMember(qual, HasMember(name))
private def typePrimaryConstrBody(clazz : Symbol, cbody: Tree, tparams: List[Symbol], enclTparams: List[Symbol], vparamss: List[List[ValDef]]): Tree = {
enclTparams foreach (sym => context.scope.enter(sym))
namer.enterValueParams(context.owner, vparamss)
typed(cbody)
}
private def validateNoCaseAncestor(clazz: Symbol) = {
if (!phase.erasedTypes) {
for (ancestor <- clazz.ancestors find (_.isCase)) {
unit.deprecationWarning(clazz.pos, (
"case class `%s' has case ancestor `%s'. Case-to-case inheritance has potentially "+
"dangerous bugs which are unlikely to be fixed. You are strongly encouraged to "+
"instead use extractors to pattern match on non-leaf nodes."
).format(clazz, ancestor))
}
}
}
def parentTypes(templ: Template): List[Tree] =
if (templ.parents.isEmpty) List()
else try {
val clazz = context.owner
var supertpt = typedTypeConstructor(templ.parents.head)
val firstParent = supertpt.tpe.typeSymbol
var mixins = templ.parents.tail map typedType
while ((supertpt.tpe.typeSymbol ne null) && supertpt.tpe.typeSymbol.initialize.isTrait) {
val supertpt1 = typedType(supertpt)
if (!supertpt1.tpe.isError) {
mixins = supertpt1 :: mixins
supertpt = TypeTree(supertpt1.tpe.parents.head) setPos supertpt.pos.focus
}
}
val supertparams = if (supertpt.hasSymbol) supertpt.symbol.typeParams else List()
var supertpe = supertpt.tpe
if (!supertparams.isEmpty)
supertpe = PolyType(supertparams, appliedType(supertpe, supertparams map (_.tpe)))
def transformSuperCall(scall: Tree): Tree = (scall: @unchecked) match {
case Apply(fn, args) =>
treeCopy.Apply(scall, transformSuperCall(fn), args map (_.duplicate))
case Select(Super(_, _), nme.CONSTRUCTOR) =>
treeCopy.Select(
scall,
atPos(supertpt.pos.focus)(New(TypeTree(supertpe)) setType supertpe),
nme.CONSTRUCTOR)
}
treeInfo.firstConstructor(templ.body) match {
case constr @ DefDef(_, _, _, vparamss, _, cbody @ Block(cstats, cunit)) =>
val (preSuperStats, rest) = cstats span (!treeInfo.isSuperConstrCall(_))
val (scall, upToSuperStats) =
if (rest.isEmpty) (EmptyTree, preSuperStats)
else (rest.head, preSuperStats :+ rest.head)
val cstats1: List[Tree] = upToSuperStats map (_.duplicate)
val cbody1 = scall match {
case Apply(_, _) =>
treeCopy.Block(cbody, cstats1.init,
if (supertparams.isEmpty) cunit.duplicate
else transformSuperCall(scall))
case _ =>
treeCopy.Block(cbody, cstats1, cunit.duplicate)
}
val outercontext = context.outer
assert(clazz != NoSymbol)
val cscope = outercontext.makeNewScope(constr, outercontext.owner)
val cbody2 = newTyper(cscope)
.typePrimaryConstrBody(clazz,
cbody1, supertparams, clazz.unsafeTypeParams, vparamss map (_.map(_.duplicate)))
scall match {
case Apply(_, _) =>
val sarg = treeInfo.firstArgument(scall)
if (sarg != EmptyTree && supertpe.typeSymbol != firstParent)
error(sarg.pos, firstParent+" is a trait; does not take constructor arguments")
if (!supertparams.isEmpty) supertpt = TypeTree(cbody2.tpe) setPos supertpt.pos.focus
case _ =>
if (!supertparams.isEmpty) error(supertpt.pos, "missing type arguments")
}
(cstats1, treeInfo.preSuperFields(templ.body)).zipped map {
(ldef, gdef) => gdef.tpt.tpe = ldef.symbol.tpe
}
case _ =>
if (!supertparams.isEmpty) error(supertpt.pos, "missing type arguments")
}
supertpt :: mixins mapConserve (tpt => checkNoEscaping.privates(clazz, tpt))
} catch {
case ex: TypeError =>
templ.tpe = null
reportTypeError(templ.pos, ex)
List(TypeTree(AnyRefClass.tpe))
}
def validateParentClasses(parents: List[Tree], selfType: Type) {
def validateParentClass(parent: Tree, superclazz: Symbol) {
if (!parent.tpe.isError) {
val psym = parent.tpe.typeSymbol.initialize
checkClassType(parent, false, true)
if (psym != superclazz) {
if (psym.isTrait) {
val ps = psym.info.parents
if (!ps.isEmpty && !superclazz.isSubClass(ps.head.typeSymbol))
error(parent.pos, "illegal inheritance; super"+superclazz+
"\n is not a subclass of the super"+ps.head.typeSymbol+
"\n of the mixin " + psym)
} else {
error(parent.pos, psym+" needs to be a trait to be mixed in")
}
}
if (psym.isFinal) {
error(parent.pos, "illegal inheritance from final "+psym)
}
if (psym.isSealed && !phase.erasedTypes) {
if (context.unit.source.file == psym.sourceFile || isValueClass(context.owner))
psym addChild context.owner
else
error(parent.pos, "illegal inheritance from sealed "+psym)
}
if (!(selfType <:< parent.tpe.typeOfThis) &&
!phase.erasedTypes &&
!context.owner.isSynthetic &&
!settings.noSelfCheck.value &&
!selfType.isErroneous &&
!parent.tpe.isErroneous)
{
error(parent.pos, "illegal inheritance;\n self-type "+
selfType+" does not conform to "+parent +
"'s selftype "+parent.tpe.typeOfThis)
if (settings.explaintypes.value) explainTypes(selfType, parent.tpe.typeOfThis)
}
if (parents exists (p => p != parent && p.tpe.typeSymbol == psym && !psym.isError))
error(parent.pos, psym+" is inherited twice")
}
}
if (!parents.isEmpty && !parents.head.tpe.isError)
for (p <- parents) validateParentClass(p, parents.head.tpe.typeSymbol)
}
def checkFinitary(classinfo: ClassInfoType) {
val clazz = classinfo.typeSymbol
for (tparam <- clazz.typeParams) {
if (classinfo.expansiveRefs(tparam) contains tparam) {
error(tparam.pos, "class graph is not finitary because type parameter "+tparam.name+" is expansively recursive")
val newinfo = ClassInfoType(
classinfo.parents map (_.instantiateTypeParams(List(tparam), List(AnyRefClass.tpe))),
classinfo.decls,
clazz)
clazz.setInfo {
clazz.info match {
case PolyType(tparams, _) => PolyType(tparams, newinfo)
case _ => newinfo
}
}
}
}
}
def typedClassDef(cdef: ClassDef): Tree = {
val clazz = cdef.symbol
val typedMods = removeAnnotations(cdef.mods)
assert(clazz != NoSymbol)
reenterTypeParams(cdef.tparams)
val tparams1 = cdef.tparams mapConserve (typedTypeDef)
val impl1 = newTyper(context.make(cdef.impl, clazz, new Scope))
.typedTemplate(cdef.impl, parentTypes(cdef.impl))
val impl2 = typerAddSyntheticMethods(impl1, clazz, context)
if ((clazz != ClassfileAnnotationClass) &&
(clazz isNonBottomSubClass ClassfileAnnotationClass))
restrictionWarning(cdef.pos, unit,
"subclassing Classfile does not\n"+
"make your annotation visible at runtime. If that is what\n"+
"you want, you must write the annotation class in Java.")
if (phase.id <= currentRun.typerPhase.id) {
for (ann <- clazz.getAnnotation(DeprecatedAttr)) {
val m = companionModuleOf(clazz, context)
if (m != NoSymbol)
m.moduleClass.addAnnotation(AnnotationInfo(ann.atp, ann.args, List()))
}
}
treeCopy.ClassDef(cdef, typedMods, cdef.name, tparams1, impl2)
.setType(NoType)
}
def typedModuleDef(mdef: ModuleDef): Tree = {
val linkedClass = companionClassOf(mdef.symbol, context)
if (linkedClass != NoSymbol)
for (c <- linkedClass.info.decl(nme.CONSTRUCTOR).alternatives)
c.initialize
val clazz = mdef.symbol.moduleClass
val maybeAddSerializable = (l: List[Tree]) =>
if (linkedClass == NoSymbol || !linkedClass.isSerializable || clazz.isSerializable) l
else {
clazz.makeSerializable()
l :+ TypeTree(SerializableClass.tpe)
}
val typedMods = removeAnnotations(mdef.mods)
assert(clazz != NoSymbol)
val impl1 = newTyper(context.make(mdef.impl, clazz, new Scope))
.typedTemplate(mdef.impl, maybeAddSerializable(parentTypes(mdef.impl)))
val impl2 = typerAddSyntheticMethods(impl1, clazz, context)
treeCopy.ModuleDef(mdef, typedMods, mdef.name, impl2) setType NoType
}
protected def typerAddSyntheticMethods(templ: Template, clazz: Symbol, context: Context): Template = {
addSyntheticMethods(templ, clazz, context)
}
def addGetterSetter(stat: Tree): List[Tree] = stat match {
case ValDef(mods, name, tpt, rhs)
if !mods.isPrivateLocal && !stat.symbol.isModuleVar =>
val isDeferred = mods.isDeferred
val value = stat.symbol
val allAnnots = value.annotations
if (!isDeferred)
value.setAnnotations(memberAnnots(allAnnots, FieldTargetClass, keepClean = !mods.isParamAccessor))
val getter = if (isDeferred) value else value.getter(value.owner)
assert(getter != NoSymbol, stat)
if (getter.isOverloaded)
error(getter.pos, getter+" is defined twice")
getter.setAnnotations(memberAnnots(allAnnots, GetterTargetClass))
if (value.isLazy) List(stat)
else {
val vdef = treeCopy.ValDef(stat, mods | PRIVATE | LOCAL, nme.getterToLocal(name), tpt, rhs)
val getterDef: DefDef = atPos(vdef.pos.focus) {
if (isDeferred) {
val r = DefDef(getter, EmptyTree)
r.tpt.asInstanceOf[TypeTree].setOriginal(tpt)
r
} else {
val rhs = gen.mkCheckInit(Select(This(value.owner), value))
val r = typed {
atPos(getter.pos.focus) {
DefDef(getter, rhs)
}
}.asInstanceOf[DefDef]
r.tpt.setPos(tpt.pos.focus)
r
}
}
checkNoEscaping.privates(getter, getterDef.tpt)
def setterDef(setter: Symbol, isBean: Boolean = false): DefDef = {
setter setAnnotations memberAnnots(allAnnots, if (isBean) BeanSetterTargetClass else SetterTargetClass)
val defTree =
if ((mods hasFlag DEFERRED) || (setter hasFlag OVERLOADED)) EmptyTree
else Assign(Select(This(value.owner), value), Ident(setter.paramss.head.head))
typedPos(vdef.pos.focus)(DefDef(setter, defTree)).asInstanceOf[DefDef]
}
val gs = new ListBuffer[DefDef]
gs.append(getterDef)
if (mods.isMutable) {
val setter = getter.setter(value.owner)
gs.append(setterDef(setter))
}
if (!forMSIL && (value.hasAnnotation(BeanPropertyAttr) ||
value.hasAnnotation(BooleanBeanPropertyAttr))) {
val nameSuffix = name.toString().capitalize
val beanGetterName =
(if (value.hasAnnotation(BooleanBeanPropertyAttr)) "is" else "get") +
nameSuffix
val beanGetter = value.owner.info.decl(beanGetterName)
if (beanGetter == NoSymbol) {
unit.error(stat.pos, "implementation limitation: the BeanProperty annotation cannot be used in a type alias or renamed import")
}
beanGetter.setAnnotations(memberAnnots(allAnnots, BeanGetterTargetClass))
if (mods.isMutable && beanGetter != NoSymbol) {
val beanSetterName = "set" + nameSuffix
val beanSetter = value.owner.info.decl(beanSetterName)
gs.append(setterDef(beanSetter, isBean = true))
}
}
if (mods.isDeferred) gs.toList else vdef :: gs.toList
}
case dd @ DocDef(comment, defn) =>
addGetterSetter(defn) map (stat => DocDef(comment, stat) setPos dd.pos)
case Annotated(annot, defn) =>
addGetterSetter(defn) map (stat => Annotated(annot, stat))
case _ =>
List(stat)
}
protected def memberAnnots(annots: List[AnnotationInfo], memberClass: Symbol, keepClean: Boolean = false) = {
def hasMatching(metaAnnots: List[AnnotationInfo], orElse: => Boolean) = {
metaAnnots.exists(_.atp.typeSymbol == memberClass) ||
(metaAnnots.forall(ann => {
val annClass = ann.atp.typeSymbol
annClass != FieldTargetClass && annClass != GetterTargetClass &&
annClass != SetterTargetClass && annClass != BeanGetterTargetClass &&
annClass != BeanSetterTargetClass && annClass != ParamTargetClass
}) && orElse)
}
def noMetaAnnot(ann: AnnotationInfo) = {
hasMatching(ann.atp.typeSymbol.annotations, keepClean)
}
annots.filter(ann => ann.atp match {
case AnnotatedType(metaAnnots, _, _) =>
hasMatching(metaAnnots, noMetaAnnot(ann))
case _ => noMetaAnnot(ann)
})
}
protected def enterSyms(txt: Context, trees: List[Tree]) = {
var txt0 = txt
for (tree <- trees) txt0 = enterSym(txt0, tree)
}
protected def enterSym(txt: Context, tree: Tree): Context =
if (txt eq context) namer.enterSym(tree)
else newNamer(txt).enterSym(tree)
def typedTemplate(templ: Template, parents1: List[Tree]): Template = {
val clazz = context.owner
val annots = clazz.annotations
if (templ.symbol == NoSymbol)
templ setSymbol clazz.newLocalDummy(templ.pos)
val self1 = templ.self match {
case vd @ ValDef(mods, name, tpt, EmptyTree) =>
val tpt1 =
checkNoEscaping.privates(
clazz.thisSym,
treeCopy.TypeTree(tpt).setOriginal(tpt) setType vd.symbol.tpe)
treeCopy.ValDef(vd, mods, name, tpt1, EmptyTree) setType NoType
}
if (self1.name != nme.WILDCARD) context.scope enter self1.symbol
val selfType =
if (clazz.isAnonymousClass && !phase.erasedTypes)
intersectionType(clazz.info.parents, clazz.owner)
else clazz.typeOfThis
assert(clazz.info.decls != EmptyScope)
enterSyms(context.outer.make(templ, clazz, clazz.info.decls), templ.body)
validateParentClasses(parents1, selfType)
if (clazz.isCase)
validateNoCaseAncestor(clazz)
if ((clazz isSubClass ClassfileAnnotationClass) && !clazz.owner.isPackageClass)
unit.error(clazz.pos, "inner classes cannot be classfile annotations")
if (!phase.erasedTypes && !clazz.info.resultType.isError)
checkFinitary(clazz.info.resultType.asInstanceOf[ClassInfoType])
val body =
if (phase.id <= currentRun.typerPhase.id && !reporter.hasErrors)
templ.body flatMap addGetterSetter
else templ.body
val body1 = typedStats(body, templ.symbol)
treeCopy.Template(templ, parents1, self1, body1) setType clazz.tpe
}
def removeAnnotations(mods: Modifiers): Modifiers =
mods.copy(annotations = Nil)
def typedValDef(vdef: ValDef): ValDef = {
val sym = vdef.symbol
val typer1 = constrTyperIf(sym.isParameter && sym.owner.isConstructor)
val typedMods = removeAnnotations(vdef.mods)
val annots = sym.annotations
var tpt1 = checkNoEscaping.privates(sym, typer1.typedType(vdef.tpt))
checkNonCyclic(vdef, tpt1)
if (sym.hasAnnotation(definitions.VolatileAttr)) {
if (!sym.isMutable)
error(vdef.pos, "values cannot be volatile")
else if (sym.isFinal)
error(vdef.pos, "final vars cannot be volatile")
}
val rhs1 =
if (vdef.rhs.isEmpty) {
if (sym.isVariable && sym.owner.isTerm && phase.id <= currentRun.typerPhase.id)
error(vdef.pos, "local variables must be initialized")
vdef.rhs
} else {
val tpt2 = if (sym.hasDefault) {
val tparams =
if (sym.owner.isConstructor) sym.owner.owner.info.typeParams
else sym.owner.tpe.typeParams
val subst = new SubstTypeMap(tparams, tparams map (_ => WildcardType)) {
override def matches(sym: Symbol, sym1: Symbol) =
if (sym.isSkolem) matches(sym.deSkolemize, sym1)
else if (sym1.isSkolem) matches(sym, sym1.deSkolemize)
else super[SubstTypeMap].matches(sym, sym1)
}
if (sym hasFlag BYNAMEPARAM)
if (tpt1.tpe.typeArgs.isEmpty) WildcardType
else subst(tpt1.tpe.typeArgs(0))
else subst(tpt1.tpe)
} else tpt1.tpe
newTyper(typer1.context.make(vdef, sym)).transformedOrTyped(vdef.rhs, EXPRmode | BYVALmode, tpt2)
}
treeCopy.ValDef(vdef, typedMods, vdef.name, tpt1, checkDead(rhs1)) setType NoType
}
def computeParamAliases(clazz: Symbol, vparamss: List[List[ValDef]], rhs: Tree) {
if (settings.debug.value) log("computing param aliases for "+clazz+":"+clazz.primaryConstructor.tpe+":"+rhs)
def decompose(call: Tree): (Tree, List[Tree]) = call match {
case Apply(fn, args) =>
val (superConstr, args1) = decompose(fn)
val params = fn.tpe.params
val args2 = if (params.isEmpty || !isRepeatedParamType(params.last.tpe)) args
else args.take(params.length - 1) :+ EmptyTree
assert(sameLength(args2, params), "mismatch " + clazz + " " + (params map (_.tpe)) + " " + args2)
(superConstr, args1 ::: args2)
case Block(stats, expr) if !stats.isEmpty =>
decompose(stats.last)
case _ =>
(call, List())
}
val (superConstr, superArgs) = decompose(rhs)
assert(superConstr.symbol ne null)
for (arg <- superArgs ; tree <- arg) {
val sym = tree.symbol
if (sym != null && (sym.info.baseClasses contains clazz)) {
if (sym.isModule)
error(tree.pos, "super constructor cannot be passed a self reference unless parameter is declared by-name")
tree match {
case This(qual) =>
error(tree.pos, "super constructor arguments cannot reference unconstructed `this`")
case _ => ()
}
}
}
if (superConstr.symbol.isPrimaryConstructor) {
val superClazz = superConstr.symbol.owner
if (!superClazz.isJavaDefined) {
val superParamAccessors = superClazz.constrParamAccessors
if (sameLength(superParamAccessors, superArgs)) {
(superParamAccessors, superArgs).zipped map { (superAcc, superArg) =>
superArg match {
case Ident(name) =>
if (vparamss.exists(_.exists(_.symbol == superArg.symbol))) {
var alias = superAcc.initialize.alias
if (alias == NoSymbol)
alias = superAcc.getter(superAcc.owner)
if (alias != NoSymbol &&
superClazz.info.nonPrivateMember(alias.name) != alias)
alias = NoSymbol
if (alias != NoSymbol) {
var ownAcc = clazz.info.decl(name).suchThat(_.isParamAccessor)
if ((ownAcc hasFlag ACCESSOR) && !ownAcc.isDeferred)
ownAcc = ownAcc.accessed
if (!ownAcc.isVariable && !alias.accessed.isVariable) {
if (settings.debug.value)
log("" + ownAcc + " has alias "+alias + alias.locationString)
ownAcc.asInstanceOf[TermSymbol].setAlias(alias)
}
}
}
case _ =>
}
()
}
}
}
}
}
def checkMethodStructuralCompatible(meth: Symbol): Unit =
if (meth.owner.isStructuralRefinement && meth.allOverriddenSymbols.isEmpty && !(meth.isPrivate || meth.hasAccessBoundary)) {
val tp: Type = meth.tpe match {
case mt: MethodType => mt
case NullaryMethodType(res) => res
case pt: PolyType => pt.resultType
case _ => NoType
}
for (paramType <- tp.paramTypes) {
if (paramType.typeSymbol.isAbstractType && !(paramType.typeSymbol.hasTransOwner(meth.owner)))
unit.error(meth.pos,"Parameter type in structural refinement may not refer to an abstract type defined outside that refinement")
else if (paramType.typeSymbol.isAbstractType && !(paramType.typeSymbol.hasTransOwner(meth)))
unit.error(meth.pos,"Parameter type in structural refinement may not refer to a type member of that refinement")
else if (paramType.isInstanceOf[ThisType] && paramType.typeSymbol == meth.owner)
unit.error(meth.pos,"Parameter type in structural refinement may not refer to the type of that refinement (self type)")
}
}
def typedUseCase(useCase: UseCase) {
def stringParser(str: String): syntaxAnalyzer.Parser = {
val file = new BatchSourceFile(context.unit.source.file, str) {
override def positionInUltimateSource(pos: Position) = {
pos.withSource(context.unit.source, useCase.pos.start)
}
}
val unit = new CompilationUnit(file)
new syntaxAnalyzer.UnitParser(unit)
}
val trees = stringParser(useCase.body+";").nonLocalDefOrDcl
val enclClass = context.enclClass.owner
def defineAlias(name: Name) =
if (context.scope.lookup(name) == NoSymbol) {
lookupVariable(name.toString.substring(1), enclClass) match {
case Some(repl) =>
silent(_.typedTypeConstructor(stringParser(repl).typ())) match {
case tpt: Tree =>
val alias = enclClass.newAliasType(useCase.pos, name.toTypeName)
val tparams = cloneSymbols(tpt.tpe.typeSymbol.typeParams, alias)
alias setInfo typeFun(tparams, appliedType(tpt.tpe, tparams map (_.tpe)))
context.scope.enter(alias)
case _ =>
}
case _ =>
}
}
for (tree <- trees; t <- tree)
t match {
case Ident(name) if name startsWith '$' => defineAlias(name)
case _ =>
}
useCase.aliases = context.scope.toList
namer.enterSyms(trees)
typedStats(trees, NoSymbol)
useCase.defined = context.scope.toList filterNot (useCase.aliases contains _)
if (settings.debug.value)
useCase.defined foreach (sym => println("defined use cases: %s:%s".format(sym, sym.tpe)))
}
def typedDefDef(ddef: DefDef): DefDef = {
val meth = ddef.symbol
reenterTypeParams(ddef.tparams)
reenterValueParams(ddef.vparamss)
if (phase.id <= currentRun.typerPhase.id && meth.isPrimaryConstructor) {
for (vparams <- ddef.vparamss; vd <- vparams) {
if (vd.mods.isParamAccessor) {
val sym = vd.symbol
sym.setAnnotations(memberAnnots(sym.annotations, ParamTargetClass, keepClean = true))
}
}
}
val tparams1 = ddef.tparams mapConserve typedTypeDef
val vparamss1 = ddef.vparamss mapConserve (_ mapConserve typedValDef)
val annots = meth.annotations
for (vparams1 <- vparamss1; vparam1 <- vparams1 dropRight 1)
if (isRepeatedParamType(vparam1.symbol.tpe))
error(vparam1.pos, "*-parameter must come last")
var tpt1 = checkNoEscaping.privates(meth, typedType(ddef.tpt))
if (!settings.YdepMethTpes.value) {
for (vparams <- vparamss1; vparam <- vparams) {
checkNoEscaping.locals(context.scope, WildcardType, vparam.tpt); ()
}
checkNoEscaping.locals(context.scope, WildcardType, tpt1)
}
checkNonCyclic(ddef, tpt1)
ddef.tpt.setType(tpt1.tpe)
val typedMods = removeAnnotations(ddef.mods)
var rhs1 =
if (ddef.name == nme.CONSTRUCTOR && !ddef.symbol.hasStaticFlag) {
if (!meth.isPrimaryConstructor &&
(!meth.owner.isClass ||
meth.owner.isModuleClass ||
meth.owner.isAnonOrRefinementClass))
error(ddef.pos, "constructor definition not allowed here")
typed(ddef.rhs)
} else {
transformedOrTyped(ddef.rhs, EXPRmode, tpt1.tpe)
}
if (meth.isPrimaryConstructor && meth.isClassConstructor &&
phase.id <= currentRun.typerPhase.id && !reporter.hasErrors)
computeParamAliases(meth.owner, vparamss1, rhs1)
if (tpt1.tpe.typeSymbol != NothingClass && !context.returnsSeen && rhs1.tpe.typeSymbol != NothingClass)
rhs1 = checkDead(rhs1)
if (phase.id <= currentRun.typerPhase.id && meth.owner.isClass &&
meth.paramss.exists(ps => ps.exists(_.hasDefaultFlag) && isRepeatedParamType(ps.last.tpe)))
error(meth.pos, "a parameter section with a `*'-parameter is not allowed to have default arguments")
if (phase.id <= currentRun.typerPhase.id) {
val allParams = meth.paramss.flatten
for (p <- allParams) {
deprecatedName(p).foreach(n => {
if (allParams.exists(p1 => p1.name == n || (p != p1 && deprecatedName(p1) == Some(n))))
error(p.pos, "deprecated parameter name "+ n +" has to be distinct from any other parameter name (deprecated or not).")
})
}
}
checkMethodStructuralCompatible(meth)
treeCopy.DefDef(ddef, typedMods, ddef.name, tparams1, vparamss1, tpt1, rhs1) setType NoType
}
def typedTypeDef(tdef: TypeDef): TypeDef = {
def typeDefTyper = {
if(tdef.tparams isEmpty) Typer.this
else newTyper(context.makeNewScope(tdef, tdef.symbol))
}
typeDefTyper.typedTypeDef0(tdef)
}
private def typedTypeDef0(tdef: TypeDef): TypeDef = {
tdef.symbol.initialize
reenterTypeParams(tdef.tparams)
val tparams1 = tdef.tparams mapConserve {typedTypeDef(_)}
val typedMods = removeAnnotations(tdef.mods)
val annots = tdef.symbol.annotations
if (settings.nospecialization.value && currentRun.compiles(tdef.symbol)) {
tdef.symbol.removeAnnotation(definitions.SpecializedClass)
tdef.symbol.deSkolemize.removeAnnotation(definitions.SpecializedClass)
}
val rhs1 = checkNoEscaping.privates(tdef.symbol, typedType(tdef.rhs))
checkNonCyclic(tdef.symbol)
if (tdef.symbol.owner.isType)
rhs1.tpe match {
case TypeBounds(lo1, hi1) =>
if (!(lo1 <:< hi1))
error(tdef.pos, "lower bound "+lo1+" does not conform to upper bound "+hi1)
case _ =>
}
treeCopy.TypeDef(tdef, typedMods, tdef.name, tparams1, rhs1) setType NoType
}
private def enterLabelDef(stat: Tree) {
stat match {
case ldef @ LabelDef(_, _, _) =>
if (ldef.symbol == NoSymbol)
ldef.symbol = namer.enterInScope(
context.owner.newLabel(ldef.pos, ldef.name) setInfo MethodType(List(), UnitClass.tpe))
case _ =>
}
}
def typedLabelDef(ldef: LabelDef): LabelDef = {
if (!nme.isLoopHeaderLabel(ldef.symbol.name) || phase.id > currentRun.typerPhase.id) {
val restpe = ldef.symbol.tpe.resultType
val rhs1 = typed(ldef.rhs, restpe)
ldef.params foreach (param => param.tpe = param.symbol.tpe)
treeCopy.LabelDef(ldef, ldef.name, ldef.params, rhs1) setType restpe
} else {
val initpe = ldef.symbol.tpe.resultType
val rhs1 = typed(ldef.rhs)
val restpe = rhs1.tpe
if (restpe == initpe) {
ldef.params foreach (param => param.tpe = param.symbol.tpe)
treeCopy.LabelDef(ldef, ldef.name, ldef.params, rhs1) setType restpe
} else {
context.scope.unlink(ldef.symbol)
val sym2 = namer.enterInScope(
context.owner.newLabel(ldef.pos, ldef.name) setInfo MethodType(List(), restpe))
val rhs2 = typed(resetAllAttrs(ldef.rhs), restpe)
ldef.params foreach (param => param.tpe = param.symbol.tpe)
treeCopy.LabelDef(ldef, ldef.name, ldef.params, rhs2) setSymbol sym2 setType restpe
}
}
}
def typedBlock(block: Block, mode: Int, pt: Type): Block = {
val syntheticPrivates = new ListBuffer[Symbol]
try {
namer.enterSyms(block.stats)
for (stat <- block.stats) enterLabelDef(stat)
if (phaseId(currentPeriod) <= currentRun.typerPhase.id) {
block match {
case block @ Block(List(classDef @ ClassDef(_, _, _, _)), newInst @ Apply(Select(New(_), _), _)) =>
val visibleMembers = pt match {
case WildcardType => classDef.symbol.info.decls.toList
case BoundedWildcardType(TypeBounds(lo, hi)) => lo.members
case _ => pt.members
}
for (member <- classDef.symbol.info.decls.toList
if member.isTerm && !member.isConstructor &&
member.allOverriddenSymbols.isEmpty &&
(!member.isPrivate && !member.hasAccessBoundary) &&
!(visibleMembers exists { visible =>
visible.name == member.name &&
member.tpe <:< visible.tpe.substThis(visible.owner, ThisType(classDef.symbol))
})
) {
member.resetFlag(PROTECTED)
member.resetFlag(LOCAL)
member.setFlag(PRIVATE | SYNTHETIC_PRIVATE)
syntheticPrivates += member
member.privateWithin = NoSymbol
}
case _ =>
}
}
val stats1 = typedStats(block.stats, context.owner)
val expr1 = typed(block.expr, mode & ~(FUNmode | QUALmode), pt)
treeCopy.Block(block, stats1, expr1)
.setType(if (treeInfo.isPureExpr(block)) expr1.tpe else expr1.tpe.deconst)
} finally {
for (sym <- syntheticPrivates) sym resetFlag SYNTHETIC_PRIVATE
}
}
def typedCase(cdef: CaseDef, pattpe: Type, pt: Type): CaseDef = {
for (Apply(_, xs) <- cdef.pat ; x <- xs dropRight 1 ; if treeInfo isStar x)
error(x.pos, "_* may only come last")
val pat1: Tree = typedPattern(cdef.pat, pattpe)
if (pat1.tpe.paramSectionCount > 0)
pat1 setType pat1.tpe.finalResultType
if (forInteractive) {
for (bind @ Bind(name, _) <- cdef.pat)
if (name.toTermName != nme.WILDCARD && bind.symbol != null && bind.symbol != NoSymbol)
namer.enterIfNotThere(bind.symbol)
}
val guard1: Tree = if (cdef.guard == EmptyTree) EmptyTree
else typed(cdef.guard, BooleanClass.tpe)
var body1: Tree = typed(cdef.body, pt)
if (!context.savedTypeBounds.isEmpty) {
body1.tpe = context.restoreTypeBounds(body1.tpe)
if (isFullyDefined(pt) && !(body1.tpe <:< pt)) {
body1 =
typed {
atPos(body1.pos) {
TypeApply(Select(body1, Any_asInstanceOf), List(TypeTree(pt)))
}
}
}
}
treeCopy.CaseDef(cdef, pat1, guard1, body1) setType body1.tpe
}
def typedCases(tree: Tree, cases: List[CaseDef], pattp: Type, pt: Type): List[CaseDef] =
cases mapConserve { cdef =>
newTyper(context.makeNewScope(cdef, context.owner)).typedCase(cdef, pattp, pt)
}
def typedFunction(fun: Function, mode: Int, pt: Type): Tree = {
val numVparams = fun.vparams.length
val codeExpected = !forMSIL && (pt.typeSymbol isNonBottomSubClass CodeClass)
if (numVparams > definitions.MaxFunctionArity)
return errorTree(fun, "implementation restricts functions to " + definitions.MaxFunctionArity + " parameters")
def decompose(pt: Type): (Symbol, List[Type], Type) =
if ((isFunctionType(pt)
||
pt.typeSymbol == PartialFunctionClass &&
numVparams == 1 && fun.body.isInstanceOf[Match])
&&
(pt.normalize.typeArgs.length - 1 == numVparams
||
fun.vparams.exists(_.tpt.isEmpty)))
(pt.typeSymbol, pt.normalize.typeArgs.init, pt.normalize.typeArgs.last)
else
(FunctionClass(numVparams), fun.vparams map (x => NoType), WildcardType)
val (clazz, argpts, respt) = decompose(if (codeExpected) pt.normalize.typeArgs.head else pt)
if (argpts.lengthCompare(numVparams) != 0)
errorTree(fun, "wrong number of parameters; expected = " + argpts.length)
else {
val vparamSyms = (fun.vparams, argpts).zipped map { (vparam, argpt) =>
if (vparam.tpt.isEmpty) {
vparam.tpt.tpe =
if (isFullyDefined(argpt)) argpt
else {
fun match {
case etaExpansion(vparams, fn, args) if !codeExpected =>
silent(_.typed(fn, forFunMode(mode), pt)) match {
case fn1: Tree if context.undetparams.isEmpty =>
val ftpe = normalize(fn1.tpe) baseType FunctionClass(numVparams)
if (isFunctionType(ftpe) && isFullyDefined(ftpe))
return typedFunction(fun, mode, ftpe)
case _ =>
}
case _ =>
}
error(vparam.pos, missingParameterTypeMsg(fun, vparam, pt))
ErrorType
}
if (!vparam.tpt.pos.isDefined) vparam.tpt setPos vparam.pos.focus
}
enterSym(context, vparam)
if (context.retyping) context.scope enter vparam.symbol
vparam.symbol
}
val vparams = fun.vparams mapConserve (typedValDef)
var body = typed(fun.body, respt)
val formals = vparamSyms map (_.tpe)
val restpe = packedType(body, fun.symbol).deconst
val funtpe = typeRef(clazz.tpe.prefix, clazz, formals :+ restpe)
val fun1 = treeCopy.Function(fun, vparams, body).setType(funtpe)
if (codeExpected) {
val liftPoint = Apply(Select(Ident(CodeModule), nme.lift_), List(fun1))
typed(atPos(fun.pos)(liftPoint))
} else fun1
}
}
def typedRefinement(stats: List[Tree]) {
namer.enterSyms(stats)
unit.toCheck += { () =>
var c = context
while (!c.reportGeneralErrors) c = c.outer
val stats1 = newTyper(c).typedStats(stats, NoSymbol)
for (stat <- stats1 if stat.isDef) {
val member = stat.symbol
if (!(context.owner.ancestors forall
(bc => member.matchingSymbol(bc, context.owner.thisType) == NoSymbol))) {
member setFlag OVERRIDE
}
}
}
}
def typedImport(imp : Import) : Import = (transformed remove imp) match {
case Some(imp1: Import) => imp1
case None => log("unhandled import: "+imp+" in "+unit); imp
}
def typedStats(stats: List[Tree], exprOwner: Symbol): List[Tree] = {
val inBlock = exprOwner == context.owner
def includesTargetPos(tree: Tree) =
tree.pos.isRange && context.unit != null && (tree.pos includes context.unit.targetPos)
val localTarget = stats exists includesTargetPos
def typedStat(stat: Tree): Tree = {
if (context.owner.isRefinementClass && !treeInfo.isDeclaration(stat))
errorTree(stat, "only declarations allowed here")
else
stat match {
case imp @ Import(_, _) =>
context = context.makeNewImport(imp)
imp.symbol.initialize
typedImport(imp)
case _ =>
if (localTarget && !includesTargetPos(stat)) {
stat
} else {
val localTyper = if (inBlock || (stat.isDef && !stat.isInstanceOf[LabelDef])) this
else newTyper(context.make(stat, exprOwner))
val result = checkDead(localTyper.typed(stat, EXPRmode | BYVALmode, WildcardType))
if (treeInfo.isSelfOrSuperConstrCall(result)) {
context.inConstructorSuffix = true
if (treeInfo.isSelfConstrCall(result) && result.symbol.pos.pointOrElse(0) >= exprOwner.enclMethod.pos.pointOrElse(0))
error(stat.pos, "called constructor's definition must precede calling constructor's definition")
}
result
}
}
}
def accesses(looker: Symbol, accessed: Symbol) = accessed.hasLocalFlag && (
accessed.isParamAccessor || (looker.hasAccessorFlag && !accessed.hasAccessorFlag && accessed.isPrivate)
)
def checkNoDoubleDefsAndAddSynthetics(stats: List[Tree]): List[Tree] = {
val scope = if (inBlock) context.scope else context.owner.info.decls
var newStats = new ListBuffer[Tree]
var needsCheck = true
var moreToAdd = true
while (moreToAdd) {
val initSize = scope.size
var e = scope.elems
while ((e ne null) && e.owner == scope) {
if (needsCheck) {
var e1 = scope.lookupNextEntry(e)
while ((e1 ne null) && e1.owner == scope) {
if (!accesses(e.sym, e1.sym) && !accesses(e1.sym, e.sym) &&
(e.sym.isType || inBlock || (e.sym.tpe matches e1.sym.tpe)))
if (!e.sym.isErroneous && !e1.sym.isErroneous && !e.sym.hasDefaultFlag &&
!e.sym.hasAnnotation(BridgeClass) && !e1.sym.hasAnnotation(BridgeClass)) {
error(e.sym.pos, e1.sym+" is defined twice"+
{if(!settings.debug.value) "" else " in "+unit.toString})
scope.unlink(e1)
}
e1 = scope.lookupNextEntry(e1)
}
}
context.unit.synthetics get e.sym foreach { tree =>
newStats += typedStat(tree)
context.unit.synthetics -= e.sym
}
e = e.next
}
needsCheck = false
moreToAdd = initSize != scope.size
}
if (newStats.isEmpty) stats
else {
def matches(stat: Tree, synt: Tree) = (stat, synt) match {
case (DefDef(_, statName, _, _, _, _), DefDef(mods, syntName, _, _, _, _)) =>
mods.hasDefaultFlag && syntName.toString.startsWith(statName.toString)
case (ClassDef(_, className, _, _), ModuleDef(_, moduleName, _)) =>
className.toTermName == moduleName
case _ => false
}
def matching(stat: Tree): List[Tree] = {
val (pos, neg) = newStats.partition(synt => matches(stat, synt))
newStats = neg
pos.toList
}
(stats foldRight List[Tree]())((stat, res) => {
stat :: matching(stat) ::: res
}) ::: newStats.toList
}
}
val result = stats mapConserve (typedStat)
if (phase.erasedTypes) result
else checkNoDoubleDefsAndAddSynthetics(result)
}
def typedArg(arg: Tree, mode: Int, newmode: Int, pt: Type): Tree = {
val typedMode = onlyStickyModes(mode) | newmode
val t = constrTyperIf((mode & SCCmode) != 0).typed(arg, typedMode, pt)
checkDead.inMode(typedMode, t)
}
def typedArgs(args: List[Tree], mode: Int) =
args mapConserve (arg => typedArg(arg, mode, 0, WildcardType))
def typedArgs(args: List[Tree], mode: Int, originalFormals: List[Type], adaptedFormals: List[Type]) = {
var newmodes = originalFormals map (tp => if (isByNameParamType(tp)) 0 else BYVALmode)
if (isVarArgTypes(originalFormals))
newmodes = newmodes.init ++ List.fill(args.length - originalFormals.length + 1)(STARmode | BYVALmode)
(args, adaptedFormals, newmodes).zipped map { (arg, formal, m) =>
typedArg(arg, mode, m, formal)
}
}
def needsInstantiation(tparams: List[Symbol], formals: List[Type], args: List[Tree]) = {
def isLowerBounded(tparam: Symbol) = {
val losym = tparam.info.bounds.lo.typeSymbol
losym != NothingClass && losym != NullClass
}
(formals, args).zipped exists {
case (formal, Function(vparams, _)) =>
(vparams exists (_.tpt.isEmpty)) &&
vparams.length <= MaxFunctionArity &&
(formal baseType FunctionClass(vparams.length) match {
case TypeRef(_, _, formalargs) =>
(formalargs, vparams).zipped.exists ((formalarg, vparam) =>
vparam.tpt.isEmpty && (tparams exists (formalarg contains))) &&
(tparams forall isLowerBounded)
case _ =>
false
})
case _ =>
false
}
}
def isNamedApplyBlock(tree: Tree) =
context.namedApplyBlockInfo exists (_._1 == tree)
def callToCompanionConstr(context: Context, calledFun: Symbol) = {
calledFun.isConstructor && {
val methCtx = context.enclMethod
(methCtx != NoContext) && {
val contextFun = methCtx.tree.symbol
contextFun.isPrimaryConstructor && contextFun.owner.isModuleClass &&
companionModuleOf(calledFun.owner, context).moduleClass == contextFun.owner
}
}
}
def doTypedApply(tree: Tree, fun0: Tree, args: List[Tree], mode: Int, pt: Type): Tree = {
var fun = fun0
if (fun.hasSymbol && fun.symbol.isOverloaded) {
def shapeType(arg: Tree): Type = arg match {
case Function(vparams, body) =>
functionType(vparams map (vparam => AnyClass.tpe), shapeType(body))
case AssignOrNamedArg(Ident(name), rhs) =>
NamedType(name, shapeType(rhs))
case _ =>
NothingClass.tpe
}
val argtypes = args map shapeType
val pre = fun.symbol.tpe.prefix
var sym = fun.symbol filter { alt =>
isApplicableSafe(context.undetparams, followApply(pre.memberType(alt)), argtypes, pt)
}
if (sym.isOverloaded) {
val sym1 = sym filter (alt => {
hasExactlyNumParams(followApply(alt.tpe), argtypes.length) ||
alt.tpe.paramss.exists(_.exists(_.hasDefault))
})
if (sym1 != NoSymbol) sym = sym1
}
if (sym != NoSymbol)
fun = adapt(fun setSymbol sym setType pre.memberType(sym), forFunMode(mode), WildcardType)
}
fun.tpe match {
case OverloadedType(pre, alts) =>
val undetparams = context.extractUndetparams()
val argtpes = new ListBuffer[Type]
val amode = forArgMode(fun, mode)
val args1 = args map {
case arg @ AssignOrNamedArg(Ident(name), rhs) =>
val rhs1 = typedArg(rhs, amode, BYVALmode, WildcardType)
argtpes += NamedType(name, rhs1.tpe.deconst)
atPos(arg.pos) { new AssignOrNamedArg(arg.lhs , rhs1) }
case arg =>
val arg1 = typedArg(arg, amode, BYVALmode, WildcardType)
argtpes += arg1.tpe.deconst
arg1
}
context.undetparams = undetparams
inferMethodAlternative(fun, undetparams, argtpes.toList, pt, varArgsOnly = treeInfo.isWildcardStarArgList(args))
doTypedApply(tree, adapt(fun, forFunMode(mode), WildcardType), args1, mode, pt)
case mt @ MethodType(params, _) =>
val paramTypes = mt.paramTypes
val formals = formalTypes(paramTypes, args.length)
def tryTupleApply: Option[Tree] = {
val tupleArgs = actualArgs(tree.pos.makeTransparent, args, formals.length)
if (!sameLength(tupleArgs, args) && !isUnitForVarArgs(args, params)) {
val savedUndetparams = context.undetparams
silent(_.doTypedApply(tree, fun, tupleArgs, mode, pt)) match {
case t: Tree =>
Some(t)
case ex =>
context.undetparams = savedUndetparams
None
}
} else None
}
def tryNamesDefaults: Tree = {
val lencmp = compareLengths(args, formals)
if (mt.isErroneous) setError(tree)
else if (inPatternMode(mode))
errorTree(tree, "wrong number of arguments for "+ treeSymTypeMsg(fun))
else if (lencmp > 0) {
tryTupleApply getOrElse errorTree(tree, "too many arguments for "+treeSymTypeMsg(fun))
} else if (lencmp == 0) {
val (namelessArgs, argPos) = removeNames(Typer.this)(args, params)
if (namelessArgs exists (_.isErroneous)) {
setError(tree)
} else if (!isIdentity(argPos) && !sameLength(formals, params))
errorTree(tree, "when using named arguments, the vararg parameter "+
"has to be specified exactly once")
else if (isIdentity(argPos) && !isNamedApplyBlock(fun)) {
doTypedApply(tree, fun, namelessArgs, mode, pt)
} else {
transformNamedApplication(Typer.this, mode, pt)(
treeCopy.Apply(tree, fun, namelessArgs), argPos)
}
} else {
val fun1 = transformNamedApplication(Typer.this, mode, pt)(fun, x => x)
if (fun1.isErroneous) setError(tree)
else {
assert(isNamedApplyBlock(fun1), fun1)
val NamedApplyInfo(qual, targs, previousArgss, _) = context.namedApplyBlockInfo.get._2
val blockIsEmpty = fun1 match {
case Block(Nil, _) =>
context.namedApplyBlockInfo = None
true
case _ => false
}
val (allArgs, missing) = addDefaults(args, qual, targs, previousArgss, params, fun.pos.focus, context)
val funSym = fun1 match { case Block(_, expr) => expr.symbol }
val lencmp2 = compareLengths(allArgs, formals)
if (!sameLength(allArgs, args) && callToCompanionConstr(context, funSym)) {
errorTree(tree, "module extending its companion class cannot use default constructor arguments")
} else if (lencmp2 > 0) {
removeNames(Typer.this)(allArgs, params)
setError(tree)
} else if (lencmp2 == 0) {
val note = "Error occurred in an application involving default arguments."
if (!(context.diagnostic contains note)) context.diagnostic = note :: context.diagnostic
doTypedApply(tree, if (blockIsEmpty) fun else fun1, allArgs, mode, pt)
} else {
tryTupleApply getOrElse errorTree(tree, notEnoughArgumentsMsg(fun, missing))
}
}
}
}
if (!sameLength(formals, args) ||
(args exists isNamed) ||
isNamedApplyBlock(fun)) {
tryNamesDefaults
} else {
val tparams = context.extractUndetparams()
if (tparams.isEmpty) {
checkDead.updateExpr(fun)
val args1 = typedArgs(args, forArgMode(fun, mode), paramTypes, formals)
val restpe = mt.resultType(args1 map (arg => gen.stableTypeFor(arg) getOrElse arg.tpe))
def ifPatternSkipFormals(tp: Type) = tp match {
case MethodType(_, rtp) if (inPatternMode(mode)) => rtp
case _ => tp
}
if (forMSIL) {
fun match {
case Select(qual, name) =>
if (isSubType(qual.tpe, DelegateClass.tpe)
&& (name == encode("+=") || name == encode("-=")))
{
val n = if (name == encode("+=")) nme.PLUS else nme.MINUS
val f = Select(qual, n)
val rhs = treeCopy.Apply(tree, f, args)
return typed(Assign(qual, rhs))
}
case _ => ()
}
}
if (fun.symbol == List_apply && args.isEmpty && !forInteractive)
atPos(tree.pos)(gen.mkNil setType restpe)
else
constfold(treeCopy.Apply(tree, fun, args1) setType ifPatternSkipFormals(restpe))
} else if (needsInstantiation(tparams, formals, args)) {
inferExprInstance(fun, tparams)
doTypedApply(tree, fun, args, mode, pt)
} else {
assert(!inPatternMode(mode))
val lenientTargs = protoTypeArgs(tparams, formals, mt.resultApprox, pt)
val strictTargs = (lenientTargs, tparams).zipped map ((targ, tparam) =>
if (targ == WildcardType) tparam.tpe else targ)
var remainingParams = paramTypes
def typedArgToPoly(arg: Tree, formal: Type): Tree = {
val lenientPt = formal.instantiateTypeParams(tparams, lenientTargs)
val newmode =
if (isByNameParamType(remainingParams.head)) POLYmode
else POLYmode | BYVALmode
if (remainingParams.tail.nonEmpty) remainingParams = remainingParams.tail
val arg1 = typedArg(arg, forArgMode(fun, mode), newmode, lenientPt)
val argtparams = context.extractUndetparams()
if (!argtparams.isEmpty) {
val strictPt = formal.instantiateTypeParams(tparams, strictTargs)
inferArgumentInstance(arg1, argtparams, strictPt, lenientPt)
}
arg1
}
val args1 = (args, formals).zipped map typedArgToPoly
if (args1 exists (_.tpe.isError)) setError(tree)
else {
if (settings.debug.value) log("infer method inst "+fun+", tparams = "+tparams+", args = "+args1.map(_.tpe)+", pt = "+pt+", lobounds = "+tparams.map(_.tpe.bounds.lo)+", parambounds = "+tparams.map(_.info))
val undetparams = inferMethodInstance(fun, tparams, args1, pt)
val result = doTypedApply(tree, fun, args1, mode, pt)
context.undetparams = undetparams
result
}
}
}
case SingleType(_, _) =>
doTypedApply(tree, fun setType fun.tpe.widen, args, mode, pt)
case ErrorType =>
setError(treeCopy.Apply(tree, fun, args))
case otpe if inPatternMode(mode) && unapplyMember(otpe).exists =>
if (args.length > MaxTupleArity)
error(fun.pos, "too many arguments for unapply pattern, maximum = "+MaxTupleArity)
def freshArgType(tp: Type): (Type, List[Symbol]) = (tp: @unchecked) match {
case MethodType(param :: _, _) =>
(param.tpe, Nil)
case PolyType(tparams, restype) =>
val tparams1 = cloneSymbols(tparams)
(freshArgType(restype)._1.substSym(tparams, tparams1), tparams1)
case OverloadedType(_, _) =>
error(fun.pos, "cannot resolve overloaded unapply")
(ErrorType, Nil)
}
val unapp = unapplyMember(otpe)
val unappType = otpe.memberType(unapp)
val argDummy = context.owner.newValue(fun.pos, nme.SELECTOR_DUMMY) setFlag SYNTHETIC setInfo pt
val arg = Ident(argDummy) setType pt
if (!isApplicableSafe(Nil, unappType, List(pt), WildcardType)) {
val (unappFormal, freeVars) = freshArgType(unappType.skolemizeExistential(context.owner, tree))
val context1 = context.makeNewScope(context.tree, context.owner)
freeVars foreach context1.scope.enter
val typer1 = newTyper(context1)
val pattp = typer1.infer.inferTypedPattern(tree.pos, unappFormal, arg.tpe)
val skolems = freeVars map { fv =>
val skolem = new TypeSkolem(context1.owner, fun.pos, fv.name.toTypeName, fv)
skolem.setInfo(fv.info.cloneInfo(skolem))
.setFlag(fv.flags | EXISTENTIAL).resetFlag(PARAM)
skolem
}
arg.tpe = pattp.substSym(freeVars, skolems)
argDummy setInfo arg.tpe
}
val fun1 = typedPos(fun.pos)(Apply(Select(fun setType null, unapp), List(arg)))
if (fun1.tpe.isErroneous) setError(tree)
else {
val formals0 = unapplyTypeList(fun1.symbol, fun1.tpe)
val formals1 = formalTypes(formals0, args.length)
if (sameLength(formals1, args)) {
val args1 = typedArgs(args, mode, formals0, formals1)
val pt1 = if (isFullyDefined(pt)) pt else makeFullyDefined(pt)
val itype = glb(List(pt1, arg.tpe))
arg.tpe = pt1
UnApply(fun1, args1) setPos tree.pos setType itype
}
else {
errorTree(tree, "wrong number of arguments for "+treeSymTypeMsg(fun))
}
}
case _ =>
errorTree(tree, fun.tpe+" does not take parameters")
}
}
def typedAnnotation(ann: Tree, mode: Int = EXPRmode, selfsym: Symbol = NoSymbol, annClass: Symbol = AnnotationClass, requireJava: Boolean = false): AnnotationInfo = {
lazy val annotationError = AnnotationInfo(ErrorType, Nil, Nil)
var hasError: Boolean = false
def error(pos: Position, msg: String) = {
context.error(pos, msg)
hasError = true
annotationError
}
def tryConst(tr: Tree, pt: Type): Option[LiteralAnnotArg] = {
val const: Constant = typed(constfold(tr), EXPRmode, pt) match {
case l @ Literal(c) if !l.isErroneous => c
case tree => tree.tpe match {
case ConstantType(c) => c
case tpe => null
}
}
def fail(msg: String) = { error(tr.pos, msg) ; None }
if (const == null)
fail("annotation argument needs to be a constant; found: " + tr)
else if (const.value == null)
fail("annotation argument cannot be null")
else
Some(LiteralAnnotArg(const))
}
def tree2ConstArg(tree: Tree, pt: Type): Option[ClassfileAnnotArg] = tree match {
case Apply(Select(New(tpt), nme.CONSTRUCTOR), args) if (pt.typeSymbol == ArrayClass) =>
error(tree.pos, "Array constants have to be specified using the `Array(...)' factory method")
None
case ann @ Apply(Select(New(tpt), nme.CONSTRUCTOR), args) =>
val annInfo = typedAnnotation(ann, mode, NoSymbol, pt.typeSymbol, true)
if (annInfo.atp.isErroneous) {
hasError = true
None
} else Some(NestedAnnotArg(annInfo))
case Apply(fun, args) =>
val typedFun = typed(fun, forFunMode(mode), WildcardType)
if (typedFun.symbol.owner == ArrayModule.moduleClass && typedFun.symbol.name == nme.apply)
pt match {
case TypeRef(_, ArrayClass, targ :: _) =>
trees2ConstArg(args, targ)
case _ =>
error(tree.pos, "found array constant, expected argument of type "+ pt)
None
}
else
tryConst(tree, pt)
case Typed(t, _) => tree2ConstArg(t, pt)
case tree =>
tryConst(tree, pt)
}
def trees2ConstArg(trees: List[Tree], pt: Type): Option[ArrayAnnotArg] = {
val args = trees.map(tree2ConstArg(_, pt))
if (args.exists(_.isEmpty)) None
else Some(ArrayAnnotArg(args.flatten.toArray))
}
val (fun, argss) = {
def extract(fun: Tree, outerArgss: List[List[Tree]]):
(Tree, List[List[Tree]]) = fun match {
case Apply(f, args) =>
extract(f, args :: outerArgss)
case Select(New(tpt), nme.CONSTRUCTOR) =>
(fun, outerArgss)
case _ =>
error(fun.pos, "unexpected tree in annotation: "+ fun)
(setError(fun), outerArgss)
}
extract(ann, List())
}
if (fun.isErroneous) annotationError
else {
val typedFun @ Select(New(tpt), _) = typed(fun, forFunMode(mode), WildcardType)
val annType = tpt.tpe
if (typedFun.isErroneous) annotationError
else if (annType.typeSymbol isNonBottomSubClass ClassfileAnnotationClass) {
val isJava = typedFun.symbol.owner.isJavaDefined
if (!annType.typeSymbol.isNonBottomSubClass(annClass)) {
error(tpt.pos, "expected annotation of type "+ annClass.tpe +", found "+ annType)
} else if (argss.length > 1) {
error(ann.pos, "multiple argument lists on classfile annotation")
} else {
val args =
if (argss.head.length == 1 && !isNamed(argss.head.head))
List(new AssignOrNamedArg(Ident(nme.value), argss.head.head))
else argss.head
val annScope = annType.decls
.filter(sym => sym.isMethod && !sym.isConstructor && sym.isJavaDefined)
val names = new collection.mutable.HashSet[Symbol]
names ++= (if (isJava) annScope.iterator
else typedFun.tpe.params.iterator)
val nvPairs = args map {
case arg @ AssignOrNamedArg(Ident(name), rhs) =>
val sym = if (isJava) annScope.lookup(name)
else typedFun.tpe.params.find(p => p.name == name).getOrElse(NoSymbol)
if (sym == NoSymbol) {
error(arg.pos, "unknown annotation argument name: " + name)
(nme.ERROR, None)
} else if (!names.contains(sym)) {
error(arg.pos, "duplicate value for annotation argument " + name)
(nme.ERROR, None)
} else {
names -= sym
if (isJava) sym.cookJavaRawInfo()
val annArg = tree2ConstArg(rhs, sym.tpe.resultType)
(sym.name, annArg)
}
case arg =>
error(arg.pos, "classfile annotation arguments have to be supplied as named arguments")
(nme.ERROR, None)
}
for (name <- names) {
if (!name.annotations.contains(AnnotationInfo(AnnotationDefaultAttr.tpe, List(), List())) &&
!name.hasDefaultFlag)
error(ann.pos, "annotation " + annType.typeSymbol.fullName + " is missing argument " + name.name)
}
if (hasError) annotationError
else AnnotationInfo(annType, List(), nvPairs map {p => (p._1, p._2.get)}).setPos(ann.pos)
}
} else if (requireJava) {
error(ann.pos, "nested classfile annotations must be defined in java; found: "+ annType)
} else {
val typedAnn = if (selfsym == NoSymbol) {
typed(ann, mode, annClass.tpe)
} else {
val funcparm = ValDef(NoMods, nme.self, TypeTree(selfsym.info), EmptyTree)
val func = Function(List(funcparm), ann.duplicate)
val fun1clazz = FunctionClass(1)
val funcType = typeRef(fun1clazz.tpe.prefix,
fun1clazz,
List(selfsym.info, annClass.tpe))
(typed(func, mode, funcType): @unchecked) match {
case t @ Function(List(arg), rhs) =>
val subs =
new TreeSymSubstituter(List(arg.symbol),List(selfsym))
subs(rhs)
}
}
def annInfo(t: Tree): AnnotationInfo = t match {
case Apply(Select(New(tpt), nme.CONSTRUCTOR), args) =>
AnnotationInfo(annType, args, List()).setPos(t.pos)
case Block(stats, expr) =>
context.warning(t.pos, "Usage of named or default arguments transformed this annotation\n"+
"constructor call into a block. The corresponding AnnotationInfo\n"+
"will contain references to local values and default getters instead\n"+
"of the actual argument trees")
annInfo(expr)
case Apply(fun, args) =>
context.warning(t.pos, "Implementation limitation: multiple argument lists on annotations are\n"+
"currently not supported; ignoring arguments "+ args)
annInfo(fun)
case _ =>
error(t.pos, "unexpected tree after typing annotation: "+ typedAnn)
}
if (annType.typeSymbol == DeprecatedAttr && argss.flatten.size < 2)
unit.deprecationWarning(ann.pos, "@deprecated now takes two arguments; see the scaladoc.")
if ((typedAnn.tpe == null) || typedAnn.tpe.isErroneous) annotationError
else annInfo(typedAnn)
}
}
}
def isRawParameter(sym: Symbol) =
sym.isTypeParameter && sym.owner.isJavaDefined
protected def existentialTransform(rawSyms: List[Symbol], tp: Type) = {
val typeParams: List[Symbol] = rawSyms map { sym =>
val name = sym.name match {
case x: TypeName => x
case x => newTypeName(x + ".type")
}
val bound = sym.existentialBound
val sowner = if (isRawParameter(sym)) context.owner else sym.owner
val quantified = sowner.newExistential(sym.pos, name)
quantified setInfo bound.cloneInfo(quantified)
}
val typeParamTypes = typeParams map (_.tpeHK)
(
typeParams map (tparam => tparam setInfo tparam.info.subst(rawSyms, typeParamTypes)),
tp.subst(rawSyms, typeParamTypes)
)
}
def packSymbols(hidden: List[Symbol], tp: Type): Type =
if (hidden.isEmpty) tp
else {
val (tparams, tp1) = existentialTransform(hidden, tp)
val res = existentialAbstraction(tparams, tp1)
res
}
def packedType(tree: Tree, owner: Symbol): Type = {
def defines(tree: Tree, sym: Symbol) =
sym.isExistentialSkolem && sym.unpackLocation == tree ||
tree.isDef && tree.symbol == sym
def isVisibleParameter(sym: Symbol) =
sym.isParameter && (sym.owner == owner) && (sym.isType || !owner.isAnonymousFunction)
def containsDef(owner: Symbol, sym: Symbol): Boolean =
(!sym.hasPackageFlag) && {
var o = sym.owner
while (o != owner && o != NoSymbol && !o.hasPackageFlag) o = o.owner
o == owner && !isVisibleParameter(sym)
}
var localSyms = collection.immutable.Set[Symbol]()
var boundSyms = collection.immutable.Set[Symbol]()
def isLocal(sym: Symbol): Boolean =
if (sym == NoSymbol || sym.isRefinementClass || sym.isLocalDummy) false
else if (owner == NoSymbol) tree exists (defines(_, sym))
else containsDef(owner, sym) || isRawParameter(sym)
def containsLocal(tp: Type): Boolean =
tp exists (t => isLocal(t.typeSymbol) || isLocal(t.termSymbol))
val normalizeLocals = new TypeMap {
def apply(tp: Type): Type = tp match {
case TypeRef(pre, sym, args) =>
if (sym.isAliasType && containsLocal(tp)) apply(tp.normalize)
else {
if (pre.isVolatile)
context.error(tree.pos, "Inferred type "+tree.tpe+" contains type selection from volatile type "+pre)
mapOver(tp)
}
case _ =>
mapOver(tp)
}
}
def addLocals(tp: Type) {
val remainingSyms = new ListBuffer[Symbol]
def addIfLocal(sym: Symbol, tp: Type) {
if (isLocal(sym) && !localSyms(sym) && !boundSyms(sym)) {
if (sym.typeParams.isEmpty) {
localSyms += sym
remainingSyms += sym
} else {
unit.error(tree.pos,
"can't existentially abstract over parameterized type " + tp)
}
}
}
for (t <- tp) {
t match {
case ExistentialType(tparams, _) =>
boundSyms ++= tparams
case AnnotatedType(annots, _, _) =>
for (annot <- annots; arg <- annot.args) {
arg match {
case Ident(_) =>
if (!arg.tpe.typeSymbol.hasFlag(EXISTENTIAL))
addIfLocal(arg.symbol, arg.tpe)
case _ => ()
}
}
case _ =>
}
addIfLocal(t.termSymbol, t)
addIfLocal(t.typeSymbol, t)
}
for (sym <- remainingSyms) addLocals(sym.existentialBound)
}
val normalizedTpe = normalizeLocals(tree.tpe)
addLocals(normalizedTpe)
packSymbols(localSyms.toList, normalizedTpe)
}
protected def typedExistentialTypeTree(tree: ExistentialTypeTree, mode: Int): Tree = {
for (wc <- tree.whereClauses)
if (wc.symbol == NoSymbol) { namer.enterSym(wc); wc.symbol setFlag EXISTENTIAL }
else context.scope enter wc.symbol
val whereClauses1 = typedStats(tree.whereClauses, context.owner)
for (vd @ ValDef(_, _, _, _) <- tree.whereClauses)
if (vd.symbol.tpe.isVolatile)
error(vd.pos, "illegal abstraction from value with volatile type "+vd.symbol.tpe)
val tpt1 = typedType(tree.tpt, mode)
val (typeParams, tpe) = existentialTransform(tree.whereClauses map (_.symbol), tpt1.tpe)
TypeTree(ExistentialType(typeParams, tpe)) setOriginal tree
}
protected def typedTypeApply(tree: Tree, mode: Int, fun: Tree, args: List[Tree]): Tree = fun.tpe match {
case OverloadedType(pre, alts) =>
inferPolyAlternatives(fun, args map (_.tpe))
val tparams = fun.symbol.typeParams
val args1 = if (sameLength(args, tparams)) {
map2Conserve(args, tparams) {
(arg, tparam) => typedHigherKindedType(arg, mode, polyType(tparam.typeParams, AnyClass.tpe))
}
} else
return errorTree(fun, "wrong number of type parameters for "+treeSymTypeMsg(fun))
typedTypeApply(tree, mode, fun, args1)
case SingleType(_, _) =>
typedTypeApply(tree, mode, fun setType fun.tpe.widen, args)
case PolyType(tparams, restpe) if tparams.nonEmpty =>
if (sameLength(tparams, args)) {
val targs = args map (_.tpe)
checkBounds(tree.pos, NoPrefix, NoSymbol, tparams, targs, "")
if (fun.symbol == Predef_classOf) {
checkClassType(args.head, true, false)
atPos(tree.pos) { gen.mkClassOf(targs.head) }
} else {
if (phase.id <= currentRun.typerPhase.id &&
fun.symbol == Any_isInstanceOf && !targs.isEmpty)
checkCheckable(tree.pos, targs.head, "")
val resultpe = restpe.instantiateTypeParams(tparams, targs)
treeCopy.TypeApply(tree, fun, args) setType resultpe
}
} else {
errorTree(tree, "wrong number of type parameters for "+treeSymTypeMsg(fun))
}
case ErrorType =>
setError(tree)
case _ =>
errorTree(tree, treeSymTypeMsg(fun)+" does not take type parameters.")
}
@inline final def deindentTyping() = context.typingIndentLevel -= 2
@inline final def indentTyping() = context.typingIndentLevel += 2
@inline final def printTyping(s: => String) = {
if (printTypings)
println(context.typingIndent + s.replaceAll("\n", "\n" + context.typingIndent))
}
@inline final def printInference(s: => String) = {
if (printInfers)
println(s)
}
protected def typed1(tree: Tree, mode: Int, pt: Type): Tree = {
def isPatternMode = inPatternMode(mode)
def ptOrLub(tps: List[Type]) = if (isFullyDefined(pt)) (pt, false) else weakLub(tps map (_.deconst))
def prefixType(fun: Tree): Type = fun match {
case Select(qualifier, _) => qualifier.tpe
case _ => NoType
}
def typedAnnotated(ann: Tree, arg1: Tree): Tree = {
val annotMode = mode & ~TYPEmode | EXPRmode
if (arg1.isType) {
if (ann.tpe == null) {
val selfsym =
if (!settings.selfInAnnots.value)
NoSymbol
else
arg1.tpe.selfsym match {
case NoSymbol =>
val sym =
context.owner.newLocalDummy(ann.pos)
.newValue(ann.pos, nme.self)
sym.setInfo(arg1.tpe.withoutAnnotations)
sym
case sym => sym
}
val ainfo = typedAnnotation(ann, annotMode, selfsym)
val atype0 = arg1.tpe.withAnnotation(ainfo)
val atype =
if ((selfsym != NoSymbol) && (ainfo.refsSymbol(selfsym)))
atype0.withSelfsym(selfsym)
else
atype0
if (ainfo.isErroneous)
arg1
else {
ann.tpe = atype
TypeTree(atype) setOriginal tree
}
} else {
TypeTree(ann.tpe) setOriginal tree
}
} else {
if (ann.tpe == null) {
val annotInfo = typedAnnotation(ann, annotMode)
ann.tpe = arg1.tpe.withAnnotation(annotInfo)
}
val atype = ann.tpe
Typed(arg1, TypeTree(atype) setOriginal tree setPos tree.pos.focus) setPos tree.pos setType atype
}
}
def typedBind(name: Name, body: Tree) = {
var vble = tree.symbol
def typedBindType(name: TypeName) = {
assert(body == EmptyTree, context.unit + " typedBind: " + name.debugString + " " + body + " " + body.getClass)
if (vble == NoSymbol)
vble =
if (isFullyDefined(pt))
context.owner.newAliasType(tree.pos, name) setInfo pt
else
context.owner.newAbstractType(tree.pos, name) setInfo TypeBounds.empty
val rawInfo = vble.rawInfo
vble = if (vble.name == tpnme.WILDCARD) context.scope.enter(vble)
else namer.enterInScope(vble)
tree setSymbol vble setType vble.tpe
}
def typedBindTerm(name: TermName) = {
if (vble == NoSymbol)
vble = context.owner.newValue(tree.pos, name)
if (vble.name.toTermName != nme.WILDCARD) {
if ((mode & ALTmode) != 0)
error(tree.pos, "illegal variable in pattern alternative")
vble = namer.enterInScope(vble)
}
val body1 = typed(body, mode, pt)
vble.setInfo(
if (treeInfo.isSequenceValued(body)) seqType(body1.tpe)
else body1.tpe)
treeCopy.Bind(tree, name, body1) setSymbol vble setType body1.tpe
}
name match {
case x: TypeName => typedBindType(x)
case x: TermName => typedBindTerm(x)
}
}
def typedArrayValue(elemtpt: Tree, elems: List[Tree]) = {
val elemtpt1 = typedType(elemtpt, mode)
val elems1 = elems mapConserve (elem => typed(elem, mode, elemtpt1.tpe))
treeCopy.ArrayValue(tree, elemtpt1, elems1)
.setType(
(if (isFullyDefined(pt) && !phase.erasedTypes) pt
else appliedType(ArrayClass.typeConstructor, List(elemtpt1.tpe))).notNull)
}
def typedAssign(lhs: Tree, rhs: Tree): Tree = {
val lhs1 = typed(lhs, EXPRmode | LHSmode, WildcardType)
val varsym = lhs1.symbol
def failMsg =
if (varsym != null && varsym.isValue) "reassignment to val"
else "assignment to non variable"
def fail = {
if (!lhs1.tpe.isError)
error(tree.pos, failMsg)
setError(tree)
}
if (varsym == null)
return fail
if (treeInfo.mayBeVarGetter(varsym)) {
treeInfo.methPart(lhs1) match {
case Select(qual, name) =>
val sel = Select(qual, nme.getterToSetter(name.toTermName)) setPos lhs.pos
val app = Apply(sel, List(rhs)) setPos tree.pos
return typed(app, mode, pt)
case _ =>
}
}
if (varsym.isVariable || varsym.isValue && phase.erasedTypes) {
val rhs1 = typed(rhs, EXPRmode | BYVALmode, lhs1.tpe)
treeCopy.Assign(tree, lhs1, checkDead(rhs1)) setType UnitClass.tpe
}
else fail
}
def typedIf(cond: Tree, thenp: Tree, elsep: Tree) = {
val cond1 = checkDead(typed(cond, EXPRmode | BYVALmode, BooleanClass.tpe))
if (elsep.isEmpty) {
val thenp1 = typed(thenp, UnitClass.tpe)
treeCopy.If(tree, cond1, thenp1, elsep) setType thenp1.tpe
} else {
var thenp1 = typed(thenp, pt)
var elsep1 = typed(elsep, pt)
val (owntype, needAdapt) = ptOrLub(List(thenp1.tpe, elsep1.tpe))
if (needAdapt) {
thenp1 = adapt(thenp1, mode, owntype)
elsep1 = adapt(elsep1, mode, owntype)
}
treeCopy.If(tree, cond1, thenp1, elsep1) setType owntype
}
}
def typedReturn(expr: Tree) = {
val enclMethod = context.enclMethod
if (enclMethod == NoContext ||
enclMethod.owner.isConstructor ||
context.enclClass.enclMethod == enclMethod
) {
errorTree(tree, "return outside method definition")
} else {
val DefDef(_, name, _, _, restpt, _) = enclMethod.tree
if (restpt.tpe eq null)
errorTree(tree, enclMethod.owner + " has return statement; needs result type")
else {
context.enclMethod.returnsSeen = true
val expr1: Tree = typed(expr, EXPRmode | BYVALmode, restpt.tpe)
if (restpt.tpe.typeSymbol == UnitClass) {
if (typed(expr).tpe.typeSymbol != UnitClass)
unit.warning(tree.pos, "enclosing method " + name + " has result type Unit: return value discarded")
}
treeCopy.Return(tree, checkDead(expr1)) setSymbol enclMethod.owner setType NothingClass.tpe
}
}
}
def typedNew(tpt: Tree) = {
val tpt1 = {
val tpt0 = typedTypeConstructor(tpt)
checkClassType(tpt0, false, true)
if (tpt0.hasSymbol && !tpt0.symbol.typeParams.isEmpty) {
context.undetparams = cloneSymbols(tpt0.symbol.typeParams)
TypeTree().setOriginal(tpt0)
.setType(appliedType(tpt0.tpe, context.undetparams map (_.tpeHK)))
} else tpt0
}
def narrowRhs(tp: Type) = { val sym = context.tree.symbol
context.tree match {
case ValDef(mods, _, _, Apply(Select(`tree`, _), _)) if !mods.isMutable && sym != null && sym != NoSymbol =>
val sym1 = if (sym.owner.isClass && sym.getter(sym.owner) != NoSymbol) sym.getter(sym.owner)
else sym.lazyAccessorOrSelf
val pre = if (sym1.owner.isClass) sym1.owner.thisType else NoPrefix
intersectionType(List(tp, singleType(pre, sym1)))
case _ => tp
}}
val tp = tpt1.tpe
val sym = tp.typeSymbol
if (sym.isAbstractType || sym.hasAbstractFlag)
error(tree.pos, sym + " is abstract; cannot be instantiated")
else if (!( tp == sym.initialize.thisSym.tpe
|| narrowRhs(tp) <:< tp.typeOfThis
|| phase.erasedTypes
)) {
error(tree.pos, sym +
" cannot be instantiated because it does not conform to its self-type "+
tp.typeOfThis)
}
treeCopy.New(tree, tpt1).setType(tp)
}
def typedEta(expr1: Tree): Tree = expr1.tpe match {
case TypeRef(_, ByNameParamClass, _) =>
val expr2 = Function(List(), expr1) setPos expr1.pos
new ChangeOwnerTraverser(context.owner, expr2.symbol).traverse(expr2)
typed1(expr2, mode, pt)
case NullaryMethodType(restpe) =>
val expr2 = Function(List(), expr1) setPos expr1.pos
new ChangeOwnerTraverser(context.owner, expr2.symbol).traverse(expr2)
typed1(expr2, mode, pt)
case PolyType(_, MethodType(formals, _)) =>
if (isFunctionType(pt)) expr1
else adapt(expr1, mode, functionType(formals map (t => WildcardType), WildcardType))
case MethodType(formals, _) =>
if (isFunctionType(pt)) expr1
else expr1 match {
case Select(qual, name) if (forMSIL &&
pt != WildcardType &&
pt != ErrorType &&
isSubType(pt, DelegateClass.tpe)) =>
val scalaCaller = newScalaCaller(pt)
addScalaCallerInfo(scalaCaller, expr1.symbol)
val n: Name = scalaCaller.name
val del = Ident(DelegateClass) setType DelegateClass.tpe
val f = Select(del, n)
val args: List[Tree] = if(expr1.symbol.isStatic) List(Literal(Constant(null)))
else List(qual)
val rhs = Apply(f, args)
typed(rhs)
case _ =>
adapt(expr1, mode, functionType(formals map (t => WildcardType), WildcardType))
}
case ErrorType =>
expr1
case _ =>
errorTree(expr1, "_ must follow method; cannot follow " + expr1.tpe)
}
def tryTypedArgs(args: List[Tree], mode: Int, other: TypeError): List[Tree] = {
val c = context.makeSilent(false)
c.retyping = true
try {
newTyper(c).typedArgs(args, mode)
} catch {
case ex: CyclicReference => throw ex
case ex: TypeError =>
null
}
}
def tryTypedApply(fun: Tree, args: List[Tree]): Tree = {
val start = startTimer(failedApplyNanos)
silent(_.doTypedApply(tree, fun, args, mode, pt)) match {
case t: Tree =>
t
case ex: TypeError =>
stopTimer(failedApplyNanos, start)
if ((fun.symbol ne null) && fun.symbol.isJavaDefined) {
val newtpe = rawToExistential(fun.tpe)
if (fun.tpe ne newtpe) {
return tryTypedApply(fun setType newtpe, args)
}
}
def treesInResult(tree: Tree): List[Tree] = tree :: (tree match {
case Block(_, r) => treesInResult(r)
case Match(_, cases) => cases
case CaseDef(_, _, r) => treesInResult(r)
case Annotated(_, r) => treesInResult(r)
case If(_, t, e) => treesInResult(t) ++ treesInResult(e)
case Try(b, catches, _) => treesInResult(b) ++ catches
case Typed(r, Function(Nil, EmptyTree)) => treesInResult(r)
case _ => Nil
})
def errorInResult(tree: Tree) = treesInResult(tree) exists (_.pos == ex.pos)
val retry = fun :: tree :: args exists errorInResult
printTyping {
val funStr = ptTree(fun) + " and " + (args map ptTree mkString ", ")
if (retry) "second try: " + funStr
else "no second try: " + funStr + " because error not in result: " + ex.pos+"!="+tree.pos
}
if (retry) {
val Select(qual, name) = fun
val args1 = tryTypedArgs(args, forArgMode(fun, mode), ex)
val qual1 =
if ((args1 ne null) && !pt.isError) adaptToArguments(qual, name, args1, pt)
else qual
if (qual1 ne qual) {
val tree1 = Apply(Select(qual1, name) setPos fun.pos, args1) setPos tree.pos
return typed1(tree1, mode | SNDTRYmode, pt)
}
}
reportTypeError(tree.pos, ex)
setError(tree)
}
}
def typedApply(fun: Tree, args: List[Tree]) = {
val stableApplication = (fun.symbol ne null) && fun.symbol.isMethod && fun.symbol.isStable
if (stableApplication && isPatternMode) {
typed1(tree, mode & ~PATTERNmode | EXPRmode, pt)
} else {
val funpt = if (isPatternMode) pt else WildcardType
val appStart = startTimer(failedApplyNanos)
val opeqStart = startTimer(failedOpEqNanos)
silent(_.typed(fun, forFunMode(mode), funpt),
if ((mode & EXPRmode) != 0) false else context.reportAmbiguousErrors,
if ((mode & EXPRmode) != 0) tree else context.tree) match {
case fun1: Tree =>
val fun2 = if (stableApplication) stabilizeFun(fun1, mode, pt) else fun1
incCounter(typedApplyCount)
def isImplicitMethod(tpe: Type) = tpe match {
case mt: MethodType => mt.isImplicit
case _ => false
}
val res =
if (phase.id <= currentRun.typerPhase.id &&
fun2.isInstanceOf[Select] &&
!isImplicitMethod(fun2.tpe) &&
((fun2.symbol eq null) || !fun2.symbol.isConstructor) &&
(mode & (EXPRmode | SNDTRYmode)) == EXPRmode) {
tryTypedApply(fun2, args)
} else {
doTypedApply(tree, fun2, args, mode, pt)
}
if (fun2.symbol == Array_apply) {
val checked = gen.mkCheckInit(res)
if (checked ne res) typed { atPos(tree.pos)(checked) }
else res
} else res
case ex: TypeError =>
fun match {
case Select(qual, name)
if !isPatternMode && nme.isOpAssignmentName(name.decode) =>
val qual1 = typedQualifier(qual)
if (treeInfo.isVariableOrGetter(qual1)) {
stopTimer(failedOpEqNanos, opeqStart)
convertToAssignment(fun, qual1, name, args, ex)
} else {
stopTimer(failedApplyNanos, appStart)
if ((qual1.symbol ne null) && qual1.symbol.isValue)
error(tree.pos, "reassignment to val")
else
reportTypeError(fun.pos, ex)
setError(tree)
}
case _ =>
stopTimer(failedApplyNanos, appStart)
reportTypeError(fun.pos, ex)
setError(tree)
}
}
}
}
def convertToAssignment(fun: Tree, qual: Tree, name: Name, args: List[Tree], ex: TypeError): Tree = {
val prefix = name.subName(0, name.length - nme.EQL.length)
def mkAssign(vble: Tree): Tree =
Assign(
vble,
Apply(
Select(vble.duplicate, prefix) setPos fun.pos.focus, args) setPos tree.pos.makeTransparent
) setPos tree.pos
def mkUpdate(table: Tree, indices: List[Tree]) = {
gen.evalOnceAll(table :: indices, context.owner, context.unit) { ts =>
val tab = ts.head
val is = ts.tail
Apply(
Select(tab(), nme.update) setPos table.pos,
((is map (i => i())) ::: List(
Apply(
Select(
Apply(
Select(tab(), nme.apply) setPos table.pos,
is map (i => i())) setPos qual.pos,
prefix) setPos fun.pos,
args) setPos tree.pos)
)
) setPos tree.pos
}
}
val tree1 = qual match {
case Ident(_) =>
mkAssign(qual)
case Select(qualqual, vname) =>
gen.evalOnce(qualqual, context.owner, context.unit) { qq =>
val qq1 = qq()
mkAssign(Select(qq1, vname) setPos qual.pos)
}
case Apply(fn, indices) =>
treeInfo.methPart(fn) match {
case Select(table, nme.apply) => mkUpdate(table, indices)
case _ => errorTree(qual, "Unexpected tree during assignment conversion.")
}
}
typed1(tree1, mode, pt)
}
def qualifyingClassSym(qual: Name): Symbol =
if (tree.symbol != NoSymbol) tree.symbol else qualifyingClass(tree, qual, false)
def typedSuper(qual: Tree, mix: TypeName) = {
val qual1 = typed(qual)
val clazz = qual1 match {
case This(_) => qual1.symbol
case _ => qual1.tpe.typeSymbol
}
def findMixinSuper(site: Type): Type = {
var ps = site.parents filter (_.typeSymbol.name == mix)
if (ps.isEmpty)
ps = site.parents filter (_.typeSymbol.toInterface.name == mix)
if (ps.isEmpty) {
if (settings.debug.value)
Console.println(site.parents map (_.typeSymbol.name))
if (phase.erasedTypes && context.enclClass.owner.isImplClass) {
restrictionError(tree.pos, unit, "traits may not select fields or methods from super[C] where C is a class")
} else {
error(tree.pos, mix+" does not name a parent class of "+clazz)
}
ErrorType
} else if (!ps.tail.isEmpty) {
error(tree.pos, "ambiguous parent class qualifier")
ErrorType
} else {
ps.head
}
}
val owntype =
if (mix.isEmpty) {
if ((mode & SUPERCONSTRmode) != 0)
if (clazz.info.parents.isEmpty) AnyRefClass.tpe
else clazz.info.parents.head
else intersectionType(clazz.info.parents)
} else {
findMixinSuper(clazz.tpe)
}
treeCopy.Super(tree, qual1, mix) setType SuperType(clazz.thisType, owntype)
}
def typedThis(qual: Name) = {
val clazz = qualifyingClassSym(qual)
if (clazz == NoSymbol) setError(tree)
else {
tree setSymbol clazz setType clazz.thisType.underlying
if (isStableContext(tree, mode, pt)) tree setType clazz.thisType
tree
}
}
def typedSelect(qual: Tree, name: Name): Tree = {
val sym =
if (tree.symbol != NoSymbol) {
if (phase.erasedTypes && qual.isInstanceOf[Super])
qual.tpe = tree.symbol.owner.tpe
if (false && settings.debug.value) {
val alts = qual.tpe.member(tree.symbol.name).alternatives
if (!(alts exists (alt =>
alt == tree.symbol || alt.isTerm && (alt.tpe matches tree.symbol.tpe))))
assert(false, "symbol "+tree.symbol+tree.symbol.locationString+" not in "+alts+" of "+qual.tpe+
"\n members = "+qual.tpe.members+
"\n type history = "+qual.tpe.termSymbol.infosString+
"\n phase = "+phase)
}
tree.symbol
} else {
member(qual, name)
}
if (sym == NoSymbol && name != nme.CONSTRUCTOR && (mode & EXPRmode) != 0) {
val qual1 =
if (member(qual, name) != NoSymbol) qual
else adaptToMemberWithArgs(tree, qual, name, mode)
if (qual1 ne qual) return typed(treeCopy.Select(tree, qual1, name), mode, pt)
}
if (!reallyExists(sym)) {
if (context.owner.toplevelClass.isJavaDefined && name.isTypeName) {
val tree1 = atPos(tree.pos) { gen.convertToSelectFromType(qual, name) }
if (tree1 != EmptyTree) return typed1(tree1, mode, pt)
}
if (settings.Xexperimental.value && (qual.tpe.widen.typeSymbol isNonBottomSubClass DynamicClass)) {
var dynInvoke = Apply(Select(qual, nme.applyDynamic), List(Literal(Constant(name.decode))))
context.tree match {
case Apply(tree1, args) if tree1 eq tree =>
;
case _ =>
dynInvoke = Apply(dynInvoke, List())
}
return typed1(util.trace("dynatype: ")(dynInvoke), mode, pt)
}
if (settings.debug.value) {
log(
"qual = "+qual+":"+qual.tpe+
"\nSymbol="+qual.tpe.termSymbol+"\nsymbol-info = "+qual.tpe.termSymbol.info+
"\nscope-id = "+qual.tpe.termSymbol.info.decls.hashCode()+"\nmembers = "+qual.tpe.members+
"\nname = "+name+"\nfound = "+sym+"\nowner = "+context.enclClass.owner
)
}
def makeErrorTree = {
val tree1 = tree match {
case Select(_, _) => treeCopy.Select(tree, qual, name)
case SelectFromTypeTree(_, _) => treeCopy.SelectFromTypeTree(tree, qual, name)
}
setError(tree1)
}
if (name == nme.ERROR && forInteractive)
return makeErrorTree
if (!qual.tpe.widen.isErroneous)
notAMember(tree, qual, name)
if (forInteractive) makeErrorTree else setError(tree)
} else {
val tree1 = tree match {
case Select(_, _) => treeCopy.Select(tree, qual, name)
case SelectFromTypeTree(_, _) => treeCopy.SelectFromTypeTree(tree, qual, name)
}
val (tree2, pre2) = makeAccessible(tree1, sym, qual.tpe, qual)
val result = stabilize(tree2, pre2, mode, pt)
def isPotentialNullDeference() = {
phase.id <= currentRun.typerPhase.id &&
!sym.isConstructor &&
!(qual.tpe <:< NotNullClass.tpe) && !qual.tpe.isNotNull &&
!(List(Any_isInstanceOf, Any_asInstanceOf) contains result.symbol)
}
if (settings.warnSelectNullable.value && isPotentialNullDeference && unit != null)
unit.warning(tree.pos, "potential null pointer dereference: "+tree)
val selection = result match {
case SelectFromTypeTree(qual@TypeTree(), name) if qual.tpe.typeArgs nonEmpty =>
treeCopy.SelectFromTypeTree(
result,
(TypeTreeWithDeferredRefCheck(){ () => val tp = qual.tpe; val sym = tp.typeSymbolDirect
checkBounds(qual.pos, tp.prefix, sym.owner, sym.typeParams, tp.typeArgs, "")
qual
}) setType qual.tpe,
name)
case accErr: Inferencer#AccessError =>
val qual1 =
try adaptToMemberWithArgs(tree, qual, name, mode)
catch { case _: TypeError => qual }
if (qual1 ne qual) typed(Select(qual1, name) setPos tree.pos, mode, pt)
else accErr.emit()
case _ =>
result
}
val isRefinableGetClass = (
selection.symbol.name == nme.getClass_
&& selection.tpe.params.isEmpty
&& qual.tpe.typeSymbol.isPublic
)
if (isRefinableGetClass)
selection setType MethodType(Nil, erasure.getClassReturnType(qual.tpe))
else
selection
}
}
def typedIdent(name: Name): Tree = {
def ambiguousError(msg: String) =
error(tree.pos, "reference to " + name + " is ambiguous;\n" + msg)
var defSym: Symbol = tree.symbol
var pre: Type = NoPrefix
var qual: Tree = EmptyTree
var inaccessibleSym: Symbol = NoSymbol
var inaccessibleExplanation: String = ""
def checkEmptyPackage(): Boolean = {
defSym = EmptyPackageClass.tpe.nonPrivateMember(name)
defSym != NoSymbol
}
def qualifies(sym: Symbol): Boolean = {
reallyExists(sym) &&
((mode & PATTERNmode | FUNmode) != (PATTERNmode | FUNmode) || !sym.isSourceMethod || sym.hasFlag(ACCESSOR))
}
if (defSym == NoSymbol) {
var defEntry: ScopeEntry = null
var cx = context
if ((mode & (PATTERNmode | TYPEPATmode)) != 0) {
cx = cx.outer
}
while (defSym == NoSymbol && cx != NoContext) {
currentRun.compileSourceFor(context.asInstanceOf[analyzer.Context], name)
pre = cx.enclClass.prefix
defEntry = cx.scope.lookupEntry(name)
if ((defEntry ne null) && qualifies(defEntry.sym)) {
defSym = defEntry.sym
}
else {
cx = cx.enclClass
val foundSym = pre.member(name) filter qualifies
defSym = foundSym filter (context.isAccessible(_, pre, false))
if (defSym == NoSymbol) {
if ((foundSym ne NoSymbol) && (inaccessibleSym eq NoSymbol)) {
inaccessibleSym = foundSym
inaccessibleExplanation = analyzer.lastAccessCheckDetails
}
cx = cx.outer
}
}
}
val symDepth = if (defEntry eq null) cx.depth
else cx.depth - (cx.scope.nestingLevel - defEntry.owner.nestingLevel)
var impSym: Symbol = NoSymbol
var imports = context.imports
while (!reallyExists(impSym) && !imports.isEmpty && imports.head.depth > symDepth) {
impSym = imports.head.importedSymbol(name)
if (!impSym.exists) imports = imports.tail
}
if (defSym.exists && impSym.exists) {
if (defSym.definedInPackage &&
(!currentRun.compiles(defSym) ||
(context.unit ne null) && defSym.sourceFile != context.unit.source.file))
defSym = NoSymbol
else if (impSym.isError || impSym.name == nme.CONSTRUCTOR)
impSym = NoSymbol
}
if (defSym.exists) {
if (impSym.exists)
ambiguousError(
"it is both defined in "+defSym.owner +
" and imported subsequently by \n"+imports.head)
else if (!defSym.owner.isClass || defSym.owner.isPackageClass || defSym.isTypeParameterOrSkolem)
pre = NoPrefix
else
qual = atPos(tree.pos.focusStart)(gen.mkAttributedQualifier(pre))
} else {
if (impSym.exists) {
var impSym1 = NoSymbol
var imports1 = imports.tail
def ambiguousImport() = {
if (!(imports.head.qual.tpe =:= imports1.head.qual.tpe))
ambiguousError(
"it is imported twice in the same scope by\n"+imports.head + "\nand "+imports1.head)
}
while (!imports1.isEmpty &&
(!imports.head.isExplicitImport(name) ||
imports1.head.depth == imports.head.depth)) {
var impSym1 = imports1.head.importedSymbol(name)
if (reallyExists(impSym1)) {
if (imports1.head.isExplicitImport(name)) {
if (imports.head.isExplicitImport(name) ||
imports1.head.depth != imports.head.depth) ambiguousImport()
impSym = impSym1
imports = imports1
} else if (!imports.head.isExplicitImport(name) &&
imports1.head.depth == imports.head.depth) ambiguousImport()
}
imports1 = imports1.tail
}
defSym = impSym
val qual0 = imports.head.qual
if (!(shortenImports && qual0.symbol.isPackage))
qual = atPos(tree.pos.focusStart)(resetPos(qual0.duplicate))
pre = qual.tpe
}
else if (settings.exposeEmptyPackage.value && checkEmptyPackage())
log("Allowing empty package member " + name + " due to settings.")
else {
if (settings.debug.value) {
log(context.imports)
}
if (inaccessibleSym eq NoSymbol) {
error(tree.pos, "not found: "+decodeWithKind(name, context.owner))
}
else new AccessError(
tree, inaccessibleSym, context.enclClass.owner.thisType,
inaccessibleExplanation
).emit()
defSym = context.owner.newErrorSymbol(name)
}
}
}
if (defSym.owner.isPackageClass) pre = defSym.owner.thisType
if (defSym.isThisSym) {
typed1(This(defSym.owner) setPos tree.pos, mode, pt)
} else {
val tree1 = if (qual == EmptyTree) tree
else atPos(tree.pos)(Select(qual, name))
val (tree2, pre2) = makeAccessible(tree1, defSym, pre, qual)
stabilize(tree2, pre2, mode, pt) match {
case accErr: Inferencer#AccessError => accErr.emit()
case result => result
}
}
}
def typedCompoundTypeTree(templ: Template) = {
val parents1 = templ.parents mapConserve (typedType(_, mode))
if (parents1 exists (_.tpe.isError)) tree setType ErrorType
else {
val decls = new Scope
val self = refinedType(parents1 map (_.tpe), context.enclClass.owner, decls, templ.pos)
newTyper(context.make(templ, self.typeSymbol, decls)).typedRefinement(templ.body)
tree setType self
}
}
def typedAppliedTypeTree(tpt: Tree, args: List[Tree]) = {
val tpt1 = typed1(tpt, mode | FUNmode | TAPPmode, WildcardType)
if (tpt1.tpe.isError) {
setError(tree)
} else if (!tpt1.hasSymbol) {
errorTree(tree, tpt1.tpe+" does not take type parameters")
} else {
val tparams = tpt1.symbol.typeParams
if (sameLength(tparams, args)) {
val args1 =
if(!tpt1.symbol.rawInfo.isComplete)
args mapConserve (typedHigherKindedType(_, mode))
else map2Conserve(args, tparams) {
(arg, tparam) =>
typedHigherKindedType(arg, mode, polyType(tparam.typeParams, AnyClass.tpe))
}
val argtypes = args1 map (_.tpe)
(args, tparams).zipped foreach { (arg, tparam) => arg match {
case Bind(_, _) =>
if (arg.symbol.isAbstractType)
arg.symbol setInfo
TypeBounds(
lub(List(arg.symbol.info.bounds.lo, tparam.info.bounds.lo.subst(tparams, argtypes))),
glb(List(arg.symbol.info.bounds.hi, tparam.info.bounds.hi.subst(tparams, argtypes))))
case _ =>
}}
val original = treeCopy.AppliedTypeTree(tree, tpt1, args1)
val result = TypeTree(appliedType(tpt1.tpe, argtypes)) setOriginal original
if(tpt1.tpe.isInstanceOf[PolyType])
(TypeTreeWithDeferredRefCheck(){ () =>
checkBounds(result.pos, tpt1.tpe.prefix, tpt1.symbol.owner, tpt1.symbol.typeParams, argtypes, "")
result
}).setType(result.tpe)
else result
} else if (tparams.isEmpty) {
errorTree(tree, tpt1.tpe+" does not take type parameters")
} else {
if (settings.debug.value) Console.println(tpt1+":"+tpt1.symbol+":"+tpt1.symbol.info)
errorTree(tree, "wrong number of type arguments for "+tpt1.tpe+", should be "+tparams.length)
}
}
}
def adaptCase(cdef: CaseDef, tpe: Type): CaseDef =
treeCopy.CaseDef(cdef, cdef.pat, cdef.guard, adapt(cdef.body, mode, tpe))
val sym: Symbol = tree.symbol
if ((sym ne null) && (sym ne NoSymbol)) sym.initialize
tree match {
case PackageDef(pid, stats) =>
val pid1 = typedQualifier(pid).asInstanceOf[RefTree]
assert(sym.moduleClass ne NoSymbol, sym)
val annots = sym.annotations
val stats1 = newTyper(context.make(tree, sym.moduleClass, sym.info.decls))
.typedStats(stats, NoSymbol)
treeCopy.PackageDef(tree, pid1, stats1) setType NoType
case tree @ ClassDef(_, _, _, _) =>
newTyper(context.makeNewScope(tree, sym)).typedClassDef(tree)
case tree @ ModuleDef(_, _, _) =>
newTyper(context.makeNewScope(tree, sym.moduleClass)).typedModuleDef(tree)
case vdef @ ValDef(_, _, _, _) =>
typedValDef(vdef)
case ddef @ DefDef(_, _, _, _, _, _) =>
newTyper(context.makeNewScope(tree, sym)).typedDefDef(ddef)
case tdef @ TypeDef(_, _, _, _) =>
typedTypeDef(tdef)
case ldef @ LabelDef(_, _, _) =>
labelTyper(ldef).typedLabelDef(ldef)
case ddef @ DocDef(comment, defn) =>
if (forScaladoc && (sym ne null) && (sym ne NoSymbol)) {
docComments(sym) = comment
comment.defineVariables(sym)
val typer1 = newTyper(context.makeNewScope(tree, context.owner))
for (useCase <- comment.useCases) {
typer1.silent(_.typedUseCase(useCase)) match {
case ex: TypeError =>
unit.warning(useCase.pos, ex.msg)
case _ =>
}
for (useCaseSym <- useCase.defined) {
if (sym.name != useCaseSym.name)
unit.warning(useCase.pos, "@usecase " + useCaseSym.name.decode + " does not match commented symbol: " + sym.name.decode)
}
}
}
typed(defn, mode, pt)
case Annotated(constr, arg) =>
typedAnnotated(constr, typed(arg, mode, pt))
case tree @ Block(_, _) =>
newTyper(context.makeNewScope(tree, context.owner))
.typedBlock(tree, mode, pt)
case Alternative(alts) =>
val alts1 = alts mapConserve (alt => typed(alt, mode | ALTmode, pt))
treeCopy.Alternative(tree, alts1) setType pt
case Star(elem) =>
checkStarPatOK(tree.pos, mode)
val elem1 = typed(elem, mode, pt)
treeCopy.Star(tree, elem1) setType makeFullyDefined(pt)
case Bind(name, body) =>
typedBind(name, body)
case UnApply(fun, args) =>
val fun1 = typed(fun)
val tpes = formalTypes(unapplyTypeList(fun.symbol, fun1.tpe), args.length)
val args1 = (args, tpes).zipped map typedPattern
treeCopy.UnApply(tree, fun1, args1) setType pt
case ArrayValue(elemtpt, elems) =>
typedArrayValue(elemtpt, elems)
case tree @ Function(_, _) =>
if (tree.symbol == NoSymbol)
tree.symbol = context.owner.newValue(tree.pos, nme.ANON_FUN_NAME)
.setFlag(SYNTHETIC).setInfo(NoType)
newTyper(context.makeNewScope(tree, tree.symbol)).typedFunction(tree, mode, pt)
case Assign(lhs, rhs) =>
typedAssign(lhs, rhs)
case AssignOrNamedArg(lhs, rhs) =>
typedAssign(lhs, rhs)
case If(cond, thenp, elsep) =>
typedIf(cond, thenp, elsep)
case tree @ Match(selector, cases) =>
if (selector == EmptyTree) {
val arity = if (isFunctionType(pt)) pt.normalize.typeArgs.length - 1 else 1
val params = for (i <- List.range(0, arity)) yield
atPos(tree.pos.focusStart) {
ValDef(Modifiers(PARAM | SYNTHETIC),
unit.freshTermName("x" + i + "$"), TypeTree(), EmptyTree)
}
val ids = for (p <- params) yield Ident(p.name)
val selector1 = atPos(tree.pos.focusStart) { if (arity == 1) ids.head else gen.mkTuple(ids) }
val body = treeCopy.Match(tree, selector1, cases)
typed1(atPos(tree.pos) { Function(params, body) }, mode, pt)
} else {
val selector1 = checkDead(typed(selector, EXPRmode | BYVALmode, WildcardType))
var cases1 = typedCases(tree, cases, selector1.tpe.widen, pt)
val (owntype, needAdapt) = ptOrLub(cases1 map (_.tpe))
if (needAdapt) {
cases1 = cases1 map (adaptCase(_, owntype))
}
treeCopy.Match(tree, selector1, cases1) setType owntype
}
case Return(expr) =>
typedReturn(expr)
case Try(block, catches, finalizer) =>
var block1 = typed(block, pt)
var catches1 = typedCases(tree, catches, ThrowableClass.tpe, pt)
val finalizer1 = if (finalizer.isEmpty) finalizer
else typed(finalizer, UnitClass.tpe)
val (owntype, needAdapt) = ptOrLub(block1.tpe :: (catches1 map (_.tpe)))
if (needAdapt) {
block1 = adapt(block1, mode, owntype)
catches1 = catches1 map (adaptCase(_, owntype))
}
treeCopy.Try(tree, block1, catches1, finalizer1) setType owntype
case Throw(expr) =>
val expr1 = typed(expr, EXPRmode | BYVALmode, ThrowableClass.tpe)
treeCopy.Throw(tree, expr1) setType NothingClass.tpe
case New(tpt: Tree) =>
typedNew(tpt)
case Typed(expr, Function(List(), EmptyTree)) =>
typedEta(checkDead(typed1(expr, mode, pt)))
case Typed(expr, tpt @ Ident(tpnme.WILDCARD_STAR)) =>
val expr0 = typed(expr, onlyStickyModes(mode), WildcardType)
def subArrayType(pt: Type) =
if (isValueClass(pt.typeSymbol) || !isFullyDefined(pt)) arrayType(pt)
else {
val tparam = context.owner freshExistential "" setInfo TypeBounds.upper(pt)
ExistentialType(List(tparam), arrayType(tparam.tpe))
}
val (expr1, baseClass) = expr0.tpe.typeSymbol match {
case ArrayClass => (adapt(expr0, onlyStickyModes(mode), subArrayType(pt)), ArrayClass)
case _ => (adapt(expr0, onlyStickyModes(mode), seqType(pt)), SeqClass)
}
expr1.tpe.baseType(baseClass) match {
case TypeRef(_, _, List(elemtp)) =>
treeCopy.Typed(tree, expr1, tpt setType elemtp) setType elemtp
case _ =>
setError(tree)
}
case Typed(expr, tpt) =>
val tpt1 = typedType(tpt, mode)
val expr1 = typed(expr, onlyStickyModes(mode), tpt1.tpe.deconst)
val owntype =
if (isPatternMode) inferTypedPattern(tpt1.pos, tpt1.tpe, pt)
else tpt1.tpe
treeCopy.Typed(tree, expr1, tpt1) setType owntype
case TypeApply(fun, args) =>
val fun1 = typed(fun, forFunMode(mode) | TAPPmode, WildcardType)
val tparams = fun1.symbol.typeParams
val args1 = if (sameLength(args, tparams)) map2Conserve(args, tparams) {
(arg, tparam) => typedHigherKindedType(arg, mode, polyType(tparam.typeParams, AnyClass.tpe))
} else {
args mapConserve (typedHigherKindedType(_, mode))
}
typedTypeApply(tree, mode, fun1, args1)
case Apply(Block(stats, expr), args) =>
typed1(atPos(tree.pos)(Block(stats, Apply(expr, args))), mode, pt)
case Apply(fun, args) =>
typedApply(fun, args) match {
case Apply(Select(New(tpt), name), args)
if (tpt.tpe != null &&
tpt.tpe.typeSymbol == ArrayClass &&
args.length == 1 &&
erasure.GenericArray.unapply(tpt.tpe).isDefined) =>
val Some((level, manifType)) = erasure.GenericArray.unapply(tpt.tpe)
if (level > MaxArrayDims)
error(tree.pos, "cannot create a generic multi-dimensional array of more than "+MaxArrayDims+" dimensions")
val newArrayApp = atPos(tree.pos) {
val manif = getManifestTree(tree.pos, manifType, false)
new ApplyToImplicitArgs(Select(manif, if (level == 1) "newArray" else "newArray"+level), args)
}
typed(newArrayApp, mode, pt)
case tree1 =>
tree1
}
case ApplyDynamic(qual, args) =>
val reflectiveCalls = !(settings.refinementMethodDispatch.value == "invoke-dynamic")
val qual1 = typed(qual, AnyRefClass.tpe)
val args1 = args mapConserve (arg => if (reflectiveCalls) typed(arg, AnyRefClass.tpe) else typed(arg))
treeCopy.ApplyDynamic(tree, qual1, args1) setType (if (reflectiveCalls) AnyRefClass.tpe else tree.symbol.info.resultType)
case Super(qual, mix) =>
typedSuper(qual, mix)
case This(qual) =>
typedThis(qual)
case Select(qual @ Super(_, _), nme.CONSTRUCTOR) =>
val qual1 =
typed(qual, EXPRmode | QUALmode | POLYmode | SUPERCONSTRmode, WildcardType)
typedSelect(qual1, nme.CONSTRUCTOR)
case Select(qual, name) =>
incCounter(typedSelectCount)
var qual1 = checkDead(typedQualifier(qual, mode))
if (name.isTypeName) qual1 = checkStable(qual1)
val tree1 =
if (name == nme.withFilter)
silent(_ => typedSelect(qual1, name)) match {
case result1: Tree =>
result1
case ex1: TypeError =>
silent(_ => typed1(Select(qual1, nme.filter) setPos tree.pos, mode, pt)) match {
case result2: Tree =>
unit.deprecationWarning(
tree.pos, "`withFilter' method does not yet exist on "+qual1.tpe.widen+
", using `filter' method instead")
result2
case ex2: TypeError =>
reportTypeError(tree.pos, ex1)
setError(tree)
}
}
else
typedSelect(qual1, name)
if (qual1.symbol == RootPackage) treeCopy.Ident(tree1, name)
else tree1
case Ident(name) =>
incCounter(typedIdentCount)
if ((name == nme.WILDCARD && (mode & (PATTERNmode | FUNmode)) == PATTERNmode) ||
(name == tpnme.WILDCARD && (mode & TYPEmode) != 0))
tree setType makeFullyDefined(pt)
else
typedIdent(name)
case Literal(value) =>
tree setType (
if (value.tag == UnitTag) UnitClass.tpe
else ConstantType(value))
case SingletonTypeTree(ref) =>
val ref1 = checkStable(
typed(ref, EXPRmode | QUALmode | (mode & TYPEPATmode), AnyRefClass.tpe))
tree setType ref1.tpe.resultType
case SelectFromTypeTree(qual, selector) =>
val qual1 = typedType(qual, mode)
if (qual1.tpe.isVolatile) error(tree.pos, "illegal type selection from volatile type "+qual.tpe)
typedSelect(qual1, selector)
case CompoundTypeTree(templ) =>
typedCompoundTypeTree(templ)
case AppliedTypeTree(tpt, args) =>
typedAppliedTypeTree(tpt, args)
case TypeBoundsTree(lo, hi) =>
val lo1 = typedType(lo, mode)
val hi1 = typedType(hi, mode)
treeCopy.TypeBoundsTree(tree, lo1, hi1) setType TypeBounds(lo1.tpe, hi1.tpe)
case etpt @ ExistentialTypeTree(_, _) =>
newTyper(context.makeNewScope(tree, context.owner)).typedExistentialTypeTree(etpt, mode)
case dc@TypeTreeWithDeferredRefCheck() => dc
case tpt @ TypeTree() =>
if (tpt.original != null)
tree setType typedType(tpt.original, mode).tpe
else
tree setType AnyClass.tpe
case Import(expr, selectors) =>
assert(forInteractive)
tree setType tree.symbol.tpe
case _ =>
abort("unexpected tree: " + tree.getClass + "\n" + tree)
}
}
def typed(tree: Tree, mode: Int, pt: Type): Tree = {
indentTyping()
def dropExistential(tp: Type): Type = tp match {
case ExistentialType(tparams, tpe) =>
if (settings.debug.value)
log("Dropping existential: " + tree + " " + tp)
new SubstWildcardMap(tparams).apply(tp)
case TypeRef(_, sym, _) if sym.isAliasType =>
val tp0 = tp.normalize
val tp1 = dropExistential(tp0)
if (tp1 eq tp0) tp else tp1
case _ => tp
}
var alreadyTyped = false
try {
if (Statistics.enabled) {
val t = currentTime()
if (pendingTreeTypes.nonEmpty) {
microsByType(pendingTreeTypes.head) += ((t - typerTime) / 1000).toInt
}
typerTime = t
pendingTreeTypes = tree.getClass :: pendingTreeTypes
}
if (context.retyping &&
(tree.tpe ne null) && (tree.tpe.isErroneous || !(tree.tpe <:< pt))) {
tree.tpe = null
if (tree.hasSymbol) tree.symbol = NoSymbol
}
alreadyTyped = tree.tpe ne null
var tree1: Tree = if (alreadyTyped) tree else {
printTyping(
ptLine("typing %s: pt = %s".format(ptTree(tree), pt),
"undetparams" -> context.undetparams,
"implicitsEnabled" -> context.implicitsEnabled,
"silent" -> !context.reportGeneralErrors,
"context.owner" -> context.owner
)
)
val tree1 = typed1(tree, mode, dropExistential(pt))
printTyping("typed %s: %s%s".format(
ptTree(tree1), tree1.tpe,
if (isSingleType(tree1.tpe)) " with underlying "+tree1.tpe.widen else "")
)
tree1
}
tree1.tpe = addAnnotations(tree1, tree1.tpe)
val result = if (tree1.isEmpty) tree1 else adapt(tree1, mode, pt, tree)
if (!alreadyTyped) {
printTyping("adapted %s: %s to %s, %s".format(
tree1, tree1.tpe.widen, pt, context.undetparamsString)
)
}
if (phase.id <= currentRun.typerPhase.id) signalDone(context.asInstanceOf[analyzer.Context], tree, result)
result
} catch {
case ex: TypeError =>
tree.tpe = null
printTyping("caught %s: while typing %s".format(ex, tree))
reportTypeError(tree.pos, ex)
setError(tree)
case ex: Exception =>
if (settings.debug.value)
Console.println("exception when typing "+tree+", pt = "+pt)
if ((context ne null) && (context.unit ne null) &&
(context.unit.source ne null) && (tree ne null))
logError("AT: " + (tree.pos).dbgString, ex)
throw ex
}
finally {
deindentTyping()
if (Statistics.enabled) {
val t = currentTime()
microsByType(pendingTreeTypes.head) += ((t - typerTime) / 1000).toInt
visitsByType(pendingTreeTypes.head) += 1
typerTime = t
pendingTreeTypes = pendingTreeTypes.tail
}
}
}
def atOwner(owner: Symbol): Typer =
newTyper(context.make(context.tree, owner))
def atOwner(tree: Tree, owner: Symbol): Typer =
newTyper(context.make(tree, owner))
def typed(tree: Tree): Tree = {
val ret = typed(tree, EXPRmode, WildcardType)
ret
}
def typedPos(pos: Position)(tree: Tree) = typed(atPos(pos)(tree))
def typed(tree: Tree, pt: Type): Tree =
typed(tree, EXPRmode, pt)
def typedQualifier(tree: Tree, mode: Int, pt: Type): Tree =
typed(tree, EXPRmode | QUALmode | POLYmode | mode & TYPEPATmode, pt)
def typedQualifier(tree: Tree, mode: Int): Tree =
typedQualifier(tree, mode, WildcardType)
def typedQualifier(tree: Tree): Tree = typedQualifier(tree, NOmode, WildcardType)
def typedOperator(tree: Tree): Tree =
typed(tree, EXPRmode | FUNmode | POLYmode | TAPPmode, WildcardType)
def typedPattern(tree: Tree, pt: Type): Tree = {
context.withImplicitsDisabled(typed(tree, PATTERNmode, pt))
}
def typedType(tree: Tree, mode: Int): Tree =
typed(tree, forTypeMode(mode), WildcardType)
def typedType(tree: Tree): Tree = typedType(tree, NOmode)
def typedHigherKindedType(tree: Tree, mode: Int, pt: Type): Tree =
if (pt.typeParams.isEmpty) typedType(tree, mode)
else typed(tree, HKmode, pt)
def typedHigherKindedType(tree: Tree, mode: Int): Tree =
typed(tree, HKmode, WildcardType)
def typedHigherKindedType(tree: Tree): Tree = typedHigherKindedType(tree, NOmode)
def typedTypeConstructor(tree: Tree, mode: Int): Tree = {
val result = typed(tree, forTypeMode(mode) | FUNmode, WildcardType)
val restpe = result.tpe.normalize
if (!phase.erasedTypes && restpe.isInstanceOf[TypeRef] && !restpe.prefix.isStable && !context.unit.isJava) {
error(tree.pos, restpe.prefix+" is not a legal prefix for a constructor")
}
if(result.tpe.typeArgs.isEmpty) {
result setType(restpe)
} else {
result
}
}
def typedTypeConstructor(tree: Tree): Tree = typedTypeConstructor(tree, NOmode)
def computeType(tree: Tree, pt: Type): Type = {
val tree1 = typed(tree, pt)
transformed(tree) = tree1
packedType(tree1, context.owner)
}
def transformedOrTyped(tree: Tree, mode: Int, pt: Type): Tree = transformed.get(tree) match {
case Some(tree1) => transformed -= tree; tree1
case None => typed(tree, mode, pt)
}
def findManifest(tp: Type, full: Boolean) = atPhase(currentRun.typerPhase) {
inferImplicit(
EmptyTree,
appliedType((if (full) FullManifestClass else PartialManifestClass).typeConstructor, List(tp)),
true, false, context)
}
def getManifestTree(pos: Position, tp: Type, full: Boolean): Tree = {
val manifestOpt = findManifest(tp, full)
if (manifestOpt.tree.isEmpty) {
error(pos, "cannot find "+(if (full) "" else "class ")+"manifest for element type "+tp)
Literal(Constant(null))
} else {
manifestOpt.tree
}
}
}
}