mapping generated for MAG,

missing generation of Organization Action set
This commit is contained in:
Sandro La Bruzzo 2024-04-05 18:12:53 +02:00
parent ef582948a7
commit 98dc042db5
9 changed files with 207 additions and 143 deletions

View File

@ -12,9 +12,9 @@
"paramRequired": true "paramRequired": true
}, },
{ {
"paramName": "w", "paramName": "ma",
"paramLongName": "workingPath", "paramLongName": "magBasePath",
"paramDescription": "The working path", "paramDescription": "The mag Base path",
"paramRequired": false "paramRequired": false
} }

View File

@ -10,12 +10,6 @@
"paramLongName": "magBasePath", "paramLongName": "magBasePath",
"paramDescription": "The base path of MAG DUMP CSV Tables", "paramDescription": "The base path of MAG DUMP CSV Tables",
"paramRequired": true "paramRequired": true
},
{
"paramName": "w",
"paramLongName": "workingPath",
"paramDescription": "The working path",
"paramRequired": false
} }
] ]

View File

@ -8,10 +8,6 @@
<name>mdstorePath</name> <name>mdstorePath</name>
<description>The base path of MAG DUMP CSV Tables</description> <description>The base path of MAG DUMP CSV Tables</description>
</property> </property>
<property>
<name>workingPath</name>
<description>The working path</description>
</property>
<property> <property>
<name>resume_from</name> <name>resume_from</name>
<value>generateOAF</value> <value>generateOAF</value>
@ -53,7 +49,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts> </spark-opts>
<arg>--magBasePath</arg><arg>${magBasePath}</arg> <arg>--magBasePath</arg><arg>${magBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn</arg> <arg>--master</arg><arg>yarn</arg>
</spark> </spark>
<ok to="generateOAF"/> <ok to="generateOAF"/>
@ -79,7 +74,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts> </spark-opts>
<arg>--mdstorePath</arg><arg>${mdstorePath}</arg> <arg>--mdstorePath</arg><arg>${mdstorePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--magBasePath</arg><arg>${magBasePath}</arg>
<arg>--master</arg><arg>yarn</arg> <arg>--master</arg><arg>yarn</arg>
</spark> </spark>
<ok to="End"/> <ok to="End"/>

View File

@ -952,7 +952,7 @@ case object Crossref2Oaf {
val targetId = getProjectId("hfri________", "1e5e62235d094afd01cd56e65112fc63") val targetId = getProjectId("hfri________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
//ERASMUS+ //ERASMUS+
case "10.13039/501100010790" => case "10.13039/501100010790" =>
generateSimpleRelationFromAward(funder, "erasmusplus_", a => a) generateSimpleRelationFromAward(funder, "erasmusplus_", a => a)
case _ => logger.debug("no match for " + funder.DOI.get) case _ => logger.debug("no match for " + funder.DOI.get)

View File

@ -4,7 +4,16 @@ import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.common.ModelConstants import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils._ import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils._
import eu.dnetlib.dhp.schema.oaf.utils.{OafMapperUtils, PidType} import eu.dnetlib.dhp.schema.oaf.utils.{OafMapperUtils, PidType}
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Instance, Journal, Publication, Result, Dataset => OafDataset} import eu.dnetlib.dhp.schema.oaf.{
Author,
DataInfo,
Instance,
Journal,
Publication,
Relation,
Result,
Dataset => OafDataset
}
import eu.dnetlib.dhp.utils.DHPUtils import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._
import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.{Dataset, Row, SparkSession}
@ -65,7 +74,41 @@ object MagUtility extends Serializable {
val mapper = new ObjectMapper() val mapper = new ObjectMapper()
val MAGCollectedFrom =keyValue(ModelConstants.MAG_ID,ModelConstants.MAG_NAME) private val MAGCollectedFrom = keyValue(ModelConstants.MAG_ID, ModelConstants.MAG_NAME)
private val MAGDataInfo: DataInfo = {
val di = new DataInfo
di.setDeletedbyinference(false)
di.setInferred(false)
di.setInvisible(false)
di.setTrust("0.9")
di.setProvenanceaction(
OafMapperUtils.qualifier(
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS
)
)
di
}
private val MAGDataInfoInvisible: DataInfo = {
val di = new DataInfo
di.setDeletedbyinference(false)
di.setInferred(false)
di.setInvisible(true)
di.setTrust("0.9")
di.setProvenanceaction(
OafMapperUtils.qualifier(
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS
)
)
di
}
val datatypedict = Map( val datatypedict = Map(
"bool" -> BooleanType, "bool" -> BooleanType,
@ -308,15 +351,14 @@ object MagUtility extends Serializable {
def getSchema(streamName: String): StructType = { def getSchema(streamName: String): StructType = {
var schema = new StructType() var schema = new StructType()
val d: Seq[String] = stream(streamName)._2 val d: Seq[String] = stream(streamName)._2
d.foreach { d.foreach { case t =>
case t => val currentType = t.split(":")
val currentType = t.split(":") val fieldName: String = currentType.head
val fieldName: String = currentType.head var fieldType: String = currentType.last
var fieldType: String = currentType.last val nullable: Boolean = fieldType.endsWith("?")
val nullable: Boolean = fieldType.endsWith("?") if (nullable)
if (nullable) fieldType = fieldType.replace("?", "")
fieldType = fieldType.replace("?", "") schema = schema.add(StructField(fieldName, datatypedict(fieldType), nullable))
schema = schema.add(StructField(fieldName, datatypedict(fieldType), nullable))
} }
schema schema
} }
@ -338,22 +380,10 @@ object MagUtility extends Serializable {
def createResultFromType(magType: Option[String], source: Option[String]): Result = { def createResultFromType(magType: Option[String], source: Option[String]): Result = {
var result: Result = null var result: Result = null
val di = new DataInfo
di.setDeletedbyinference(false) if (magType == null || magType.orNull == null) {
di.setInferred(false)
di.setInvisible(false)
di.setTrust("0.9")
di.setProvenanceaction(
OafMapperUtils.qualifier(
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.SYSIMPORT_ACTIONSET,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS
)
)
if (magType == null || magType.orNull ==null) {
result = new Publication result = new Publication
result.setDataInfo(di) result.setDataInfo(MAGDataInfo)
val i = new Instance val i = new Instance
i.setInstancetype( i.setInstancetype(
qualifier( qualifier(
@ -386,7 +416,7 @@ object MagUtility extends Serializable {
result = new Publication result = new Publication
qualifier("0043", "Journal", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE) qualifier("0043", "Journal", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE)
case "patent" => case "patent" =>
if (source!= null && source.orNull != null) { if (source != null && source.orNull != null) {
val s = source.get.toLowerCase val s = source.get.toLowerCase
if (s.contains("patent") || s.contains("brevet")) { if (s.contains("patent") || s.contains("brevet")) {
result = new Publication result = new Publication
@ -404,9 +434,11 @@ object MagUtility extends Serializable {
ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE ModelConstants.DNET_PUBLICATION_RESOURCE
) )
} else if (s.contains("proceedings") || s.contains("conference") || s.contains("workshop") || s.contains( } else if (
"symposium" s.contains("proceedings") || s.contains("conference") || s.contains("workshop") || s.contains(
)) { "symposium"
)
) {
result = new Publication result = new Publication
qualifier( qualifier(
"0001", "0001",
@ -419,7 +451,7 @@ object MagUtility extends Serializable {
case "repository" => case "repository" =>
result = new Publication() result = new Publication()
di.setInvisible(true) result.setDataInfo(MAGDataInfoInvisible)
qualifier( qualifier(
"0038", "0038",
"Other literature type", "Other literature type",
@ -454,7 +486,8 @@ object MagUtility extends Serializable {
} }
if (result != null) { if (result != null) {
result.setDataInfo(di) if (result.getDataInfo == null)
result.setDataInfo(MAGDataInfo)
val i = new Instance val i = new Instance
i.setInstancetype(tp) i.setInstancetype(tp)
result.setInstance(List(i).asJava) result.setInstance(List(i).asJava)
@ -466,14 +499,13 @@ object MagUtility extends Serializable {
def convertMAGtoOAF(paper: MAGPaper): String = { def convertMAGtoOAF(paper: MAGPaper): String = {
// FILTER all the MAG paper with no URL // FILTER all the MAG paper with no URL
if (paper.urls.orNull == null ) if (paper.urls.orNull == null)
return null return null
val result = createResultFromType(paper.docType, paper.originalVenue) val result = createResultFromType(paper.docType, paper.originalVenue)
if (result == null) if (result == null)
return null return null
result.setCollectedfrom(List(MAGCollectedFrom).asJava) result.setCollectedfrom(List(MAGCollectedFrom).asJava)
val pidList = List( val pidList = List(
structuredProperty( structuredProperty(
@ -582,30 +614,32 @@ object MagUtility extends Serializable {
val instance = result.getInstance().get(0) val instance = result.getInstance().get(0)
instance.setPid(pidList.asJava) instance.setPid(pidList.asJava)
if(paper.doi.orNull != null) if (paper.doi.orNull != null)
instance.setAlternateIdentifier( instance.setAlternateIdentifier(
List( List(
structuredProperty( structuredProperty(
paper.doi.get, paper.doi.get,
qualifier( qualifier(
PidType.doi.toString, PidType.doi.toString,
PidType.doi.toString, PidType.doi.toString,
ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES,
ModelConstants.DNET_PID_TYPES ModelConstants.DNET_PID_TYPES
), ),
null null
) )
).asJava ).asJava
) )
instance.setUrl(paper.urls.get.asJava) instance.setUrl(paper.urls.get.asJava)
instance.setHostedby(ModelConstants.UNKNOWN_REPOSITORY) instance.setHostedby(ModelConstants.UNKNOWN_REPOSITORY)
instance.setCollectedfrom(MAGCollectedFrom) instance.setCollectedfrom(MAGCollectedFrom)
instance.setAccessright(accessRight( instance.setAccessright(
ModelConstants.UNKNOWN, accessRight(
ModelConstants.NOT_AVAILABLE, ModelConstants.UNKNOWN,
ModelConstants.DNET_ACCESS_MODES, ModelConstants.NOT_AVAILABLE,
ModelConstants.DNET_ACCESS_MODES ModelConstants.DNET_ACCESS_MODES,
)) ModelConstants.DNET_ACCESS_MODES
)
)
if (paper.authors.orNull != null && paper.authors.get.nonEmpty) if (paper.authors.orNull != null && paper.authors.get.nonEmpty)
result.setAuthor( result.setAuthor(
@ -646,6 +680,31 @@ object MagUtility extends Serializable {
.asJava .asJava
) )
mapper.writeValueAsString(result) mapper.writeValueAsString(result)
}
def generateAffiliationRelations(paperAffiliation: Row): List[Relation] = {
val affId = s"20|mag_________::${DHPUtils.md5(paperAffiliation.getAs[Long]("AffiliationId").toString)}"
val oafId = s"50|mag_________::${DHPUtils.md5(paperAffiliation.getAs[Long]("PaperId").toString)}"
val r: Relation = new Relation
r.setSource(oafId)
r.setTarget(affId)
r.setRelType(ModelConstants.RESULT_ORGANIZATION)
r.setRelClass(ModelConstants.HAS_AUTHOR_INSTITUTION)
r.setSubRelType(ModelConstants.AFFILIATION)
r.setDataInfo(MAGDataInfo)
r.setCollectedfrom(List(MAGCollectedFrom).asJava)
val r1: Relation = new Relation
r1.setTarget(oafId)
r1.setSource(affId)
r1.setRelType(ModelConstants.RESULT_ORGANIZATION)
r1.setRelClass(ModelConstants.IS_AUTHOR_INSTITUTION_OF)
r1.setSubRelType(ModelConstants.AFFILIATION)
r1.setDataInfo(MAGDataInfo)
r1.setCollectedfrom(List(MAGCollectedFrom).asJava)
List(r, r1)
} }
def convertInvertedIndexString(json_input: String): String = { def convertInvertedIndexString(json_input: String): String = {

View File

@ -15,15 +15,12 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
override def run(): Unit = { override def run(): Unit = {
val magBasePath: String = parser.get("magBasePath") val magBasePath: String = parser.get("magBasePath")
log.info("found parameters magBasePath: {}", magBasePath) log.info("found parameters magBasePath: {}", magBasePath)
val workingPath: String = parser.get("workingPath") generatedDenormalizedMAGTable(spark, magBasePath)
log.info("found parameters workingPath: {}", workingPath)
generatedDenormalizedMAGTable(spark, magBasePath, workingPath)
} }
private def generatedDenormalizedMAGTable( private def generatedDenormalizedMAGTable(
spark: SparkSession, spark: SparkSession,
magBasePath: String, magBasePath: String
workingPath: String
): Unit = { ): Unit = {
import spark.implicits._ import spark.implicits._
@ -114,27 +111,6 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
step2.count() step2.count()
step1.unpersist() step1.unpersist()
// val fos = MagUtility
// .loadMagEntity(spark, "FieldsOfStudy", magBasePath)
// .select($"FieldOfStudyId".alias("fos"), $"DisplayName", $"MainType")
//
// val paperFieldsOfStudy = MagUtility
// .loadMagEntity(spark, "PaperFieldsOfStudy", magBasePath)
// .select($"FieldOfStudyId", $"Score", $"PaperId")
//
// val paperFoS = paperFieldsOfStudy
// .join(broadcast(fos), fos("fos") === paperFieldsOfStudy("FieldOfStudyId"))
// .groupBy("PaperId")
// .agg(collect_set(struct("FieldOfStudyId", "DisplayName", "MainType", "Score")).as("FoS"))
//
// val step3 = step2
// .join(paperFoS, step2("PaperId") === paperFoS("PaperId"), "left")
// .select(step2("*"), paperFoS("FoS"))
// .cache()
// step3.count()
//
// step2.unpersist()
val journals = MagUtility val journals = MagUtility
.loadMagEntity(spark, "Journals", magBasePath) .loadMagEntity(spark, "Journals", magBasePath)
.select( .select(
@ -204,7 +180,7 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
) )
.write .write
.mode("OverWrite") .mode("OverWrite")
.save(s"$workingPath/mag") .save(s"$magBasePath/mag_denormalized")
step3.unpersist() step3.unpersist()
} }
} }

View File

@ -1,7 +1,9 @@
package eu.dnetlib.dhp.collection.mag package eu.dnetlib.dhp.collection.mag
import eu.dnetlib.dhp.application.AbstractScalaApplication import eu.dnetlib.dhp.application.AbstractScalaApplication
import eu.dnetlib.dhp.schema.oaf.{Publication, Result} import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Result}
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructType}
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession} import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory} import org.slf4j.{Logger, LoggerFactory}
@ -14,26 +16,49 @@ class SparkMAGtoOAF(propertyPath: String, args: Array[String], log: Logger)
override def run(): Unit = { override def run(): Unit = {
val mdstorePath: String = parser.get("mdstorePath") val mdstorePath: String = parser.get("mdstorePath")
log.info("found parameters mdstorePath: {}", mdstorePath) log.info("found parameters mdstorePath: {}", mdstorePath)
val workingPath: String = parser.get("workingPath") val magBasePath: String = parser.get("magBasePath")
log.info("found parameters workingPath: {}", workingPath) log.info("found parameters magBasePath: {}", magBasePath)
convertMAG(spark, workingPath, mdstorePath) convertMAG(spark, magBasePath, mdstorePath)
generateAffiliations(spark, magBasePath, mdstorePath)
} }
def convertMAG(spark: SparkSession, workingPath: String, mdStorePath: String): Unit = { def convertMAG(spark: SparkSession, magBasePath: String, mdStorePath: String): Unit = {
import spark.implicits._ import spark.implicits._
spark.read spark.read
.load(s"$workingPath/mag") .load(s"$magBasePath/mag_denormalized")
.as[MAGPaper] .as[MAGPaper]
.map(s => MagUtility.convertMAGtoOAF(s)) .map(s => MagUtility.convertMAGtoOAF(s))
.filter(s => s != null)
.write .write
.option("compression", "gzip") .option("compression", "gzip")
.mode(SaveMode.Overwrite) .mode(SaveMode.Overwrite)
.text(mdStorePath) .text(mdStorePath)
} }
def generateAffiliations(spark: SparkSession, magBasePath: String, mdStorePath: String): Unit = {
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
val schema = new StructType()
.add(StructField("id", StringType))
.add(StructField("originalId", ArrayType(StringType)))
val generatedMag =
spark.read.schema(schema).json(mdStorePath).selectExpr("explode(originalId) as PaperId").distinct()
val paperAuthorAffiliations = MagUtility
.loadMagEntity(spark, "PaperAuthorAffiliations", magBasePath)
.where(col("AffiliationId").isNotNull)
.select("PaperId", "AffiliationId")
.distinct
paperAuthorAffiliations
.join(generatedMag, paperAuthorAffiliations("PaperId") === generatedMag("PaperId"), "leftsemi")
.flatMap(r => MagUtility.generateAffiliationRelations(r))
.write
.option("compression", "gzip")
.mode(SaveMode.Append)
.json(mdStorePath)
}
} }
object SparkMAGtoOAF { object SparkMAGtoOAF {

View File

@ -7,7 +7,6 @@ import org.junit.jupiter.api.extension.ExtendWith
import org.mockito.junit.jupiter.MockitoExtension import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.{Logger, LoggerFactory} import org.slf4j.{Logger, LoggerFactory}
@ExtendWith(Array(classOf[MockitoExtension])) @ExtendWith(Array(classOf[MockitoExtension]))
class CrossrefMappingTest extends AbstractVocabularyTest { class CrossrefMappingTest extends AbstractVocabularyTest {
@ -19,6 +18,4 @@ class CrossrefMappingTest extends AbstractVocabularyTest {
super.setUpVocabulary() super.setUpVocabulary()
} }
} }

View File

@ -3,62 +3,80 @@ package eu.dnetlib.dhp.collection.mag
import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.{Dataset, Publication, Result} import eu.dnetlib.dhp.schema.oaf.{Dataset, Publication, Result}
import org.apache.spark.sql.SparkSession import org.apache.spark.sql.SparkSession
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
class MAGMappingTest { class MAGMappingTest {
val mapper = new ObjectMapper() val mapper = new ObjectMapper()
def mappingTest(): Unit = {
// @Test val spark = SparkSession
// def mappingTest(): Unit = { .builder()
// .appName("Test")
// val spark = SparkSession .master("local[*]")
// .builder() .getOrCreate()
// .appName("Test")
// .master("local[*]")
// .getOrCreate()
//
// new SparkMAGtoOAF(null,null,null).convertMAG(spark,"/home/sandro/Downloads", "/home/sandro/Downloads/mag_oaf")
//
// }
val s = new SparkMAGtoOAF(null, null, null)
s.convertMAG(spark, "/home/sandro/Downloads/mag_test", "/home/sandro/Downloads/mag_oaf")
s.generateAffiliations(spark, "/home/sandro/Downloads/mag_test", "/home/sandro/Downloads/mag_oaf")
}
@Test @Test
def mappingMagType(): Unit = { def mappingMagType(): Unit = {
checkResult[Publication](MagUtility.createResultFromType(null, null), invisible = false, "Other literature type")
checkResult[Publication](MagUtility.createResultFromType(null, null), invisible = false,"Other literature type") checkResult[Publication](
checkResult[Publication](MagUtility.createResultFromType(Some("BookChapter"), null), invisible = false,"Part of book or chapter of book") MagUtility.createResultFromType(Some("BookChapter"), null),
checkResult[Publication](MagUtility.createResultFromType(Some("Book"), null), invisible = false,"Book") invisible = false,
checkResult[Publication](MagUtility.createResultFromType(Some("Repository"), null), invisible = true,"Other literature type") "Part of book or chapter of book"
checkResult[Publication](MagUtility.createResultFromType(Some("Thesis"), null), invisible = false,"Thesis") )
checkResult[Publication](MagUtility.createResultFromType(Some("Conference"), null), invisible = false,"Article") checkResult[Publication](MagUtility.createResultFromType(Some("Book"), null), invisible = false, "Book")
checkResult[Publication](MagUtility.createResultFromType(Some("Journal"), null), invisible = false,"Journal") checkResult[Publication](
checkResult[Dataset](MagUtility.createResultFromType(Some("Dataset"), null), invisible = false,"Dataset") MagUtility.createResultFromType(Some("Repository"), null),
checkResult[Publication](MagUtility.createResultFromType(Some("Patent"), Some("Patent Department of the Navy")), invisible = false,"Patent") invisible = true,
checkResult[Publication](MagUtility.createResultFromType(Some("Patent"), Some("Brevet Department of the Navy")), invisible = false,"Patent") "Other literature type"
checkResult[Publication](MagUtility.createResultFromType(Some("Patent"), Some("Journal of the Navy")), invisible = false,"Journal") )
checkResult[Publication](MagUtility.createResultFromType(Some("Patent"), Some("Proceedings of the Navy")), invisible = false,"Article") checkResult[Publication](MagUtility.createResultFromType(Some("Thesis"), null), invisible = false, "Thesis")
checkResult[Dataset](MagUtility.createResultFromType(Some("Dataset"), null), invisible = false,"Dataset") checkResult[Publication](MagUtility.createResultFromType(Some("Conference"), null), invisible = false, "Article")
checkResult[Publication](MagUtility.createResultFromType(Some("Journal"), null), invisible = false, "Journal")
checkResult[Dataset](MagUtility.createResultFromType(Some("Dataset"), null), invisible = false, "Dataset")
checkResult[Publication](
MagUtility.createResultFromType(Some("Patent"), Some("Patent Department of the Navy")),
invisible = false,
"Patent"
)
checkResult[Publication](
MagUtility.createResultFromType(Some("Patent"), Some("Brevet Department of the Navy")),
invisible = false,
"Patent"
)
checkResult[Publication](
MagUtility.createResultFromType(Some("Patent"), Some("Journal of the Navy")),
invisible = false,
"Journal"
)
checkResult[Publication](
MagUtility.createResultFromType(Some("Patent"), Some("Proceedings of the Navy")),
invisible = false,
"Article"
)
checkResult[Dataset](MagUtility.createResultFromType(Some("Dataset"), null), invisible = false, "Dataset")
assertNull(MagUtility.createResultFromType(Some("Patent"), null)) assertNull(MagUtility.createResultFromType(Some("Patent"), null))
assertNull(MagUtility.createResultFromType(Some("Patent"), Some("Some name "))) assertNull(MagUtility.createResultFromType(Some("Patent"), Some("Some name ")))
} }
def checkResult[T](r: Result, invisible: Boolean, typeName: String): Unit = {
def checkResult[T](r:Result, invisible:Boolean, typeName:String): Unit = {
assertNotNull(r) assertNotNull(r)
assertTrue(r.isInstanceOf[T]) assertTrue(r.isInstanceOf[T])
assertNotNull(r.getDataInfo) assertNotNull(r.getDataInfo)
assertEquals( invisible ,r.getDataInfo.getInvisible) assertEquals(invisible, r.getDataInfo.getInvisible)
assertNotNull(r.getInstance()) assertNotNull(r.getInstance())
assertTrue(r.getInstance().size()>0) assertTrue(r.getInstance().size() > 0)
assertNotNull(r.getInstance().get(0).getInstancetype) assertNotNull(r.getInstance().get(0).getInstancetype)
assertEquals(typeName, r.getInstance().get(0).getInstancetype.getClassname) assertEquals(typeName, r.getInstance().get(0).getInstancetype.getClassname)