diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/convert_MAG_to_OAF_properties.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/convert_MAG_to_OAF_properties.json
index ff2322a76..48926f3e9 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/convert_MAG_to_OAF_properties.json
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/convert_MAG_to_OAF_properties.json
@@ -12,9 +12,9 @@
"paramRequired": true
},
{
- "paramName": "w",
- "paramLongName": "workingPath",
- "paramDescription": "The working path",
+ "paramName": "ma",
+ "paramLongName": "magBasePath",
+ "paramDescription": "The mag Base path",
"paramRequired": false
}
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/create_MAG_denormalized_table_properties.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/create_MAG_denormalized_table_properties.json
index 95a4a8517..886a94741 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/create_MAG_denormalized_table_properties.json
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/create_MAG_denormalized_table_properties.json
@@ -10,12 +10,6 @@
"paramLongName": "magBasePath",
"paramDescription": "The base path of MAG DUMP CSV Tables",
"paramRequired": true
- },
- {
- "paramName": "w",
- "paramLongName": "workingPath",
- "paramDescription": "The working path",
- "paramRequired": false
}
]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/oozie_app/workflow.xml
index 91a429d35..511eacb5f 100644
--- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/oozie_app/workflow.xml
+++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/mag/oozie_app/workflow.xml
@@ -8,10 +8,6 @@
mdstorePath
The base path of MAG DUMP CSV Tables
-
- workingPath
- The working path
-
resume_from
generateOAF
@@ -53,7 +49,6 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--magBasePath${magBasePath}
- --workingPath${workingPath}
--masteryarn
@@ -79,7 +74,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--mdstorePath${mdstorePath}
- --workingPath${workingPath}
+ --magBasePath${magBasePath}
--masteryarn
diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala
index abc22a2c6..44c82e256 100644
--- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala
+++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/crossref/Crossref2Oaf.scala
@@ -952,7 +952,7 @@ case object Crossref2Oaf {
val targetId = getProjectId("hfri________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
- //ERASMUS+
+ //ERASMUS+
case "10.13039/501100010790" =>
generateSimpleRelationFromAward(funder, "erasmusplus_", a => a)
case _ => logger.debug("no match for " + funder.DOI.get)
diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/MagUtility.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/MagUtility.scala
index 3551bb215..aeb7ae1b1 100644
--- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/MagUtility.scala
+++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/MagUtility.scala
@@ -4,7 +4,16 @@ import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils._
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 org.apache.spark.sql.types._
import org.apache.spark.sql.{Dataset, Row, SparkSession}
@@ -65,7 +74,41 @@ object MagUtility extends Serializable {
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(
"bool" -> BooleanType,
@@ -308,15 +351,14 @@ object MagUtility extends Serializable {
def getSchema(streamName: String): StructType = {
var schema = new StructType()
val d: Seq[String] = stream(streamName)._2
- d.foreach {
- case t =>
- val currentType = t.split(":")
- val fieldName: String = currentType.head
- var fieldType: String = currentType.last
- val nullable: Boolean = fieldType.endsWith("?")
- if (nullable)
- fieldType = fieldType.replace("?", "")
- schema = schema.add(StructField(fieldName, datatypedict(fieldType), nullable))
+ d.foreach { case t =>
+ val currentType = t.split(":")
+ val fieldName: String = currentType.head
+ var fieldType: String = currentType.last
+ val nullable: Boolean = fieldType.endsWith("?")
+ if (nullable)
+ fieldType = fieldType.replace("?", "")
+ schema = schema.add(StructField(fieldName, datatypedict(fieldType), nullable))
}
schema
}
@@ -338,22 +380,10 @@ object MagUtility extends Serializable {
def createResultFromType(magType: Option[String], source: Option[String]): Result = {
var result: Result = null
- 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
- )
- )
- if (magType == null || magType.orNull ==null) {
+
+ if (magType == null || magType.orNull == null) {
result = new Publication
- result.setDataInfo(di)
+ result.setDataInfo(MAGDataInfo)
val i = new Instance
i.setInstancetype(
qualifier(
@@ -386,7 +416,7 @@ object MagUtility extends Serializable {
result = new Publication
qualifier("0043", "Journal", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE)
case "patent" =>
- if (source!= null && source.orNull != null) {
+ if (source != null && source.orNull != null) {
val s = source.get.toLowerCase
if (s.contains("patent") || s.contains("brevet")) {
result = new Publication
@@ -404,9 +434,11 @@ object MagUtility extends Serializable {
ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE
)
- } else if (s.contains("proceedings") || s.contains("conference") || s.contains("workshop") || s.contains(
- "symposium"
- )) {
+ } else if (
+ s.contains("proceedings") || s.contains("conference") || s.contains("workshop") || s.contains(
+ "symposium"
+ )
+ ) {
result = new Publication
qualifier(
"0001",
@@ -419,7 +451,7 @@ object MagUtility extends Serializable {
case "repository" =>
result = new Publication()
- di.setInvisible(true)
+ result.setDataInfo(MAGDataInfoInvisible)
qualifier(
"0038",
"Other literature type",
@@ -454,7 +486,8 @@ object MagUtility extends Serializable {
}
if (result != null) {
- result.setDataInfo(di)
+ if (result.getDataInfo == null)
+ result.setDataInfo(MAGDataInfo)
val i = new Instance
i.setInstancetype(tp)
result.setInstance(List(i).asJava)
@@ -466,14 +499,13 @@ object MagUtility extends Serializable {
def convertMAGtoOAF(paper: MAGPaper): String = {
// FILTER all the MAG paper with no URL
- if (paper.urls.orNull == null )
+ if (paper.urls.orNull == null)
return null
val result = createResultFromType(paper.docType, paper.originalVenue)
if (result == null)
return null
-
result.setCollectedfrom(List(MAGCollectedFrom).asJava)
val pidList = List(
structuredProperty(
@@ -582,30 +614,32 @@ object MagUtility extends Serializable {
val instance = result.getInstance().get(0)
instance.setPid(pidList.asJava)
- if(paper.doi.orNull != null)
- instance.setAlternateIdentifier(
- List(
- structuredProperty(
- paper.doi.get,
- qualifier(
- PidType.doi.toString,
- PidType.doi.toString,
- ModelConstants.DNET_PID_TYPES,
- ModelConstants.DNET_PID_TYPES
- ),
- null
- )
- ).asJava
- )
+ if (paper.doi.orNull != null)
+ instance.setAlternateIdentifier(
+ List(
+ structuredProperty(
+ paper.doi.get,
+ qualifier(
+ PidType.doi.toString,
+ PidType.doi.toString,
+ ModelConstants.DNET_PID_TYPES,
+ ModelConstants.DNET_PID_TYPES
+ ),
+ null
+ )
+ ).asJava
+ )
instance.setUrl(paper.urls.get.asJava)
instance.setHostedby(ModelConstants.UNKNOWN_REPOSITORY)
instance.setCollectedfrom(MAGCollectedFrom)
- instance.setAccessright(accessRight(
- ModelConstants.UNKNOWN,
- ModelConstants.NOT_AVAILABLE,
- ModelConstants.DNET_ACCESS_MODES,
- ModelConstants.DNET_ACCESS_MODES
- ))
+ instance.setAccessright(
+ accessRight(
+ ModelConstants.UNKNOWN,
+ ModelConstants.NOT_AVAILABLE,
+ ModelConstants.DNET_ACCESS_MODES,
+ ModelConstants.DNET_ACCESS_MODES
+ )
+ )
if (paper.authors.orNull != null && paper.authors.get.nonEmpty)
result.setAuthor(
@@ -646,6 +680,31 @@ object MagUtility extends Serializable {
.asJava
)
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 = {
diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkCreateMagDenormalizedTable.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkCreateMagDenormalizedTable.scala
index 5f1099de0..948ac257d 100644
--- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkCreateMagDenormalizedTable.scala
+++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkCreateMagDenormalizedTable.scala
@@ -15,15 +15,12 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
override def run(): Unit = {
val magBasePath: String = parser.get("magBasePath")
log.info("found parameters magBasePath: {}", magBasePath)
- val workingPath: String = parser.get("workingPath")
- log.info("found parameters workingPath: {}", workingPath)
- generatedDenormalizedMAGTable(spark, magBasePath, workingPath)
+ generatedDenormalizedMAGTable(spark, magBasePath)
}
private def generatedDenormalizedMAGTable(
spark: SparkSession,
- magBasePath: String,
- workingPath: String
+ magBasePath: String
): Unit = {
import spark.implicits._
@@ -114,27 +111,6 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
step2.count()
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
.loadMagEntity(spark, "Journals", magBasePath)
.select(
@@ -204,7 +180,7 @@ class SparkCreateMagDenormalizedTable(propertyPath: String, args: Array[String],
)
.write
.mode("OverWrite")
- .save(s"$workingPath/mag")
+ .save(s"$magBasePath/mag_denormalized")
step3.unpersist()
}
}
diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkMAGtoOAF.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkMAGtoOAF.scala
index fc2aefaf1..2fde0ab18 100644
--- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkMAGtoOAF.scala
+++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/mag/SparkMAGtoOAF.scala
@@ -1,7 +1,9 @@
package eu.dnetlib.dhp.collection.mag
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.slf4j.{Logger, LoggerFactory}
@@ -14,26 +16,49 @@ class SparkMAGtoOAF(propertyPath: String, args: Array[String], log: Logger)
override def run(): Unit = {
val mdstorePath: String = parser.get("mdstorePath")
log.info("found parameters mdstorePath: {}", mdstorePath)
- val workingPath: String = parser.get("workingPath")
- log.info("found parameters workingPath: {}", workingPath)
- convertMAG(spark, workingPath, mdstorePath)
+ val magBasePath: String = parser.get("magBasePath")
+ log.info("found parameters magBasePath: {}", magBasePath)
+ 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._
-
-
spark.read
- .load(s"$workingPath/mag")
+ .load(s"$magBasePath/mag_denormalized")
.as[MAGPaper]
.map(s => MagUtility.convertMAGtoOAF(s))
+ .filter(s => s != null)
.write
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.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 {
diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala
index 296c30395..ed43bb1a1 100644
--- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala
+++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/crossref/CrossrefMappingTest.scala
@@ -7,7 +7,6 @@ import org.junit.jupiter.api.extension.ExtendWith
import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.{Logger, LoggerFactory}
-
@ExtendWith(Array(classOf[MockitoExtension]))
class CrossrefMappingTest extends AbstractVocabularyTest {
@@ -19,6 +18,4 @@ class CrossrefMappingTest extends AbstractVocabularyTest {
super.setUpVocabulary()
}
-
-
}
diff --git a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/mag/MAGMappingTest.scala b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/mag/MAGMappingTest.scala
index 0790f1400..47105b732 100644
--- a/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/mag/MAGMappingTest.scala
+++ b/dhp-workflows/dhp-aggregation/src/test/scala/eu/dnetlib/dhp/collection/mag/MAGMappingTest.scala
@@ -3,62 +3,80 @@ package eu.dnetlib.dhp.collection.mag
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.{Dataset, Publication, Result}
import org.apache.spark.sql.SparkSession
-import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Assertions._
-
-
+import org.junit.jupiter.api.Test
class MAGMappingTest {
val mapper = new ObjectMapper()
+ def mappingTest(): Unit = {
-// @Test
-// def mappingTest(): Unit = {
-//
-// val spark = SparkSession
-// .builder()
-// .appName("Test")
-// .master("local[*]")
-// .getOrCreate()
-//
-// new SparkMAGtoOAF(null,null,null).convertMAG(spark,"/home/sandro/Downloads", "/home/sandro/Downloads/mag_oaf")
-//
-// }
+ val spark = SparkSession
+ .builder()
+ .appName("Test")
+ .master("local[*]")
+ .getOrCreate()
+ 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
def mappingMagType(): Unit = {
-
- checkResult[Publication](MagUtility.createResultFromType(null, null), invisible = false,"Other literature type")
- checkResult[Publication](MagUtility.createResultFromType(Some("BookChapter"), null), invisible = false,"Part of book or chapter of book")
- checkResult[Publication](MagUtility.createResultFromType(Some("Book"), null), invisible = false,"Book")
- checkResult[Publication](MagUtility.createResultFromType(Some("Repository"), null), invisible = true,"Other literature type")
- 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("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")
+ checkResult[Publication](MagUtility.createResultFromType(null, null), invisible = false, "Other literature type")
+ checkResult[Publication](
+ MagUtility.createResultFromType(Some("BookChapter"), null),
+ invisible = false,
+ "Part of book or chapter of book"
+ )
+ checkResult[Publication](MagUtility.createResultFromType(Some("Book"), null), invisible = false, "Book")
+ checkResult[Publication](
+ MagUtility.createResultFromType(Some("Repository"), null),
+ invisible = true,
+ "Other literature type"
+ )
+ 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("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"), 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)
assertTrue(r.isInstanceOf[T])
assertNotNull(r.getDataInfo)
- assertEquals( invisible ,r.getDataInfo.getInvisible)
+ assertEquals(invisible, r.getDataInfo.getInvisible)
assertNotNull(r.getInstance())
- assertTrue(r.getInstance().size()>0)
+ assertTrue(r.getInstance().size() > 0)
assertNotNull(r.getInstance().get(0).getInstancetype)
assertEquals(typeName, r.getInstance().get(0).getInstancetype.getClassname)