Merge branch 'master' of https://code-repo.d4science.org/D-Net/dnet-hadoop
This commit is contained in:
commit
5e8509bef7
|
@ -6,7 +6,7 @@
|
|||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<artifactId>dhp</artifactId>
|
||||
<version>1.2.4-SNAPSHOT</version>
|
||||
<relativePath>../</relativePath>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<artifactId>dhp-schemas</artifactId>
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
package eu.dnetlib.dhp.schema.orcid;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class OrcidDOI {
|
||||
private String doi;
|
||||
private List<AuthorData> authors;
|
||||
|
||||
public String getDoi() {
|
||||
return doi;
|
||||
}
|
||||
|
||||
public void setDoi(String doi) {
|
||||
this.doi = doi;
|
||||
}
|
||||
|
||||
public List<AuthorData> getAuthors() {
|
||||
return authors;
|
||||
}
|
||||
|
||||
public void setAuthors(List<AuthorData> authors) {
|
||||
this.authors = authors;
|
||||
}
|
||||
}
|
|
@ -200,7 +200,7 @@ case object Crossref2Oaf {
|
|||
a.setSurname(family)
|
||||
a.setFullname(s"$given $family")
|
||||
if (StringUtils.isNotBlank(orcid))
|
||||
a.setPid(List(createSP(orcid, ORCID, PID_TYPES)).asJava)
|
||||
a.setPid(List(createSP(orcid, ORCID, PID_TYPES, generateDataInfo())).asJava)
|
||||
|
||||
a
|
||||
}
|
||||
|
@ -248,7 +248,7 @@ case object Crossref2Oaf {
|
|||
|
||||
|
||||
def snsfRule(award:String): String = {
|
||||
var tmp1 = StringUtils.substringAfter(award,"_")
|
||||
val tmp1 = StringUtils.substringAfter(award,"_")
|
||||
val tmp2 = StringUtils.substringBefore(tmp1,"/")
|
||||
logger.debug(s"From $award to $tmp2")
|
||||
tmp2
|
||||
|
@ -294,7 +294,7 @@ case object Crossref2Oaf {
|
|||
}
|
||||
|
||||
def getProjectId (nsPrefix:String, targetId:String):String = {
|
||||
"40|$nsPrefix::$targetId"
|
||||
s"40|$nsPrefix::$targetId"
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -2,6 +2,7 @@ package eu.dnetlib.doiboost.crossref
|
|||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.hadoop.io.{IntWritable, Text}
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
|
@ -12,21 +13,23 @@ import org.slf4j.{Logger, LoggerFactory}
|
|||
|
||||
object CrossrefDataset {
|
||||
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
||||
|
||||
def extractTimestamp(input:String): Long = {
|
||||
|
||||
def to_item(input:String):CrossrefDT = {
|
||||
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: json4s.JValue = parse(input)
|
||||
|
||||
(json\"indexed"\"timestamp").extractOrElse[Long](0)
|
||||
val ts:Long = (json \ "indexed" \ "timestamp").extract[Long]
|
||||
val doi:String = (json \ "DOI").extract[String]
|
||||
CrossrefDT(doi, input, ts)
|
||||
|
||||
}
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
||||
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(CrossrefDataset.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref_to_dataset_params.json")))
|
||||
parser.parseArgument(args)
|
||||
|
@ -49,9 +52,8 @@ object CrossrefDataset {
|
|||
if (a == null)
|
||||
return b
|
||||
|
||||
val tb = extractTimestamp(b.json)
|
||||
val ta = extractTimestamp(a.json)
|
||||
if(ta >tb) {
|
||||
|
||||
if(a.timestamp >b.timestamp) {
|
||||
return a
|
||||
}
|
||||
b
|
||||
|
@ -63,9 +65,7 @@ object CrossrefDataset {
|
|||
if (a == null)
|
||||
return b
|
||||
|
||||
val tb = extractTimestamp(b.json)
|
||||
val ta = extractTimestamp(a.json)
|
||||
if(ta >tb) {
|
||||
if(a.timestamp >b.timestamp) {
|
||||
return a
|
||||
}
|
||||
b
|
||||
|
@ -78,15 +78,21 @@ object CrossrefDataset {
|
|||
override def finish(reduction: CrossrefDT): CrossrefDT = reduction
|
||||
}
|
||||
|
||||
val sourcePath:String = parser.get("sourcePath")
|
||||
val targetPath:String = parser.get("targetPath")
|
||||
val workingPath:String = parser.get("workingPath")
|
||||
|
||||
val ds:Dataset[CrossrefDT] = spark.read.load(sourcePath).as[CrossrefDT]
|
||||
|
||||
ds.groupByKey(_.doi)
|
||||
val main_ds:Dataset[CrossrefDT] = spark.read.load(s"$workingPath/crossref_ds").as[CrossrefDT]
|
||||
|
||||
|
||||
val update =
|
||||
spark.createDataset(spark.sparkContext.sequenceFile(s"$workingPath/index_update", classOf[IntWritable], classOf[Text])
|
||||
.map(i =>CrossrefImporter.decompressBlob(i._2.toString))
|
||||
.map(i =>to_item(i)))
|
||||
|
||||
main_ds.union(update).groupByKey(_.doi)
|
||||
.agg(crossrefAggregator.toColumn)
|
||||
.map(s=>s._2)
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/crossref_ds_updated")
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -34,85 +34,21 @@ object SparkMapDumpIntoOAF {
|
|||
implicit val mapEncoderRelatons: Encoder[Relation] = Encoders.kryo[Relation]
|
||||
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
|
||||
|
||||
val sc = spark.sparkContext
|
||||
val targetPath = parser.get("targetPath")
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
spark.read.load(parser.get("sourcePath")).as[CrossrefDT]
|
||||
.flatMap(k => Crossref2Oaf.convert(k.json))
|
||||
.filter(o => o != null)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$targetPath/mixObject")
|
||||
|
||||
|
||||
val ds:Dataset[Oaf] = spark.read.load(s"$targetPath/mixObject").as[Oaf]
|
||||
|
||||
ds.filter(o => o.isInstanceOf[Publication]).map(o => o.asInstanceOf[Publication]).write.save(s"$targetPath/publication")
|
||||
ds.filter(o => o.isInstanceOf[Publication]).map(o => o.asInstanceOf[Publication]).write.mode(SaveMode.Overwrite).save(s"$targetPath/crossrefPublication")
|
||||
|
||||
ds.filter(o => o.isInstanceOf[Relation]).map(o => o.asInstanceOf[Relation]).write.save(s"$targetPath/relation")
|
||||
ds.filter(o => o.isInstanceOf[Relation]).map(o => o.asInstanceOf[Relation]).write.mode(SaveMode.Overwrite).save(s"$targetPath/crossrefRelation")
|
||||
|
||||
ds.filter(o => o.isInstanceOf[OafDataset]).map(o => o.asInstanceOf[OafDataset]).write.save(s"$targetPath/dataset")
|
||||
|
||||
|
||||
|
||||
//
|
||||
//
|
||||
//
|
||||
// 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 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")
|
||||
ds.filter(o => o.isInstanceOf[OafDataset]).map(o => o.asInstanceOf[OafDataset]).write.mode(SaveMode.Overwrite).save(s"$targetPath/crossrefDataset")
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -16,88 +16,86 @@
|
|||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
<!-- <property>-->
|
||||
<!-- <name>timestamp</name>-->
|
||||
<!-- <description>Timestamp for incremental Harvesting</description>-->
|
||||
<!-- </property>-->
|
||||
<property>
|
||||
<name>timestamp</name>
|
||||
<description>Timestamp for incremental Harvesting</description>
|
||||
</property>
|
||||
|
||||
</parameters>
|
||||
|
||||
<start to="ExtractCrossrefToOAF"/>
|
||||
<start to="ImportCrossRef"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<!-- <action name="ResetWorkingPath">-->
|
||||
<!-- <fs>-->
|
||||
<!-- <delete path='${workingPath}/input/crossref/index_dump'/>-->
|
||||
<!--<!– <mkdir path='${workingPath}/input/crossref'/>–>-->
|
||||
<!-- </fs>-->
|
||||
<!-- <ok to="ImportCrossRef"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
<action name="ImportCrossRef">
|
||||
<java>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>
|
||||
<arg>-t</arg><arg>${workingPath}/input/crossref/index_update</arg>
|
||||
<arg>-n</arg><arg>${nameNode}</arg>
|
||||
<arg>-ts</arg><arg>${timestamp}</arg>
|
||||
</java>
|
||||
<ok to="GenerateDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="GenerateDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ExtractCrossrefToOAF</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>/data/doiboost/input/crossref</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="RenameDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="RenameDataset">
|
||||
<fs>
|
||||
<delete path='${workingPath}/input/crossref/crossref_ds'/>
|
||||
<move source="${workingPath}/input/crossref/crossref_ds_updated"
|
||||
target="${workingPath}/input/crossref/crossref_ds"/>
|
||||
</fs>
|
||||
<ok to="ConvertCrossrefToOAF"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<!-- <action name="ImportCrossRef">-->
|
||||
<!-- <java>-->
|
||||
<!-- <job-tracker>${jobTracker}</job-tracker>-->
|
||||
<!-- <name-node>${nameNode}</name-node>-->
|
||||
<!-- <main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>-->
|
||||
<!-- <arg>-t</arg><arg>${workingPath}/input/crossref/index_dump_1</arg>-->
|
||||
<!-- <arg>-n</arg><arg>${nameNode}</arg>-->
|
||||
<!-- <arg>-ts</arg><arg>${timestamp}</arg>-->
|
||||
<!-- </java>-->
|
||||
<!-- <ok to="End"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
|
||||
<action name="ExtractCrossrefToOAF">
|
||||
<action name="ConvertCrossrefToOAF">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ExtractCrossrefToOAF</name>
|
||||
<name>ConvertCrossrefToOAF</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingPath}/input/crossref/crossref_ds</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/input/crossref</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/process/</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<!-- <action name="GenerateDataset">-->
|
||||
<!-- <spark xmlns="uri:oozie:spark-action:0.2">-->
|
||||
<!-- <master>yarn-cluster</master>-->
|
||||
<!-- <mode>cluster</mode>-->
|
||||
<!-- <name>ExtractCrossrefToOAF</name>-->
|
||||
<!-- <class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>-->
|
||||
<!-- <jar>dhp-doiboost-${projectVersion}.jar</jar>-->
|
||||
<!-- <spark-opts>-->
|
||||
<!-- --executor-memory=${sparkExecutorMemory}-->
|
||||
<!-- --executor-cores=${sparkExecutorCores}-->
|
||||
<!-- --driver-memory=${sparkDriverMemory}-->
|
||||
<!-- ${sparkExtraOPT}-->
|
||||
<!-- </spark-opts>-->
|
||||
<!-- <arg>--sourcePath</arg><arg>/data/doiboost/crossref/cr_dataset</arg>-->
|
||||
<!-- <arg>--targetPath</arg><arg>/data/doiboost/crossref/crossrefDataset</arg>-->
|
||||
<!-- <arg>--master</arg><arg>yarn-cluster</arg>-->
|
||||
<!-- </spark>-->
|
||||
<!-- <ok to="End"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -1,6 +1,5 @@
|
|||
[
|
||||
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true},
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the working dir path", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working dir path", "paramRequired": true},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
|
||||
]
|
|
@ -7,7 +7,7 @@
|
|||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<artifactId>dhp</artifactId>
|
||||
<version>1.2.4-SNAPSHOT</version>
|
||||
<relativePath>../</relativePath>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<artifactId>dhp-workflows</artifactId>
|
||||
|
|
Loading…
Reference in New Issue