Skip to content

Commit

Permalink
[gosrc2cpg] - Fix for unwanted error because of malformed CPG creation (
Browse files Browse the repository at this point in the history
joernio#3930)

1. It seems creating NewMethodNode directly was causing this issue. As
the `methodNode()` from `NodeBuilder` was also setting some offset. Not
sure what exactly it does. So tried creating the Dummy `ParserNodeInfo`
object and used it to call `NodeBuilder` methods.
2. Also refactored the code to use class instance of GoGlobal instead of
Singleton object, which required to clear the state before each unit
test execution.
  • Loading branch information
pandurangpatil authored Dec 11, 2023
1 parent 216c88a commit c742637
Show file tree
Hide file tree
Showing 17 changed files with 64 additions and 90 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -28,18 +28,19 @@ class GoSrc2Cpg extends X2CpgFrontend[Config] {
def createCpg(config: Config): Try[Cpg] = {
withNewEmptyCpg(config.outputPath, config) { (cpg, config) =>
File.usingTemporaryDirectory("gosrc2cpgOut") { tmpDir =>
val goGlobal = GoGlobal()
new MetaDataPass(cpg, Languages.GOLANG, config.inputPath).createAndApply()
val astGenResult = new AstGenRunner(config).execute(tmpDir).asInstanceOf[GoAstGenRunnerResult]
val goMod = new GoModHelper(
Some(config),
astGenResult.parsedModFile.flatMap(modFile => GoAstJsonParser.readModFile(Paths.get(modFile)).map(x => x))
)
if config.fetchDependencies then new DownloadDependenciesPass(goMod).process()
if config.fetchDependencies then new DownloadDependenciesPass(goMod, goGlobal).process()
val astCreators =
new MethodAndTypeCacheBuilderPass(Some(cpg), astGenResult.parsedFiles, config, goMod).process()
new AstCreationPass(cpg, astCreators, config, report).createAndApply()
if GoGlobal.pkgLevelVarAndConstantAstMap.size() > 0 then
new PackageCtorCreationPass(cpg, config).createAndApply()
new MethodAndTypeCacheBuilderPass(Some(cpg), astGenResult.parsedFiles, config, goMod, goGlobal).process()
new AstCreationPass(cpg, astCreators, report).createAndApply()
if goGlobal.pkgLevelVarAndConstantAstMap.size() > 0 then
new PackageCtorCreationPass(cpg, config, goGlobal).createAndApply()
report.print()
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
package io.joern.gosrc2cpg.astcreation

import io.joern.gosrc2cpg.datastructures.GoGlobal
import io.joern.gosrc2cpg.model.GoModHelper
import io.joern.gosrc2cpg.parser.ParserAst.*
import io.joern.gosrc2cpg.parser.{ParserKeys, ParserNodeInfo}
Expand All @@ -16,9 +17,13 @@ import ujson.Value

import scala.collection.mutable

class AstCreator(val relPathFileName: String, val parserResult: ParserResult, goMod: GoModHelper)(implicit
withSchemaValidation: ValidationMode
) extends AstCreatorBase(relPathFileName)
class AstCreator(
val relPathFileName: String,
val parserResult: ParserResult,
val goMod: GoModHelper,
val goGlobal: GoGlobal
)(implicit withSchemaValidation: ValidationMode)
extends AstCreatorBase(relPathFileName)
with AstCreatorHelper
with AstForGenDeclarationCreator
with AstForExpressionCreator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ trait AstCreatorHelper { this: AstCreator =>
}

protected def resolveAliasToFullName(alias: String, typeOrMethodName: String): String = {
s"${aliasToNameSpaceMapping.getOrElse(alias, GoGlobal.aliasToNameSpaceMapping.getOrDefault(alias, s"${XDefines.Unknown}.<$alias>"))}.$typeOrMethodName"
s"${aliasToNameSpaceMapping.getOrElse(alias, goGlobal.aliasToNameSpaceMapping.getOrDefault(alias, s"${XDefines.Unknown}.<$alias>"))}.$typeOrMethodName"
}
protected def generateTypeFullName(
typeName: Option[String] = None,
Expand Down Expand Up @@ -247,12 +247,6 @@ trait AstCreatorHelper { this: AstCreator =>
}
}

protected def registerType(typeName: String): String = {
val fixedTypeName = fixQualifiedName(StringUtils.normalizeSpace(typeName))
GoGlobal.usedTypes.putIfAbsent(fixedTypeName, true)
fixedTypeName
}

protected def fixQualifiedName(name: String): String =
name.stripPrefix(Defines.qualifiedNameSeparator).replace(Defines.qualifiedNameSeparator, ".")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,7 +144,7 @@ trait AstForGenDeclarationCreator(implicit withSchemaValidation: ValidationMode)
None,
Some(typeFullName)
)
GoGlobal.recordPkgLevelVarAndConstantAst(fullyQualifiedPackage, callAst(cNode, arguments), relPathFileName)
goGlobal.recordPkgLevelVarAndConstantAst(fullyQualifiedPackage, callAst(cNode, arguments), relPathFileName)
case _ =>

}
Expand All @@ -160,7 +160,7 @@ trait AstForGenDeclarationCreator(implicit withSchemaValidation: ValidationMode)
val node = localNode(localParserNode, name, localParserNode.code, typeFullNameStr)

if recordVar then
GoGlobal.recordStructTypeMemberType(s"$fullyQualifiedPackage${Defines.dot}$name", typeFullNameStr)
goGlobal.recordStructTypeMemberType(s"$fullyQualifiedPackage${Defines.dot}$name", typeFullNameStr)
else scope.addToScope(name, (node, typeFullNameStr))
Ast(node)
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ trait AstForLambdaCreator(implicit withSchemaValidation: ValidationMode) { this:
if baseFullName == fullyQualifiedPackage then
typeDeclNode_.astParentType(NodeTypes.TYPE_DECL).astParentFullName(fullyQualifiedPackage)
else typeDeclNode_.astParentType(NodeTypes.METHOD).astParentFullName(baseFullName)
val structTypes = Option(GoGlobal.lambdaSignatureToLambdaTypeMap.get(signature)) match {
val structTypes = Option(goGlobal.lambdaSignatureToLambdaTypeMap.get(signature)) match {
case Some(types) => types.map(_._1)
case None => Seq.empty
}
Expand All @@ -49,7 +49,7 @@ trait AstForLambdaCreator(implicit withSchemaValidation: ValidationMode) { this:
methodNode_.astParentType(NodeTypes.TYPE_DECL)
methodNode_.astParentFullName(fullName)
Ast.storeInDiffGraph(astForMethod, diffGraph)
GoGlobal.recordFullNameToReturnType(fullName, returnTypeStr, signature)
goGlobal.recordFullNameToReturnType(fullName, returnTypeStr, signature)
Seq(Ast(methodRefNode(funcLiteral, funcLiteral.code, fullName, fullName)))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,15 +96,15 @@ trait AstForMethodCallExpressionCreator(implicit withSchemaValidation: Validatio
// This assumption will be invalid when another package is imported with alias "."
val methodFullName = s"$fullyQualifiedPackage.$methodName"
val (returnTypeFullNameCache, signatureCache) =
GoGlobal.methodFullNameReturnTypeMap
goGlobal.methodFullNameReturnTypeMap
.getOrDefault(methodFullName, (Defines.anyTypeName, s"$methodFullName()"))
val (signature, fullName, returnTypeFullName) =
Defines.builtinFunctions.getOrElse(methodName, (signatureCache, methodFullName, returnTypeFullNameCache))
val lambdaOption = scope.lookupVariable(methodName)
val (postLambdaFullname, postLambdaSignature, postLambdaReturnTypeFullName) = lambdaOption match
case Some((_, lambdaTypeFullName)) =>
val (lambdaReturnTypeFullNameCache, lambdaSignatureCache) =
GoGlobal.methodFullNameReturnTypeMap
goGlobal.methodFullNameReturnTypeMap
.getOrDefault(lambdaTypeFullName, (returnTypeFullName, signature))
if (lambdaSignatureCache == signature) then
// This means we didn't find the lambda signature in methodFullNameReturnTypeMap cache.
Expand All @@ -126,7 +126,7 @@ trait AstForMethodCallExpressionCreator(implicit withSchemaValidation: Validatio
val callMethodFullName =
resolveAliasToFullName(alias, methodName)
val (returnTypeFullNameCache, signatureCache) =
GoGlobal.methodFullNameReturnTypeMap
goGlobal.methodFullNameReturnTypeMap
.getOrDefault(
callMethodFullName,
(s"$callMethodFullName.${Defines.ReturnType}.${XDefines.Unknown}", s"$callMethodFullName()")
Expand All @@ -152,7 +152,7 @@ trait AstForMethodCallExpressionCreator(implicit withSchemaValidation: Validatio
.stripPrefix("*")
val callMethodFullName = s"$receiverTypeFullName.$methodName"
val (returnTypeFullNameCache, signatureCache) =
GoGlobal.methodFullNameReturnTypeMap
goGlobal.methodFullNameReturnTypeMap
.getOrDefault(
callMethodFullName,
(s"$receiverTypeFullName.$methodName.${Defines.ReturnType}.${XDefines.Unknown}", s"$callMethodFullName()")
Expand Down
Original file line number Diff line number Diff line change
@@ -1,49 +1,30 @@
package io.joern.gosrc2cpg.astcreation

import io.joern.gosrc2cpg.parser.ParserAst.Unknown
import io.joern.gosrc2cpg.parser.ParserNodeInfo
import io.joern.x2cpg.astgen.AstGenNodeBuilder
import io.joern.x2cpg.{Ast, AstCreatorBase, ValidationMode, Defines as XDefines}
import io.shiftleft.codepropertygraph.generated.{EvaluationStrategies, NodeTypes}
import io.shiftleft.codepropertygraph.generated.nodes.{NewBlock, NewMethod, NewMethodReturn}
import io.shiftleft.codepropertygraph.generated.NodeTypes
import org.apache.commons.lang.StringUtils
import overflowdb.BatchedUpdate.DiffGraphBuilder
import ujson.Value

import scala.collection.immutable.Set

class AstForPackageConstructorCreator(val pacakgePath: String, statements: Set[(Ast, String)])(implicit
withSchemaValidation: ValidationMode
) extends AstCreatorBase(pacakgePath) {
) extends AstCreatorBase(pacakgePath)
with AstGenNodeBuilder[AstForPackageConstructorCreator] {

override def createAst(): DiffGraphBuilder = {
val name = StringUtils.normalizeSpace(s"$pacakgePath${XDefines.StaticInitMethodName}")
val fakeGlobalMethod =
NewMethod()
.name(name)
.code(name)
.fullName(name)
.filename(pacakgePath)
.astParentType(NodeTypes.TYPE_DECL)
.astParentFullName(pacakgePath)
.isExternal(false)
.lineNumber(0)
.columnNumber(0)
.lineNumberEnd(0)
.columnNumberEnd(0)

val blockNode_ = NewBlock()
.code(name)
.typeFullName(Defines.voidTypeName)
.lineNumber(0)
.columnNumber(0)

val declsAsts = statements.map(_._1).toList
val name = StringUtils.normalizeSpace(s"$pacakgePath${XDefines.StaticInitMethodName}")
val node = ParserNodeInfo(Unknown, Value("{}"), name, Some(0), Some(0), Some(0), Some(0))
val ctorMethod = methodNode(node, name, name, name, None, pacakgePath, Some(NodeTypes.TYPE_DECL), Some(pacakgePath))
val blockNode_ = blockNode(node, Defines.empty, Defines.voidTypeName)
val declsAsts = statements.map(_._1).toList
setArgumentIndices(declsAsts)

val methodReturn = NewMethodReturn()
.typeFullName(Defines.voidTypeName)
.code("RET")
.evaluationStrategy(EvaluationStrategies.BY_VALUE)
.lineNumber(0)
.columnNumber(0)
val ctorAst = methodAst(fakeGlobalMethod, Seq.empty, blockAst(blockNode_, declsAsts), methodReturn)
val methodReturn = methodReturnNode(node, Defines.anyTypeName)
val ctorAst = methodAst(ctorMethod, Seq.empty, blockAst(blockNode_, declsAsts), methodReturn)
Ast.storeInDiffGraph(ctorAst, diffGraph)
diffGraph
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ trait AstForPrimitivesCreator(implicit withSchemaValidation: ValidationMode) { t
Ast(node).withRefEdge(node, variable)
case _ =>
// If its not local node then check if its global member variable of package TypeDecl
Option(GoGlobal.structTypeMemberTypeMapping.get(s"$fullyQualifiedPackage${Defines.dot}$identifierName")) match
Option(goGlobal.structTypeMemberTypeMapping.get(s"$fullyQualifiedPackage${Defines.dot}$identifierName")) match
case Some(fieldTypeFullName) => astForPackageGlobalFieldAccess(fieldTypeFullName, identifierName, ident)
case _ =>
// TODO: something is wrong here. Refer to SwitchTests -> "be correct for switch case 4"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ trait AstForStatementsCreator(implicit withSchemaValidation: ValidationMode) { t
val code = s"for ${initParserNode.code};${condParserNode.code};${iterParserNode.code}"
val forNode = controlStructureNode(forStmt, ControlStructureTypes.FOR, code)

val initAstBlock = blockNode(forStmt, Defines.empty, registerType(Defines.voidTypeName))
val initAstBlock = blockNode(forStmt, Defines.empty, Defines.voidTypeName)
scope.pushNewScope(initAstBlock)
val initAst = blockAst(initAstBlock, astsForStatement(initParserNode, 1).toList)
scope.popScope()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ trait AstForTypeDeclCreator(implicit withSchemaValidation: ValidationMode) { thi

protected def processFuncType(typeNode: ParserNodeInfo, typeDeclFullName: String): Seq[Ast] = {
val (signature, returnTypeFullName, _, _, _) = generateLambdaSignature(typeNode)
GoGlobal.recordLambdaSigntureToLambdaType(signature, typeDeclFullName, returnTypeFullName)
goGlobal.recordLambdaSigntureToLambdaType(signature, typeDeclFullName, returnTypeFullName)
Seq.empty
}

Expand All @@ -39,7 +39,7 @@ trait AstForTypeDeclCreator(implicit withSchemaValidation: ValidationMode) { thi
.map(fieldInfo => {
val fieldNodeInfo = createParserNodeInfo(fieldInfo)
val fieldName = fieldNodeInfo.json(ParserKeys.Name).str
GoGlobal.recordStructTypeMemberType(typeDeclFullName + Defines.dot + fieldName, typeFullName)
goGlobal.recordStructTypeMemberType(typeDeclFullName + Defines.dot + fieldName, typeFullName)
Ast(memberNode(typeInfo, fieldName, fieldNodeInfo.code, typeFullName))
})
})
Expand All @@ -62,7 +62,7 @@ trait AstForTypeDeclCreator(implicit withSchemaValidation: ValidationMode) { thi
val receiverFullName = resolveAliasToFullName(alias, fieldIdentifier)
(
astForNode(xnode),
GoGlobal.structTypeMemberTypeMapping.getOrDefault(
goGlobal.structTypeMemberTypeMapping.getOrDefault(
receiverFullName,
s"$receiverFullName${Defines.dot}${Defines.FieldAccess}${Defines.dot}${XDefines.Unknown}"
)
Expand All @@ -75,7 +75,7 @@ trait AstForTypeDeclCreator(implicit withSchemaValidation: ValidationMode) { thi
private def receiverAstAndFullName(xnode: ParserNodeInfo, fieldIdentifier: String): (Seq[Ast], String) = {
val identifierAsts = astForNode(xnode)
val receiverTypeFullName = getTypeFullNameFromAstNode(identifierAsts)
val fieldTypeFullName = GoGlobal.structTypeMemberTypeMapping.getOrDefault(
val fieldTypeFullName = goGlobal.structTypeMemberTypeMapping.getOrDefault(
s"$receiverTypeFullName${Defines.dot}$fieldIdentifier",
s"$receiverTypeFullName${Defines.dot}$fieldIdentifier${Defines.dot}${Defines.FieldAccess}${Defines.dot}${XDefines.Unknown}"
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ trait CacheBuilder(implicit withSchemaValidation: ValidationMode) { this: AstCre

cpgOpt.map { _ =>
// We don't want to process this part when third party dependencies are being processed.
val result = GoGlobal.recordAliasToNamespaceMapping(declaredPackageName, fullyQualifiedPackage)
val result = goGlobal.recordAliasToNamespaceMapping(declaredPackageName, fullyQualifiedPackage)
if (result == null) {
// if result is null that means item got added first time otherwise it has been already added to global map
val rootNode = createParserNodeInfo(parserResult.json)
Expand Down Expand Up @@ -163,7 +163,7 @@ trait CacheBuilder(implicit withSchemaValidation: ValidationMode) { this: AstCre
val params = funcDeclVal(ParserKeys.Type)(ParserKeys.Params)(ParserKeys.List)
val signature =
s"$methodFullname(${parameterSignature(params, genericTypeMethodMap)})$returnTypeStr"
GoGlobal.recordFullNameToReturnType(methodFullname, returnTypeStr, signature)
goGlobal.recordFullNameToReturnType(methodFullname, returnTypeStr, signature)
(name, methodFullname, signature, params, receiverInfo, genericTypeMethodMap)
}
}
Original file line number Diff line number Diff line change
@@ -1,12 +1,10 @@
package io.joern.gosrc2cpg.datastructures

import io.joern.gosrc2cpg.astcreation.Defines
import io.joern.x2cpg.datastructures.Global
import io.joern.x2cpg.Ast

import java.util.concurrent.ConcurrentHashMap
import scala.jdk.CollectionConverters.EnumerationHasAsScala

object GoGlobal extends Global {
class GoGlobal {

/** This map will only contain the mapping for those packages whose package name is different from the enclosing
* folder name
Expand Down Expand Up @@ -86,11 +84,4 @@ object GoGlobal extends Global {
}
}
}

def typesSeen(): List[String] = {
val types = usedTypes.keys().asScala.filterNot(_ == Defines.anyTypeName).toList
usedTypes.clear()
types
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ package io.joern.gosrc2cpg.passes

import io.joern.gosrc2cpg.Config
import io.joern.gosrc2cpg.astcreation.AstCreator
import io.joern.gosrc2cpg.datastructures.GoGlobal
import io.joern.gosrc2cpg.parser.GoAstJsonParser
import io.joern.x2cpg.astgen.ParserResult
import io.joern.x2cpg.SourceFiles
Expand All @@ -14,7 +15,7 @@ import org.slf4j.{Logger, LoggerFactory}
import java.nio.file.Paths
import scala.util.{Failure, Success, Try}

class AstCreationPass(cpg: Cpg, astCreators: Seq[AstCreator], config: Config, report: Report)
class AstCreationPass(cpg: Cpg, astCreators: Seq[AstCreator], report: Report)
extends ConcurrentWriterCpgPass[AstCreator](cpg) {
private val logger: Logger = LoggerFactory.getLogger(classOf[AstCreationPass])
override def generateParts(): Array[AstCreator] = astCreators.toArray
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ package io.joern.gosrc2cpg.passes

import better.files.File
import io.joern.gosrc2cpg.Config
import io.joern.gosrc2cpg.datastructures.GoGlobal
import io.joern.gosrc2cpg.model.GoModHelper
import io.joern.gosrc2cpg.parser.GoAstJsonParser
import io.joern.gosrc2cpg.utils.AstGenRunner
Expand All @@ -13,7 +14,7 @@ import java.io.File as JFile
import java.nio.file.Paths
import scala.util.{Failure, Success, Try}

class DownloadDependenciesPass(parentGoMod: GoModHelper) {
class DownloadDependenciesPass(parentGoMod: GoModHelper, goGlobal: GoGlobal) {
private val logger = LoggerFactory.getLogger(getClass)
def process(): Unit = {
File.usingTemporaryDirectory("go-temp-download") { tmpDir =>
Expand Down Expand Up @@ -52,7 +53,7 @@ class DownloadDependenciesPass(parentGoMod: GoModHelper) {
Some(config),
astGenResult.parsedModFile.flatMap(modFile => GoAstJsonParser.readModFile(Paths.get(modFile)).map(x => x))
)
new MethodAndTypeCacheBuilderPass(None, astGenResult.parsedFiles, config, goMod).process()
new MethodAndTypeCacheBuilderPass(None, astGenResult.parsedFiles, config, goMod, goGlobal).process()
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ package io.joern.gosrc2cpg.passes

import io.joern.gosrc2cpg.Config
import io.joern.gosrc2cpg.astcreation.AstCreator
import io.joern.gosrc2cpg.datastructures.GoGlobal
import io.joern.gosrc2cpg.model.GoModHelper
import io.joern.gosrc2cpg.parser.GoAstJsonParser
import io.joern.x2cpg.SourceFiles
Expand All @@ -13,14 +14,20 @@ import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration.Duration
import scala.concurrent.{Await, Future}

class MethodAndTypeCacheBuilderPass(cpgOpt: Option[Cpg], astFiles: List[String], config: Config, goMod: GoModHelper) {
class MethodAndTypeCacheBuilderPass(
cpgOpt: Option[Cpg],
astFiles: List[String],
config: Config,
goMod: GoModHelper,
goGlobal: GoGlobal
) {
def process(): Seq[AstCreator] = {
val futures = astFiles
.map(file => {
Future {
val parserResult = GoAstJsonParser.readFile(Paths.get(file))
val relPathFileName = SourceFiles.toRelativePath(parserResult.fullPath, config.inputPath)
val astCreator = new AstCreator(relPathFileName, parserResult, goMod)(config.schemaValidation)
val astCreator = new AstCreator(relPathFileName, parserResult, goMod, goGlobal)(config.schemaValidation)
val diffGraph = astCreator.buildCache(cpgOpt)
(astCreator, diffGraph)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,13 +9,13 @@ import io.shiftleft.passes.ConcurrentWriterCpgPass

import scala.jdk.CollectionConverters.*

class PackageCtorCreationPass(cpg: Cpg, config: Config)
class PackageCtorCreationPass(cpg: Cpg, config: Config, goGlobal: GoGlobal)
extends ConcurrentWriterCpgPass[(String, Set[(Ast, String)])](cpg) {
override def generateParts(): Array[(String, Set[(Ast, String)])] =
GoGlobal.pkgLevelVarAndConstantAstMap
goGlobal.pkgLevelVarAndConstantAstMap
.keys()
.asScala
.map(key => (key, GoGlobal.pkgLevelVarAndConstantAstMap.get(key)))
.map(key => (key, goGlobal.pkgLevelVarAndConstantAstMap.get(key)))
.toArray

override def runOnPart(diffGraph: DiffGraphBuilder, part: (String, Set[(Ast, String)])): Unit = {
Expand Down
Loading

0 comments on commit c742637

Please sign in to comment.