fixed Crossref Mapping

This commit is contained in:
Sandro La Bruzzo 2020-05-20 17:05:46 +02:00
parent b771d67e9d
commit 5818abaab4
4 changed files with 111 additions and 99 deletions

View File

@ -137,6 +137,9 @@ case object Crossref2Oaf {
if (StringUtils.isNotBlank(issuedDate)) {
result.setDateofacceptance(asField(issuedDate))
}
else {
result.setDateofacceptance(asField(createdDate.getValue))
}
result.setRelevantdate(List(createdDate, postedDate, acceptedDate, publishedOnlineDate, publishedPrintDate).filter(p => p != null).asJava)
@ -233,6 +236,16 @@ case object Crossref2Oaf {
val queue = new mutable.Queue[Relation]
def snfRule(award:String): String = {
var tmp1 = StringUtils.substringAfter(award,"_")
val tmp2 = StringUtils.substringBefore(tmp1,"/")
logger.debug(s"From $award to $tmp2")
tmp2
}
def extractECAward(award: String): String = {
val awardECRegex: Regex = "[0-9]{4,9}".r
if (awardECRegex.findAllIn(award).hasNext)
@ -298,7 +311,7 @@ case object Crossref2Oaf {
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/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snfRule)
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__________" )

View File

@ -31,100 +31,99 @@ object SparkPreProcessMAG {
implicit val mapEncoderPubs: Encoder[Publication] = org.apache.spark.sql.Encoders.kryo[Publication]
implicit val tupleForJoinEncoder: Encoder[(String, Publication)] = Encoders.tuple(Encoders.STRING, mapEncoderPubs)
// logger.info("Phase 1) make uninque DOI in Papers:")
// val d: Dataset[MagPapers] = spark.read.load(s"${parser.get("sourcePath")}/Papers").as[MagPapers]
//
//
// // Filtering Papers with DOI, and since for the same DOI we have multiple version of item with different PapersId we get the last one
// val result: RDD[MagPapers] = d.where(col("Doi").isNotNull).rdd.map { p: MagPapers => Tuple2(p.Doi, p) }.reduceByKey { case (p1: MagPapers, p2: MagPapers) =>
// var r = if (p1 == null) p2 else p1
// if (p1 != null && p2 != null) {
// if (p1.CreatedDate != null && p2.CreatedDate != null) {
// if (p1.CreatedDate.before(p2.CreatedDate))
// r = p1
// else
// r = p2
// } else {
// r = if (p1.CreatedDate == null) p2 else p1
// }
// }
// r
// }.map(_._2)
//
// val distinctPaper: Dataset[MagPapers] = spark.createDataset(result)
// distinctPaper.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/Papers_distinct")
// logger.info(s"Total number of element: ${result.count()}")
//
// logger.info("Phase 3) Group Author by PaperId")
// val authors = spark.read.load(s"$sourcePath/Authors").as[MagAuthor]
//
// val affiliation = spark.read.load(s"$sourcePath/Affiliations").as[MagAffiliation]
//
// 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)) }
// .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)
// } else
// mpa
// }).groupBy("PaperId").agg(collect_list(struct($"author", $"affiliation")).as("authors"))
// .write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_1_paper_authors")
//
// logger.info("Phase 4) create First Version of publication Entity with Paper Journal and Authors")
//
// val journals = spark.read.load(s"$sourcePath/Journals").as[MagJournal]
//
// val papers = spark.read.load((s"${parser.get("targetPath")}/Papers_distinct")).as[MagPapers]
//
// val paperWithAuthors = spark.read.load(s"${parser.get("targetPath")}/merge_step_1_paper_authors").as[MagPaperWithAuthorList]
//
// val firstJoin = papers.joinWith(journals, papers("JournalId").equalTo(journals("JournalId")), "left")
// firstJoin.joinWith(paperWithAuthors, firstJoin("_1.PaperId").equalTo(paperWithAuthors("PaperId")), "left")
// .map { a: ((MagPapers, MagJournal), MagPaperWithAuthorList) => ConversionUtil.createOAFFromJournalAuthorPaper(a) }.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_2")
//
//
// var magPubs: Dataset[(String, Publication)] = spark.read.load(s"${parser.get("targetPath")}/merge_step_2").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
//
// val paperUrlDataset = spark.read.load(s"$sourcePath/PaperUrls").as[MagPaperUrl].groupBy("PaperId").agg(collect_list(struct("sourceUrl")).as("instances")).as[MagUrl]
//
//
// logger.info("Phase 5) enrich publication with URL and Instances")
//
// magPubs.joinWith(paperUrlDataset, col("_1").equalTo(paperUrlDataset("PaperId")), "left")
// .map { a: ((String, Publication), MagUrl) => ConversionUtil.addInstances((a._1._2, a._2)) }
// .write.mode(SaveMode.Overwrite)
// .save(s"${parser.get("targetPath")}/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")
//
// val paperAbstract = spark.read.load((s"${parser.get("targetPath")}/PaperAbstract")).as[MagPaperAbstract]
//
//
// magPubs = spark.read.load(s"${parser.get("targetPath")}/merge_step_3").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
//
// magPubs.joinWith(paperAbstract, col("_1").equalTo(paperAbstract("PaperId")), "left").map(p => {
// val pub = p._1._2
// val abst = p._2
// if (abst != null) {
// pub.setDescription(List(asField(abst.IndexedAbstract)).asJava)
// }
// pub
// }
// ).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_4")
//
logger.info("Phase 1) make uninque DOI in Papers:")
val d: Dataset[MagPapers] = spark.read.load(s"${parser.get("sourcePath")}/Papers").as[MagPapers]
// Filtering Papers with DOI, and since for the same DOI we have multiple version of item with different PapersId we get the last one
val result: RDD[MagPapers] = d.where(col("Doi").isNotNull).rdd.map { p: MagPapers => Tuple2(p.Doi, p) }.reduceByKey { case (p1: MagPapers, p2: MagPapers) =>
var r = if (p1 == null) p2 else p1
if (p1 != null && p2 != null) {
if (p1.CreatedDate != null && p2.CreatedDate != null) {
if (p1.CreatedDate.before(p2.CreatedDate))
r = p1
else
r = p2
} else {
r = if (p1.CreatedDate == null) p2 else p1
}
}
r
}.map(_._2)
val distinctPaper: Dataset[MagPapers] = spark.createDataset(result)
distinctPaper.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/Papers_distinct")
logger.info(s"Total number of element: ${result.count()}")
logger.info("Phase 3) Group Author by PaperId")
val authors = spark.read.load(s"$sourcePath/Authors").as[MagAuthor]
val affiliation = spark.read.load(s"$sourcePath/Affiliations").as[MagAffiliation]
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)) }
.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)
} else
mpa
}).groupBy("PaperId").agg(collect_list(struct($"author", $"affiliation")).as("authors"))
.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_1_paper_authors")
logger.info("Phase 4) create First Version of publication Entity with Paper Journal and Authors")
val journals = spark.read.load(s"$sourcePath/Journals").as[MagJournal]
val papers = spark.read.load((s"${parser.get("targetPath")}/Papers_distinct")).as[MagPapers]
val paperWithAuthors = spark.read.load(s"${parser.get("targetPath")}/merge_step_1_paper_authors").as[MagPaperWithAuthorList]
val firstJoin = papers.joinWith(journals, papers("JournalId").equalTo(journals("JournalId")), "left")
firstJoin.joinWith(paperWithAuthors, firstJoin("_1.PaperId").equalTo(paperWithAuthors("PaperId")), "left")
.map { a: ((MagPapers, MagJournal), MagPaperWithAuthorList) => ConversionUtil.createOAFFromJournalAuthorPaper(a) }.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_2")
var magPubs: Dataset[(String, Publication)] = spark.read.load(s"${parser.get("targetPath")}/merge_step_2").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
val paperUrlDataset = spark.read.load(s"$sourcePath/PaperUrls").as[MagPaperUrl].groupBy("PaperId").agg(collect_list(struct("sourceUrl")).as("instances")).as[MagUrl]
logger.info("Phase 5) enrich publication with URL and Instances")
magPubs.joinWith(paperUrlDataset, col("_1").equalTo(paperUrlDataset("PaperId")), "left")
.map { a: ((String, Publication), MagUrl) => ConversionUtil.addInstances((a._1._2, a._2)) }
.write.mode(SaveMode.Overwrite)
.save(s"${parser.get("targetPath")}/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")
val paperAbstract = spark.read.load((s"${parser.get("targetPath")}/PaperAbstract")).as[MagPaperAbstract]
magPubs = spark.read.load(s"${parser.get("targetPath")}/merge_step_3").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
magPubs.joinWith(paperAbstract, col("_1").equalTo(paperAbstract("PaperId")), "left").map(p => {
val pub = p._1._2
val abst = p._2
if (abst != null) {
pub.setDescription(List(asField(abst.IndexedAbstract)).asJava)
}
pub
}
).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/merge_step_4")
logger.info("Phase 7) Enrich Publication with FieldOfStudy")
val magPubs = spark.read.load(s"${parser.get("targetPath")}/merge_step_4").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
magPubs = spark.read.load(s"${parser.get("targetPath")}/merge_step_4").as[Publication].map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
val fos = spark.read.load(s"$sourcePath/FieldsOfStudy").select($"FieldOfStudyId".alias("fos"), $"DisplayName", $"MainType")
@ -135,7 +134,6 @@ object SparkPreProcessMAG {
.groupBy($"PaperId").agg(collect_list(struct($"FieldOfStudyId", $"DisplayName", $"MainType", $"Score")).as("subjects"))
.as[MagFieldOfStudy]
magPubs.joinWith(paperField, col("_1").equalTo(paperField("PaperId")), "left").
map(item => {
val publication = item._1._2

View File

@ -90,9 +90,9 @@ class CrossrefMappingTest {
val rels:List[Relation] = resultList.filter(p => p.isInstanceOf[Relation]).map(r=> r.asInstanceOf[Relation])
assertEquals(rels.size, 4)
rels.foreach(s => logger.info(s.getTarget))
rels.foreach(s => logger.info(s.getTarget))
assertEquals(rels.size, 3 )
}

View File

@ -179,12 +179,13 @@
],
"funder": [
{
"DOI": "10.13039/501100001711",
"name": "Swiss National Science Foundation (Schweizerische Nationalfonds)",
"doi-asserted-by": "publisher",
"DOI": "10.13039/501100000781",
"name": "European Research Council",
"award": [
"284236-REPCOLLAB",
"FP7/2007-2013"
"CR32I3_156724",
"31003A_173281/1",
"200021_165850"
]
}
],