forked from D-Net/dnet-hadoop
merged manually changes on stable_id for doiboost into master
This commit is contained in:
parent
a801999e75
commit
1adfc41d23
|
@ -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-common</artifactId>
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
package eu.dnetlib.doiboost.crossref
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf._
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
|
||||
|
@ -13,12 +14,13 @@ import org.slf4j.{Logger, LoggerFactory}
|
|||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.matching.Regex
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils
|
||||
|
||||
case class CrossrefDT(doi: String, json:String, timestamp: Long) {}
|
||||
|
||||
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, sequence:Option[String], ORCID: Option[String], affiliation: Option[mappingAffiliation]) {}
|
||||
|
||||
case class mappingFunder(name: String, DOI: Option[String], award: Option[List[String]]) {}
|
||||
|
||||
|
@ -154,7 +156,12 @@ case object Crossref2Oaf {
|
|||
|
||||
//Mapping Author
|
||||
val authorList: List[mappingAuthor] = (json \ "author").extractOrElse[List[mappingAuthor]](List())
|
||||
result.setAuthor(authorList.map(a => generateAuhtor(a.given.orNull, a.family, a.ORCID.orNull)).asJava)
|
||||
|
||||
|
||||
|
||||
val sorted_list = authorList.sortWith((a:mappingAuthor, b:mappingAuthor) => a.sequence.isDefined && a.sequence.get.equalsIgnoreCase("first"))
|
||||
|
||||
result.setAuthor(sorted_list.zipWithIndex.map{case (a, index) => generateAuhtor(a.given.orNull, a.family, a.ORCID.orNull, index)}.asJava)
|
||||
|
||||
// Mapping instance
|
||||
val instance = new Instance()
|
||||
|
@ -170,14 +177,14 @@ case object Crossref2Oaf {
|
|||
|
||||
if(has_review != JNothing) {
|
||||
instance.setRefereed(
|
||||
createQualifier("0001", "peerReviewed", "dnet:review_levels", "dnet:review_levels"))
|
||||
OafUtils.createQualifier("0001", "peerReviewed", ModelConstants.DNET_REVIEW_LEVELS, ModelConstants.DNET_REVIEW_LEVELS))
|
||||
}
|
||||
|
||||
|
||||
instance.setAccessright(getRestrictedQualifier())
|
||||
result.setInstance(List(instance).asJava)
|
||||
instance.setInstancetype(createQualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), "dnet:publication_resource", "dnet:publication_resource"))
|
||||
result.setResourcetype(createQualifier(cobjCategory.substring(0, 4),"dnet:dataCite_resource"))
|
||||
instance.setInstancetype(OafUtils.createQualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
result.setResourcetype(OafUtils.createQualifier(cobjCategory.substring(0, 4),"dnet:dataCite_resource"))
|
||||
|
||||
instance.setCollectedfrom(createCrossrefCollectedFrom())
|
||||
if (StringUtils.isNotBlank(issuedDate)) {
|
||||
|
@ -194,13 +201,14 @@ case object Crossref2Oaf {
|
|||
}
|
||||
|
||||
|
||||
def generateAuhtor(given: String, family: String, orcid: String): Author = {
|
||||
def generateAuhtor(given: String, family: String, orcid: String, index:Int): Author = {
|
||||
val a = new Author
|
||||
a.setName(given)
|
||||
a.setSurname(family)
|
||||
a.setFullname(s"$given $family")
|
||||
a.setRank(index+1)
|
||||
if (StringUtils.isNotBlank(orcid))
|
||||
a.setPid(List(createSP(orcid, ORCID_PENDING, PID_TYPES, generateDataInfo())).asJava)
|
||||
a.setPid(List(createSP(orcid, ModelConstants.ORCID_PENDING, ModelConstants.DNET_PID_TYPES, generateDataInfo())).asJava)
|
||||
|
||||
a
|
||||
}
|
||||
|
@ -221,7 +229,7 @@ case object Crossref2Oaf {
|
|||
val result = generateItemFromType(objectType, objectSubType)
|
||||
if (result == null)
|
||||
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"))
|
||||
mappingResult(result, json, cOBJCategory)
|
||||
|
||||
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package eu.dnetlib.doiboost.mag
|
||||
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, StructuredProperty}
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil
|
||||
import org.json4s
|
||||
|
@ -31,11 +32,11 @@ case class MagAffiliation(AffiliationId: Long, Rank: Int, NormalizedName: String
|
|||
case class MagPaperAuthorAffiliation(PaperId: Long, AuthorId: Long, AffiliationId: Option[Long], AuthorSequenceNumber: Int, OriginalAuthor: String, OriginalAffiliation: String) {}
|
||||
|
||||
|
||||
case class MagAuthorAffiliation(author: MagAuthor, affiliation:String)
|
||||
case class MagAuthorAffiliation(author: MagAuthor, affiliation:String, sequenceNumber:Int)
|
||||
|
||||
case class MagPaperWithAuthorList(PaperId: Long, authors: List[MagAuthorAffiliation]) {}
|
||||
|
||||
case class MagPaperAuthorDenormalized(PaperId: Long, author: MagAuthor, affiliation:String) {}
|
||||
case class MagPaperAuthorDenormalized(PaperId: Long, author: MagAuthor, affiliation:String, sequenceNumber:Int) {}
|
||||
|
||||
case class MagPaperUrl(PaperId: Long, SourceType: Option[Int], SourceUrl: Option[String], LanguageCode: Option[String]) {}
|
||||
|
||||
|
@ -202,12 +203,12 @@ case object ConversionUtil {
|
|||
val authorsOAF = authors.authors.map { f: MagAuthorAffiliation =>
|
||||
|
||||
val a: eu.dnetlib.dhp.schema.oaf.Author = new eu.dnetlib.dhp.schema.oaf.Author
|
||||
|
||||
a.setRank(f.sequenceNumber)
|
||||
if (f.author.DisplayName.isDefined)
|
||||
a.setFullname(f.author.DisplayName.get)
|
||||
|
||||
if(f.affiliation!= null)
|
||||
a.setAffiliation(List(asField(f.affiliation)).asJava)
|
||||
a.setPid(List(createSP(s"https://academic.microsoft.com/#/detail/${f.author.AuthorId}", "URL", PID_TYPES)).asJava)
|
||||
a.setPid(List(createSP(s"https://academic.microsoft.com/#/detail/${f.author.AuthorId}", "URL", ModelConstants.DNET_PID_TYPES)).asJava)
|
||||
a
|
||||
}
|
||||
pub.setAuthor(authorsOAF.asJava)
|
||||
|
@ -274,7 +275,7 @@ case object ConversionUtil {
|
|||
a.setAffiliation(List(asField(f.affiliation)).asJava)
|
||||
|
||||
|
||||
a.setPid(List(createSP(s"https://academic.microsoft.com/#/detail/${f.author.AuthorId}", "URL", PID_TYPES)).asJava)
|
||||
a.setPid(List(createSP(s"https://academic.microsoft.com/#/detail/${f.author.AuthorId}", "URL", ModelConstants.DNET_PID_TYPES)).asJava)
|
||||
|
||||
a
|
||||
|
||||
|
|
|
@ -6,10 +6,10 @@ import org.apache.spark.SparkConf
|
|||
import org.apache.spark.sql.{SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import org.apache.spark.sql.functions._
|
||||
|
||||
object SparkImportMagIntoDataset {
|
||||
val datatypedict = Map(
|
||||
"bool" -> BooleanType,
|
||||
"int" -> IntegerType,
|
||||
"uint" -> IntegerType,
|
||||
"long" -> LongType,
|
||||
|
@ -25,11 +25,10 @@ object SparkImportMagIntoDataset {
|
|||
"AuthorExtendedAttributes" -> Tuple2("mag/AuthorExtendedAttributes.txt", Seq("AuthorId:long", "AttributeType:int", "AttributeValue:string")),
|
||||
"Authors" -> Tuple2("mag/Authors.txt", Seq("AuthorId:long", "Rank:uint", "NormalizedName:string", "DisplayName:string", "LastKnownAffiliationId:long?", "PaperCount:long", "PaperFamilyCount:long", "CitationCount:long", "CreatedDate:DateTime")),
|
||||
"ConferenceInstances" -> Tuple2("mag/ConferenceInstances.txt", Seq("ConferenceInstanceId:long", "NormalizedName:string", "DisplayName:string", "ConferenceSeriesId:long", "Location:string", "OfficialUrl:string", "StartDate:DateTime?", "EndDate:DateTime?", "AbstractRegistrationDate:DateTime?", "SubmissionDeadlineDate:DateTime?", "NotificationDueDate:DateTime?", "FinalVersionDueDate:DateTime?", "PaperCount:long", "PaperFamilyCount:long" ,"CitationCount:long", "Latitude:float?", "Longitude:float?", "CreatedDate:DateTime")),
|
||||
"ConferenceSeries" -> Tuple2("mag/ConferenceSeries.txt", Seq("ConferenceSeriesId:long", "Rank:uint", "NormalizedName:string", "DisplayName:string", "PaperCount:long", "CitationCount:long", "CreatedDate:DateTime")),
|
||||
"ConferenceSeries" -> Tuple2("mag/ConferenceSeries.txt", Seq("ConferenceSeriesId:long", "Rank:uint", "NormalizedName:string", "DisplayName:string", "PaperCount:long", "PaperFamilyCount:long", "CitationCount:long", "CreatedDate:DateTime")),
|
||||
"EntityRelatedEntities" -> Tuple2("advanced/EntityRelatedEntities.txt", Seq("EntityId:long", "EntityType:string", "RelatedEntityId:long", "RelatedEntityType:string", "RelatedType:int", "Score:float")),
|
||||
"FieldOfStudyChildren" -> Tuple2("advanced/FieldOfStudyChildren.txt", Seq("FieldOfStudyId:long", "ChildFieldOfStudyId:long")),
|
||||
"FieldOfStudyExtendedAttributes" -> Tuple2("advanced/FieldOfStudyExtendedAttributes.txt", Seq("FieldOfStudyId:long", "AttributeType:int", "AttributeValue:string")),
|
||||
// ['FieldOfStudyId:long', 'Rank:uint', 'NormalizedName:string', 'DisplayName:string', 'MainType:string', 'Level:int', 'PaperCount:long', 'PaperFamilyCount:long', 'CitationCount:long', 'CreatedDate:DateTime']
|
||||
"FieldsOfStudy" -> Tuple2("advanced/FieldsOfStudy.txt", Seq("FieldOfStudyId:long", "Rank:uint", "NormalizedName:string", "DisplayName:string", "MainType:string", "Level:int", "PaperCount:long", "PaperFamilyCount:long", "CitationCount:long", "CreatedDate:DateTime")),
|
||||
"Journals" -> Tuple2("mag/Journals.txt", Seq("JournalId:long", "Rank:uint", "NormalizedName:string", "DisplayName:string", "Issn:string", "Publisher:string", "Webpage:string", "PaperCount:long", "PaperFamilyCount:long" ,"CitationCount:long", "CreatedDate:DateTime")),
|
||||
"PaperAbstractsInvertedIndex" -> Tuple2("nlp/PaperAbstractsInvertedIndex.txt.*", Seq("PaperId:long", "IndexedAbstract:string")),
|
||||
|
@ -37,6 +36,7 @@ object SparkImportMagIntoDataset {
|
|||
"PaperCitationContexts" -> Tuple2("nlp/PaperCitationContexts.txt", Seq("PaperId:long", "PaperReferenceId:long", "CitationContext:string")),
|
||||
"PaperExtendedAttributes" -> Tuple2("mag/PaperExtendedAttributes.txt", Seq("PaperId:long", "AttributeType:int", "AttributeValue:string")),
|
||||
"PaperFieldsOfStudy" -> Tuple2("advanced/PaperFieldsOfStudy.txt", Seq("PaperId:long", "FieldOfStudyId:long", "Score:float")),
|
||||
"PaperMeSH" -> Tuple2("advanced/PaperMeSH.txt", Seq("PaperId:long", "DescriptorUI:string", "DescriptorName:string", "QualifierUI:string", "QualifierName:string", "IsMajorTopic:bool")),
|
||||
"PaperRecommendations" -> Tuple2("advanced/PaperRecommendations.txt", Seq("PaperId:long", "RecommendedPaperId:long", "Score:float")),
|
||||
"PaperReferences" -> Tuple2("mag/PaperReferences.txt", Seq("PaperId:long", "PaperReferenceId:long")),
|
||||
"PaperResources" -> Tuple2("mag/PaperResources.txt", Seq("PaperId:long", "ResourceType:int", "ResourceUrl:string", "SourceUrl:string", "RelationshipType:int")),
|
||||
|
|
|
@ -58,16 +58,16 @@ object SparkProcessMAG {
|
|||
val paperAuthorAffiliation = spark.read.load(s"$sourcePath/PaperAuthorAffiliations").as[MagPaperAuthorAffiliation]
|
||||
|
||||
paperAuthorAffiliation.joinWith(authors, paperAuthorAffiliation("AuthorId").equalTo(authors("AuthorId")))
|
||||
.map { case (a: MagPaperAuthorAffiliation, b: MagAuthor) => (a.AffiliationId, MagPaperAuthorDenormalized(a.PaperId, b, null)) }
|
||||
.map { case (a: MagPaperAuthorAffiliation, b: MagAuthor) => (a.AffiliationId, MagPaperAuthorDenormalized(a.PaperId, b, null, a.AuthorSequenceNumber)) }
|
||||
.joinWith(affiliation, affiliation("AffiliationId").equalTo(col("_1")), "left")
|
||||
.map(s => {
|
||||
val mpa = s._1._2
|
||||
val af = s._2
|
||||
if (af != null) {
|
||||
MagPaperAuthorDenormalized(mpa.PaperId, mpa.author, af.DisplayName)
|
||||
MagPaperAuthorDenormalized(mpa.PaperId, mpa.author, af.DisplayName, mpa.sequenceNumber)
|
||||
} else
|
||||
mpa
|
||||
}).groupBy("PaperId").agg(collect_list(struct($"author", $"affiliation")).as("authors"))
|
||||
}).groupBy("PaperId").agg(collect_list(struct($"author", $"affiliation", $"sequenceNumber")).as("authors"))
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/merge_step_1_paper_authors")
|
||||
|
||||
logger.info("Phase 4) create First Version of publication Entity with Paper Journal and Authors")
|
||||
|
@ -115,10 +115,9 @@ object SparkProcessMAG {
|
|||
.save(s"$workingPath/merge_step_3")
|
||||
|
||||
|
||||
// logger.info("Phase 6) Enrich Publication with description")
|
||||
// val pa = spark.read.load(s"${parser.get("sourcePath")}/PaperAbstractsInvertedIndex").as[MagPaperAbstract]
|
||||
// pa.map(ConversionUtil.transformPaperAbstract).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/PaperAbstract")
|
||||
|
||||
// logger.info("Phase 6) Enrich Publication with description")
|
||||
// val pa = spark.read.load(s"${parser.get("sourcePath")}/PaperAbstractsInvertedIndex").as[MagPaperAbstract]
|
||||
// pa.map(ConversionUtil.transformPaperAbstract).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/PaperAbstract")
|
||||
val paperAbstract = spark.read.load((s"$workingPath/PaperAbstract")).as[MagPaperAbstract]
|
||||
|
||||
|
||||
|
|
|
@ -1,17 +1,25 @@
|
|||
package eu.dnetlib.doiboost.orcid
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Publication}
|
||||
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
|
||||
import eu.dnetlib.dhp.schema.orcid.{AuthorData, OrcidDOI}
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{ORCID, PID_TYPES, createSP, generateDataInfo, generateIdentifier}
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{createSP, generateDataInfo}
|
||||
import org.apache.commons.lang.StringUtils
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.json4s
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST._
|
||||
import org.json4s.jackson.JsonMethods._
|
||||
|
||||
|
||||
case class ORCIDItem(oid:String,name:String,surname:String,creditName:String,errorCode:String){}
|
||||
case class ORCIDItem(doi:String, authors:List[OrcidAuthor]){}
|
||||
case class OrcidAuthor(oid:String, name:Option[String], surname:Option[String], creditName:Option[String], otherNames:Option[List[String]], errorCode:Option[String]){}
|
||||
case class OrcidWork(oid:String, doi:String)
|
||||
|
||||
|
||||
|
||||
|
||||
|
@ -44,16 +52,64 @@ object ORCIDToOAF {
|
|||
}
|
||||
|
||||
|
||||
def convertTOOAF(input:OrcidDOI) :Publication = {
|
||||
val doi = input.getDoi
|
||||
def strValid(s:Option[String]) : Boolean = {
|
||||
s.isDefined && s.get.nonEmpty
|
||||
}
|
||||
|
||||
def authorValid(author:OrcidAuthor): Boolean ={
|
||||
if (strValid(author.name) && strValid(author.surname)) {
|
||||
return true
|
||||
}
|
||||
if (strValid(author.surname)) {
|
||||
return true
|
||||
}
|
||||
if (strValid(author.creditName)) {
|
||||
return true
|
||||
|
||||
}
|
||||
false
|
||||
}
|
||||
|
||||
|
||||
def extractDOIWorks(input:String): List[OrcidWork] = {
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: json4s.JValue = parse(input)
|
||||
|
||||
val oid = (json \ "workDetail" \"oid").extractOrElse[String](null)
|
||||
if (oid == null)
|
||||
return List()
|
||||
val doi:List[(String, String)] = for {
|
||||
JObject(extIds) <- json \ "workDetail" \"extIds"
|
||||
JField("type", JString(typeValue)) <- extIds
|
||||
JField("value", JString(value)) <- extIds
|
||||
if "doi".equalsIgnoreCase(typeValue)
|
||||
} yield (typeValue, value)
|
||||
if (doi.nonEmpty) {
|
||||
return doi.map(l =>OrcidWork(oid, l._2))
|
||||
}
|
||||
List()
|
||||
}
|
||||
|
||||
def convertORCIDAuthor(input:String): OrcidAuthor = {
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: json4s.JValue = parse(input)
|
||||
|
||||
(json \"authorData" ).extractOrElse[OrcidAuthor](null)
|
||||
}
|
||||
|
||||
|
||||
def convertTOOAF(input:ORCIDItem) :Publication = {
|
||||
val doi = input.doi
|
||||
val pub:Publication = new Publication
|
||||
pub.setPid(List(createSP(doi.toLowerCase, "doi", PID_TYPES)).asJava)
|
||||
pub.setPid(List(createSP(doi.toLowerCase, "doi", ModelConstants.DNET_PID_TYPES)).asJava)
|
||||
pub.setDataInfo(generateDataInfo())
|
||||
pub.setId(generateIdentifier(pub, doi.toLowerCase))
|
||||
|
||||
pub.setId(DoiBoostMappingUtil.generateIdentifier(pub, doi.toLowerCase))
|
||||
|
||||
try{
|
||||
|
||||
val l:List[Author]= input.getAuthors.asScala.map(a=> {
|
||||
generateAuthor(a.getName, a.getSurname, a.getCreditName, a.getOid)
|
||||
val l:List[Author]= input.authors.map(a=> {
|
||||
generateAuthor(a)
|
||||
})(collection.breakOut)
|
||||
|
||||
pub.setAuthor(l.asJava)
|
||||
|
@ -74,16 +130,20 @@ object ORCIDToOAF {
|
|||
di
|
||||
}
|
||||
|
||||
def generateAuthor(given: String, family: String, fullName:String, orcid: String): Author = {
|
||||
def generateAuthor(o : OrcidAuthor): Author = {
|
||||
val a = new Author
|
||||
a.setName(given)
|
||||
a.setSurname(family)
|
||||
if (fullName!= null && fullName.nonEmpty)
|
||||
a.setFullname(fullName)
|
||||
else
|
||||
a.setFullname(s"$given $family")
|
||||
if (StringUtils.isNotBlank(orcid))
|
||||
a.setPid(List(createSP(orcid, ORCID, PID_TYPES, generateOricPIDDatainfo())).asJava)
|
||||
if (strValid(o.name)) {
|
||||
a.setName(o.name.get.capitalize)
|
||||
}
|
||||
if (strValid(o.surname)) {
|
||||
a.setSurname(o.surname.get.capitalize)
|
||||
}
|
||||
if(strValid(o.name) && strValid(o.surname))
|
||||
a.setFullname(s"${o.name.get.capitalize} ${o.surname.get.capitalize}")
|
||||
else if (strValid(o.creditName))
|
||||
a.setFullname(o.creditName.get)
|
||||
if (StringUtils.isNotBlank(o.oid))
|
||||
a.setPid(List(createSP(o.oid, ModelConstants.ORCID, ModelConstants.DNET_PID_TYPES, generateOricPIDDatainfo())).asJava)
|
||||
|
||||
a
|
||||
}
|
||||
|
|
|
@ -1,72 +1,49 @@
|
|||
package eu.dnetlib.doiboost.orcid
|
||||
|
||||
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication
|
||||
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
|
||||
import eu.dnetlib.doiboost.mag.ConversionUtil
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkConvertORCIDToOAF {
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkConvertORCIDToOAF.getClass)
|
||||
|
||||
def getPublicationAggregator(): Aggregator[(String, Publication), Publication, Publication] = new Aggregator[(String, Publication), Publication, Publication]{
|
||||
|
||||
override def zero: Publication = new Publication()
|
||||
|
||||
override def reduce(b: Publication, a: (String, Publication)): Publication = {
|
||||
b.mergeFrom(a._2)
|
||||
b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
|
||||
override def merge(wx: Publication, wy: Publication): Publication = {
|
||||
wx.mergeFrom(wy)
|
||||
wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: Publication): Publication = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[Publication] =
|
||||
Encoders.kryo(classOf[Publication])
|
||||
|
||||
override def outputEncoder: Encoder[Publication] =
|
||||
Encoders.kryo(classOf[Publication])
|
||||
}
|
||||
|
||||
def run(spark:SparkSession,sourcePath:String, targetPath:String):Unit = {
|
||||
def run(spark:SparkSession,sourcePath:String,workingPath:String, targetPath:String):Unit = {
|
||||
import spark.implicits._
|
||||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
implicit val mapOrcid: Encoder[OrcidDOI] = Encoders.kryo[OrcidDOI]
|
||||
implicit val tupleForJoinEncoder: Encoder[(String, Publication)] = Encoders.tuple(Encoders.STRING, mapEncoderPubs)
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
mapper.getDeserializationConfig.withFeatures(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES)
|
||||
val inputRDD:RDD[OrcidAuthor] = spark.sparkContext.textFile(s"$sourcePath/authors").map(s => ORCIDToOAF.convertORCIDAuthor(s)).filter(s => s!= null).filter(s => ORCIDToOAF.authorValid(s))
|
||||
|
||||
val dataset:Dataset[OrcidDOI] = spark.createDataset(spark.sparkContext.textFile(sourcePath).map(s => mapper.readValue(s,classOf[OrcidDOI])))
|
||||
spark.createDataset(inputRDD).as[OrcidAuthor].write.mode(SaveMode.Overwrite).save(s"$workingPath/author")
|
||||
|
||||
val res = spark.sparkContext.textFile(s"$sourcePath/works").flatMap(s => ORCIDToOAF.extractDOIWorks(s)).filter(s => s!= null)
|
||||
|
||||
spark.createDataset(res).as[OrcidWork].write.mode(SaveMode.Overwrite).save(s"$workingPath/works")
|
||||
|
||||
val authors :Dataset[OrcidAuthor] = spark.read.load(s"$workingPath/author").as[OrcidAuthor]
|
||||
|
||||
val works :Dataset[OrcidWork] = spark.read.load(s"$workingPath/works").as[OrcidWork]
|
||||
|
||||
works.joinWith(authors, authors("oid").equalTo(works("oid")))
|
||||
.map(i =>{
|
||||
val doi = i._1.doi
|
||||
val author = i._2
|
||||
(doi, author)
|
||||
}).groupBy(col("_1").alias("doi"))
|
||||
.agg(collect_list(col("_2")).alias("authors"))
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/orcidworksWithAuthor")
|
||||
|
||||
val dataset: Dataset[ORCIDItem] =spark.read.load(s"$workingPath/orcidworksWithAuthor").as[ORCIDItem]
|
||||
|
||||
logger.info("Converting ORCID to OAF")
|
||||
dataset.map(o => ORCIDToOAF.convertTOOAF(o)).filter(p=>p!=null)
|
||||
.map(d => (d.getId, d))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(getPublicationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
}
|
||||
dataset.map(o => ORCIDToOAF.convertTOOAF(o)).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
}
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json")))
|
||||
parser.parseArgument(args)
|
||||
|
@ -78,10 +55,10 @@ def run(spark:SparkSession,sourcePath:String, targetPath:String):Unit = {
|
|||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
val workingPath = parser.get("workingPath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
run(spark, sourcePath, targetPath)
|
||||
run(spark, sourcePath, workingPath, targetPath)
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -54,6 +54,11 @@
|
|||
</property>
|
||||
|
||||
<!-- MAG Parameters -->
|
||||
<property>
|
||||
<name>MAGDumpPath</name>
|
||||
<description>the MAG dump working path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>inputPathMAG</name>
|
||||
<description>the MAG working path</description>
|
||||
|
@ -69,6 +74,11 @@
|
|||
<!-- ORCID Parameters -->
|
||||
<property>
|
||||
<name>inputPathOrcid</name>
|
||||
<description>the ORCID input path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>workingPathOrcid</name>
|
||||
<description>the ORCID working path</description>
|
||||
</property>
|
||||
|
||||
|
@ -132,7 +142,10 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${inputPathCrossref}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
|
@ -147,6 +160,43 @@
|
|||
<move source="${inputPathCrossref}/crossref_ds_updated"
|
||||
target="${inputPathCrossref}/crossref_ds"/>
|
||||
</fs>
|
||||
<ok to="ResetMagWorkingPath"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<!-- MAG SECTION -->
|
||||
<action name="ResetMagWorkingPath">
|
||||
<fs>
|
||||
<delete path="${inputPathMAG}/dataset"/>
|
||||
<delete path="${inputPathMAG}/process"/>
|
||||
</fs>
|
||||
<ok to="ConvertMagToDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertMagToDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${MAGDumpPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="ConvertCrossrefToOAF"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
@ -164,46 +214,15 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="ResetMagWorkingPath"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<!-- MAG SECTION -->
|
||||
<action name="ResetMagWorkingPath">
|
||||
<fs>
|
||||
<delete path="${inputPathMAG}/dataset"/>
|
||||
<delete path="${inputPathMAG}/process"/>
|
||||
<delete path="${inputPathMAG}/dataset"/>
|
||||
</fs>
|
||||
<ok to="ConvertMagToDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertMagToDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</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>${inputPathMAG}/input</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="ProcessMAG"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
@ -216,11 +235,14 @@
|
|||
<class>eu.dnetlib.doiboost.mag.SparkProcessMAG</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-memory=${sparkExecutorIntersectionMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--workingPath</arg><arg>${inputPathMAG}/process</arg>
|
||||
|
@ -245,10 +267,14 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathUnpayWall}/uw_extracted</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/uwPublication</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="ProcessORCID"/>
|
||||
|
@ -268,10 +294,14 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathOrcid}</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPathOrcid}</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/orcidPublication</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="CreateDOIBoost"/>
|
||||
|
@ -291,11 +321,15 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--hostedByMapPath</arg><arg>${hostedByMapPath}</arg>
|
||||
<arg>--affiliationPath</arg><arg>${inputPathMAG}/process/Affiliations</arg>
|
||||
<arg>--paperAffiliationPath</arg><arg>${inputPathMAG}/process/PaperAuthorAffiliations</arg>
|
||||
<arg>--affiliationPath</arg><arg>${inputPathMAG}/dataset/Affiliations</arg>
|
||||
<arg>--paperAffiliationPath</arg><arg>${inputPathMAG}/dataset/PaperAuthorAffiliations</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
|
@ -316,7 +350,10 @@
|
|||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--dbPublicationPath</arg><arg>${workingPath}/doiBoostPublicationFiltered</arg>
|
||||
<arg>--dbDatasetPath</arg><arg>${workingPath}/crossrefDataset</arg>
|
||||
|
|
Loading…
Reference in New Issue