diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 157373db6..6c2c70419 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -1,5 +1,7 @@ package eu.dnetlib.doiboost.crossref +import java.util + import eu.dnetlib.dhp.schema.oaf._ import eu.dnetlib.dhp.utils.DHPUtils import org.apache.commons.lang.StringUtils @@ -7,14 +9,21 @@ import org.json4s import org.json4s.DefaultFormats import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ -import org.slf4j.Logger -import scala.collection.JavaConverters._ +import org.slf4j.{Logger, LoggerFactory} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.matching.Regex + +case class mappingAffiliation(name: String) {} -case class mappingAffiliation(name: String) case class mappingAuthor(given: Option[String], family: String, ORCID: Option[String], affiliation: Option[mappingAffiliation]) {} +case class mappingFunder(name: String, DOI: Option[String], award: Option[List[String]]) {} + case object Crossref2Oaf { + val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass) //STATIC STRING val MAG = "MAG" @@ -83,7 +92,6 @@ case object Crossref2Oaf { "report" -> "0017 Report" ) - def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats @@ -143,13 +151,10 @@ case object Crossref2Oaf { //Mapping AUthor - val authorList: List[mappingAuthor] = (json \ "author").extractOrElse[List[mappingAuthor]](List()) - result.setAuthor(authorList.map(a => generateAuhtor(a.given.orNull, a.family, a.ORCID.orNull)).asJava) // Mapping instance - val instance = new Instance() val license = for { JString(lic) <- json \ "license" \ "URL" @@ -182,37 +187,146 @@ case object Crossref2Oaf { a.setFullname(s"${given} ${family}") if (StringUtils.isNotBlank(orcid)) a.setPid(List(createSP(orcid, ORCID, PID_TYPES)).asJava) + a } - def convert(input: String, logger: Logger): Result = { + def convert(input: String): List[Oaf] = { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats lazy val json: json4s.JValue = parse(input) + var resultList: List[Oaf] = List() + + val objectType = (json \ "type").extractOrElse[String](null) val objectSubType = (json \ "subtype").extractOrElse[String](null) if (objectType == null) - return null + return resultList val result = generateItemFromType(objectType, objectSubType) if (result == null) - return result + return List() val cOBJCategory = mappingCrossrefSubType.getOrElse(objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type")); -// logger.debug(mappingCrossrefType(objectType)) -// logger.debug(cOBJCategory) - mappingResult(result, json, cOBJCategory) + val funderList: List[mappingFunder] = (json \ "funder").extractOrElse[List[mappingFunder]](List()) + + if (funderList.nonEmpty) { + resultList = resultList ::: mappingFunderToRelations(funderList, result.getId, createCollectedFrom(), result.getDataInfo, result.getLastupdatetimestamp) + } + + result match { case publication: Publication => convertPublication(publication, json, cOBJCategory) case dataset: Dataset => convertDataset(dataset) } + resultList = resultList ::: List(result) + resultList + } - result + + def mappingFunderToRelations(funders: List[mappingFunder], sourceId: String, cf: KeyValue, di: DataInfo, ts: Long): List[Relation] = { + + val queue = new mutable.Queue[Relation] + + + def extractECAward(award: String): String = { + val awardECRegex: Regex = "[0-9]{4,9}".r + if (awardECRegex.findAllIn(award).hasNext) + return awardECRegex.findAllIn(award).max + null + } + + + def generateRelation(sourceId:String, targetId:String, nsPrefix:String) :Relation = { + + val r = new Relation + r.setSource(sourceId) + r.setTarget(s"$nsPrefix::$targetId") + r.setRelType("resultProject") + r.setRelClass("isProducedBy") + r.setSubRelType("outcome") + r.setCollectedfrom(List(cf).asJava) + r.setDataInfo(di) + r.setLastupdatetimestamp(ts) + r + } + + + def generateSimpleRelationFromAward(funder: mappingFunder, nsPrefix: String, extractField: String => String): Unit = { + if (funder.award.isDefined && funder.award.get.nonEmpty) + funder.award.get.map(extractField).filter(a => a!= null && a.nonEmpty).foreach( + award => { + val targetId = DHPUtils.md5(award) + queue += generateRelation(sourceId, targetId, nsPrefix) + } + ) + } + + if (funders != null) + funders.foreach(funder => { + if (funder.DOI.isDefined && funder.DOI.get.nonEmpty) { + funder.DOI.get match { + case "10.13039/100010663" | + "10.13039/100010661" | + "10.13039/501100007601" | + "10.13039/501100000780" | + "10.13039/100010665" => generateSimpleRelationFromAward(funder, "corda__h2020", extractECAward) + case "10.13039/100011199" | + "10.13039/100004431" | + "10.13039/501100004963" | + "10.13039/501100000780" => generateSimpleRelationFromAward(funder, "corda_______", extractECAward) + case "10.13039/501100000781" => generateSimpleRelationFromAward(funder, "corda_______", extractECAward) + generateSimpleRelationFromAward(funder, "corda__h2020", extractECAward) + case "10.13039/100000001" => generateSimpleRelationFromAward(funder, "nsf_________", a => a) + case "10.13039/501100001665" => generateSimpleRelationFromAward(funder, "anr_________", a => a) + case "10.13039/501100002341" => generateSimpleRelationFromAward(funder, "aka_________", a => a) + case "10.13039/501100001602" => generateSimpleRelationFromAward(funder, "aka_________", a => a.replace("SFI", "")) + case "10.13039/501100000923" => generateSimpleRelationFromAward(funder, "arc_________", a => a) + case "10.13039/501100000038"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "nserc_______" ) + case "10.13039/501100000155"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "sshrc_______" ) + case "10.13039/501100000024"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "cihr________" ) + case "10.13039/501100002848" => generateSimpleRelationFromAward(funder, "conicytf____", a => a) + case "10.13039/501100003448" => generateSimpleRelationFromAward(funder, "gsrt________", extractECAward) + case "10.13039/501100010198" => generateSimpleRelationFromAward(funder, "sgov________", a=>a) + case "10.13039/501100004564" => generateSimpleRelationFromAward(funder, "mestd_______", extractECAward) + case "10.13039/501100003407" => generateSimpleRelationFromAward(funder, "miur________", a=>a) + queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "miur________" ) + case "10.13039/501100006588" | + "10.13039/501100004488" => generateSimpleRelationFromAward(funder, "irb_hr______", a=>a.replaceAll("Project No.", "").replaceAll("HRZZ-","") ) + case "10.13039/501100006769"=> generateSimpleRelationFromAward(funder, "rsf_________", a=>a) + case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", extractECAward) + case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a) + case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a) + case "10.13039/100004440"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" ) + case _ => logger.debug("no match for "+funder.DOI.get ) + + + } + + + } else { + funder.name match { + case "European Union’s Horizon 2020 research and innovation program" => generateSimpleRelationFromAward(funder, "corda__h2020", extractECAward) + case "European Union's" => + generateSimpleRelationFromAward(funder, "corda__h2020", extractECAward) + generateSimpleRelationFromAward(funder, "corda_______", extractECAward) + case "The French National Research Agency (ANR)" | + "The French National Research Agency" => generateSimpleRelationFromAward(funder, "anr_________", a => a) + case "CONICYT, Programa de Formación de Capital Humano Avanzado" => generateSimpleRelationFromAward(funder, "conicytf____", extractECAward) + case "Wellcome Trust Masters Fellowship" => queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" ) + case _ => logger.debug("no match for "+funder.name ) + + } + } + + } + ) + queue.toList } def convertDataset(dataset: Dataset): Unit = { @@ -247,7 +361,7 @@ case object Crossref2Oaf { JField("value", JString(vl)) <- issn_type } yield Tuple2(tp, vl) - val volume = (json \ "volume").extractOrElse[String] (null) + val volume = (json \ "volume").extractOrElse[String](null) if (containerTitles.nonEmpty) { val journal = new Journal journal.setName(containerTitles.head) @@ -259,19 +373,15 @@ case object Crossref2Oaf { case "print" => journal.setIssnPrinted(tp._2) } }) - } journal.setVol(volume) - - val page = (json \ "page").extractOrElse[String] (null) - if(page!= null ) { + val page = (json \ "page").extractOrElse[String](null) + if (page != null) { val pp = page.split("-") journal.setSp(pp.head) if (pp.size > 1) journal.setEp(pp(1)) } - - publication.setJournal(journal) } @@ -341,6 +451,15 @@ case object Crossref2Oaf { } + def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = { + val sp = new StructuredProperty + sp.setQualifier(createQualifier(classId, schemeId)) + sp.setValue(value) + sp.setDataInfo(dataInfo) + sp + + } + def createCollectedFrom(): KeyValue = { val cf = new KeyValue diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala index 3374f2969..8cf9223f6 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala @@ -1,11 +1,12 @@ package eu.dnetlib.doiboost.crossref import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.schema.oaf.Publication +import eu.dnetlib.dhp.schema.oaf +import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Result} import org.apache.commons.io.IOUtils import org.apache.hadoop.io.{IntWritable, Text} import org.apache.spark.SparkConf -import org.apache.spark.sql.{Dataset, Encoders, SaveMode, SparkSession} +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} import org.slf4j.{Logger, LoggerFactory} @@ -26,42 +27,46 @@ object SparkMapDumpIntoOAF { .config(conf) .appName(SparkMapDumpIntoOAF.getClass.getSimpleName) .master(parser.get("master")).getOrCreate() - import spark.implicits._ - implicit val mapEncoder = Encoders.bean(classOf[Publication]) + + implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo(classOf[Publication]) + implicit val mapEncoderRelatons: Encoder[Relation] = Encoders.kryo(classOf[Relation]) + implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo(classOf[eu.dnetlib.dhp.schema.oaf.Dataset]) val sc = spark.sparkContext - - val total = sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text]) - .map(k => k._2.toString).map(CrossrefImporter.decompressBlob) - .map(k => Crossref2Oaf.convert(k, logger)) - .filter(k => k != null && k.isInstanceOf[Publication]) - .map(k => k.asInstanceOf[Publication]) - - - val ds: Dataset[Publication] = spark.createDataset(total) val targetPath = parser.get("targetPath") - ds.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication") - logger.info(s"total Item :${total}") + sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text]) + .map(k => k._2.toString).map(CrossrefImporter.decompressBlob) + .flatMap(k => Crossref2Oaf.convert(k)).saveAsObjectFile(s"${targetPath}/mixObject") + + + val inputRDD = sc.objectFile[Oaf](s"${targetPath}/mixObject").filter(p=> p!= null) + val total = inputRDD.count() + + val totalPub = inputRDD.filter(p => p.isInstanceOf[Publication]).count() + val totalDat = inputRDD.filter(p => p.isInstanceOf[eu.dnetlib.dhp.schema.oaf.Dataset]).count() + val totalRel = inputRDD.filter(p => p.isInstanceOf[eu.dnetlib.dhp.schema.oaf.Relation]).count() + + + logger.info(s"Created $total") + logger.info(s"totalPub $totalPub") + logger.info(s"totalDat $totalDat") + logger.info(s"totalRel $totalRel") + val pubs: Dataset[Publication] = spark.createDataset(inputRDD.filter(k => k != null && k.isInstanceOf[Publication]) + .map(k => k.asInstanceOf[Publication])) + pubs.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication") + + val ds: Dataset[eu.dnetlib.dhp.schema.oaf.Dataset] = spark.createDataset(inputRDD.filter(k => k != null && k.isInstanceOf[eu.dnetlib.dhp.schema.oaf.Dataset]) + .map(k => k.asInstanceOf[eu.dnetlib.dhp.schema.oaf.Dataset])) + ds.write.mode(SaveMode.Overwrite).save(s"${targetPath}/dataset") + + val rels: Dataset[Relation] = spark.createDataset(inputRDD.filter(k => k != null && k.isInstanceOf[Relation]) + .map(k => k.asInstanceOf[Relation])) + rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations") + + - // lazy val json: json4s.JValue = parse(item) - // - // - // val references = for { - // JArray(references) <- json \\ "reference" - // JObject(reference) <- references - // JField("first-page", JString(firstPage)) <- reference - // JField("author", JString(author)) <- reference - // } yield Reference(author, firstPage) - // - // - // - // - // logger.info((json \ "created" \ "timestamp").extractOrElse("missing")) - // logger.info(references.toString()) - // - // logger.info((json \ "type").extractOrElse("missing")) } diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/CrossrefMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/CrossrefMappingTest.scala new file mode 100644 index 000000000..65e22ab9f --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/CrossrefMappingTest.scala @@ -0,0 +1,312 @@ +package eu.dnetlib.doiboost + +import com.fasterxml.jackson.databind.SerializationFeature +import eu.dnetlib.dhp.schema.oaf.{Dataset, KeyValue, Oaf, Publication, Relation, Result} +import eu.dnetlib.doiboost.crossref.{Crossref2Oaf, SparkMapDumpIntoOAF} +import org.apache.spark.{SparkConf, sql} +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.codehaus.jackson.map.ObjectMapper +import org.junit.jupiter.api.Test + +import scala.io.Source +import org.junit.jupiter.api.Assertions._ +import org.slf4j.{Logger, LoggerFactory} + +import scala.collection.JavaConverters._ + + +class CrossrefMappingTest { + + val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass) + val mapper = new ObjectMapper() + + + + @Test + def testRelSpark() :Unit = { + val conf: SparkConf = new SparkConf() + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(SparkMapDumpIntoOAF.getClass.getSimpleName) + .master("local[*]").getOrCreate() + + import spark.implicits._ + implicit val mapEncoderRelations: Encoder[Relation] = Encoders.kryo(classOf[Relation]) + implicit val mapEncoderPublication: Encoder[Publication] = Encoders.kryo(classOf[Publication]) + implicit val mapEncoderTupleJoinPubs: Encoder[(String, Publication)] = Encoders.tuple(Encoders.STRING, mapEncoderPublication) + implicit val mapEncoderTupleJoinRels: Encoder[(String, Relation)] = Encoders.tuple(Encoders.STRING, mapEncoderRelations) + + val relations:sql.Dataset[Relation] = spark.read.load("/data/doiboost/relations").as[Relation] + val publications :sql.Dataset[Publication] = spark.read.load("/data/doiboost/publication").as[Publication] + val ds1 = publications.map(p => Tuple2(p.getId, p)) + val ds2 = relations.map(p => Tuple2(p.getSource, p)) + val total =ds1.joinWith(ds2, ds1.col("_1")===ds2.col("_1")).count() + println(s"total : $total") + } + + + @Test + def testFunderRelationshipsMapping(): Unit = { + val template = Source.fromInputStream(getClass.getResourceAsStream("article_funder_template.json")).mkString + val funder_doi = Source.fromInputStream(getClass.getResourceAsStream("funder_doi")).mkString + val funder_name = Source.fromInputStream(getClass.getResourceAsStream("funder_doi")).mkString + + + for (line <- funder_doi.lines) { + val json = template.replace("%s", line) + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + assertTrue(resultList.nonEmpty) + checkRelation(resultList) + } + for (line <- funder_name.lines) { + val json = template.replace("%s", line) + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + assertTrue(resultList.nonEmpty) + checkRelation(resultList) + } + } + + def checkRelation(generatedOAF: List[Oaf]): Unit = { + + val rels: List[Relation] = generatedOAF.filter(p => p.isInstanceOf[Relation]).asInstanceOf[List[Relation]] + assertFalse(rels.isEmpty) + rels.foreach(relation => { + val relJson = mapper.writeValueAsString(relation) + + assertNotNull(relation.getSource, s"Source of relation null $relJson") + assertNotNull(relation.getTarget, s"Target of relation null $relJson") + assertFalse(relation.getTarget.isEmpty, s"Target is empty: $relJson") + assertFalse(relation.getRelClass.isEmpty, s"RelClass is empty: $relJson") + assertFalse(relation.getRelType.isEmpty, s"RelType is empty: $relJson") + assertFalse(relation.getSubRelType.isEmpty, s"SubRelType is empty: $relJson") + + }) + + } + + + @Test + def testConvertBookFromCrossRef2Oaf(): Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("book.json")).mkString + assertNotNull(json) + + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + val items = resultList.filter(p => p.isInstanceOf[Result]) + + assert(items.nonEmpty) + assert(items.size == 1) + val result: Result = items.head.asInstanceOf[Result] + assertNotNull(result) + + logger.info(mapper.writeValueAsString(result)); + + assertNotNull(result.getDataInfo, "Datainfo test not null Failed"); + assertNotNull( + result.getDataInfo.getProvenanceaction, + "DataInfo/Provenance test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassid.isEmpty, + "DataInfo/Provenance/classId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassname.isEmpty, + "DataInfo/Provenance/className test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemeid.isEmpty, + "DataInfo/Provenance/SchemeId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemename.isEmpty, + "DataInfo/Provenance/SchemeName test not null Failed"); + + assertNotNull(result.getCollectedfrom, "CollectedFrom test not null Failed"); + assertFalse(result.getCollectedfrom.isEmpty); + + val collectedFromList = result.getCollectedfrom.asScala + assert(collectedFromList.exists(c => c.getKey.equalsIgnoreCase("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")), "Wrong collected from assertion") + + assert(collectedFromList.exists(c => c.getValue.equalsIgnoreCase("crossref")), "Wrong collected from assertion") + + + val relevantDates = result.getRelevantdate.asScala + + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("created")), "Missing relevant date of type created") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("published-online")), "Missing relevant date of type published-online") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("published-print")), "Missing relevant date of type published-print") + val rels = resultList.filter(p => p.isInstanceOf[Relation]) + assert(rels.isEmpty) + } + + + @Test + def testConvertPreprintFromCrossRef2Oaf(): Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("preprint.json")).mkString + assertNotNull(json) + + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + val items = resultList.filter(p => p.isInstanceOf[Publication]) + + assert(items.nonEmpty) + assert(items.size == 1) + val result: Result = items.head.asInstanceOf[Publication] + assertNotNull(result) + + logger.info(mapper.writeValueAsString(result)); + + assertNotNull(result.getDataInfo, "Datainfo test not null Failed"); + assertNotNull( + result.getDataInfo.getProvenanceaction, + "DataInfo/Provenance test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassid.isEmpty, + "DataInfo/Provenance/classId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassname.isEmpty, + "DataInfo/Provenance/className test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemeid.isEmpty, + "DataInfo/Provenance/SchemeId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemename.isEmpty, + "DataInfo/Provenance/SchemeName test not null Failed"); + + assertNotNull(result.getCollectedfrom, "CollectedFrom test not null Failed"); + assertFalse(result.getCollectedfrom.isEmpty); + + val collectedFromList = result.getCollectedfrom.asScala + assert(collectedFromList.exists(c => c.getKey.equalsIgnoreCase("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")), "Wrong collected from assertion") + + assert(collectedFromList.exists(c => c.getValue.equalsIgnoreCase("crossref")), "Wrong collected from assertion") + + + val relevantDates = result.getRelevantdate.asScala + + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("created")), "Missing relevant date of type created") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("available")), "Missing relevant date of type available") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("accepted")), "Missing relevant date of type accepted") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("published-online")), "Missing relevant date of type published-online") + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("published-print")), "Missing relevant date of type published-print") + val rels = resultList.filter(p => p.isInstanceOf[Relation]) + assert(rels.isEmpty) + } + + + @Test + def testConvertDatasetFromCrossRef2Oaf(): Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("dataset.json")).mkString + assertNotNull(json) + + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + val items = resultList.filter(p => p.isInstanceOf[Dataset]) + + assert(items.nonEmpty) + assert(items.size == 1) + val result: Result = items.head.asInstanceOf[Dataset] + assertNotNull(result) + + logger.info(mapper.writeValueAsString(result)); + + assertNotNull(result.getDataInfo, "Datainfo test not null Failed"); + assertNotNull( + result.getDataInfo.getProvenanceaction, + "DataInfo/Provenance test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassid.isEmpty, + "DataInfo/Provenance/classId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassname.isEmpty, + "DataInfo/Provenance/className test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemeid.isEmpty, + "DataInfo/Provenance/SchemeId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemename.isEmpty, + "DataInfo/Provenance/SchemeName test not null Failed"); + + assertNotNull(result.getCollectedfrom, "CollectedFrom test not null Failed"); + assertFalse(result.getCollectedfrom.isEmpty); + } + + @Test + def testConvertArticleFromCrossRef2Oaf(): Unit = { + val json = Source.fromInputStream(getClass.getResourceAsStream("article.json")).mkString + assertNotNull(json) + + assertFalse(json.isEmpty); + + val resultList: List[Oaf] = Crossref2Oaf.convert(json) + + assertTrue(resultList.nonEmpty) + + val items = resultList.filter(p => p.isInstanceOf[Publication]) + + assert(items.nonEmpty) + assert(items.size == 1) + val result: Result = items.head.asInstanceOf[Publication] + assertNotNull(result) + + logger.info(mapper.writeValueAsString(result)); + + assertNotNull(result.getDataInfo, "Datainfo test not null Failed"); + assertNotNull( + result.getDataInfo.getProvenanceaction, + "DataInfo/Provenance test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassid.isEmpty, + "DataInfo/Provenance/classId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getClassname.isEmpty, + "DataInfo/Provenance/className test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemeid.isEmpty, + "DataInfo/Provenance/SchemeId test not null Failed"); + assertFalse( + result.getDataInfo.getProvenanceaction.getSchemename.isEmpty, + "DataInfo/Provenance/SchemeName test not null Failed"); + + assertNotNull(result.getCollectedfrom, "CollectedFrom test not null Failed"); + assertFalse(result.getCollectedfrom.isEmpty); + + val collectedFromList = result.getCollectedfrom.asScala + assert(collectedFromList.exists(c => c.getKey.equalsIgnoreCase("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")), "Wrong collected from assertion") + + assert(collectedFromList.exists(c => c.getValue.equalsIgnoreCase("crossref")), "Wrong collected from assertion") + + + val relevantDates = result.getRelevantdate.asScala + + assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("created")), "Missing relevant date of type created") + + val rels = resultList.filter(p => p.isInstanceOf[Relation]).asInstanceOf[List[Relation]] + assertFalse(rels.isEmpty) + rels.foreach(relation => { + assertNotNull(relation) + assertFalse(relation.getSource.isEmpty) + assertFalse(relation.getTarget.isEmpty) + assertFalse(relation.getRelClass.isEmpty) + assertFalse(relation.getRelType.isEmpty) + assertFalse(relation.getSubRelType.isEmpty) + + }) + + + } + +} diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/DoiBoostTest.java b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/DoiBoostTest.java index 49f9ef912..12228599c 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/DoiBoostTest.java +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/DoiBoostTest.java @@ -1,25 +1,18 @@ package eu.dnetlib.doiboost; -import static org.junit.jupiter.api.Assertions.*; - import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.doiboost.crossref.Crossref2Oaf; import eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF; -import java.io.IOException; -import java.util.List; -import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Test; -import org.junit.platform.commons.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DoiBoostTest { + final ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT); Logger logger = LoggerFactory.getLogger(DoiBoostTest.class); + @Test public void test() throws Exception { // SparkDownloadContentFromCrossref.main(null); @@ -29,213 +22,4 @@ public class DoiBoostTest { "-m local[*] -s file:///data/doiboost/crossref_dump.seq -t /data/doiboost" .split(" ")); } - - @Test - public void testConvertDatasetCrossRef2Oaf() throws IOException { - final String json = IOUtils.toString(getClass().getResourceAsStream("dataset.json")); - ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT); - assertNotNull(json); - assertFalse(StringUtils.isBlank(json)); - final Result result = Crossref2Oaf.convert(json, logger); - - logger.info(mapper.writeValueAsString(result)); - } - - @Test - public void testConvertPreprintCrossRef2Oaf() throws IOException { - - final String json = IOUtils.toString(getClass().getResourceAsStream("preprint.json")); - ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT); - assertNotNull(json); - assertFalse(StringUtils.isBlank(json)); - - final Result result = Crossref2Oaf.convert(json, logger); - assertNotNull(result); - - assertNotNull(result.getDataInfo(), "Datainfo test not null Failed"); - assertNotNull( - result.getDataInfo().getProvenanceaction(), - "DataInfo/Provenance test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassid()), - "DataInfo/Provenance/classId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassname()), - "DataInfo/Provenance/className test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemeid()), - "DataInfo/Provenance/SchemeId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemename()), - "DataInfo/Provenance/SchemeName test not null Failed"); - - assertNotNull(result.getCollectedfrom(), "CollectedFrom test not null Failed"); - assertTrue(result.getCollectedfrom().size() > 0); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch( - c -> - c.getKey() - .equalsIgnoreCase( - "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"))); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch(c -> c.getValue().equalsIgnoreCase("crossref"))); - - assertTrue( - result.getRelevantdate().stream() - .anyMatch(d -> d.getQualifier().getClassid().equalsIgnoreCase("created"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> d.getQualifier().getClassid().equalsIgnoreCase("available"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch(d -> d.getQualifier().getClassid().equalsIgnoreCase("accepted"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> - d.getQualifier() - .getClassid() - .equalsIgnoreCase("published-online"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> - d.getQualifier() - .getClassid() - .equalsIgnoreCase("published-print"))); - - logger.info(mapper.writeValueAsString(result)); - } - - @Test - public void testConvertArticleCrossRef2Oaf() throws IOException { - - final String json = IOUtils.toString(getClass().getResourceAsStream("article.json")); - ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT); - assertNotNull(json); - assertFalse(StringUtils.isBlank(json)); - final Result result = Crossref2Oaf.convert(json, logger); - assertNotNull(result); - - assertNotNull(result.getDataInfo(), "Datainfo test not null Failed"); - assertNotNull( - result.getDataInfo().getProvenanceaction(), - "DataInfo/Provenance test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassid()), - "DataInfo/Provenance/classId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassname()), - "DataInfo/Provenance/className test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemeid()), - "DataInfo/Provenance/SchemeId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemename()), - "DataInfo/Provenance/SchemeName test not null Failed"); - - assertNotNull(result.getCollectedfrom(), "CollectedFrom test not null Failed"); - assertTrue(result.getCollectedfrom().size() > 0); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch( - c -> - c.getKey() - .equalsIgnoreCase( - "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"))); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch(c -> c.getValue().equalsIgnoreCase("crossref"))); - - assertTrue( - result.getRelevantdate().stream() - .anyMatch(d -> d.getQualifier().getClassid().equalsIgnoreCase("created"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> - d.getQualifier() - .getClassid() - .equalsIgnoreCase("published-online"))); - // assertTrue( - // result.getRelevantdate().stream() - // .anyMatch( - // d -> - // d.getQualifier() - // .getClassid() - // .equalsIgnoreCase("published-print"))); - - logger.info(mapper.writeValueAsString(result)); - } - - @Test - public void testConvertBooktCrossRef2Oaf() throws IOException { - - final String json = IOUtils.toString(getClass().getResourceAsStream("book.json")); - ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT); - assertNotNull(json); - assertFalse(StringUtils.isBlank(json)); - final Result result = Crossref2Oaf.convert(json, logger); - assertNotNull(result); - logger.info(mapper.writeValueAsString(result)); - - assertNotNull(result.getDataInfo(), "Datainfo test not null Failed"); - assertNotNull( - result.getDataInfo().getProvenanceaction(), - "DataInfo/Provenance test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassid()), - "DataInfo/Provenance/classId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getClassname()), - "DataInfo/Provenance/className test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemeid()), - "DataInfo/Provenance/SchemeId test not null Failed"); - assertFalse( - StringUtils.isBlank(result.getDataInfo().getProvenanceaction().getSchemename()), - "DataInfo/Provenance/SchemeName test not null Failed"); - - assertNotNull(result.getCollectedfrom(), "CollectedFrom test not null Failed"); - assertTrue(result.getCollectedfrom().size() > 0); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch( - c -> - c.getKey() - .equalsIgnoreCase( - "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2"))); - assertTrue( - result.getCollectedfrom().stream() - .anyMatch(c -> c.getValue().equalsIgnoreCase("crossref"))); - - assertTrue( - result.getRelevantdate().stream() - .anyMatch(d -> d.getQualifier().getClassid().equalsIgnoreCase("created"))); - - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> - d.getQualifier() - .getClassid() - .equalsIgnoreCase("published-online"))); - assertTrue( - result.getRelevantdate().stream() - .anyMatch( - d -> - d.getQualifier() - .getClassid() - .equalsIgnoreCase("published-print"))); - } - - @Test - public void testPath() throws Exception { - final String json = IOUtils.toString(getClass().getResourceAsStream("response.json")); - final List res = JsonPath.read(json, "$.hits.hits[*]._source.blob"); - System.out.println(res.size()); - } } diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article.json index afef13b69..e0dc0db39 100644 --- a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article.json +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article.json @@ -170,5 +170,6 @@ "container-title": [ "Ecl\u00e9tica Qu\u00edmica Journal" ], - "page": "41-50" + "page": "41-50", + "funder": [{"DOI": "10.13039/100010663","name": "H2020 European Research Council","doi-asserted-by": "publisher","award": ["677749"]}] } \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article_funder_template.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article_funder_template.json new file mode 100644 index 000000000..1a49109ec --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/article_funder_template.json @@ -0,0 +1,175 @@ +{ + "DOI": "10.26850/1678-4618eqj.v35.1.2010.p41-46", + "issued": { + "date-parts": [ + [ + 2018, + 1, + 15 + ] + ] + }, + "abstract": "A qualitative spot-test and tandem quantitative analysis of dipyrone in the bulk drugand in pharmaceutical preparations is proposed. The formation of a reddish-violet\u00a0 color indicates a positive result. In sequence a quantitative procedure can be performed in the same flask. The quantitative results obtained were statistically compared with those obtained with the method indicated by the Brazilian\u00a0 Pharmacopoeia, using the Student\u2019s t and the F tests. Considering the concentration in a 100 \u03bcL aliquot, the qualitative visual limit of detection is about 5\u00d710-6 g; instrumental LOD \u2245 1.4\u00d710-4 mol L-1 ; LOQ \u2245 4.5\u00d710-4 mol L-1.", + "prefix": "10.26850", + "author": [ + { + "authenticated-orcid": false, + "given": "Matthieu", + "family": "Tubino", + "sequence": "first", + "affiliation": [], + "ORCID": "http://orcid.org/0000-0002-1987-3907" + }, + { + "affiliation": [], + "given": "A. C.", + "family": "Biondo", + "sequence": "additional" + }, + { + "authenticated-orcid": false, + "given": "Marta Maria Duarte Carvalho", + "family": "Vila", + "sequence": "additional", + "affiliation": [], + "ORCID": "http://orcid.org/0000-0002-0198-7076" + }, + { + "authenticated-orcid": false, + "given": "Leonardo", + "family": "Pezza", + "sequence": "additional", + "affiliation": [], + "ORCID": "http://orcid.org/0000-0003-0197-7369" + }, + { + "authenticated-orcid": false, + "given": "Helena Redigolo", + "family": "Pezza", + "sequence": "additional", + "affiliation": [], + "ORCID": "http://orcid.org/0000-0001-5564-1639" + } + ], + "reference-count": 0, + "ISSN": [ + "1678-4618" + ], + "member": "11395", + "source": "Crossref", + "score": 1.0, + "deposited": { + "timestamp": 1540823529000, + "date-time": "2018-10-29T14:32:09Z", + "date-parts": [ + [ + 2018, + 10, + 29 + ] + ] + }, + "indexed": { + "timestamp": 1540825815212, + "date-time": "2018-10-29T15:10:15Z", + "date-parts": [ + [ + 2018, + 10, + 29 + ] + ] + }, + "type": "journal-article", + "published-online": { + "date-parts": [ + [ + 2018, + 1, + 15 + ] + ] + }, + "URL": "http://dx.doi.org/10.26850/1678-4618eqj.v35.1.2010.p41-46", + "is-referenced-by-count": 0, + "volume": "35", + "issn-type": [ + { + "type": "electronic", + "value": "1678-4618" + } + ], + "link": [ + { + "URL": "http://revista.iq.unesp.br/ojs/index.php/ecletica/article/viewFile/191/149", + "intended-application": "text-mining", + "content-version": "vor", + "content-type": "application/pdf" + }, + { + "URL": "http://revista.iq.unesp.br/ojs/index.php/ecletica/article/viewFile/191/149", + "intended-application": "similarity-checking", + "content-version": "vor", + "content-type": "unspecified" + } + ], + "journal-issue": { + "issue": "1", + "published-online": { + "date-parts": [ + [ + 2018, + 1, + 15 + ] + ] + } + }, + "references-count": 0, + "short-container-title": [ + "Eclet. Quim. J." + ], + "publisher": "Ecletica Quimica Journal", + "content-domain": { + "domain": [], + "crossmark-restriction": false + }, + "license": [ + { + "URL": "http://creativecommons.org/licenses/by/4.0", + "start": { + "timestamp": 1515974400000, + "date-time": "2018-01-15T00:00:00Z", + "date-parts": [ + [ + 2018, + 1, + 15 + ] + ] + }, + "content-version": "unspecified", + "delay-in-days": 0 + } + ], + "created": { + "timestamp": 1517590842000, + "date-time": "2018-02-02T17:00:42Z", + "date-parts": [ + [ + 2018, + 2, + 2 + ] + ] + }, + "issue": "1", + "title": [ + "Spot-test identification and rapid quantitative sequential analys is of dipyrone" + ], + "container-title": [ + "Ecl\u00e9tica Qu\u00edmica Journal" + ], + %s + "page": "41-50" +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_doi b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_doi new file mode 100644 index 000000000..4183b3b2d --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_doi @@ -0,0 +1,34 @@ +"funder": [{"DOI": "10.13039/100010663","name": "H2020 European Research Council","doi-asserted-by": "publisher","award": ["677749"]}], +"funder": [{"name": "European Union’s Horizon 2020 research and innovation program","award": ["296801","304995","675395"]}], +"funder": [{"DOI": "10.13039/100010661","name": "Horizon 2020 Framework Programme","doi-asserted-by": "publisher","award": ["722467", "H2020-FETOPEN-2015-CSA 712689","773830 (2018–2022)"]}], +"funder": [{"DOI": "10.13039/501100007601","name": "Horizon 2020","doi-asserted-by": "publisher","award": ["645119"]}], +"funder": [{"DOI": "10.13039/100010665","name": "H2020 Marie Skłodowska-Curie Actions","doi-asserted-by": "publisher","award": ["840267"]}], +"funder": [{"DOI": "10.13039/100011199","name": "FP7 Ideas: European Research Council","doi-asserted-by": "publisher","award": ["226438"]}], +"funder": [{"DOI": "10.13039/100004431","name": "Directorate-General for Research and Innovation","doi-asserted-by": "publisher","award": ["321427"]}], +"funder": [{"DOI": "10.13039/501100004963","name": "Seventh Framework Programme","doi-asserted-by": "publisher","award": ["287818","612538"]}], +"funder": [{"DOI": "10.13039/501100000781","name": "European Research Council","doi-asserted-by": "publisher","award": ["340185"]}], +"funder": [{"name": "European Union's","award": ["763909"]}], +"funder": [{"DOI": "10.13039/501100000780","name": "European Commission","doi-asserted-by": "publisher","award": ["645119", "H2020-FETOPEN-2015-CSA_712689"]}], +"funder": [{"DOI": "10.13039/100000001","name": "National Science Foundation","doi-asserted-by": "publisher","award": ["1639552,1634422","ID0EEMBI7182"]}], +"funder": [{"name": "The French National Research Agency (ANR)","award": ["ID0E4QBI7183","ANR-11-JS56-01501","ID0E3VBI7184","ANR-13-BS06-0008"]}], +"funder": [{"DOI": "10.13039/501100001665","name": "Agence Nationale de la Recherche","doi-asserted-by": "publisher","award": ["ANR-14-ASTR-0004-01"]}], +"funder": [{"DOI": "10.13039/501100002341","name": "Academy of Finland","doi-asserted-by": "publisher","award": ["294337","292335","31444","250114","292482"]}], +"funder": [{"DOI": "10.13039/501100001602","name": "Science Foundation Ireland","doi-asserted-by": "publisher","award": ["16/SP/3829","12/RC/2302_P2","SFI/09/IN.I/12974"]}], +"funder": [{"DOI": "10.13039/501100000923","name": "Australian Research Council","doi-asserted-by": "publisher","award": ["LP110200134"]}], +"funder": [{"DOI": "10.13039/501100000038","name": "NSERC","doi-asserted-by": "crossref","award": []}], +"funder": [{"DOI": "10.13039/501100000155","name": "Social Sciences and Humanities Research Council of Canada","doi-asserted-by": "publisher","award": []}], +"funder": [{"DOI": "10.13039/501100000024","name": "Canadian Institutes for Health Research","doi-asserted-by": "crossref","award": ["HIB-126784","HHP-111405"]}], +"funder": [{"DOI": "10.13039/501100002848","name": "Comisión Nacional de Investigación Científica y Tecnológica","doi-asserted-by": "publisher","award": ["15130011"]}], +"funder": [{"DOI": "10.13039/501100003448","name": "General Secretariat for Research and Technology","doi-asserted-by": "publisher","award": ["MIS: 380170"]}], +"funder": [{"DOI": "10.13039/501100010198","name": "Ministerio de Economía, Industria y Competitividad, Gobierno de España","doi-asserted-by": "publisher","award": ["ECO2017-89715-P"]}], +"funder": [{"DOI": "10.13039/501100004564","name": "Ministarstvo Prosvete, Nauke i Tehnološkog Razvoja","doi-asserted-by": "publisher","award": ["TR34008"]}], +"funder": [{"DOI": "10.13039/501100003407","name": "MIUR","doi-asserted-by": "publisher","award": ["20158A9CBM"]}], +"funder": [{"DOI": "10.13039/501100003407","name": "MIUR","doi-asserted-by": "publisher","award": []}], +"funder": [{"DOI": "10.13039/501100006588","name": "Ministarstvo Znanosti, Obrazovanja i Sporta","doi-asserted-by": "publisher","award": ["037-0372790-2799", "Project No. 125-1253008-1350"]}], +"funder": [{"DOI": "10.13039/501100006588","name": "Ministry of Science, Education and Sports","doi-asserted-by": "publisher","award": ["181-1811096-1093"]}], +"funder": [{"DOI": "10.13039/501100004488","name": "Hrvatska Zaklada za Znanost","doi-asserted-by": "publisher","award": ["HRZZ-IP-2013-11-3013", "UIP-2014-09-4744"]}], +"funder": [{"DOI": "10.13039/501100006769","name": "Russian Science Foundation","doi-asserted-by": "publisher","award": ["17-11-01027"]}], +"funder": [{"DOI": "10.13039/501100001711","name": "Swiss National Science Foundation (Schweizerische Nationalfonds)","doi-asserted-by": "publisher","award": ["CR32I3_156724", "31003A_173281/1"]}], +"funder": [{"DOI": "10.13039/501100004410","name": "Türkiye Bilimsel ve Teknolojik Araştirma Kurumu","doi-asserted-by": "publisher","award": ["113M552"]}], +"funder": [{"DOI": "10.13039/100004440","name": "Wellcome Trust","doi-asserted-by": "crossref","award": ["095127","079080"]}], +"funder": [{"DOI": "10.13039/100004440","name": "Wellcome Trust","doi-asserted-by": "crossref","award": []}], \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_name b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_name new file mode 100644 index 000000000..b5c9232ac --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/funder_name @@ -0,0 +1,5 @@ +"funder": [{"name": "Wellcome Trust Masters Fellowship","award": ["090633"]}], +"funder": [{"name": "CONICYT, Programa de Formación de Capital Humano Avanzado","award": ["#72140079"]}], +"funder": [{"name": "European Union's","award": ["763909"]}], +"funder": [{"name": "European Union’s Horizon 2020 research and innovation program","award": ["296801","304995","675395"]}], +"funder": [{"name": "The French National Research Agency (ANR)","award": ["ID0E4QBI7183","ANR-11-JS56-01501","ID0E3VBI7184","ANR-13-BS06-0008"]}], diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java index d597f42a4..cfdff369c 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java @@ -36,6 +36,7 @@ public class SparkIndexCollectionOnES { final String index = parser.get("index"); final String idPath = parser.get("idPath"); final String type = parser.get("type"); + final String indexHost = parser.get("esHost"); final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); @@ -59,7 +60,8 @@ public class SparkIndexCollectionOnES { else inputRdd = sc.textFile(sourcePath); Map esCfg = new HashMap<>(); - esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54"); + // esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54"); + esCfg.put("es.nodes", indexHost); esCfg.put("es.mapping.id", idPath); esCfg.put("es.batch.write.retry.count", "8"); esCfg.put("es.batch.write.retry.wait", "60s"); diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/provision/index_on_es.json b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/provision/index_on_es.json index 905b6d514..f70f7dd79 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/provision/index_on_es.json +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/provision/index_on_es.json @@ -17,6 +17,13 @@ "paramDescription": "the index name", "paramRequired": true }, + { + "paramName": "h", + "paramLongName": "esHost", + "paramDescription": "the index host name", + "paramRequired": true + }, + { "paramName": "t", diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml index 9fc86e014..4f5c7bbf6 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/dhp/sx/index/oozie_app/workflow.xml @@ -16,6 +16,11 @@ index index name + + + indexHost + index host name + @@ -37,6 +42,7 @@ -mt yarn-cluster --sourcePath${workingDirPath}/summary --index${index}_object + --esHost${indexHost} --idPathid --typesummary @@ -57,6 +63,7 @@ -mt yarn-cluster --sourcePath${workingDirPath}/scholix_json --index${index}_scholix + --esHost${indexHost} --idPathidentifier --typescholix