forked from antonis.lempesis/dnet-hadoop
Merge branch 'master' of https://code-repo.d4science.org/D-Net/dnet-hadoop into orcid-no-doi
This commit is contained in:
commit
f2df3ead74
|
@ -14,7 +14,7 @@ import scala.collection.JavaConverters._
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import scala.util.matching.Regex
|
import scala.util.matching.Regex
|
||||||
|
|
||||||
case class CrossrefDT(doi: String, json:String) {}
|
case class CrossrefDT(doi: String, json:String, timestamp: Long) {}
|
||||||
|
|
||||||
case class mappingAffiliation(name: String) {}
|
case class mappingAffiliation(name: String) {}
|
||||||
|
|
||||||
|
@ -265,18 +265,20 @@ case object Crossref2Oaf {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def generateRelation(sourceId:String, targetId:String, nsPrefix:String) :Relation = {
|
def generateRelation(sourceId:String, targetId:String, relClass:String) :Relation = {
|
||||||
|
|
||||||
val r = new Relation
|
val r = new Relation
|
||||||
r.setSource(sourceId)
|
r.setSource(sourceId)
|
||||||
r.setTarget(s"40|$nsPrefix::$targetId")
|
r.setTarget(targetId)
|
||||||
r.setRelType("resultProject")
|
r.setRelType("resultProject")
|
||||||
r.setRelClass("isProducedBy")
|
r.setRelClass(relClass)
|
||||||
r.setSubRelType("outcome")
|
r.setSubRelType("outcome")
|
||||||
r.setCollectedfrom(List(cf).asJava)
|
r.setCollectedfrom(List(cf).asJava)
|
||||||
r.setDataInfo(di)
|
r.setDataInfo(di)
|
||||||
r.setLastupdatetimestamp(ts)
|
r.setLastupdatetimestamp(ts)
|
||||||
r
|
r
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -284,12 +286,18 @@ case object Crossref2Oaf {
|
||||||
if (funder.award.isDefined && funder.award.get.nonEmpty)
|
if (funder.award.isDefined && funder.award.get.nonEmpty)
|
||||||
funder.award.get.map(extractField).filter(a => a!= null && a.nonEmpty).foreach(
|
funder.award.get.map(extractField).filter(a => a!= null && a.nonEmpty).foreach(
|
||||||
award => {
|
award => {
|
||||||
val targetId = DHPUtils.md5(award)
|
val targetId = getProjectId(nsPrefix, DHPUtils.md5(award))
|
||||||
queue += generateRelation(sourceId, targetId, nsPrefix)
|
queue += generateRelation(sourceId, targetId , "isProducedBy")
|
||||||
|
queue += generateRelation(targetId , sourceId, "produces")
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def getProjectId (nsPrefix:String, targetId:String):String = {
|
||||||
|
"40|$nsPrefix::$targetId"
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
if (funders != null)
|
if (funders != null)
|
||||||
funders.foreach(funder => {
|
funders.foreach(funder => {
|
||||||
if (funder.DOI.isDefined && funder.DOI.get.nonEmpty) {
|
if (funder.DOI.isDefined && funder.DOI.get.nonEmpty) {
|
||||||
|
@ -310,22 +318,33 @@ case object Crossref2Oaf {
|
||||||
case "10.13039/501100002341" => generateSimpleRelationFromAward(funder, "aka_________", 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/501100001602" => generateSimpleRelationFromAward(funder, "aka_________", a => a.replace("SFI", ""))
|
||||||
case "10.13039/501100000923" => generateSimpleRelationFromAward(funder, "arc_________", a => a)
|
case "10.13039/501100000923" => generateSimpleRelationFromAward(funder, "arc_________", a => a)
|
||||||
case "10.13039/501100000038"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "nserc_______" )
|
case "10.13039/501100000038"=> val targetId = getProjectId("nserc_______" , "1e5e62235d094afd01cd56e65112fc63")
|
||||||
case "10.13039/501100000155"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "sshrc_______" )
|
queue += generateRelation(sourceId, targetId, "isProducedBy" )
|
||||||
case "10.13039/501100000024"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "cihr________" )
|
queue += generateRelation(targetId, sourceId, "produces" )
|
||||||
|
case "10.13039/501100000155"=> val targetId = getProjectId("sshrc_______" , "1e5e62235d094afd01cd56e65112fc63")
|
||||||
|
queue += generateRelation(sourceId,targetId, "isProducedBy" )
|
||||||
|
queue += generateRelation(targetId,sourceId, "produces" )
|
||||||
|
case "10.13039/501100000024"=> val targetId = getProjectId("cihr________" , "1e5e62235d094afd01cd56e65112fc63")
|
||||||
|
queue += generateRelation(sourceId,targetId, "isProducedBy" )
|
||||||
|
queue += generateRelation(targetId,sourceId, "produces" )
|
||||||
case "10.13039/501100002848" => generateSimpleRelationFromAward(funder, "conicytf____", a => a)
|
case "10.13039/501100002848" => generateSimpleRelationFromAward(funder, "conicytf____", a => a)
|
||||||
case "10.13039/501100003448" => generateSimpleRelationFromAward(funder, "gsrt________", extractECAward)
|
case "10.13039/501100003448" => generateSimpleRelationFromAward(funder, "gsrt________", extractECAward)
|
||||||
case "10.13039/501100010198" => generateSimpleRelationFromAward(funder, "sgov________", a=>a)
|
case "10.13039/501100010198" => generateSimpleRelationFromAward(funder, "sgov________", a=>a)
|
||||||
case "10.13039/501100004564" => generateSimpleRelationFromAward(funder, "mestd_______", extractECAward)
|
case "10.13039/501100004564" => generateSimpleRelationFromAward(funder, "mestd_______", extractECAward)
|
||||||
case "10.13039/501100003407" => generateSimpleRelationFromAward(funder, "miur________", a=>a)
|
case "10.13039/501100003407" => generateSimpleRelationFromAward(funder, "miur________", a=>a)
|
||||||
queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "miur________" )
|
val targetId = getProjectId("miur________" , "1e5e62235d094afd01cd56e65112fc63")
|
||||||
|
queue += generateRelation(sourceId,targetId, "isProducedBy" )
|
||||||
|
queue += generateRelation(targetId,sourceId, "produces" )
|
||||||
case "10.13039/501100006588" |
|
case "10.13039/501100006588" |
|
||||||
"10.13039/501100004488" => generateSimpleRelationFromAward(funder, "irb_hr______", a=>a.replaceAll("Project No.", "").replaceAll("HRZZ-","") )
|
"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/501100006769"=> generateSimpleRelationFromAward(funder, "rsf_________", a=>a)
|
||||||
case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snsfRule)
|
case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snsfRule)
|
||||||
case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a)
|
case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a)
|
||||||
case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a)
|
case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a)
|
||||||
case "10.13039/100004440"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" )
|
case "10.13039/100004440"=> val targetId = getProjectId("wt__________" , "1e5e62235d094afd01cd56e65112fc63")
|
||||||
|
queue += generateRelation(sourceId,targetId, "isProducedBy" )
|
||||||
|
queue += generateRelation(targetId,sourceId, "produces" )
|
||||||
|
|
||||||
case _ => logger.debug("no match for "+funder.DOI.get )
|
case _ => logger.debug("no match for "+funder.DOI.get )
|
||||||
|
|
||||||
|
|
||||||
|
@ -341,7 +360,9 @@ case object Crossref2Oaf {
|
||||||
case "The French National Research Agency (ANR)" |
|
case "The French National Research Agency (ANR)" |
|
||||||
"The French National Research Agency" => generateSimpleRelationFromAward(funder, "anr_________", a => a)
|
"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 "CONICYT, Programa de Formación de Capital Humano Avanzado" => generateSimpleRelationFromAward(funder, "conicytf____", extractECAward)
|
||||||
case "Wellcome Trust Masters Fellowship" => queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" )
|
case "Wellcome Trust Masters Fellowship" => val targetId = getProjectId("wt__________", "1e5e62235d094afd01cd56e65112fc63")
|
||||||
|
queue += generateRelation(sourceId, targetId, "isProducedBy" )
|
||||||
|
queue += generateRelation(targetId, sourceId, "produces" )
|
||||||
case _ => logger.debug("no match for "+funder.name )
|
case _ => logger.debug("no match for "+funder.name )
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,69 +29,90 @@ object SparkMapDumpIntoOAF {
|
||||||
.appName(SparkMapDumpIntoOAF.getClass.getSimpleName)
|
.appName(SparkMapDumpIntoOAF.getClass.getSimpleName)
|
||||||
.master(parser.get("master")).getOrCreate()
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||||
implicit val mapEncoderRelatons: Encoder[Relation] = Encoders.kryo[Relation]
|
implicit val mapEncoderRelatons: Encoder[Relation] = Encoders.kryo[Relation]
|
||||||
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
|
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
|
||||||
|
|
||||||
val sc = spark.sparkContext
|
val sc = spark.sparkContext
|
||||||
val targetPath = parser.get("targetPath")
|
val targetPath = parser.get("targetPath")
|
||||||
|
import spark.implicits._
|
||||||
|
|
||||||
|
|
||||||
sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text])
|
spark.read.load(parser.get("sourcePath")).as[CrossrefDT]
|
||||||
.map(k => k._2.toString).map(CrossrefImporter.decompressBlob)
|
.flatMap(k => Crossref2Oaf.convert(k.json))
|
||||||
.flatMap(k => Crossref2Oaf.convert(k)).saveAsObjectFile(s"${targetPath}/mixObject")
|
.filter(o => o != null)
|
||||||
|
.write.mode(SaveMode.Overwrite).save(s"$targetPath/mixObject")
|
||||||
val inputRDD = sc.objectFile[Oaf](s"${targetPath}/mixObject").filter(p=> p!= null)
|
|
||||||
|
|
||||||
val distinctPubs:RDD[Publication] = inputRDD.filter(k => k != null && k.isInstanceOf[Publication])
|
|
||||||
.map(k => k.asInstanceOf[Publication]).map { p: Publication => Tuple2(p.getId, p) }.reduceByKey { case (p1: Publication, p2: Publication) =>
|
|
||||||
var r = if (p1 == null) p2 else p1
|
|
||||||
if (p1 != null && p2 != null) {
|
|
||||||
if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
|
|
||||||
if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
|
|
||||||
r = p2
|
|
||||||
else
|
|
||||||
r = p1
|
|
||||||
} else {
|
|
||||||
r = if (p1.getLastupdatetimestamp == null) p2 else p1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
r
|
|
||||||
}.map(_._2)
|
|
||||||
|
|
||||||
val pubs:Dataset[Publication] = spark.createDataset(distinctPubs)
|
|
||||||
pubs.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication")
|
|
||||||
|
|
||||||
|
|
||||||
val distincDatasets:RDD[OafDataset] = inputRDD.filter(k => k != null && k.isInstanceOf[OafDataset])
|
val ds:Dataset[Oaf] = spark.read.load(s"$targetPath/mixObject").as[Oaf]
|
||||||
.map(k => k.asInstanceOf[OafDataset]).map(p => Tuple2(p.getId, p)).reduceByKey { case (p1: OafDataset, p2: OafDataset) =>
|
|
||||||
var r = if (p1 == null) p2 else p1
|
|
||||||
if (p1 != null && p2 != null) {
|
|
||||||
if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
|
|
||||||
if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
|
|
||||||
r = p2
|
|
||||||
else
|
|
||||||
r = p1
|
|
||||||
} else {
|
|
||||||
r = if (p1.getLastupdatetimestamp == null) p2 else p1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
r
|
|
||||||
}.map(_._2)
|
|
||||||
|
|
||||||
spark.createDataset(distincDatasets).write.mode(SaveMode.Overwrite).save(s"${targetPath}/dataset")
|
ds.filter(o => o.isInstanceOf[Publication]).map(o => o.asInstanceOf[Publication]).write.save(s"$targetPath/publication")
|
||||||
|
|
||||||
|
ds.filter(o => o.isInstanceOf[Relation]).map(o => o.asInstanceOf[Relation]).write.save(s"$targetPath/relation")
|
||||||
|
|
||||||
|
ds.filter(o => o.isInstanceOf[OafDataset]).map(o => o.asInstanceOf[OafDataset]).write.save(s"$targetPath/dataset")
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
val distinctRels =inputRDD.filter(k => k != null && k.isInstanceOf[Relation])
|
//
|
||||||
.map(k => k.asInstanceOf[Relation]).map(r=> (s"${r.getSource}::${r.getTarget}",r))
|
//
|
||||||
.reduceByKey { case (p1: Relation, p2: Relation) =>
|
//
|
||||||
if (p1 == null) p2 else p1
|
// sc.sequenceFile(parser.get("sourcePath"), classOf[IntWritable], classOf[Text])
|
||||||
}.map(_._2)
|
// .map(k => k._2.toString).map(CrossrefImporter.decompressBlob)
|
||||||
|
// .flatMap(k => Crossref2Oaf.convert(k)).saveAsObjectFile(s"${targetPath}/mixObject")
|
||||||
val rels: Dataset[Relation] = spark.createDataset(distinctRels)
|
//
|
||||||
|
// val inputRDD = sc.objectFile[Oaf](s"${targetPath}/mixObject").filter(p=> p!= null)
|
||||||
rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations")
|
//
|
||||||
|
// val distinctPubs:RDD[Publication] = inputRDD.filter(k => k != null && k.isInstanceOf[Publication])
|
||||||
|
// .map(k => k.asInstanceOf[Publication]).map { p: Publication => Tuple2(p.getId, p) }.reduceByKey { case (p1: Publication, p2: Publication) =>
|
||||||
|
// var r = if (p1 == null) p2 else p1
|
||||||
|
// if (p1 != null && p2 != null) {
|
||||||
|
// if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
|
||||||
|
// if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
|
||||||
|
// r = p2
|
||||||
|
// else
|
||||||
|
// r = p1
|
||||||
|
// } else {
|
||||||
|
// r = if (p1.getLastupdatetimestamp == null) p2 else p1
|
||||||
|
// }
|
||||||
|
// }
|
||||||
|
// r
|
||||||
|
// }.map(_._2)
|
||||||
|
//
|
||||||
|
// val pubs:Dataset[Publication] = spark.createDataset(distinctPubs)
|
||||||
|
// pubs.write.mode(SaveMode.Overwrite).save(s"${targetPath}/publication")
|
||||||
|
//
|
||||||
|
//
|
||||||
|
// val distincDatasets:RDD[OafDataset] = inputRDD.filter(k => k != null && k.isInstanceOf[OafDataset])
|
||||||
|
// .map(k => k.asInstanceOf[OafDataset]).map(p => Tuple2(p.getId, p)).reduceByKey { case (p1: OafDataset, p2: OafDataset) =>
|
||||||
|
// var r = if (p1 == null) p2 else p1
|
||||||
|
// if (p1 != null && p2 != null) {
|
||||||
|
// if (p1.getLastupdatetimestamp != null && p2.getLastupdatetimestamp != null) {
|
||||||
|
// if (p1.getLastupdatetimestamp < p2.getLastupdatetimestamp)
|
||||||
|
// r = p2
|
||||||
|
// else
|
||||||
|
// r = p1
|
||||||
|
// } else {
|
||||||
|
// r = if (p1.getLastupdatetimestamp == null) p2 else p1
|
||||||
|
// }
|
||||||
|
// }
|
||||||
|
// r
|
||||||
|
// }.map(_._2)
|
||||||
|
//
|
||||||
|
// spark.createDataset(distincDatasets).write.mode(SaveMode.Overwrite).save(s"${targetPath}/dataset")
|
||||||
|
//
|
||||||
|
//
|
||||||
|
//
|
||||||
|
// val distinctRels =inputRDD.filter(k => k != null && k.isInstanceOf[Relation])
|
||||||
|
// .map(k => k.asInstanceOf[Relation]).map(r=> (s"${r.getSource}::${r.getTarget}",r))
|
||||||
|
// .reduceByKey { case (p1: Relation, p2: Relation) =>
|
||||||
|
// if (p1 == null) p2 else p1
|
||||||
|
// }.map(_._2)
|
||||||
|
//
|
||||||
|
// val rels: Dataset[Relation] = spark.createDataset(distinctRels)
|
||||||
|
//
|
||||||
|
// rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -16,10 +16,10 @@
|
||||||
<name>sparkExecutorCores</name>
|
<name>sparkExecutorCores</name>
|
||||||
<description>number of cores used by single executor</description>
|
<description>number of cores used by single executor</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<!-- <property>-->
|
||||||
<name>timestamp</name>
|
<!-- <name>timestamp</name>-->
|
||||||
<description>Timestamp for incremental Harvesting</description>
|
<!-- <description>Timestamp for incremental Harvesting</description>-->
|
||||||
</property>
|
<!-- </property>-->
|
||||||
|
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
|
@ -30,29 +30,29 @@
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
<action name="ResetWorkingPath">
|
<!-- <action name="ResetWorkingPath">-->
|
||||||
<fs>
|
<!-- <fs>-->
|
||||||
<delete path='${workingPath}/input/crossref/index_dump'/>
|
<!-- <delete path='${workingPath}/input/crossref/index_dump'/>-->
|
||||||
<!-- <mkdir path='${workingPath}/input/crossref'/>-->
|
<!--<!– <mkdir path='${workingPath}/input/crossref'/>–>-->
|
||||||
</fs>
|
<!-- </fs>-->
|
||||||
<ok to="ImportCrossRef"/>
|
<!-- <ok to="ImportCrossRef"/>-->
|
||||||
<error to="Kill"/>
|
<!-- <error to="Kill"/>-->
|
||||||
</action>
|
<!-- </action>-->
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
<action name="ImportCrossRef">
|
<!-- <action name="ImportCrossRef">-->
|
||||||
<java>
|
<!-- <java>-->
|
||||||
<job-tracker>${jobTracker}</job-tracker>
|
<!-- <job-tracker>${jobTracker}</job-tracker>-->
|
||||||
<name-node>${nameNode}</name-node>
|
<!-- <name-node>${nameNode}</name-node>-->
|
||||||
<main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>
|
<!-- <main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>-->
|
||||||
<arg>-t</arg><arg>${workingPath}/input/crossref/index_dump_1</arg>
|
<!-- <arg>-t</arg><arg>${workingPath}/input/crossref/index_dump_1</arg>-->
|
||||||
<arg>-n</arg><arg>${nameNode}</arg>
|
<!-- <arg>-n</arg><arg>${nameNode}</arg>-->
|
||||||
<arg>-ts</arg><arg>${timestamp}</arg>
|
<!-- <arg>-ts</arg><arg>${timestamp}</arg>-->
|
||||||
</java>
|
<!-- </java>-->
|
||||||
<ok to="End"/>
|
<!-- <ok to="End"/>-->
|
||||||
<error to="Kill"/>
|
<!-- <error to="Kill"/>-->
|
||||||
</action>
|
<!-- </action>-->
|
||||||
|
|
||||||
|
|
||||||
<action name="ExtractCrossrefToOAF">
|
<action name="ExtractCrossrefToOAF">
|
||||||
|
@ -68,7 +68,7 @@
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
${sparkExtraOPT}
|
${sparkExtraOPT}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${workingPath}/input/crossref/index_dump,${workingPath}/input/crossref/index_dump_1,${workingPath}/crossref/index_dump</arg>
|
<arg>--sourcePath</arg><arg>${workingPath}/input/crossref/crossref_ds</arg>
|
||||||
<arg>--targetPath</arg><arg>${workingPath}/input/crossref</arg>
|
<arg>--targetPath</arg><arg>${workingPath}/input/crossref</arg>
|
||||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
@ -78,26 +78,26 @@
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
<action name="GenerateDataset">
|
<!-- <action name="GenerateDataset">-->
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<!-- <spark xmlns="uri:oozie:spark-action:0.2">-->
|
||||||
<master>yarn-cluster</master>
|
<!-- <master>yarn-cluster</master>-->
|
||||||
<mode>cluster</mode>
|
<!-- <mode>cluster</mode>-->
|
||||||
<name>ExtractCrossrefToOAF</name>
|
<!-- <name>ExtractCrossrefToOAF</name>-->
|
||||||
<class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>
|
<!-- <class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>-->
|
||||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
<!-- <jar>dhp-doiboost-${projectVersion}.jar</jar>-->
|
||||||
<spark-opts>
|
<!-- <spark-opts>-->
|
||||||
--executor-memory=${sparkExecutorMemory}
|
<!-- --executor-memory=${sparkExecutorMemory}-->
|
||||||
--executor-cores=${sparkExecutorCores}
|
<!-- --executor-cores=${sparkExecutorCores}-->
|
||||||
--driver-memory=${sparkDriverMemory}
|
<!-- --driver-memory=${sparkDriverMemory}-->
|
||||||
${sparkExtraOPT}
|
<!-- ${sparkExtraOPT}-->
|
||||||
</spark-opts>
|
<!-- </spark-opts>-->
|
||||||
<arg>--sourcePath</arg><arg>/data/doiboost/crossref/cr_dataset</arg>
|
<!-- <arg>--sourcePath</arg><arg>/data/doiboost/crossref/cr_dataset</arg>-->
|
||||||
<arg>--targetPath</arg><arg>/data/doiboost/crossref/crossrefDataset</arg>
|
<!-- <arg>--targetPath</arg><arg>/data/doiboost/crossref/crossrefDataset</arg>-->
|
||||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
<!-- <arg>--master</arg><arg>yarn-cluster</arg>-->
|
||||||
</spark>
|
<!-- </spark>-->
|
||||||
<ok to="End"/>
|
<!-- <ok to="End"/>-->
|
||||||
<error to="Kill"/>
|
<!-- <error to="Kill"/>-->
|
||||||
</action>
|
<!-- </action>-->
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -1,63 +0,0 @@
|
||||||
package eu.dnetlib.dhp.doiboost
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation}
|
|
||||||
import org.apache.spark.SparkContext
|
|
||||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
|
|
||||||
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
|
|
||||||
import org.json4s
|
|
||||||
import org.json4s.DefaultFormats
|
|
||||||
import org.json4s.jackson.JsonMethods._
|
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
class QueryTest {
|
|
||||||
|
|
||||||
def extract_payload(input:String) :String = {
|
|
||||||
|
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
|
||||||
lazy val json: json4s.JValue = parse(input)
|
|
||||||
|
|
||||||
|
|
||||||
compact(render((json \ "payload")))
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def has_ands(r:Relation) :Boolean = {
|
|
||||||
|
|
||||||
r.getCollectedfrom!= null && r.getCollectedfrom.asScala.count(k => k.getValue.contains("Australian")) > 0
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
def hasInstanceWithUrl(p:Publication):Boolean = {
|
|
||||||
val c = p.getInstance.asScala.map(i => i.getUrl!= null && !i.getUrl.isEmpty).size
|
|
||||||
!(!p.getInstance.isEmpty && c == p.getInstance().size)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def hasNullAccessRights(p:Publication):Boolean = {
|
|
||||||
val c = p.getInstance.asScala.map(i => i.getAccessright!= null && i.getAccessright.getClassname.nonEmpty).size
|
|
||||||
!p.getInstance.isEmpty && c == p.getInstance().size()
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def myQuery(spark:SparkSession, sc:SparkContext): Unit = {
|
|
||||||
implicit val mapEncoderPub: Encoder[Publication] = Encoders.kryo[Publication]
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
val mapper = new ObjectMapper()
|
|
||||||
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
|
||||||
|
|
||||||
|
|
||||||
val ds:Dataset[Publication] = spark.read.load("/tmp/p").as[Publication]
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
ds.filter(p =>p.getBestaccessright!= null && p.getBestaccessright.getClassname.nonEmpty).count()
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -1,10 +1,7 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.graph.clean;
|
package eu.dnetlib.dhp.oa.graph.clean;
|
||||||
|
|
||||||
import java.util.HashSet;
|
import java.util.*;
|
||||||
import java.util.LinkedHashMap;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -18,7 +15,9 @@ import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
|
|
||||||
public class CleaningFunctions {
|
public class CleaningFunctions {
|
||||||
|
|
||||||
|
public static final String DOI_URL_PREFIX_REGEX = "(^http(s?):\\/\\/)(((dx\\.)?doi\\.org)|(handle\\.test\\.datacite\\.org))\\/";
|
||||||
public static final String ORCID_PREFIX_REGEX = "^http(s?):\\/\\/orcid\\.org\\/";
|
public static final String ORCID_PREFIX_REGEX = "^http(s?):\\/\\/orcid\\.org\\/";
|
||||||
|
public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)";
|
||||||
|
|
||||||
public static final Set<String> PID_BLACKLIST = new HashSet<>();
|
public static final Set<String> PID_BLACKLIST = new HashSet<>();
|
||||||
|
|
||||||
|
@ -79,7 +78,7 @@ public class CleaningFunctions {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static <T extends Oaf> T fixDefaults(T value) {
|
public static <T extends Oaf> T fixDefaults(T value) {
|
||||||
if (value instanceof Datasource) {
|
if (value instanceof Datasource) {
|
||||||
// nothing to clean here
|
// nothing to clean here
|
||||||
} else if (value instanceof Project) {
|
} else if (value instanceof Project) {
|
||||||
|
@ -112,6 +111,29 @@ public class CleaningFunctions {
|
||||||
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
|
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
|
||||||
.filter(sp -> Objects.nonNull(sp.getQualifier()))
|
.filter(sp -> Objects.nonNull(sp.getQualifier()))
|
||||||
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
|
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
|
||||||
|
.map(CleaningFunctions::cleanValue)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
if (Objects.nonNull(r.getTitle())) {
|
||||||
|
r
|
||||||
|
.setTitle(
|
||||||
|
r
|
||||||
|
.getTitle()
|
||||||
|
.stream()
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
|
||||||
|
.map(CleaningFunctions::cleanValue)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
if (Objects.nonNull(r.getDescription())) {
|
||||||
|
r
|
||||||
|
.setDescription(
|
||||||
|
r
|
||||||
|
.getDescription()
|
||||||
|
.stream()
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
|
||||||
|
.map(CleaningFunctions::cleanValue)
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
if (Objects.nonNull(r.getPid())) {
|
if (Objects.nonNull(r.getPid())) {
|
||||||
|
@ -125,10 +147,7 @@ public class CleaningFunctions {
|
||||||
.filter(sp -> !PID_BLACKLIST.contains(sp.getValue().trim().toLowerCase()))
|
.filter(sp -> !PID_BLACKLIST.contains(sp.getValue().trim().toLowerCase()))
|
||||||
.filter(sp -> Objects.nonNull(sp.getQualifier()))
|
.filter(sp -> Objects.nonNull(sp.getQualifier()))
|
||||||
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
|
.filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid()))
|
||||||
.map(sp -> {
|
.map(CleaningFunctions::normalizePidValue)
|
||||||
sp.setValue(StringUtils.trim(sp.getValue()));
|
|
||||||
return sp;
|
|
||||||
})
|
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {
|
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {
|
||||||
|
@ -211,6 +230,16 @@ public class CleaningFunctions {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected static StructuredProperty cleanValue(StructuredProperty s) {
|
||||||
|
s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " "));
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static Field<String> cleanValue(Field<String> s) {
|
||||||
|
s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " "));
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
|
||||||
// HELPERS
|
// HELPERS
|
||||||
|
|
||||||
private static void fixVocabName(Qualifier q, String vocabularyName) {
|
private static void fixVocabName(Qualifier q, String vocabularyName) {
|
||||||
|
@ -226,4 +255,24 @@ public class CleaningFunctions {
|
||||||
classid, classname, scheme, scheme);
|
classid, classname, scheme, scheme);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method that normalises PID values on a per-type basis.
|
||||||
|
* @param pid the PID whose value will be normalised.
|
||||||
|
* @return the PID containing the normalised value.
|
||||||
|
*/
|
||||||
|
public static StructuredProperty normalizePidValue(StructuredProperty pid) {
|
||||||
|
String value = Optional
|
||||||
|
.ofNullable(pid.getValue())
|
||||||
|
.map(String::trim)
|
||||||
|
.orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty"));
|
||||||
|
switch (pid.getQualifier().getClassid()) {
|
||||||
|
|
||||||
|
// TODO add cleaning for more PID types as needed
|
||||||
|
case "doi":
|
||||||
|
pid.setValue(value.toLowerCase().replaceAll(DOI_URL_PREFIX_REGEX, ""));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
return pid;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -12,6 +12,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.common.MakeTarArchive;
|
||||||
|
|
||||||
public class MakeTar implements Serializable {
|
public class MakeTar implements Serializable {
|
||||||
|
|
||||||
|
@ -41,71 +42,71 @@ public class MakeTar implements Serializable {
|
||||||
|
|
||||||
FileSystem fileSystem = FileSystem.get(conf);
|
FileSystem fileSystem = FileSystem.get(conf);
|
||||||
|
|
||||||
makeTArArchive(fileSystem, inputPath, outputPath);
|
MakeTarArchive.tarMaxSize(fileSystem, inputPath, outputPath, "scholix_dump", 25);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void makeTArArchive(FileSystem fileSystem, String inputPath, String outputPath) throws IOException {
|
// public static void makeTArArchive(FileSystem fileSystem, String inputPath, String outputPath) throws IOException {
|
||||||
|
//
|
||||||
RemoteIterator<LocatedFileStatus> dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath));
|
// RemoteIterator<LocatedFileStatus> dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath));
|
||||||
|
//
|
||||||
while (dir_iterator.hasNext()) {
|
// while (dir_iterator.hasNext()) {
|
||||||
LocatedFileStatus fileStatus = dir_iterator.next();
|
// LocatedFileStatus fileStatus = dir_iterator.next();
|
||||||
|
//
|
||||||
Path p = fileStatus.getPath();
|
// Path p = fileStatus.getPath();
|
||||||
String p_string = p.toString();
|
// String p_string = p.toString();
|
||||||
String entity = p_string.substring(p_string.lastIndexOf("/") + 1);
|
// String entity = p_string.substring(p_string.lastIndexOf("/") + 1);
|
||||||
|
//
|
||||||
write(fileSystem, p_string, outputPath + "/" + entity + ".tar", entity);
|
// write(fileSystem, p_string, outputPath + "/" + entity + ".tar", entity);
|
||||||
}
|
// }
|
||||||
|
//
|
||||||
}
|
// }
|
||||||
|
//
|
||||||
private static void write(FileSystem fileSystem, String inputPath, String outputPath, String dir_name)
|
// private static void write(FileSystem fileSystem, String inputPath, String outputPath, String dir_name)
|
||||||
throws IOException {
|
// throws IOException {
|
||||||
|
//
|
||||||
Path hdfsWritePath = new Path(outputPath);
|
// Path hdfsWritePath = new Path(outputPath);
|
||||||
FSDataOutputStream fsDataOutputStream = null;
|
// FSDataOutputStream fsDataOutputStream = null;
|
||||||
if (fileSystem.exists(hdfsWritePath)) {
|
// if (fileSystem.exists(hdfsWritePath)) {
|
||||||
fileSystem.delete(hdfsWritePath, true);
|
// fileSystem.delete(hdfsWritePath, true);
|
||||||
|
//
|
||||||
}
|
// }
|
||||||
fsDataOutputStream = fileSystem.create(hdfsWritePath);
|
// fsDataOutputStream = fileSystem.create(hdfsWritePath);
|
||||||
|
//
|
||||||
TarArchiveOutputStream ar = new TarArchiveOutputStream(fsDataOutputStream.getWrappedStream());
|
// TarArchiveOutputStream ar = new TarArchiveOutputStream(fsDataOutputStream.getWrappedStream());
|
||||||
|
//
|
||||||
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
// RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
||||||
.listFiles(
|
// .listFiles(
|
||||||
new Path(inputPath), true);
|
// new Path(inputPath), true);
|
||||||
|
//
|
||||||
while (fileStatusListIterator.hasNext()) {
|
// while (fileStatusListIterator.hasNext()) {
|
||||||
LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
// LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
||||||
|
//
|
||||||
Path p = fileStatus.getPath();
|
// Path p = fileStatus.getPath();
|
||||||
String p_string = p.toString();
|
// String p_string = p.toString();
|
||||||
if (!p_string.endsWith("_SUCCESS")) {
|
// if (!p_string.endsWith("_SUCCESS")) {
|
||||||
String name = p_string.substring(p_string.lastIndexOf("/") + 1);
|
// String name = p_string.substring(p_string.lastIndexOf("/") + 1);
|
||||||
TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name + ".json.gz");
|
// TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name + ".json.gz");
|
||||||
entry.setSize(fileStatus.getLen());
|
// entry.setSize(fileStatus.getLen());
|
||||||
ar.putArchiveEntry(entry);
|
// ar.putArchiveEntry(entry);
|
||||||
|
//
|
||||||
InputStream is = fileSystem.open(fileStatus.getPath());
|
// InputStream is = fileSystem.open(fileStatus.getPath());
|
||||||
|
//
|
||||||
BufferedInputStream bis = new BufferedInputStream(is);
|
// BufferedInputStream bis = new BufferedInputStream(is);
|
||||||
|
//
|
||||||
int count;
|
// int count;
|
||||||
byte data[] = new byte[1024];
|
// byte data[] = new byte[1024];
|
||||||
while ((count = bis.read(data, 0, data.length)) != -1) {
|
// while ((count = bis.read(data, 0, data.length)) != -1) {
|
||||||
ar.write(data, 0, count);
|
// ar.write(data, 0, count);
|
||||||
}
|
// }
|
||||||
bis.close();
|
// bis.close();
|
||||||
ar.closeArchiveEntry();
|
// ar.closeArchiveEntry();
|
||||||
|
//
|
||||||
}
|
// }
|
||||||
|
//
|
||||||
}
|
// }
|
||||||
|
//
|
||||||
ar.close();
|
// ar.close();
|
||||||
}
|
// }
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class SendToZenodoHDFS implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
zenodoApiClient.sendMretadata(metadata);
|
zenodoApiClient.sendMretadata(metadata);
|
||||||
zenodoApiClient.publish();
|
// zenodoApiClient.publish();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -8,10 +8,10 @@
|
||||||
<name>targetPath</name>
|
<name>targetPath</name>
|
||||||
<description>the target path</description>
|
<description>the target path</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<!-- <property>-->
|
||||||
<name>metadata</name>
|
<!-- <name>metadata</name>-->
|
||||||
<description>the metadata</description>
|
<!-- <description>the metadata</description>-->
|
||||||
</property>
|
<!-- </property>-->
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="send_zenodo"/>
|
<start to="send_zenodo"/>
|
||||||
|
@ -34,20 +34,20 @@
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
|
||||||
<action name="send_zenodo">
|
<!-- <action name="send_zenodo">-->
|
||||||
<java>
|
<!-- <java>-->
|
||||||
<main-class>eu.dnetlib.dhp.export.zenodo.SendToZenodoHDFS</main-class>
|
<!-- <main-class>eu.dnetlib.dhp.export.zenodo.SendToZenodoHDFS</main-class>-->
|
||||||
<arg>--hdfsPath</arg><arg>/user/dnet.scholexplorer/scholix/provision/scholix.tar/scholix-2020-10-16.tar</arg>
|
<!-- <arg>--hdfsPath</arg><arg>/user/dnet.scholexplorer/scholix/provision/scholix.tar/scholix-2020-10-16.tar</arg>-->
|
||||||
<arg>--nameNode</arg><arg>${nameNode}</arg>
|
<!-- <arg>--nameNode</arg><arg>${nameNode}</arg>-->
|
||||||
<arg>--accessToken</arg><arg>b6ddrY6b77WxcDEevn9gqVE5sL5sDNjdUijt75W3o7cQo5vpFFI48dMiu8Gv</arg>
|
<!-- <arg>--accessToken</arg><arg>b6ddrY6b77WxcDEevn9gqVE5sL5sDNjdUijt75W3o7cQo5vpFFI48dMiu8Gv</arg>-->
|
||||||
<arg>--connectionUrl</arg><arg>https://zenodo.org/api/deposit/depositions</arg>
|
<!-- <arg>--connectionUrl</arg><arg>https://zenodo.org/api/deposit/depositions</arg>-->
|
||||||
<arg>--metadata</arg><arg>${metadata}</arg>
|
<!-- <arg>--metadata</arg><arg>${metadata}</arg>-->
|
||||||
<arg>--conceptRecordId</arg><arg>1200252</arg>
|
<!-- <arg>--conceptRecordId</arg><arg>1200252</arg>-->
|
||||||
<arg>--newDeposition</arg><arg>false</arg>
|
<!-- <arg>--newDeposition</arg><arg>false</arg>-->
|
||||||
</java>
|
<!-- </java>-->
|
||||||
<ok to="End"/>
|
<!-- <ok to="End"/>-->
|
||||||
<error to="Kill"/>
|
<!-- <error to="Kill"/>-->
|
||||||
</action>
|
<!-- </action>-->
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
Loading…
Reference in New Issue