forked from D-Net/dnet-hadoop
implemented unit test
This commit is contained in:
parent
fdc0523e4c
commit
09f161f1f4
|
@ -1,5 +1,7 @@
|
||||||
package eu.dnetlib.doiboost.crossref
|
package eu.dnetlib.doiboost.crossref
|
||||||
|
|
||||||
|
import java.util
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf._
|
import eu.dnetlib.dhp.schema.oaf._
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils
|
import eu.dnetlib.dhp.utils.DHPUtils
|
||||||
import org.apache.commons.lang.StringUtils
|
import org.apache.commons.lang.StringUtils
|
||||||
|
@ -7,14 +9,21 @@ import org.json4s
|
||||||
import org.json4s.DefaultFormats
|
import org.json4s.DefaultFormats
|
||||||
import org.json4s.JsonAST._
|
import org.json4s.JsonAST._
|
||||||
import org.json4s.jackson.JsonMethods._
|
import org.json4s.jackson.JsonMethods._
|
||||||
import org.slf4j.Logger
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
|
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 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 {
|
case object Crossref2Oaf {
|
||||||
|
val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass)
|
||||||
|
|
||||||
//STATIC STRING
|
//STATIC STRING
|
||||||
val MAG = "MAG"
|
val MAG = "MAG"
|
||||||
|
@ -83,7 +92,6 @@ case object Crossref2Oaf {
|
||||||
"report" -> "0017 Report"
|
"report" -> "0017 Report"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = {
|
def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = {
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||||
|
|
||||||
|
@ -143,13 +151,10 @@ case object Crossref2Oaf {
|
||||||
|
|
||||||
|
|
||||||
//Mapping AUthor
|
//Mapping AUthor
|
||||||
|
|
||||||
val authorList: List[mappingAuthor] = (json \ "author").extractOrElse[List[mappingAuthor]](List())
|
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)
|
result.setAuthor(authorList.map(a => generateAuhtor(a.given.orNull, a.family, a.ORCID.orNull)).asJava)
|
||||||
|
|
||||||
// Mapping instance
|
// Mapping instance
|
||||||
|
|
||||||
val instance = new Instance()
|
val instance = new Instance()
|
||||||
val license = for {
|
val license = for {
|
||||||
JString(lic) <- json \ "license" \ "URL"
|
JString(lic) <- json \ "license" \ "URL"
|
||||||
|
@ -182,37 +187,146 @@ case object Crossref2Oaf {
|
||||||
a.setFullname(s"${given} ${family}")
|
a.setFullname(s"${given} ${family}")
|
||||||
if (StringUtils.isNotBlank(orcid))
|
if (StringUtils.isNotBlank(orcid))
|
||||||
a.setPid(List(createSP(orcid, ORCID, PID_TYPES)).asJava)
|
a.setPid(List(createSP(orcid, ORCID, PID_TYPES)).asJava)
|
||||||
|
|
||||||
a
|
a
|
||||||
}
|
}
|
||||||
|
|
||||||
def convert(input: String, logger: Logger): Result = {
|
def convert(input: String): List[Oaf] = {
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||||
lazy val json: json4s.JValue = parse(input)
|
lazy val json: json4s.JValue = parse(input)
|
||||||
|
|
||||||
|
|
||||||
|
var resultList: List[Oaf] = List()
|
||||||
|
|
||||||
|
|
||||||
val objectType = (json \ "type").extractOrElse[String](null)
|
val objectType = (json \ "type").extractOrElse[String](null)
|
||||||
val objectSubType = (json \ "subtype").extractOrElse[String](null)
|
val objectSubType = (json \ "subtype").extractOrElse[String](null)
|
||||||
if (objectType == null)
|
if (objectType == null)
|
||||||
return null
|
return resultList
|
||||||
|
|
||||||
|
|
||||||
val result = generateItemFromType(objectType, objectSubType)
|
val result = generateItemFromType(objectType, objectSubType)
|
||||||
if (result == null)
|
if (result == null)
|
||||||
return result
|
return List()
|
||||||
val cOBJCategory = mappingCrossrefSubType.getOrElse(objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type"));
|
val cOBJCategory = mappingCrossrefSubType.getOrElse(objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type"));
|
||||||
// logger.debug(mappingCrossrefType(objectType))
|
|
||||||
// logger.debug(cOBJCategory)
|
|
||||||
|
|
||||||
mappingResult(result, json, 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 {
|
result match {
|
||||||
case publication: Publication => convertPublication(publication, json, cOBJCategory)
|
case publication: Publication => convertPublication(publication, json, cOBJCategory)
|
||||||
case dataset: Dataset => convertDataset(dataset)
|
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 = {
|
def convertDataset(dataset: Dataset): Unit = {
|
||||||
|
@ -259,10 +373,8 @@ case object Crossref2Oaf {
|
||||||
case "print" => journal.setIssnPrinted(tp._2)
|
case "print" => journal.setIssnPrinted(tp._2)
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
}
|
}
|
||||||
journal.setVol(volume)
|
journal.setVol(volume)
|
||||||
|
|
||||||
val page = (json \ "page").extractOrElse[String](null)
|
val page = (json \ "page").extractOrElse[String](null)
|
||||||
if (page != null) {
|
if (page != null) {
|
||||||
val pp = page.split("-")
|
val pp = page.split("-")
|
||||||
|
@ -270,8 +382,6 @@ case object Crossref2Oaf {
|
||||||
if (pp.size > 1)
|
if (pp.size > 1)
|
||||||
journal.setEp(pp(1))
|
journal.setEp(pp(1))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
publication.setJournal(journal)
|
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 = {
|
def createCollectedFrom(): KeyValue = {
|
||||||
|
|
||||||
val cf = new KeyValue
|
val cf = new KeyValue
|
||||||
|
|
|
@ -1,11 +1,12 @@
|
||||||
package eu.dnetlib.doiboost.crossref
|
package eu.dnetlib.doiboost.crossref
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
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.commons.io.IOUtils
|
||||||
import org.apache.hadoop.io.{IntWritable, Text}
|
import org.apache.hadoop.io.{IntWritable, Text}
|
||||||
import org.apache.spark.SparkConf
|
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}
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
|
||||||
|
|
||||||
|
@ -26,42 +27,46 @@ object SparkMapDumpIntoOAF {
|
||||||
.config(conf)
|
.config(conf)
|
||||||
.appName(SparkMapDumpIntoOAF.getClass.getSimpleName)
|
.appName(SparkMapDumpIntoOAF.getClass.getSimpleName)
|
||||||
.master(parser.get("master")).getOrCreate()
|
.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 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")
|
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"))
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
|
|
||||||
|
})
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1,25 +1,18 @@
|
||||||
package eu.dnetlib.doiboost;
|
package eu.dnetlib.doiboost;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.*;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.SerializationFeature;
|
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 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.jupiter.api.Test;
|
||||||
import org.junit.platform.commons.util.StringUtils;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
public class DoiBoostTest {
|
public class DoiBoostTest {
|
||||||
|
|
||||||
|
final ObjectMapper mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT);
|
||||||
Logger logger = LoggerFactory.getLogger(DoiBoostTest.class);
|
Logger logger = LoggerFactory.getLogger(DoiBoostTest.class);
|
||||||
|
|
||||||
|
@Test
|
||||||
public void test() throws Exception {
|
public void test() throws Exception {
|
||||||
|
|
||||||
// SparkDownloadContentFromCrossref.main(null);
|
// SparkDownloadContentFromCrossref.main(null);
|
||||||
|
@ -29,213 +22,4 @@ public class DoiBoostTest {
|
||||||
"-m local[*] -s file:///data/doiboost/crossref_dump.seq -t /data/doiboost"
|
"-m local[*] -s file:///data/doiboost/crossref_dump.seq -t /data/doiboost"
|
||||||
.split(" "));
|
.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());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -170,5 +170,6 @@
|
||||||
"container-title": [
|
"container-title": [
|
||||||
"Ecl\u00e9tica Qu\u00edmica Journal"
|
"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"]}]
|
||||||
}
|
}
|
|
@ -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"
|
||||||
|
}
|
|
@ -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": []}],
|
|
@ -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"]}],
|
|
@ -36,6 +36,7 @@ public class SparkIndexCollectionOnES {
|
||||||
final String index = parser.get("index");
|
final String index = parser.get("index");
|
||||||
final String idPath = parser.get("idPath");
|
final String idPath = parser.get("idPath");
|
||||||
final String type = parser.get("type");
|
final String type = parser.get("type");
|
||||||
|
final String indexHost = parser.get("esHost");
|
||||||
|
|
||||||
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
||||||
|
|
||||||
|
@ -59,7 +60,8 @@ public class SparkIndexCollectionOnES {
|
||||||
else inputRdd = sc.textFile(sourcePath);
|
else inputRdd = sc.textFile(sourcePath);
|
||||||
|
|
||||||
Map<String, String> esCfg = new HashMap<>();
|
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.mapping.id", idPath);
|
||||||
esCfg.put("es.batch.write.retry.count", "8");
|
esCfg.put("es.batch.write.retry.count", "8");
|
||||||
esCfg.put("es.batch.write.retry.wait", "60s");
|
esCfg.put("es.batch.write.retry.wait", "60s");
|
||||||
|
|
|
@ -17,6 +17,13 @@
|
||||||
"paramDescription": "the index name",
|
"paramDescription": "the index name",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
"paramName": "h",
|
||||||
|
"paramLongName": "esHost",
|
||||||
|
"paramDescription": "the index host name",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
|
||||||
|
|
||||||
{
|
{
|
||||||
"paramName": "t",
|
"paramName": "t",
|
||||||
|
|
|
@ -16,6 +16,11 @@
|
||||||
<name>index</name>
|
<name>index</name>
|
||||||
<description>index name</description>
|
<description>index name</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>indexHost</name>
|
||||||
|
<description>index host name</description>
|
||||||
|
</property>
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="indexSummary"/>
|
<start to="indexSummary"/>
|
||||||
|
@ -37,6 +42,7 @@
|
||||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||||
<arg>--sourcePath</arg><arg>${workingDirPath}/summary</arg>
|
<arg>--sourcePath</arg><arg>${workingDirPath}/summary</arg>
|
||||||
<arg>--index</arg><arg>${index}_object</arg>
|
<arg>--index</arg><arg>${index}_object</arg>
|
||||||
|
<arg>--esHost</arg><arg>${indexHost}</arg>
|
||||||
<arg>--idPath</arg><arg>id</arg>
|
<arg>--idPath</arg><arg>id</arg>
|
||||||
<arg>--type</arg><arg>summary</arg>
|
<arg>--type</arg><arg>summary</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
@ -57,6 +63,7 @@
|
||||||
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
<arg>-mt</arg> <arg>yarn-cluster</arg>
|
||||||
<arg>--sourcePath</arg><arg>${workingDirPath}/scholix_json</arg>
|
<arg>--sourcePath</arg><arg>${workingDirPath}/scholix_json</arg>
|
||||||
<arg>--index</arg><arg>${index}_scholix</arg>
|
<arg>--index</arg><arg>${index}_scholix</arg>
|
||||||
|
<arg>--esHost</arg><arg>${indexHost}</arg>
|
||||||
<arg>--idPath</arg><arg>identifier</arg>
|
<arg>--idPath</arg><arg>identifier</arg>
|
||||||
<arg>--type</arg><arg>scholix</arg>
|
<arg>--type</arg><arg>scholix</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
|
Loading…
Reference in New Issue