implemented unit test

This commit is contained in:
Sandro La Bruzzo 2020-04-29 13:13:02 +02:00
parent fdc0523e4c
commit 09f161f1f4
11 changed files with 724 additions and 273 deletions

View File

@ -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 Unions 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

View File

@ -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"))
}

View File

@ -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)
})
}
}

View File

@ -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<String> res = JsonPath.read(json, "$.hits.hits[*]._source.blob");
System.out.println(res.size());
}
}

View File

@ -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"]}]
}

View File

@ -0,0 +1,175 @@
{
"DOI": "10.26850/1678-4618eqj.v35.1.2010.p41-46",
"issued": {
"date-parts": [
[
2018,
1,
15
]
]
},
"abstract": "<jats:p>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.</jats:p>",
"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"
}

View File

@ -0,0 +1,34 @@
"funder": [{"DOI": "10.13039/100010663","name": "H2020 European Research Council","doi-asserted-by": "publisher","award": ["677749"]}],
"funder": [{"name": "European Unions 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 (20182022)"]}],
"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": []}],

View File

@ -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 Unions 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"]}],

View File

@ -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<String, String> 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");

View File

@ -17,6 +17,13 @@
"paramDescription": "the index name",
"paramRequired": true
},
{
"paramName": "h",
"paramLongName": "esHost",
"paramDescription": "the index host name",
"paramRequired": true
},
{
"paramName": "t",

View File

@ -16,6 +16,11 @@
<name>index</name>
<description>index name</description>
</property>
<property>
<name>indexHost</name>
<description>index host name</description>
</property>
</parameters>
<start to="indexSummary"/>
@ -37,6 +42,7 @@
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${workingDirPath}/summary</arg>
<arg>--index</arg><arg>${index}_object</arg>
<arg>--esHost</arg><arg>${indexHost}</arg>
<arg>--idPath</arg><arg>id</arg>
<arg>--type</arg><arg>summary</arg>
</spark>
@ -57,6 +63,7 @@
<arg>-mt</arg> <arg>yarn-cluster</arg>
<arg>--sourcePath</arg><arg>${workingDirPath}/scholix_json</arg>
<arg>--index</arg><arg>${index}_scholix</arg>
<arg>--esHost</arg><arg>${indexHost}</arg>
<arg>--idPath</arg><arg>identifier</arg>
<arg>--type</arg><arg>scholix</arg>
</spark>