Skip to content

Commit

Permalink
[javasrc2cpg] Clean up assignment creation code (joernio#4129)
Browse files Browse the repository at this point in the history
* Add working refactored variable declarations

* Fix static field assigns

* Deduplicate assignment creation

* Remove debug code
  • Loading branch information
johannescoetzee authored Feb 6, 2024
1 parent 334989f commit be077da
Show file tree
Hide file tree
Showing 9 changed files with 148 additions and 178 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ private[declarations] trait AstForMethodsCreator { this: AstCreator =>
fieldDeclarations.flatMap { fieldDeclaration =>
fieldDeclaration.getVariables.asScala.filter(_.getInitializer.isPresent).toList.flatMap { variableDeclaration =>
scope.pushFieldDeclScope(fieldDeclaration.isStatic, variableDeclaration.getNameAsString)
val assignmentAsts = assignmentsForVarDecl(variableDeclaration :: Nil)
val assignmentAsts = astsForVariableDeclarator(variableDeclaration, fieldDeclaration)
scope.popFieldDeclScope()
assignmentAsts
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ private[declarations] trait AstForTypeDeclsCreator { this: AstCreator =>
staticFields.flatMap { field =>
field.getVariables.asScala.toList.flatMap { variable =>
scope.pushFieldDeclScope(isStatic = true, name = variable.getNameAsString)
val assignment = assignmentsForVarDecl(variable :: Nil)
val assignment = astsForVariableDeclarator(variable, field)
scope.popFieldDeclScope()
assignment
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -210,8 +210,14 @@ trait AstForCallExpressionsCreator { this: AstCreator =>
val blockAst =
blockAstForConstructorInvocation(line(expr), column(expr), allocNode, initCall, argumentAsts, isInnerType)

val parentIsSimpleAssign = expr.getParentNode.toScala
.collect { case assignExpr: AssignExpr =>
assignExpr.getTarget.isInstanceOf[NameExpr]
}
.getOrElse(false)

expr.getParentNode.toScala match {
case Some(parent) if parent.isInstanceOf[VariableDeclarator] || parent.isInstanceOf[AssignExpr] =>
case Some(parent) if parent.isInstanceOf[VariableDeclarator] || parentIsSimpleAssign =>
val partialConstructor = PartialConstructor(typeFullName, initCall, argumentAsts, blockAst)
partialConstructorQueue.append(partialConstructor)
Ast(allocNode)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ trait AstForExpressionsCreator
case x: SuperExpr => Seq(astForSuperExpr(x, expectedType))
case x: ThisExpr => Seq(astForThisExpr(x, expectedType))
case x: UnaryExpr => Seq(astForUnaryExpr(x, expectedType))
case x: VariableDeclarationExpr => astsForVariableDecl(x)
case x: VariableDeclarationExpr => astsForVariableDeclarationExpr(x)
case x => Seq(unknownAst(x))
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import io.joern.javasrc2cpg.scope.Scope.{
NotInScope,
ScopeMember,
ScopeParameter,
ScopeStaticImport,
ScopeVariable,
SimpleVariable
}
Expand Down Expand Up @@ -51,11 +50,6 @@ trait AstForNameExpressionsCreator { this: AstCreator =>
column(nameExpr)
)

case SimpleVariable(variable: ScopeStaticImport) =>
val targetName = variable.typeFullName.stripSuffix(s".${variable.name}")
val typeFullName = expressionReturnTypeFullName(nameExpr).map(typeInfoCalc.registerType)
fieldAccessAst(targetName, Some(targetName), variable.name, typeFullName, line(nameExpr), column(nameExpr))

case SimpleVariable(variable) =>
val identifier = identifierNode(nameExpr, name, name, typeFullName.getOrElse(TypeConstants.Any))
val captured = variable.node match {
Expand Down
Original file line number Diff line number Diff line change
@@ -1,20 +1,29 @@
package io.joern.javasrc2cpg.astcreation.expressions

import com.github.javaparser.ast.Node
import com.github.javaparser.ast.body.VariableDeclarator
import com.github.javaparser.ast.expr.AssignExpr.Operator
import com.github.javaparser.ast.expr.{AssignExpr, VariableDeclarationExpr}
import com.github.javaparser.ast.expr.{AssignExpr, Expression, VariableDeclarationExpr}
import com.github.javaparser.resolution.types.ResolvedType
import io.joern.javasrc2cpg.astcreation.expressions.AstForCallExpressionsCreator.PartialConstructor
import io.joern.javasrc2cpg.astcreation.{AstCreator, ExpectedType}
import io.joern.javasrc2cpg.scope.Scope.{ScopeMember, ScopeParameter, ScopeStaticImport, SimpleVariable}
import io.joern.javasrc2cpg.scope.Scope.{
NewVariableNode,
ScopeLocal,
ScopeMember,
ScopeParameter,
ScopeVariable,
SimpleVariable,
newVariableNodeType
}
import io.joern.javasrc2cpg.typesolvers.TypeInfoCalculator.TypeConstants
import io.joern.javasrc2cpg.util.NameConstants
import io.joern.x2cpg.passes.frontend.TypeNodePass
import io.joern.x2cpg.utils.AstPropertiesUtil.*
import io.joern.x2cpg.utils.NodeBuilders.newOperatorCallNode
import io.joern.x2cpg.{Ast, Defines}
import io.shiftleft.codepropertygraph.generated.nodes.{NewCall, NewFieldIdentifier, NewIdentifier, NewLocal}
import io.shiftleft.codepropertygraph.generated.{EdgeTypes, Operators}
import io.shiftleft.codepropertygraph.generated.nodes.{NewCall, NewFieldIdentifier, NewIdentifier, NewLocal, NewMember}
import io.shiftleft.codepropertygraph.generated.{DispatchTypes, EdgeTypes, Operators}
import org.slf4j.LoggerFactory

import scala.jdk.CollectionConverters.*
Expand All @@ -25,7 +34,7 @@ import io.joern.javasrc2cpg.scope.JavaScopeElement.PartialInit
trait AstForVarDeclAndAssignsCreator { this: AstCreator =>
private val logger = LoggerFactory.getLogger(this.getClass())

private[expressions] def astsForAssignExpr(expr: AssignExpr, expectedExprType: ExpectedType): Seq[Ast] = {
private[expressions] def astsForAssignExpr(expr: AssignExpr, expectedType: ExpectedType): Seq[Ast] = {
val operatorName = expr.getOperator match {
case Operator.ASSIGN => Operators.assignment
case Operator.PLUS => Operators.assignmentPlus
Expand All @@ -42,150 +51,24 @@ trait AstForVarDeclAndAssignsCreator { this: AstCreator =>
}

val maybeResolvedType = Try(expr.getTarget.calculateResolvedType()).toOption
val expectedType = maybeResolvedType
val expectedInitializerType = maybeResolvedType
.map { resolvedType =>
ExpectedType(typeInfoCalc.fullName(resolvedType), Some(resolvedType))
}
.getOrElse(expectedExprType) // resolved target type should be more accurate
val targetAst = astsForExpression(expr.getTarget, expectedType)
val argsAsts = astsForExpression(expr.getValue, expectedType)
val valueType = argsAsts.headOption.flatMap(_.rootType)

val typeFullName =
targetAst.headOption
.flatMap(_.rootType)
.orElse(valueType)
.orElse(expectedType.fullName)
.getOrElse(TypeConstants.Any)

val code = s"${targetAst.rootCodeOrEmpty} ${expr.getOperator.asString} ${argsAsts.rootCodeOrEmpty}"

val callNode = newOperatorCallNode(operatorName, code, Some(typeFullName), line(expr), column(expr))

if (partialConstructorQueue.isEmpty) {
val assignAst = callAst(callNode, targetAst ++ argsAsts)
Seq(assignAst)
} else {
if (partialConstructorQueue.size > 1) {
logger.warn("BUG: Received multiple partial constructors from assignment. Dropping all but the first.")
}
val partialConstructor = partialConstructorQueue.head
partialConstructorQueue.clear()

targetAst.flatMap(_.root).toList match {
case List(identifier: NewIdentifier) =>
// In this case we have a simple assign. No block needed.
// e.g. Foo f = new Foo();
val initAst = completeInitForConstructor(partialConstructor, Ast(identifier.copy))
Seq(callAst(callNode, targetAst ++ argsAsts), initAst)

case _ =>
// In this case the left hand side is more complex than an identifier, so
// we need to contain the constructor in a block.
// e.g. items[10] = new Foo();
val valueAst = partialConstructor.blockAst
Seq(callAst(callNode, targetAst ++ Seq(valueAst)))
}
}
}

private[expressions] def astsForVariableDecl(varDecl: VariableDeclarationExpr): Seq[Ast] = {
val locals = localsForVarDecl(varDecl)
val localAsts = locals.map { Ast(_) }

locals.foreach { local =>
scope.enclosingBlock.get.addLocal(local)
}

val assignments =
assignmentsForVarDecl(varDecl.getVariables.asScala)

localAsts ++ assignments
}

private def localsForVarDecl(varDecl: VariableDeclarationExpr): List[NewLocal] = {
varDecl.getVariables.asScala.map { variable =>
val name = variable.getName.toString
val typeFullName =
tryWithSafeStackOverflow(
scope
.lookupType(variable.getTypeAsString)
.orElse(typeInfoCalc.fullName(variable.getType))
).toOption.flatten.getOrElse(TypeConstants.Any)
val code = s"${variable.getType} $name"
NewLocal()
.name(name)
.code(code)
.typeFullName(typeFullName)
.lineNumber(line(varDecl))
.columnNumber(column(varDecl))
}.toList
}

def assignmentsForVarDecl(variables: Iterable[VariableDeclarator]): Seq[Ast] = {
val variablesWithInitializers =
variables.filter(_.getInitializer.toScala.isDefined)
val assignments = variablesWithInitializers.flatMap { variable =>
val name = variable.getName.toString
val initializer = variable.getInitializer.toScala.get // Won't crash because of filter
val javaParserVarType = variable.getTypeAsString
val typeFullName =
scope
.lookupType(javaParserVarType, includeWildcards = false)
.orElse(tryWithSafeStackOverflow(typeInfoCalc.fullName(variable.getType)).toOption.flatten)
// TODO: Surely the variable being declared can't already be in scope?
.orElse(scope.lookupVariable(name).typeFullName)

// Need the actual resolvedType here for when the RHS is a lambda expression.
val resolvedExpectedType =
tryWithSafeStackOverflow(symbolSolver.toResolvedType(variable.getType, classOf[ResolvedType])).toOption
val initializerAsts = astsForExpression(initializer, ExpectedType(typeFullName, resolvedExpectedType))

val typeName = typeFullName
.map(TypeNodePass.fullToShortName)
.getOrElse(s"${Defines.UnresolvedNamespace}.${variable.getTypeAsString}")
val code = s"$typeName $name = ${initializerAsts.rootCodeOrEmpty}"

val callNode = newOperatorCallNode(Operators.assignment, code, typeFullName, line(variable), column(variable))

val targetAst = scope.lookupVariable(name).getVariable() match {
case Some(member: ScopeMember) =>
val baseName =
if (member.isStatic)
scope.enclosingTypeDecl.name.getOrElse(NameConstants.Unknown)
else
NameConstants.This

val thisType = scope.enclosingTypeDecl.fullName.getOrElse(TypeConstants.Any)
fieldAccessAst(
baseName,
scope.enclosingTypeDecl.fullName,
name,
Some(member.typeFullName),
line(variable),
column(variable)
)

case Some(staticImport: ScopeStaticImport) =>
val targetName = staticImport.typeFullName.stripSuffix(s".${staticImport.name}")
val fieldName = staticImport.name
fieldAccessAst(targetName, Some(targetName), fieldName, typeFullName, line(variable), column(variable))

case maybeCorrespNode =>
val identifier = identifierNode(variable, name, name, typeFullName.getOrElse(TypeConstants.Any))
Ast(identifier).withRefEdges(identifier, maybeCorrespNode.map(_.node).toList)
}

val declAst = callAst(callNode, Seq(targetAst) ++ initializerAsts)

// Since all partial constructors will be dealt with here, don't pass them up.
val constructorAsts = partialConstructorQueue.map(completeInitForConstructor(_, copyAstForVarDeclInit(targetAst)))
partialConstructorQueue.clear()

Seq(declAst) ++ constructorAsts
}

assignments.toList
.getOrElse(expectedType) // resolved target type should be more accurate

// TODO What to do if target is empty?
val targetAst = astsForExpression(expr.getTarget, expectedInitializerType).head

astsForAssignment(
expr,
targetAst,
expr.getValue,
operatorName,
expr.getOperator.asString,
expectedInitializerType,
varDeclType = None
)
}

private[expressions] def completeInitForConstructor(partialConstructor: PartialConstructor, targetAst: Ast): Ast = {
Expand Down Expand Up @@ -242,4 +125,102 @@ trait AstForVarDeclAndAssignsCreator { this: AstCreator =>
Ast()
}
}

private[expressions] def astsForVariableDeclarationExpr(
variableDeclarationExpr: VariableDeclarationExpr
): Seq[Ast] = {
val variableDeclaratorAsts = variableDeclarationExpr.getVariables.asScala
.map(astsForVariableDeclarator(_, variableDeclarationExpr))
.toList
variableDeclaratorAsts.flatMap(_.headOption) ++ variableDeclaratorAsts.flatMap(_.drop(1))
}

def astsForVariableDeclarator(variableDeclarator: VariableDeclarator, originNode: Node): Seq[Ast] = {
val typeFullName = tryWithSafeStackOverflow(
scope
.lookupType(variableDeclarator.getTypeAsString, includeWildcards = false)
.orElse(typeInfoCalc.fullName(variableDeclarator.getType))
).toOption.flatten

val (correspondingNode, localAst): (NewVariableNode, Option[Ast]) =
scope.lookupVariable(variableDeclarator.getNameAsString).variableNode.map((_, None)).getOrElse {
val localCode = s"${variableDeclarator.getTypeAsString} ${variableDeclarator.getNameAsString}"
val local =
localNode(
originNode,
variableDeclarator.getNameAsString,
localCode,
typeFullName.getOrElse(TypeConstants.Any)
)

scope.enclosingBlock.foreach(_.addLocal(local))

(local, Some(Ast(local)))
}

val assignmentTarget = correspondingNode match {
case member: NewMember =>
val name =
if (scope.isEnclosingScopeStatic)
scope.enclosingTypeDecl.map(_.typeDecl.name).getOrElse(NameConstants.Unknown)
else NameConstants.This
fieldAccessAst(
name,
scope.enclosingTypeDecl.fullName,
correspondingNode.name,
Some(newVariableNodeType(correspondingNode)),
line(originNode),
column(originNode)
)

case variable =>
val node = identifierNode(variableDeclarator, variable.name, variable.name, newVariableNodeType(variable))
Ast(node).withRefEdge(node, correspondingNode)
}

val assignmentAsts = variableDeclarator.getInitializer.toScala.toList.flatMap { initializer =>

val expectedType =
tryWithSafeStackOverflow(
symbolSolver.toResolvedType(variableDeclarator.getType, classOf[ResolvedType])
).toOption

astsForAssignment(
variableDeclarator,
assignmentTarget,
initializer,
Operators.assignment,
"=",
ExpectedType(typeFullName, expectedType),
Some(variableDeclarator.getTypeAsString)
)
}

localAst.toList ++ assignmentAsts
}

private def astsForAssignment(
node: Node,
target: Ast,
initializer: Expression,
operatorName: String,
symbol: String,
expectedType: ExpectedType,
varDeclType: Option[String]
): List[Ast] = {
val codeTypePrefix = varDeclType.map(_ + " ").getOrElse("")
val code = s"$codeTypePrefix${target.rootCodeOrEmpty} $symbol ${initializer.toString}"
val assignmentNode = callNode(node, code, operatorName, operatorName, DispatchTypes.STATIC_DISPATCH)

target.rootType.foreach(assignmentNode.typeFullName(_))

val initializerAsts = astsForExpression(initializer, expectedType)

val constructorAsts = partialConstructorQueue.map(completeInitForConstructor(_, copyAstForVarDeclInit(target)))
partialConstructorQueue.clear()

val assignmentAst = callAst(assignmentNode, target :: initializerAsts.toList)

assignmentAst :: constructorAsts.toList
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -75,10 +75,6 @@ object JavaScopeElement {

class NamespaceScope(val namespace: NewNamespaceBlock) extends JavaScopeElement with TypeDeclContainer {
val isStatic = false

def addStaticImport(staticImport: NewImport): Unit = {
addVariableToScope(ScopeStaticImport(staticImport))
}
}

class BlockScope extends JavaScopeElement {
Expand Down
Loading

0 comments on commit be077da

Please sign in to comment.