diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java index d759f0d55..1efa86586 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java @@ -7,6 +7,10 @@ import eu.dnetlib.dhp.schema.oaf.Qualifier; public class ModelConstants { + public static final String ORCID = "orcid"; + public static final String ORCID_PENDING = "orcid_pending"; + public static final String ORCID_CLASSNAME = "Open Researcher and Contributor ID"; + public static final String DNET_SUBJECT_TYPOLOGIES = "dnet:subject_classification_typologies"; public static final String DNET_RESULT_TYPOLOGIES = "dnet:result_typologies"; public static final String DNET_PUBLICATION_RESOURCE = "dnet:publication_resource"; diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/AuthorData.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/orcid/AuthorData.java similarity index 64% rename from dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/AuthorData.java rename to dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/orcid/AuthorData.java index 29551c347..6c94cdb13 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/AuthorData.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/orcid/AuthorData.java @@ -1,7 +1,14 @@ -package eu.dnetlib.doiboost.orcid.model; +package eu.dnetlib.dhp.schema.orcid; import java.io.Serializable; +import java.util.List; + +import com.google.common.collect.Lists; + +/** + * This class models the data that are retrieved from orcid publication + */ public class AuthorData implements Serializable { @@ -10,6 +17,7 @@ public class AuthorData implements Serializable { private String surname; private String creditName; private String errorCode; + private List otherNames; public String getErrorCode() { return errorCode; @@ -50,4 +58,15 @@ public class AuthorData implements Serializable { public void setOid(String oid) { this.oid = oid; } + + public List getOtherNames() { + return otherNames; + } + + public void setOtherNames(List otherNames) { + if (this.otherNames == null) { + this.otherNames = Lists.newArrayList(); + } + this.otherNames = otherNames; + } } diff --git a/dhp-workflows/dhp-doiboost/pom.xml b/dhp-workflows/dhp-doiboost/pom.xml index 3299c1496..624dd7b31 100644 --- a/dhp-workflows/dhp-doiboost/pom.xml +++ b/dhp-workflows/dhp-doiboost/pom.xml @@ -14,7 +14,7 @@ net.alchim31.maven scala-maven-plugin - 4.0.1 + ${net.alchim31.maven.version} scala-compile-first @@ -51,7 +51,6 @@ org.apache.httpcomponents httpclient - 4.3.4 eu.dnetlib.dhp @@ -84,6 +83,11 @@ spark-sql_2.11 + + org.apache.commons + commons-text + + diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 096217a55..d1f6d8613 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -14,7 +14,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable 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) {} @@ -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 r.setSource(sourceId) - r.setTarget(s"40|$nsPrefix::$targetId") + r.setTarget(targetId) r.setRelType("resultProject") - r.setRelClass("isProducedBy") + r.setRelClass(relClass) r.setSubRelType("outcome") r.setCollectedfrom(List(cf).asJava) r.setDataInfo(di) r.setLastupdatetimestamp(ts) r + + } @@ -284,12 +286,18 @@ case object Crossref2Oaf { if (funder.award.isDefined && funder.award.get.nonEmpty) funder.award.get.map(extractField).filter(a => a!= null && a.nonEmpty).foreach( award => { - val targetId = DHPUtils.md5(award) - queue += generateRelation(sourceId, targetId, nsPrefix) + val targetId = getProjectId(nsPrefix, DHPUtils.md5(award)) + queue += generateRelation(sourceId, targetId , "isProducedBy") + queue += generateRelation(targetId , sourceId, "produces") } ) } + def getProjectId (nsPrefix:String, targetId:String):String = { + "40|$nsPrefix::$targetId" + } + + if (funders != null) funders.foreach(funder => { 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/501100001602" => generateSimpleRelationFromAward(funder, "aka_________", a => a.replace("SFI", "")) case "10.13039/501100000923" => generateSimpleRelationFromAward(funder, "arc_________", a => a) - case "10.13039/501100000038"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "nserc_______" ) - case "10.13039/501100000155"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "sshrc_______" ) - case "10.13039/501100000024"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "cihr________" ) + case "10.13039/501100000038"=> val targetId = getProjectId("nserc_______" , "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, "isProducedBy" ) + 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/501100003448" => generateSimpleRelationFromAward(funder, "gsrt________", extractECAward) case "10.13039/501100010198" => generateSimpleRelationFromAward(funder, "sgov________", a=>a) case "10.13039/501100004564" => generateSimpleRelationFromAward(funder, "mestd_______", extractECAward) case "10.13039/501100003407" => generateSimpleRelationFromAward(funder, "miur________", a=>a) - queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "miur________" ) + val targetId = getProjectId("miur________" , "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId,targetId, "isProducedBy" ) + queue += generateRelation(targetId,sourceId, "produces" ) 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________", snsfRule) case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a) case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a) - case "10.13039/100004440"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" ) + case "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 ) @@ -341,7 +360,9 @@ case object Crossref2Oaf { case "The French National Research Agency (ANR)" | "The French National Research Agency" => generateSimpleRelationFromAward(funder, "anr_________", a => a) case "CONICYT, Programa de Formación de Capital Humano Avanzado" => generateSimpleRelationFromAward(funder, "conicytf____", extractECAward) - case "Wellcome Trust Masters Fellowship" => queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" ) + case "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 ) } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala index fac4c90b4..08319058c 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/crossref/SparkMapDumpIntoOAF.scala @@ -29,69 +29,90 @@ object SparkMapDumpIntoOAF { .appName(SparkMapDumpIntoOAF.getClass.getSimpleName) .master(parser.get("master")).getOrCreate() + implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf] implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication] 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._ - 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") + 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 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) + val ds:Dataset[Oaf] = spark.read.load(s"$targetPath/mixObject").as[Oaf] - 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 - }.map(_._2) - - val rels: Dataset[Relation] = spark.createDataset(distinctRels) - - rels.write.mode(SaveMode.Overwrite).save(s"${targetPath}/relations") +// +// +// +// 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") } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ActivitiesDecompressor.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ActivitiesDecompressor.java index 570fdef17..420c363ec 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ActivitiesDecompressor.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ActivitiesDecompressor.java @@ -17,11 +17,12 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.hadoop.io.compress.GzipCodec; import org.mortbay.log.Log; -import eu.dnetlib.doiboost.orcid.json.JsonWriter; import eu.dnetlib.doiboost.orcid.model.WorkData; import eu.dnetlib.doiboost.orcid.xml.XMLRecordParser; +import eu.dnetlib.doiboost.orcidnodoi.json.JsonWriter; public class ActivitiesDecompressor { @@ -143,4 +144,64 @@ public class ActivitiesDecompressor { Log.info("Error from Orcid found: " + errorFromOrcidFound); Log.info("Error parsing xml work found: " + xmlParserErrorFound); } + + public static void extractXML(Configuration conf, String inputUri, Path outputPath) + throws Exception { + String uri = inputUri; + FileSystem fs = FileSystem.get(URI.create(uri), conf); + Path inputPath = new Path(uri); + CompressionCodecFactory factory = new CompressionCodecFactory(conf); + CompressionCodec codec = factory.getCodec(inputPath); + if (codec == null) { + System.err.println("No codec found for " + uri); + System.exit(1); + } + CompressionCodecFactory.removeSuffix(uri, codec.getDefaultExtension()); + InputStream gzipInputStream = null; + try { + gzipInputStream = codec.createInputStream(fs.open(inputPath)); + int counter = 0; + try (TarArchiveInputStream tais = new TarArchiveInputStream(gzipInputStream)) { + TarArchiveEntry entry = null; + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(outputPath), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class), + SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new GzipCodec()))) { + while ((entry = tais.getNextTarEntry()) != null) { + String filename = entry.getName(); + if (entry.isDirectory() || !filename.contains("works")) { + } else { + counter++; + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); + String line; + StringBuffer buffer = new StringBuffer(); + while ((line = br.readLine()) != null) { + buffer.append(line); + } + String xml = buffer.toString(); + String[] filenameParts = filename.split("/"); + final Text key = new Text( + XMLRecordParser + .retrieveOrcidIdFromActivity( + xml.getBytes(), filenameParts[filenameParts.length - 1])); + final Text value = new Text(xml); + writer.append(key, value); + if ((counter % 100000) == 0) { + Log.info("Current xml works extracted: " + counter); + } + } + } + } + } + Log.info("Activities extraction completed"); + Log.info("Total XML works parsed: " + counter); + } finally { + Log.debug("Closing gzip stream"); + IOUtils.closeStream(gzipInputStream); + } + } + } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLActivitiesData.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLActivitiesData.java new file mode 100644 index 000000000..c834efa20 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLActivitiesData.java @@ -0,0 +1,54 @@ + +package eu.dnetlib.doiboost.orcid; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.mortbay.log.Log; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork; + +public class ExtractXMLActivitiesData extends OrcidDSManager { + private String outputWorksPath; + private String activitiesFileNameTarGz; + + public static void main(String[] args) throws IOException, Exception { + ExtractXMLActivitiesData extractXMLActivitiesData = new ExtractXMLActivitiesData(); + extractXMLActivitiesData.loadArgs(args); + extractXMLActivitiesData.extractWorks(); + } + + private void loadArgs(String[] args) throws IOException, Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + GenOrcidAuthorWork.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/gen_orcid_works-no-doi_from_activities.json"))); + parser.parseArgument(args); + + hdfsServerUri = parser.get("hdfsServerUri"); + Log.info("HDFS URI: " + hdfsServerUri); + workingPath = parser.get("workingPath"); + Log.info("Working Path: " + workingPath); + activitiesFileNameTarGz = parser.get("activitiesFileNameTarGz"); + Log.info("Activities File Name: " + activitiesFileNameTarGz); + outputWorksPath = parser.get("outputWorksPath"); + Log.info("Output Author Work Data: " + outputWorksPath); + } + + private void extractWorks() throws Exception { + Configuration conf = initConfigurationObject(); + FileSystem fs = initFileSystemObject(conf); + String tarGzUri = hdfsServerUri.concat(workingPath).concat(activitiesFileNameTarGz); + Path outputPath = new Path( + hdfsServerUri + .concat(workingPath) + .concat(outputWorksPath)); + ActivitiesDecompressor.extractXML(conf, tarGzUri, outputPath); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLSummariesData.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLSummariesData.java new file mode 100644 index 000000000..843889108 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/ExtractXMLSummariesData.java @@ -0,0 +1,56 @@ + +package eu.dnetlib.doiboost.orcid; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.mortbay.log.Log; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork; + +public class ExtractXMLSummariesData extends OrcidDSManager { + + private String outputAuthorsPath; + private String summariesFileNameTarGz; + + public static void main(String[] args) throws IOException, Exception { + ExtractXMLSummariesData extractXMLSummariesData = new ExtractXMLSummariesData(); + extractXMLSummariesData.loadArgs(args); + extractXMLSummariesData.extractAuthors(); + } + + private void loadArgs(String[] args) throws IOException, Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + GenOrcidAuthorWork.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/gen_orcid_authors_from_summaries.json"))); + parser.parseArgument(args); + + hdfsServerUri = parser.get("hdfsServerUri"); + Log.info("HDFS URI: " + hdfsServerUri); + workingPath = parser.get("workingPath"); + Log.info("Working Path: " + workingPath); + summariesFileNameTarGz = parser.get("summariesFileNameTarGz"); + Log.info("Summaries File Name: " + summariesFileNameTarGz); + outputAuthorsPath = parser.get("outputAuthorsPath"); + Log.info("Output Authors Data: " + outputAuthorsPath); + } + + public void extractAuthors() throws Exception { + Configuration conf = initConfigurationObject(); + FileSystem fs = initFileSystemObject(conf); + String tarGzUri = hdfsServerUri.concat(workingPath).concat(summariesFileNameTarGz); + Path outputPath = new Path( + hdfsServerUri + .concat(workingPath) + .concat(outputAuthorsPath) + .concat("xml_authors.seq")); + SummariesDecompressor.extractXML(conf, tarGzUri, outputPath); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidAuthorsDOIsDataGen.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidAuthorsDOIsDataGen.java index 70528a8f6..2ec4fe59d 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidAuthorsDOIsDataGen.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidAuthorsDOIsDataGen.java @@ -25,8 +25,8 @@ public class OrcidAuthorsDOIsDataGen extends OrcidDSManager { public void generateAuthorsDOIsData() throws Exception { Configuration conf = initConfigurationObject(); FileSystem fs = initFileSystemObject(conf); - String tarGzUri = hdfsServerUri.concat(hdfsOrcidDefaultPath).concat(activitiesFileNameTarGz); - Path outputPath = new Path(hdfsServerUri.concat(hdfsOrcidDefaultPath).concat(outputAuthorsDOIsPath)); + String tarGzUri = hdfsServerUri.concat(workingPath).concat(activitiesFileNameTarGz); + Path outputPath = new Path(hdfsServerUri.concat(workingPath).concat(outputAuthorsDOIsPath)); ActivitiesDecompressor.parseGzActivities(conf, tarGzUri, outputPath); } @@ -41,8 +41,8 @@ public class OrcidAuthorsDOIsDataGen extends OrcidDSManager { hdfsServerUri = parser.get("hdfsServerUri"); Log.info("HDFS URI: " + hdfsServerUri); - hdfsOrcidDefaultPath = parser.get("hdfsOrcidDefaultPath"); - Log.info("Default Path: " + hdfsOrcidDefaultPath); + workingPath = parser.get("workingPath"); + Log.info("Default Path: " + workingPath); activitiesFileNameTarGz = parser.get("activitiesFileNameTarGz"); Log.info("Activities File Name: " + activitiesFileNameTarGz); outputAuthorsDOIsPath = parser.get("outputAuthorsDOIsPath"); diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDSManager.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDSManager.java index 4f846bdf3..bf13db021 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDSManager.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDSManager.java @@ -15,7 +15,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; public class OrcidDSManager { protected String hdfsServerUri; - protected String hdfsOrcidDefaultPath; + protected String workingPath; private String summariesFileNameTarGz; private String outputAuthorsPath; @@ -28,10 +28,10 @@ public class OrcidDSManager { public void generateAuthors() throws Exception { Configuration conf = initConfigurationObject(); FileSystem fs = initFileSystemObject(conf); - String tarGzUri = hdfsServerUri.concat(hdfsOrcidDefaultPath).concat(summariesFileNameTarGz); + String tarGzUri = hdfsServerUri.concat(workingPath).concat(summariesFileNameTarGz); Path outputPath = new Path( hdfsServerUri - .concat(hdfsOrcidDefaultPath) + .concat(workingPath) .concat(outputAuthorsPath) .concat("authors.seq")); SummariesDecompressor.parseGzSummaries(conf, tarGzUri, outputPath); @@ -41,22 +41,18 @@ public class OrcidDSManager { // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI - conf.set("fs.defaultFS", hdfsServerUri.concat(hdfsOrcidDefaultPath)); + conf.set("fs.defaultFS", hdfsServerUri.concat(workingPath)); // Because of Maven conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); return conf; } - protected FileSystem initFileSystemObject(Configuration conf) { + protected FileSystem initFileSystemObject(Configuration conf) throws IOException { // Get the filesystem - HDFS + // if there is an exception, it will be propagate FileSystem fs = null; - try { - fs = FileSystem.get(URI.create(hdfsServerUri.concat(hdfsOrcidDefaultPath)), conf); - } catch (IOException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } + fs = FileSystem.get(URI.create(hdfsServerUri.concat(workingPath)), conf); return fs; } @@ -66,13 +62,13 @@ public class OrcidDSManager { .toString( OrcidDSManager.class .getResourceAsStream( - "/eu/dnetlib/dhp/doiboost/create_orcid_authors_data.json"))); + "/eu/dnetlib/dhp/doiboost/gen_orcid_authors_from_summaries.json"))); parser.parseArgument(args); hdfsServerUri = parser.get("hdfsServerUri"); Log.info("HDFS URI: " + hdfsServerUri); - hdfsOrcidDefaultPath = parser.get("hdfsOrcidDefaultPath"); - Log.info("Default Path: " + hdfsOrcidDefaultPath); + workingPath = parser.get("workingPath"); + Log.info("Working Path: " + workingPath); summariesFileNameTarGz = parser.get("summariesFileNameTarGz"); Log.info("Summaries File Name: " + summariesFileNameTarGz); outputAuthorsPath = parser.get("outputAuthorsPath"); diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDownloader.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDownloader.java index 2e1a199da..be727ab9f 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDownloader.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/OrcidDownloader.java @@ -1,14 +1,15 @@ package eu.dnetlib.doiboost.orcid; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; +import java.io.*; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; import java.util.List; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -16,6 +17,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; @@ -27,10 +29,10 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; public class OrcidDownloader extends OrcidDSManager { static final int REQ_LIMIT = 24; -// static final int REQ_MAX_TEST = 100; - static final int RECORD_PARSED_COUNTER_LOG_INTERVAL = 10000; + static final int REQ_MAX_TEST = -1; + static final int RECORD_PARSED_COUNTER_LOG_INTERVAL = 500; static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; - static final String lastUpdate = "2019-09-30 00:00:00"; + static final String lastUpdate = "2020-09-29 00:00:00"; private String lambdaFileName; private String outputPath; private String token; @@ -41,7 +43,7 @@ public class OrcidDownloader extends OrcidDSManager { orcidDownloader.parseLambdaFile(); } - private String downloadRecord(String orcidId) { + private String downloadRecord(String orcidId) throws IOException { try (CloseableHttpClient client = HttpClients.createDefault()) { HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record"); httpGet.addHeader("Accept", "application/vnd.orcid+xml"); @@ -49,17 +51,23 @@ public class OrcidDownloader extends OrcidDSManager { CloseableHttpResponse response = client.execute(httpGet); if (response.getStatusLine().getStatusCode() != 200) { Log - .warn( + .info( "Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode()); return new String(""); } - return IOUtils.toString(response.getEntity().getContent()); - - } catch (Throwable e) { - Log.warn("Downloading " + orcidId, e.getMessage()); - +// return IOUtils.toString(response.getEntity().getContent()); + return xmlStreamToString(response.getEntity().getContent()); } - return new String(""); + } + + private String xmlStreamToString(InputStream xmlStream) throws IOException { + BufferedReader br = new BufferedReader(new InputStreamReader(xmlStream)); + String line; + StringBuffer buffer = new StringBuffer(); + while ((line = br.readLine()) != null) { + buffer.append(line); + } + return buffer.toString(); } public void parseLambdaFile() throws Exception { @@ -69,97 +77,94 @@ public class OrcidDownloader extends OrcidDSManager { long startDownload = 0; Configuration conf = initConfigurationObject(); FileSystem fs = initFileSystemObject(conf); - String lambdaFileUri = hdfsServerUri.concat(hdfsOrcidDefaultPath).concat(lambdaFileName); + String lambdaFileUri = hdfsServerUri.concat(workingPath).concat(lambdaFileName); Path hdfsreadpath = new Path(lambdaFileUri); FSDataInputStream lambdaFileStream = fs.open(hdfsreadpath); Path hdfsoutputPath = new Path( hdfsServerUri - .concat(hdfsOrcidDefaultPath) + .concat(workingPath) .concat(outputPath) - .concat("orcid_records.seq")); - - try (SequenceFile.Writer writer = SequenceFile - .createWriter( - conf, - SequenceFile.Writer.file(hdfsoutputPath), - SequenceFile.Writer.keyClass(Text.class), - SequenceFile.Writer.valueClass(Text.class))) { - - try (BufferedReader br = new BufferedReader(new InputStreamReader(lambdaFileStream))) { - String line; - int nReqTmp = 0; + .concat("updated_xml_authors.seq")); + try (TarArchiveInputStream tais = new TarArchiveInputStream( + new GzipCompressorInputStream(lambdaFileStream))) { + TarArchiveEntry entry = null; + StringBuilder sb = new StringBuilder(); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(hdfsoutputPath), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class), + SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new GzipCodec()))) { startDownload = System.currentTimeMillis(); - long startReqTmp = System.currentTimeMillis(); - while ((line = br.readLine()) != null) { - parsedRecordsCounter++; - // skip headers line - if (parsedRecordsCounter == 1) { - continue; - } - String[] values = line.split(","); - List recordInfo = Arrays.asList(values); - String orcidId = recordInfo.get(0); - if (isModified(orcidId, recordInfo.get(3))) { - String record = downloadRecord(orcidId); - downloadedRecordsCounter++; - if (!record.isEmpty()) { - String compressRecord = ArgumentApplicationParser.compressArgument(record); - final Text key = new Text(recordInfo.get(0)); - final Text value = new Text(compressRecord); - - try { + while ((entry = tais.getNextTarEntry()) != null) { + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); // Read directly from tarInput + String line; + while ((line = br.readLine()) != null) { + String[] values = line.split(","); + List recordInfo = Arrays.asList(values); + int nReqTmp = 0; + long startReqTmp = System.currentTimeMillis(); + // skip headers line + if (parsedRecordsCounter == 0) { + parsedRecordsCounter++; + continue; + } + parsedRecordsCounter++; + String orcidId = recordInfo.get(0); + if (isModified(orcidId, recordInfo.get(3))) { + String record = downloadRecord(orcidId); + downloadedRecordsCounter++; + if (!record.isEmpty()) { +// String compressRecord = ArgumentApplicationParser.compressArgument(record); + final Text key = new Text(recordInfo.get(0)); + final Text value = new Text(record); writer.append(key, value); savedRecordsCounter++; - } catch (IOException e) { - Log.warn("Writing to sequence file: " + e.getMessage()); - Log.warn(e); - throw new RuntimeException(e); + } + } else { + break; + } + long endReq = System.currentTimeMillis(); + nReqTmp++; + if (nReqTmp == REQ_LIMIT) { + long reqSessionDuration = endReq - startReqTmp; + if (reqSessionDuration <= 1000) { + Log + .info( + "\nreqSessionDuration: " + + reqSessionDuration + + " nReqTmp: " + + nReqTmp + + " wait ...."); + Thread.sleep(1000 - reqSessionDuration); + } else { + nReqTmp = 0; + startReqTmp = System.currentTimeMillis(); + } + } + if ((parsedRecordsCounter % RECORD_PARSED_COUNTER_LOG_INTERVAL) == 0) { + Log + .info( + "Current parsed: " + + parsedRecordsCounter + + " downloaded: " + + downloadedRecordsCounter + + " saved: " + + savedRecordsCounter); + if (REQ_MAX_TEST != -1 && parsedRecordsCounter > REQ_MAX_TEST) { + break; } } } - long endReq = System.currentTimeMillis(); - nReqTmp++; - if (nReqTmp == REQ_LIMIT) { - long reqSessionDuration = endReq - startReqTmp; - if (reqSessionDuration <= 1000) { - Log - .warn( - "\nreqSessionDuration: " - + reqSessionDuration - + " nReqTmp: " - + nReqTmp - + " wait ...."); - Thread.sleep(1000 - reqSessionDuration); - } else { - nReqTmp = 0; - startReqTmp = System.currentTimeMillis(); - } - } - -// if (parsedRecordsCounter > REQ_MAX_TEST) { -// break; -// } - if ((parsedRecordsCounter % RECORD_PARSED_COUNTER_LOG_INTERVAL) == 0) { - Log - .info( - "Current parsed: " - + parsedRecordsCounter - + " downloaded: " - + downloadedRecordsCounter - + " saved: " - + savedRecordsCounter); -// if (parsedRecordsCounter > REQ_MAX_TEST) { -// break; -// } - } + long endDownload = System.currentTimeMillis(); + long downloadTime = endDownload - startDownload; + Log.info("Download time: " + ((downloadTime / 1000) / 60) + " minutes"); } - long endDownload = System.currentTimeMillis(); - long downloadTime = endDownload - startDownload; - Log.info("Download time: " + ((downloadTime / 1000) / 60) + " minutes"); } } - lambdaFileStream.close(); Log.info("Download started at: " + new Date(startDownload).toString()); + Log.info("Download ended at: " + new Date(System.currentTimeMillis()).toString()); Log.info("Parsed Records Counter: " + parsedRecordsCounter); Log.info("Downloaded Records Counter: " + downloadedRecordsCounter); Log.info("Saved Records Counter: " + savedRecordsCounter); @@ -176,8 +181,8 @@ public class OrcidDownloader extends OrcidDSManager { hdfsServerUri = parser.get("hdfsServerUri"); Log.info("HDFS URI: " + hdfsServerUri); - hdfsOrcidDefaultPath = parser.get("hdfsOrcidDefaultPath"); - Log.info("Default Path: " + hdfsOrcidDefaultPath); + workingPath = parser.get("workingPath"); + Log.info("Default Path: " + workingPath); lambdaFileName = parser.get("lambdaFileName"); Log.info("Lambda File Name: " + lambdaFileName); outputPath = parser.get("outputPath"); @@ -185,7 +190,7 @@ public class OrcidDownloader extends OrcidDSManager { token = parser.get("token"); } - private boolean isModified(String orcidId, String modifiedDate) { + public boolean isModified(String orcidId, String modifiedDate) { Date modifiedDateDt = null; Date lastUpdateDt = null; try { @@ -195,7 +200,7 @@ public class OrcidDownloader extends OrcidDSManager { modifiedDateDt = new SimpleDateFormat(DATE_FORMAT).parse(modifiedDate); lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate); } catch (Exception e) { - Log.warn("[" + orcidId + "] Parsing date: ", e.getMessage()); + Log.info("[" + orcidId + "] Parsing date: ", e.getMessage()); return true; } return modifiedDateDt.after(lastUpdateDt); diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkDownloadOrcidAuthors.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkDownloadOrcidAuthors.java new file mode 100644 index 000000000..598835a00 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkDownloadOrcidAuthors.java @@ -0,0 +1,188 @@ + +package eu.dnetlib.doiboost.orcid; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.util.LongAccumulator; +import org.mortbay.log.Log; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData; +import scala.Tuple2; + +public class SparkDownloadOrcidAuthors { + + static Logger logger = LoggerFactory.getLogger(SparkDownloadOrcidAuthors.class); + static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; + static final String lastUpdate = "2020-09-29 00:00:00"; + + public static void main(String[] args) throws IOException, Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkDownloadOrcidAuthors.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/download_orcid_data.json"))); + parser.parseArgument(args); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); + final String workingPath = parser.get("workingPath"); + logger.info("workingPath: ", workingPath); + final String outputPath = parser.get("outputPath"); + logger.info("outputPath: ", outputPath); + final String token = parser.get("token"); + final String lambdaFileName = parser.get("lambdaFileName"); + logger.info("lambdaFileName: ", lambdaFileName); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + LongAccumulator parsedRecordsAcc = spark.sparkContext().longAccumulator("parsed_records"); + LongAccumulator modifiedRecordsAcc = spark.sparkContext().longAccumulator("to_download_records"); + LongAccumulator downloadedRecordsAcc = spark.sparkContext().longAccumulator("downloaded_records"); + LongAccumulator errorHTTP403Acc = spark.sparkContext().longAccumulator("error_HTTP_403"); + LongAccumulator errorHTTP409Acc = spark.sparkContext().longAccumulator("error_HTTP_409"); + LongAccumulator errorHTTP503Acc = spark.sparkContext().longAccumulator("error_HTTP_503"); + LongAccumulator errorHTTP525Acc = spark.sparkContext().longAccumulator("error_HTTP_525"); + LongAccumulator errorHTTPGenericAcc = spark.sparkContext().longAccumulator("error_HTTP_Generic"); + + logger.info("Retrieving data from lamda sequence file"); + JavaPairRDD lamdaFileRDD = sc + .sequenceFile(workingPath + lambdaFileName, Text.class, Text.class); + logger.info("Data retrieved: " + lamdaFileRDD.count()); + + Function, Boolean> isModifiedAfterFilter = data -> { + String orcidId = data._1().toString(); + String lastModifiedDate = data._2().toString(); + parsedRecordsAcc.add(1); + if (isModified(orcidId, lastModifiedDate)) { + modifiedRecordsAcc.add(1); + return true; + } + return false; + }; + + Function, Tuple2> downloadRecordFunction = data -> { + String orcidId = data._1().toString(); + String lastModifiedDate = data._2().toString(); + final DownloadedRecordData downloaded = new DownloadedRecordData(); + downloaded.setOrcidId(orcidId); + downloaded.setLastModifiedDate(lastModifiedDate); + try (CloseableHttpClient client = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record"); + httpGet.addHeader("Accept", "application/vnd.orcid+xml"); + httpGet.addHeader("Authorization", String.format("Bearer %s", token)); + long startReq = System.currentTimeMillis(); + CloseableHttpResponse response = client.execute(httpGet); + long endReq = System.currentTimeMillis(); + long reqTime = endReq - startReq; + if (reqTime < 1000) { + Thread.sleep(1000 - reqTime); + } + int statusCode = response.getStatusLine().getStatusCode(); + downloaded.setStatusCode(statusCode); + if (statusCode != 200) { + switch (statusCode) { + case 403: + errorHTTP403Acc.add(1); + case 409: + errorHTTP409Acc.add(1); + case 503: + errorHTTP503Acc.add(1); + throw new RuntimeException("Orcid request rate limit reached (HTTP 503)"); + case 525: + errorHTTP525Acc.add(1); + default: + errorHTTPGenericAcc.add(1); + logger + .info( + "Downloading " + orcidId + " status code: " + + response.getStatusLine().getStatusCode()); + } + return downloaded.toTuple2(); + } + downloadedRecordsAcc.add(1); + downloaded + .setCompressedData( + ArgumentApplicationParser + .compressArgument(IOUtils.toString(response.getEntity().getContent()))); + } catch (Throwable e) { + logger.info("Downloading " + orcidId, e.getMessage()); + downloaded.setErrorMessage(e.getMessage()); + return downloaded.toTuple2(); + } + return downloaded.toTuple2(); + }; + + sc.hadoopConfiguration().set("mapreduce.output.fileoutputformat.compress", "true"); + + logger.info("Start execution ..."); + JavaPairRDD authorsModifiedRDD = lamdaFileRDD.filter(isModifiedAfterFilter); + logger.info("Authors modified count: " + authorsModifiedRDD.count()); + logger.info("Start downloading ..."); + authorsModifiedRDD + .repartition(10) + .map(downloadRecordFunction) + .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2()))) + .saveAsNewAPIHadoopFile( + workingPath.concat(outputPath), + Text.class, + Text.class, + SequenceFileOutputFormat.class, + sc.hadoopConfiguration()); + logger.info("parsedRecordsAcc: " + parsedRecordsAcc.value().toString()); + logger.info("modifiedRecordsAcc: " + modifiedRecordsAcc.value().toString()); + logger.info("downloadedRecordsAcc: " + downloadedRecordsAcc.value().toString()); + logger.info("errorHTTP403Acc: " + errorHTTP403Acc.value().toString()); + logger.info("errorHTTP409Acc: " + errorHTTP409Acc.value().toString()); + logger.info("errorHTTP503Acc: " + errorHTTP503Acc.value().toString()); + logger.info("errorHTTP525Acc: " + errorHTTP525Acc.value().toString()); + logger.info("errorHTTPGenericAcc: " + errorHTTPGenericAcc.value().toString()); + }); + + } + + private static boolean isModified(String orcidId, String modifiedDate) { + Date modifiedDateDt = null; + Date lastUpdateDt = null; + try { + if (modifiedDate.length() != 19) { + modifiedDate = modifiedDate.substring(0, 19); + } + modifiedDateDt = new SimpleDateFormat(DATE_FORMAT).parse(modifiedDate); + lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate); + } catch (Exception e) { + logger.info("[" + orcidId + "] Parsing date: ", e.getMessage()); + return true; + } + return modifiedDateDt.after(lastUpdateDt); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenLastModifiedSeq.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenLastModifiedSeq.java new file mode 100644 index 000000000..f710635ab --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenLastModifiedSeq.java @@ -0,0 +1,99 @@ + +package eu.dnetlib.doiboost.orcid; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.net.URI; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.mortbay.log.Log; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + +public class SparkGenLastModifiedSeq { + private static String hdfsServerUri; + private static String workingPath; + private static String outputPath; + private static String lambdaFileName; + + public static void main(String[] args) throws IOException, Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkGenLastModifiedSeq.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/download_orcid_data.json"))); + parser.parseArgument(args); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + hdfsServerUri = parser.get("hdfsServerUri"); + workingPath = parser.get("workingPath"); + outputPath = parser.get("outputPath"); + lambdaFileName = parser.get("lambdaFileName"); + String lambdaFileUri = hdfsServerUri.concat(workingPath).concat(lambdaFileName); + + SparkConf sparkConf = new SparkConf(); + runWithSparkSession( + sparkConf, + isSparkSessionManaged, + spark -> { + int rowsNum = 0; + Path output = new Path( + hdfsServerUri + .concat(workingPath) + .concat(outputPath)); + Path hdfsreadpath = new Path(lambdaFileUri); + Configuration conf = new Configuration(); + conf.set("fs.defaultFS", hdfsServerUri.concat(workingPath)); + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + FileSystem fs = FileSystem.get(URI.create(hdfsServerUri.concat(workingPath)), conf); + FSDataInputStream lambdaFileStream = fs.open(hdfsreadpath); + try (TarArchiveInputStream tais = new TarArchiveInputStream( + new GzipCompressorInputStream(lambdaFileStream))) { + TarArchiveEntry entry = null; + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(output), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class), + SequenceFile.Writer.compression(SequenceFile.CompressionType.BLOCK, new GzipCodec()))) { + while ((entry = tais.getNextTarEntry()) != null) { + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); + String line; + while ((line = br.readLine()) != null) { + String[] values = line.split(","); + List recordInfo = Arrays.asList(values); + String orcidId = recordInfo.get(0); + final Text key = new Text(orcidId); + final Text value = new Text(recordInfo.get(3)); + writer.append(key, value); + rowsNum++; + } + } + } + } + Log.info("Saved rows from lamda csv tar file: " + rowsNum); + }); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenerateDoiAuthorList.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenerateDoiAuthorList.java index b4239bba2..011c153ec 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenerateDoiAuthorList.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkGenerateDoiAuthorList.java @@ -13,9 +13,6 @@ import java.util.stream.Stream; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.compress.GzipCodec; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -33,7 +30,7 @@ import com.google.gson.JsonElement; import com.google.gson.JsonParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.doiboost.orcid.model.AuthorData; +import eu.dnetlib.dhp.schema.orcid.AuthorData; import eu.dnetlib.doiboost.orcid.model.WorkData; import scala.Tuple2; diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkOrcidGenerateAuthors.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkOrcidGenerateAuthors.java deleted file mode 100644 index 4e18ab840..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkOrcidGenerateAuthors.java +++ /dev/null @@ -1,165 +0,0 @@ - -package eu.dnetlib.doiboost.orcid; - -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.io.IOException; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.List; -import java.util.Optional; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClients; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.util.LongAccumulator; -import org.mortbay.log.Log; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.doiboost.orcid.model.DownloadedRecordData; -import scala.Tuple2; - -public class SparkOrcidGenerateAuthors { - - static final String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; - static final String lastUpdate = "2019-09-30 00:00:00"; - - public static void main(String[] args) throws IOException, Exception { - Logger logger = LoggerFactory.getLogger(SparkOrcidGenerateAuthors.class); - logger.info("[ SparkOrcidGenerateAuthors STARTED]"); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkOrcidGenerateAuthors.class - .getResourceAsStream( - "/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json"))); - parser.parseArgument(args); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String workingPath = parser.get("workingPath"); - logger.info("workingPath: ", workingPath); - final String outputAuthorsPath = parser.get("outputAuthorsPath"); - logger.info("outputAuthorsPath: ", outputAuthorsPath); - final String token = parser.get("token"); - - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - LongAccumulator parsedRecordsAcc = sc.sc().longAccumulator("parsedRecords"); - LongAccumulator modifiedRecordsAcc = sc.sc().longAccumulator("modifiedRecords"); - LongAccumulator downloadedRecordsAcc = sc.sc().longAccumulator("downloadedRecords"); - LongAccumulator alreadyDownloadedRecords = sc.sc().longAccumulator("alreadyDownloadedRecords"); - JavaRDD lamdaFileRDD = sc.textFile(workingPath + "lamdafiles"); - - JavaRDD downloadedRDD = sc.textFile(workingPath + "downloaded"); - Function getOrcidIdFunction = line -> { - try { - String[] values = line.split(","); - return values[0].substring(1); - } catch (Exception e) { - return new String(""); - } - }; - List downloadedRecords = downloadedRDD.map(getOrcidIdFunction).collect(); - - Function isModifiedAfterFilter = line -> { - String[] values = line.split(","); - String orcidId = values[0]; - parsedRecordsAcc.add(1); - if (isModified(orcidId, values[3])) { - modifiedRecordsAcc.add(1); - return true; - } - return false; - }; - Function isNotDownloadedFilter = line -> { - String[] values = line.split(","); - String orcidId = values[0]; - if (downloadedRecords.contains(orcidId)) { - alreadyDownloadedRecords.add(1); - return false; - } - return true; - }; - Function> downloadRecordFunction = line -> { - String[] values = line.split(","); - String orcidId = values[0]; - String modifiedDate = values[3]; - return downloadRecord(orcidId, modifiedDate, token, downloadedRecordsAcc); - }; - - lamdaFileRDD - .filter(isModifiedAfterFilter) - .filter(isNotDownloadedFilter) - .map(downloadRecordFunction) - .rdd() - .saveAsTextFile(workingPath.concat(outputAuthorsPath)); - }); - - } - - private static boolean isModified(String orcidId, String modifiedDate) { - Date modifiedDateDt = null; - Date lastUpdateDt = null; - try { - if (modifiedDate.length() != 19) { - modifiedDate = modifiedDate.substring(0, 19); - } - modifiedDateDt = new SimpleDateFormat(DATE_FORMAT).parse(modifiedDate); - lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate); - } catch (Exception e) { - Log.warn("[" + orcidId + "] Parsing date: ", e.getMessage()); - return true; - } - return modifiedDateDt.after(lastUpdateDt); - } - - private static Tuple2 downloadRecord(String orcidId, String modifiedDate, String token, - LongAccumulator downloadedRecordsAcc) { - final DownloadedRecordData data = new DownloadedRecordData(); - data.setOrcidId(orcidId); - data.setModifiedDate(modifiedDate); - try (CloseableHttpClient client = HttpClients.createDefault()) { - HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record"); - httpGet.addHeader("Accept", "application/vnd.orcid+xml"); - httpGet.addHeader("Authorization", String.format("Bearer %s", token)); - CloseableHttpResponse response = client.execute(httpGet); - int statusCode = response.getStatusLine().getStatusCode(); - data.setStatusCode(statusCode); - if (statusCode != 200) { - Log - .warn( - "Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode()); - return data.toTuple2(); - } - downloadedRecordsAcc.add(1); - data - .setCompressedData( - ArgumentApplicationParser.compressArgument(IOUtils.toString(response.getEntity().getContent()))); - } catch (Throwable e) { - Log.warn("Downloading " + orcidId, e.getMessage()); - data.setErrorMessage(e.getMessage()); - return data.toTuple2(); - } - return data.toTuple2(); - } -} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPartitionLambdaFile.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPartitionLambdaFile.java deleted file mode 100644 index ca6f0f6c4..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPartitionLambdaFile.java +++ /dev/null @@ -1,50 +0,0 @@ - -package eu.dnetlib.doiboost.orcid; - -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.io.IOException; -import java.util.Optional; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; - -public class SparkPartitionLambdaFile { - - public static void main(String[] args) throws IOException, Exception { - Logger logger = LoggerFactory.getLogger(SparkOrcidGenerateAuthors.class); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkOrcidGenerateAuthors.class - .getResourceAsStream( - "/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json"))); - parser.parseArgument(args); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - final String workingPath = parser.get("workingPath"); - - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD lamdaFileRDD = sc.textFile(workingPath + "last_modified.csv"); - - lamdaFileRDD - .repartition(20) - .saveAsTextFile(workingPath.concat("lamdafiles")); - }); - } - -} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SummariesDecompressor.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SummariesDecompressor.java index f0bbb5c32..c16899977 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SummariesDecompressor.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SummariesDecompressor.java @@ -17,11 +17,12 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.apache.hadoop.io.compress.GzipCodec; import org.mortbay.log.Log; -import eu.dnetlib.doiboost.orcid.json.JsonWriter; -import eu.dnetlib.doiboost.orcid.model.AuthorData; +import eu.dnetlib.dhp.schema.orcid.AuthorData; import eu.dnetlib.doiboost.orcid.xml.XMLRecordParser; +import eu.dnetlib.doiboost.orcidnodoi.json.JsonWriter; public class SummariesDecompressor { @@ -56,6 +57,7 @@ public class SummariesDecompressor { int nameFound = 0; int surnameFound = 0; int creditNameFound = 0; + int otherNamesFound = 0; int errorFromOrcidFound = 0; int xmlParserErrorFound = 0; try (TarArchiveInputStream tais = new TarArchiveInputStream(gzipInputStream)) { @@ -117,6 +119,9 @@ public class SummariesDecompressor { if (authorData.getCreditName() != null) { creditNameFound += 1; } + if (authorData.getOtherNames() != null && authorData.getOtherNames().size() > 1) { + otherNamesFound += authorData.getOtherNames().size(); + } } else { Log.warn("Data not retrievable [" + entry.getName() + "] " + buffer.toString()); @@ -152,7 +157,71 @@ public class SummariesDecompressor { Log.info("Name found: " + nameFound); Log.info("Surname found: " + surnameFound); Log.info("Credit name found: " + creditNameFound); + Log.info("Other names found: " + otherNamesFound); Log.info("Error from Orcid found: " + errorFromOrcidFound); Log.info("Error parsing xml record found: " + xmlParserErrorFound); } + + public static void extractXML(Configuration conf, String inputUri, Path outputPath) + throws Exception { + String uri = inputUri; + FileSystem fs = FileSystem.get(URI.create(uri), conf); + Path inputPath = new Path(uri); + CompressionCodecFactory factory = new CompressionCodecFactory(conf); + CompressionCodec codec = factory.getCodec(inputPath); + if (codec == null) { + System.err.println("No codec found for " + uri); + System.exit(1); + } + CompressionCodecFactory.removeSuffix(uri, codec.getDefaultExtension()); + InputStream gzipInputStream = null; + try { + gzipInputStream = codec.createInputStream(fs.open(inputPath)); + int counter = 0; + try (TarArchiveInputStream tais = new TarArchiveInputStream(gzipInputStream)) { + TarArchiveEntry entry = null; + CompressionCodec Codec = new GzipCodec(); + org.apache.hadoop.io.SequenceFile.Writer.Option optCom = SequenceFile.Writer + .compression(SequenceFile.CompressionType.RECORD, Codec); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(outputPath), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class), optCom)) { + while ((entry = tais.getNextTarEntry()) != null) { + String filename = entry.getName(); + if (entry.isDirectory()) { + Log.debug("Directory entry name: " + entry.getName()); + } else { + Log.debug("XML record entry name: " + entry.getName()); + counter++; + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); + String line; + StringBuffer buffer = new StringBuffer(); + while ((line = br.readLine()) != null) { + buffer.append(line); + } + String xml = buffer.toString(); + final Text key = new Text( + XMLRecordParser + .retrieveOrcidIdFromSummary( + xml.getBytes(), filename.split("/")[2].substring(0, 19))); + final Text value = new Text(xml); + writer.append(key, value); + } + if ((counter % 100000) == 0) { + Log.info("Current xml records extracted: " + counter); + } + } + } + } + Log.info("Summaries extract completed"); + Log.info("Total XML records parsed: " + counter); + + } finally { + Log.debug("Closing gzip stream"); + IOUtils.closeStream(gzipInputStream); + } + } } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonHelper.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonHelper.java new file mode 100644 index 000000000..94f7d8c91 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonHelper.java @@ -0,0 +1,13 @@ + +package eu.dnetlib.doiboost.orcid.json; + +import com.google.gson.Gson; + +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; + +public class JsonHelper { + + public static String createOidWork(WorkDataNoDoi workData) { + return new Gson().toJson(workData); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonWriter.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonWriter.java deleted file mode 100644 index 35676d5ba..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/json/JsonWriter.java +++ /dev/null @@ -1,28 +0,0 @@ - -package eu.dnetlib.doiboost.orcid.json; - -import com.google.gson.JsonObject; - -import eu.dnetlib.doiboost.orcid.model.AuthorData; -import eu.dnetlib.doiboost.orcid.model.WorkData; - -public class JsonWriter { - - public static String create(AuthorData authorData) { - JsonObject author = new JsonObject(); - author.addProperty("oid", authorData.getOid()); - author.addProperty("name", authorData.getName()); - author.addProperty("surname", authorData.getSurname()); - if (authorData.getCreditName() != null) { - author.addProperty("creditname", authorData.getCreditName()); - } - return author.toString(); - } - - public static String create(WorkData workData) { - JsonObject work = new JsonObject(); - work.addProperty("oid", workData.getOid()); - work.addProperty("doi", workData.getDoi()); - return work.toString(); - } -} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/DownloadedRecordData.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/DownloadedRecordData.java index f66ef82a2..da1a79b19 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/DownloadedRecordData.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/model/DownloadedRecordData.java @@ -3,8 +3,6 @@ package eu.dnetlib.doiboost.orcid.model; import java.io.Serializable; -import org.apache.hadoop.io.Text; - import com.google.gson.JsonObject; import scala.Tuple2; @@ -12,7 +10,7 @@ import scala.Tuple2; public class DownloadedRecordData implements Serializable { private String orcidId; - private String modifiedDate; + private String lastModifiedDate; private String statusCode; private String compressedData; private String errorMessage; @@ -20,7 +18,7 @@ public class DownloadedRecordData implements Serializable { public Tuple2 toTuple2() { JsonObject data = new JsonObject(); data.addProperty("statusCode", getStatusCode()); - data.addProperty("modifiedDate", getModifiedDate()); + data.addProperty("lastModifiedDate", getLastModifiedDate()); if (getCompressedData() != null) { data.addProperty("compressedData", getCompressedData()); } @@ -66,11 +64,11 @@ public class DownloadedRecordData implements Serializable { this.compressedData = compressedData; } - public String getModifiedDate() { - return modifiedDate; + public String getLastModifiedDate() { + return lastModifiedDate; } - public void setModifiedDate(String modifiedDate) { - this.modifiedDate = modifiedDate; + public void setLastModifiedDate(String lastModifiedDate) { + this.lastModifiedDate = lastModifiedDate; } } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParser.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParser.java index 2e43f4d3e..cc9abb621 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParser.java +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParser.java @@ -4,6 +4,8 @@ package eu.dnetlib.doiboost.orcid.xml; import java.util.Arrays; import java.util.List; +import org.mortbay.log.Log; + import com.ximpleware.AutoPilot; import com.ximpleware.EOFException; import com.ximpleware.EncodingException; @@ -14,7 +16,7 @@ import com.ximpleware.VTDNav; import eu.dnetlib.dhp.parser.utility.VtdException; import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; -import eu.dnetlib.doiboost.orcid.model.AuthorData; +import eu.dnetlib.dhp.schema.orcid.AuthorData; import eu.dnetlib.doiboost.orcid.model.WorkData; public class XMLRecordParser { @@ -81,6 +83,12 @@ public class XMLRecordParser { if (!creditNames.isEmpty()) { authorData.setCreditName(creditNames.get(0)); } + + final List otherNames = VtdUtilityParser.getTextValue(ap, vn, "//other-name:content"); + if (!otherNames.isEmpty()) { + authorData.setOtherNames(otherNames); + } + return authorData; } @@ -120,4 +128,33 @@ public class XMLRecordParser { } return workData; } + + public static String retrieveOrcidIdFromSummary(byte[] bytes, String defaultValue) + throws VtdException, ParseException { + return retrieveOrcidId(bytes, defaultValue, NS_RECORD, NS_RECORD_URL, "//record:record", "path").substring(1); + } + + public static String retrieveOrcidIdFromActivity(byte[] bytes, String defaultValue) + throws VtdException, ParseException { + return retrieveOrcidId(bytes, defaultValue, NS_WORK, NS_WORK_URL, "//work:work", "put-code"); + } + + private static String retrieveOrcidId(byte[] bytes, String defaultValue, String ns, String nsUrl, String xpath, + String idAttributeName) + throws VtdException, ParseException { + final VTDGen vg = new VTDGen(); + vg.setDoc(bytes); + vg.parse(true); + final VTDNav vn = vg.getNav(); + final AutoPilot ap = new AutoPilot(vn); + ap.declareXPathNameSpace(ns, nsUrl); + List recordNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, xpath, Arrays.asList(idAttributeName)); + if (!recordNodes.isEmpty()) { + return (recordNodes.get(0).getAttributes().get(idAttributeName)); + } + Log.info("id not found - default: " + defaultValue); + return defaultValue; + } } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/ActivitiesDumpReader.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/ActivitiesDumpReader.java new file mode 100644 index 000000000..c2cfafd87 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/ActivitiesDumpReader.java @@ -0,0 +1,154 @@ + +package eu.dnetlib.doiboost.orcidnodoi; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URI; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.mortbay.log.Log; + +import eu.dnetlib.doiboost.orcid.json.JsonHelper; +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; +import eu.dnetlib.doiboost.orcidnodoi.xml.XMLRecordParserNoDoi; + +/** + * This class write on hdfs one sequence file, the key is an orcid identifier and the + * value is an orcid publication in json format + */ + +public class ActivitiesDumpReader { + + private static final int MAX_XML_WORKS_PARSED = -1; + private static final int XML_WORKS_PARSED_COUNTER_LOG_INTERVAL = 100000; + + public static void parseGzActivities(Configuration conf, String inputUri, Path outputPath) + throws Exception { + String uri = inputUri; + FileSystem fs = FileSystem.get(URI.create(uri), conf); + Path inputPath = new Path(uri); + CompressionCodecFactory factory = new CompressionCodecFactory(conf); + CompressionCodec codec = factory.getCodec(inputPath); + if (codec == null) { + System.err.println("No codec found for " + uri); + System.exit(1); + } + CompressionCodecFactory.removeSuffix(uri, codec.getDefaultExtension()); + InputStream gzipInputStream = null; + try { + gzipInputStream = codec.createInputStream(fs.open(inputPath)); + parseTarActivities(fs, conf, gzipInputStream, outputPath); + + } finally { + Log.debug("Closing gzip stream"); + IOUtils.closeStream(gzipInputStream); + } + } + + private static void parseTarActivities( + FileSystem fs, Configuration conf, InputStream gzipInputStream, Path outputPath) { + int counter = 0; + int noDoiFound = 0; + int errorFromOrcidFound = 0; + int xmlParserErrorFound = 0; + try (TarArchiveInputStream tais = new TarArchiveInputStream(gzipInputStream)) { + TarArchiveEntry entry = null; + + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(outputPath), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class))) { + while ((entry = tais.getNextTarEntry()) != null) { + String filename = entry.getName(); + StringBuffer buffer = new StringBuffer(); + try { + if (entry.isDirectory() || !filename.contains("works")) { + + } else { + Log.debug("XML work entry name: " + entry.getName()); + counter++; + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); // Read directly from + // tarInput + String line; + buffer = new StringBuffer(); + while ((line = br.readLine()) != null) { + buffer.append(line); + } + WorkDataNoDoi workDataNoDoi = XMLRecordParserNoDoi + .VTDParseWorkData(buffer.toString().getBytes()); + if (workDataNoDoi != null) { + if (workDataNoDoi.getErrorCode() != null) { + errorFromOrcidFound += 1; + Log + .debug( + "error from Orcid with code " + + workDataNoDoi.getErrorCode() + + " for entry " + + entry.getName()); + continue; + } + boolean isDoiFound = workDataNoDoi + .getExtIds() + .stream() + .filter(e -> e.getType() != null) + .anyMatch(e -> e.getType().equals("doi")); + if (!isDoiFound) { + String jsonData = JsonHelper.createOidWork(workDataNoDoi); + Log.debug("oid: " + workDataNoDoi.getOid() + " data: " + jsonData); + + final Text key = new Text(workDataNoDoi.getOid()); + final Text value = new Text(jsonData); + + try { + writer.append(key, value); + } catch (IOException e) { + Log.debug("Writing to sequence file: " + e.getMessage()); + Log.debug(e); + throw new RuntimeException(e); + } + noDoiFound += 1; + } + + } else { + Log.warn("Data not retrievable [" + entry.getName() + "] " + buffer.toString()); + xmlParserErrorFound += 1; + } + } + } catch (Exception e) { + throw new Exception(filename, e); + } + + if ((counter % XML_WORKS_PARSED_COUNTER_LOG_INTERVAL) == 0) { + Log.info("Current xml works parsed: " + counter); + } + + if ((MAX_XML_WORKS_PARSED > -1) && (counter > MAX_XML_WORKS_PARSED)) { + break; + } + } + } + } catch (Exception e) { + Log.warn("Parsing work from gzip archive: " + e.getMessage()); + Log.warn(e); + throw new RuntimeException(e); + } + Log.info("Activities parse completed"); + Log.info("Total XML works parsed: " + counter); + Log.info("Total no doi work found: " + noDoiFound); + Log.info("Error from Orcid found: " + errorFromOrcidFound); + Log.info("Error parsing xml work found: " + xmlParserErrorFound); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/GenOrcidAuthorWork.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/GenOrcidAuthorWork.java new file mode 100644 index 000000000..d3e9aeaef --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/GenOrcidAuthorWork.java @@ -0,0 +1,57 @@ + +package eu.dnetlib.doiboost.orcidnodoi; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.mortbay.log.Log; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.doiboost.orcid.OrcidDSManager; + +/** + * This job generates one sequence file, the key is an orcid identifier and the + * value is an orcid publication in json format + */ + +public class GenOrcidAuthorWork extends OrcidDSManager { + + private String activitiesFileNameTarGz; + private String outputWorksPath; + + public static void main(String[] args) throws IOException, Exception { + GenOrcidAuthorWork genOrcidAuthorWork = new GenOrcidAuthorWork(); + genOrcidAuthorWork.loadArgs(args); + genOrcidAuthorWork.generateAuthorsDOIsData(); + } + + public void generateAuthorsDOIsData() throws Exception { + Configuration conf = initConfigurationObject(); + FileSystem fs = initFileSystemObject(conf); + String tarGzUri = hdfsServerUri.concat(workingPath).concat(activitiesFileNameTarGz); + Path outputPath = new Path(hdfsServerUri.concat(workingPath).concat(outputWorksPath)); + ActivitiesDumpReader.parseGzActivities(conf, tarGzUri, outputPath); + } + + private void loadArgs(String[] args) throws IOException, Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + GenOrcidAuthorWork.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/gen_orcid_works-no-doi_from_activities.json"))); + parser.parseArgument(args); + + hdfsServerUri = parser.get("hdfsServerUri"); + Log.info("HDFS URI: " + hdfsServerUri); + workingPath = parser.get("workingPath"); + Log.info("Working Path: " + workingPath); + activitiesFileNameTarGz = parser.get("activitiesFileNameTarGz"); + Log.info("Activities File Name: " + activitiesFileNameTarGz); + outputWorksPath = parser.get("outputWorksPath"); + Log.info("Output Author Work Data: " + outputWorksPath); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/SparkGenEnrichedOrcidWorks.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/SparkGenEnrichedOrcidWorks.java new file mode 100644 index 000000000..a92d534d8 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/SparkGenEnrichedOrcidWorks.java @@ -0,0 +1,180 @@ + +package eu.dnetlib.doiboost.orcidnodoi; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.IOException; +import java.util.Objects; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import com.google.gson.JsonElement; +import com.google.gson.JsonParser; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.orcid.AuthorData; +import eu.dnetlib.doiboost.orcid.json.JsonHelper; +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; +import eu.dnetlib.doiboost.orcidnodoi.oaf.PublicationToOaf; +import eu.dnetlib.doiboost.orcidnodoi.similarity.AuthorMatcher; +import scala.Tuple2; + +/** + * This spark job generates one parquet file, containing orcid publications dataset + */ + +public class SparkGenEnrichedOrcidWorks { + + static Logger logger = LoggerFactory.getLogger(SparkGenEnrichedOrcidWorks.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws IOException, Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkGenEnrichedOrcidWorks.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/gen_enriched_orcid_works_parameters.json"))); + parser.parseArgument(args); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + final String workingPath = parser.get("workingPath"); + final String outputEnrichedWorksPath = parser.get("outputEnrichedWorksPath"); + final String outputWorksPath = parser.get("outputWorksPath"); + final String hdfsServerUri = parser.get("hdfsServerUri"); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaPairRDD summariesRDD = sc + .sequenceFile(workingPath + "authors/authors.seq", Text.class, Text.class); + Dataset summariesDataset = spark + .createDataset( + summariesRDD.map(seq -> loadAuthorFromJson(seq._1(), seq._2())).rdd(), + Encoders.bean(AuthorData.class)); + logger.info("Authors data loaded: " + summariesDataset.count()); + + JavaPairRDD activitiesRDD = sc + .sequenceFile(workingPath + outputWorksPath + "*.seq", Text.class, Text.class); + Dataset activitiesDataset = spark + .createDataset( + activitiesRDD.map(seq -> loadWorkFromJson(seq._1(), seq._2())).rdd(), + Encoders.bean(WorkDataNoDoi.class)); + logger.info("Works data loaded: " + activitiesDataset.count()); + + JavaRDD> enrichedWorksRDD = activitiesDataset + .joinWith( + summariesDataset, + activitiesDataset.col("oid").equalTo(summariesDataset.col("oid")), "inner") + .map( + (MapFunction, Tuple2>) value -> { + WorkDataNoDoi w = value._1; + AuthorData a = value._2; + AuthorMatcher.match(a, w.getContributors()); + return new Tuple2<>(a.getOid(), JsonHelper.createOidWork(w)); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .filter(Objects::nonNull) + .toJavaRDD(); + logger.info("Enriched works RDD ready."); + + final LongAccumulator parsedPublications = spark.sparkContext().longAccumulator("parsedPublications"); + final LongAccumulator enrichedPublications = spark + .sparkContext() + .longAccumulator("enrichedPublications"); + final LongAccumulator errorsGeneric = spark.sparkContext().longAccumulator("errorsGeneric"); + final LongAccumulator errorsInvalidTitle = spark.sparkContext().longAccumulator("errorsInvalidTitle"); + final LongAccumulator errorsNotFoundAuthors = spark + .sparkContext() + .longAccumulator("errorsNotFoundAuthors"); + final LongAccumulator errorsInvalidType = spark.sparkContext().longAccumulator("errorsInvalidType"); + final PublicationToOaf publicationToOaf = new PublicationToOaf( + parsedPublications, + enrichedPublications, + errorsGeneric, + errorsInvalidTitle, + errorsNotFoundAuthors, + errorsInvalidType); + JavaRDD oafPublicationRDD = enrichedWorksRDD + .map( + e -> { + return (Publication) publicationToOaf + .generatePublicationActionsFromJson(e._2()); + }) + .filter(p -> p != null); + + sc.hadoopConfiguration().set("mapreduce.output.fileoutputformat.compress", "true"); + + oafPublicationRDD + .mapToPair( + p -> new Tuple2<>(p.getClass().toString(), + OBJECT_MAPPER.writeValueAsString(new AtomicAction<>(Publication.class, (Publication) p)))) + .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2()))) + .saveAsNewAPIHadoopFile( + workingPath.concat(outputEnrichedWorksPath), + Text.class, + Text.class, + SequenceFileOutputFormat.class, + sc.hadoopConfiguration()); + + logger.info("parsedPublications: " + parsedPublications.value().toString()); + logger.info("enrichedPublications: " + enrichedPublications.value().toString()); + logger.info("errorsGeneric: " + errorsGeneric.value().toString()); + logger.info("errorsInvalidTitle: " + errorsInvalidTitle.value().toString()); + logger.info("errorsNotFoundAuthors: " + errorsNotFoundAuthors.value().toString()); + logger.info("errorsInvalidType: " + errorsInvalidType.value().toString()); + }); + } + + private static AuthorData loadAuthorFromJson(Text orcidId, Text json) { + AuthorData authorData = new AuthorData(); + authorData.setOid(orcidId.toString()); + JsonElement jElement = new JsonParser().parse(json.toString()); + authorData.setName(getJsonValue(jElement, "name")); + authorData.setSurname(getJsonValue(jElement, "surname")); + authorData.setCreditName(getJsonValue(jElement, "creditname")); + return authorData; + } + + private static WorkDataNoDoi loadWorkFromJson(Text orcidId, Text json) { + + WorkDataNoDoi workData = new Gson().fromJson(json.toString(), WorkDataNoDoi.class); + return workData; + } + + private static String getJsonValue(JsonElement jElement, String property) { + if (jElement.getAsJsonObject().has(property)) { + JsonElement name = null; + name = jElement.getAsJsonObject().get(property); + if (name != null && !name.isJsonNull()) { + return name.getAsString(); + } + } + return new String(""); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/json/JsonWriter.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/json/JsonWriter.java new file mode 100644 index 000000000..982fb6316 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/json/JsonWriter.java @@ -0,0 +1,31 @@ + +package eu.dnetlib.doiboost.orcidnodoi.json; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.JsonObject; + +import eu.dnetlib.dhp.schema.orcid.AuthorData; +import eu.dnetlib.doiboost.orcid.model.WorkData; + +/** + * This class converts an object to json and viceversa + */ + +public class JsonWriter { + + public static final com.fasterxml.jackson.databind.ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .setSerializationInclusion(JsonInclude.Include.NON_NULL);; + + public static String create(AuthorData authorData) throws JsonProcessingException { + return OBJECT_MAPPER.writeValueAsString(authorData); + } + + public static String create(WorkData workData) { + JsonObject work = new JsonObject(); + work.addProperty("oid", workData.getOid()); + work.addProperty("doi", workData.getDoi()); + return work.toString(); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/Contributor.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/Contributor.java new file mode 100644 index 000000000..9222c1cc4 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/Contributor.java @@ -0,0 +1,58 @@ + +package eu.dnetlib.doiboost.orcidnodoi.model; + +import java.io.Serializable; + +import eu.dnetlib.dhp.schema.orcid.AuthorData; + +/** + * This class models the data related to a contributor, that are retrieved from an orcid publication + */ + +public class Contributor extends AuthorData implements Serializable { + private String sequence; + private String role; + private transient boolean simpleMatch = false; + private transient Double score = 0.0; + private transient boolean bestMatch = false; + + public String getSequence() { + return sequence; + } + + public void setSequence(String sequence) { + this.sequence = sequence; + } + + public String getRole() { + return role; + } + + public void setRole(String role) { + this.role = role; + } + + public boolean isSimpleMatch() { + return simpleMatch; + } + + public void setSimpleMatch(boolean simpleMatch) { + this.simpleMatch = simpleMatch; + } + + public Double getScore() { + return score; + } + + public void setScore(Double score) { + this.score = score; + } + + public boolean isBestMatch() { + return bestMatch; + } + + public void setBestMatch(boolean bestMatch) { + this.bestMatch = bestMatch; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/ExternalId.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/ExternalId.java new file mode 100644 index 000000000..7fe50ce25 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/ExternalId.java @@ -0,0 +1,36 @@ + +package eu.dnetlib.doiboost.orcidnodoi.model; + +/** + * This class models the data related to external id, that are retrieved from an orcid publication + */ + +public class ExternalId { + private String type; + private String value; + private String relationShip; + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public String getRelationShip() { + return relationShip; + } + + public void setRelationShip(String relationShip) { + this.relationShip = relationShip; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/PublicationDate.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/PublicationDate.java new file mode 100644 index 000000000..5f794d8eb --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/PublicationDate.java @@ -0,0 +1,36 @@ + +package eu.dnetlib.doiboost.orcidnodoi.model; + +/** + * This class models the data related to a publication date, that are retrieved from an orcid publication + */ + +public class PublicationDate { + private String year; + private String month; + private String day; + + public String getYear() { + return year; + } + + public void setYear(String year) { + this.year = year; + } + + public String getMonth() { + return month; + } + + public void setMonth(String month) { + this.month = month; + } + + public String getDay() { + return day; + } + + public void setDay(String day) { + this.day = day; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/WorkDataNoDoi.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/WorkDataNoDoi.java new file mode 100644 index 000000000..58f992d12 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/model/WorkDataNoDoi.java @@ -0,0 +1,104 @@ + +package eu.dnetlib.doiboost.orcidnodoi.model; + +import java.io.Serializable; +import java.util.List; + +/** + * This class models the data that are retrieved from orcid publication + */ + +public class WorkDataNoDoi implements Serializable { + + private String oid; + private String id; + private String sourceName; + private String type; + private List titles; + private List urls; + List extIds; + List publicationDates; + List contributors; + + public String getOid() { + return oid; + } + + public void setOid(String oid) { + this.oid = oid; + } + + public String getErrorCode() { + return errorCode; + } + + public void setErrorCode(String errorCode) { + this.errorCode = errorCode; + } + + private String errorCode; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public List getTitles() { + return titles; + } + + public void setTitles(List titles) { + this.titles = titles; + } + + public String getSourceName() { + return sourceName; + } + + public void setSourceName(String sourceName) { + this.sourceName = sourceName; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public List getUrls() { + return urls; + } + + public void setUrls(List urls) { + this.urls = urls; + } + + public List getExtIds() { + return extIds; + } + + public void setExtIds(List extIds) { + this.extIds = extIds; + } + + public List getPublicationDates() { + return publicationDates; + } + + public void setPublicationDates(List publicationDates) { + this.publicationDates = publicationDates; + } + + public List getContributors() { + return contributors; + } + + public void setContributors(List contributors) { + this.contributors = contributors; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/oaf/PublicationToOaf.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/oaf/PublicationToOaf.java new file mode 100644 index 000000000..1aed66dfd --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/oaf/PublicationToOaf.java @@ -0,0 +1,543 @@ + +package eu.dnetlib.doiboost.orcidnodoi.oaf; + +import static eu.dnetlib.doiboost.orcidnodoi.util.DumpToActionsUtility.*; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.*; + +import eu.dnetlib.dhp.common.PacePerson; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.doiboost.orcidnodoi.util.DumpToActionsUtility; +import eu.dnetlib.doiboost.orcidnodoi.util.Pair; + +/** + * This class converts an orcid publication from json format to oaf + */ + +public class PublicationToOaf implements Serializable { + + static Logger logger = LoggerFactory.getLogger(PublicationToOaf.class); + + public static final String ORCID = "ORCID"; + public final static String orcidPREFIX = "orcid_______"; + public static final String OPENAIRE_PREFIX = "openaire____"; + public static final String SEPARATOR = "::"; + + private final LongAccumulator parsedPublications; + private final LongAccumulator enrichedPublications; + private final LongAccumulator errorsGeneric; + private final LongAccumulator errorsInvalidTitle; + private final LongAccumulator errorsNotFoundAuthors; + private final LongAccumulator errorsInvalidType; + + public PublicationToOaf( + LongAccumulator parsedPublications, + LongAccumulator enrichedPublications, + LongAccumulator errorsGeneric, + LongAccumulator errorsInvalidTitle, + LongAccumulator errorsNotFoundAuthors, + LongAccumulator errorsInvalidType) { + this.parsedPublications = parsedPublications; + this.enrichedPublications = enrichedPublications; + this.errorsGeneric = errorsGeneric; + this.errorsInvalidTitle = errorsInvalidTitle; + this.errorsNotFoundAuthors = errorsNotFoundAuthors; + this.errorsInvalidType = errorsInvalidType; + } + + public PublicationToOaf() { + this.parsedPublications = null; + this.enrichedPublications = null; + this.errorsGeneric = null; + this.errorsInvalidTitle = null; + this.errorsNotFoundAuthors = null; + this.errorsInvalidType = null; + } + + private static Map> datasources = new HashMap>() { + + { + put(ORCID.toLowerCase(), new Pair<>(ORCID, OPENAIRE_PREFIX + SEPARATOR + "orcid")); + + } + }; + + // json external id will be mapped to oaf:pid/@classid Map to oaf:pid/@classname + private static Map> externalIds = new HashMap>() { + + { + put("ark".toLowerCase(), new Pair<>("ark", "ark")); + put("arxiv".toLowerCase(), new Pair<>("arxiv", "arXiv")); + put("pmc".toLowerCase(), new Pair<>("pmc", "pmc")); + put("pmid".toLowerCase(), new Pair<>("pmid", "pmid")); + put("source-work-id".toLowerCase(), new Pair<>("orcidworkid", "orcidworkid")); + put("urn".toLowerCase(), new Pair<>("urn", "urn")); + } + }; + + static Map> typologiesMapping; + + static { + try { + final String tt = IOUtils + .toString( + PublicationToOaf.class + .getResourceAsStream( + "/eu/dnetlib/dhp/doiboost/orcidnodoi/mappings/typologies.json")); + typologiesMapping = new Gson().fromJson(tt, Map.class); + } catch (Exception e) { + throw new RuntimeException("loading typologies", e); + } + } + + public static final String PID_TYPES = "dnet:pid_types"; + + public Oaf generatePublicationActionsFromJson(final String json) { + try { + if (parsedPublications != null) { + parsedPublications.add(1); + } + JsonElement jElement = new JsonParser().parse(json); + JsonObject jObject = jElement.getAsJsonObject(); + return generatePublicationActionsFromDump(jObject); + } catch (Throwable t) { + logger.error("creating publication: " + t.getMessage()); + if (errorsGeneric != null) { + errorsGeneric.add(1); + } + return null; + } + } + + public Oaf generatePublicationActionsFromDump(final JsonObject rootElement) { + + if (!isValid(rootElement)) { + return null; + } + + Publication publication = new Publication(); + + final DataInfo dataInfo = new DataInfo(); + dataInfo.setDeletedbyinference(false); + dataInfo.setInferred(false); + dataInfo.setTrust("0.9"); + dataInfo + .setProvenanceaction( + mapQualifier( + "sysimport:actionset:orcidworks-no-doi", + "sysimport:actionset:orcidworks-no-doi", + "dnet:provenanceActions", + "dnet:provenanceActions")); + publication.setDataInfo(dataInfo); + + publication.setLastupdatetimestamp(new Date().getTime()); + + publication.setDateofcollection("2020-10-14"); + publication.setDateoftransformation(DumpToActionsUtility.now_ISO8601()); + + // Adding external ids + externalIds + .keySet() + .stream() + .forEach(jsonExtId -> { + final String classid = externalIds.get(jsonExtId.toLowerCase()).getValue(); + final String classname = externalIds.get(jsonExtId.toLowerCase()).getKey(); + final String extId = getStringValue(rootElement, jsonExtId); + if (StringUtils.isNotBlank(extId)) { + publication + .getExternalReference() + .add( + convertExtRef(extId, classid, classname, "dnet:pid_types", "dnet:pid_types")); + } + }); + + // Adding source + final String source = getStringValue(rootElement, "sourceName"); + if (StringUtils.isNotBlank(source)) { + Field sourceField = mapStringField(source, null); + if (sourceField == null) { + publication.setSource(null); + } else { + publication.setSource(Arrays.asList(sourceField)); + } + } + + // Adding titles + final List titles = createRepeatedField(rootElement, "titles"); + if (titles == null || titles.isEmpty()) { + if (errorsInvalidTitle != null) { + errorsInvalidTitle.add(1); + } + return null; + } + Qualifier q = mapQualifier("main title", "main title", "dnet:dataCite_title", "dnet:dataCite_title"); + publication + .setTitle( + titles + .stream() + .map(t -> { + return mapStructuredProperty(t, q, null); + }) + .filter(s -> s != null) + .collect(Collectors.toList())); + // Adding identifier + final String id = getStringValue(rootElement, "id"); + String sourceId = null; + if (id != null) { + publication.setOriginalId(Arrays.asList(id)); + sourceId = String.format("50|%s" + SEPARATOR + "%s", orcidPREFIX, DHPUtils.md5(id.toLowerCase())); + } else { + String mergedTitle = titles.stream().map(Object::toString).collect(Collectors.joining(",")); + sourceId = String.format("50|%s" + SEPARATOR + "%s", orcidPREFIX, DHPUtils.md5(mergedTitle.toLowerCase())); + } + publication.setId(sourceId); + + // Adding relevant date + settingRelevantDate(rootElement, publication, "publication_date", "issued", true); + + // Adding collectedfrom + publication.setCollectedfrom(Arrays.asList(createCollectedFrom())); + + // Adding type + final String type = getStringValue(rootElement, "type"); + String cobjValue = ""; + if (StringUtils.isNotBlank(type)) { + publication.setResourcetype(mapQualifier(type, type, "dnet:dataCite_resource", "dnet:dataCite_resource")); + + final String typeValue = typologiesMapping.get(type).get("value"); + cobjValue = typologiesMapping.get(type).get("cobj"); + final Instance instance = new Instance(); + + // Adding hostedby + instance.setHostedby(createHostedBy()); + + // Adding url + final List urls = createRepeatedField(rootElement, "urls"); + if (urls != null && !urls.isEmpty()) { + instance.setUrl(urls); + } else { + dataInfo.setInvisible(true); + } + + final String pubDate = getPublicationDate(rootElement, "publicationDates"); + if (StringUtils.isNotBlank(pubDate)) { + instance.setDateofacceptance(mapStringField(pubDate, null)); + } + + instance.setCollectedfrom(createCollectedFrom()); + + // Adding accessright + instance.setAccessright(mapQualifier("UNKNOWN", "UNKNOWN", "dnet:access_modes", "dnet:access_modes")); + + // Adding type + instance + .setInstancetype( + mapQualifier(cobjValue, typeValue, "dnet:publication_resource", "dnet:publication_resource")); + + publication.setInstance(Arrays.asList(instance)); + } else { + if (errorsInvalidType != null) { + errorsInvalidType.add(1); + } + return null; + } + + // Adding authors + final List authors = createAuthors(rootElement); + if (authors != null && authors.size() > 0) { + publication.setAuthor(authors); + } else { + if (errorsNotFoundAuthors != null) { + errorsNotFoundAuthors.add(1); + } + return null; + } + String classValue = getDefaultResulttype(cobjValue); + publication + .setResulttype(mapQualifier(classValue, classValue, "dnet:result_typologies", "dnet:result_typologies")); + if (enrichedPublications != null) { + enrichedPublications.add(1); + } + return publication; + } + + public List createAuthors(final JsonObject root) { + + final String authorsJSONFieldName = "contributors"; + + if (root.has(authorsJSONFieldName) && root.get(authorsJSONFieldName).isJsonArray()) { + + final List authors = new ArrayList<>(); + final JsonArray jsonAuthors = root.getAsJsonArray(authorsJSONFieldName); + int firstCounter = 0; + int defaultCounter = 0; + int rank = 1; + int currentRank = 0; + + for (final JsonElement item : jsonAuthors) { + final JsonObject jsonAuthor = item.getAsJsonObject(); + final Author author = new Author(); + if (item.isJsonObject()) { + final String creditname = getStringValue(jsonAuthor, "creditName"); + final String surname = getStringValue(jsonAuthor, "surname"); + final String name = getStringValue(jsonAuthor, "name"); + final String oid = getStringValue(jsonAuthor, "oid"); + final String seq = getStringValue(jsonAuthor, "sequence"); + if (StringUtils.isNotBlank(seq)) { + if (seq.equals("first")) { + firstCounter += 1; + rank = firstCounter; + + } else if (seq.equals("additional")) { + rank = currentRank + 1; + } else { + defaultCounter += 1; + rank = defaultCounter; + } + } + if (StringUtils.isNotBlank(oid)) { + author.setPid(Arrays.asList(mapAuthorId(oid))); + author.setFullname(name + " " + surname); + if (StringUtils.isNotBlank(name)) { + author.setName(name); + } + if (StringUtils.isNotBlank(surname)) { + author.setSurname(surname); + } + } else { + PacePerson p = new PacePerson(creditname, false); + if (p.isAccurate()) { + author.setName(p.getNormalisedFirstName()); + author.setSurname(p.getNormalisedSurname()); + author.setFullname(p.getNormalisedFullname()); + } else { + author.setFullname(creditname); + } + } + } + author.setRank(rank); + authors.add(author); + currentRank = rank; + } + return authors; + + } + return null; + } + + private List createRepeatedField(final JsonObject rootElement, final String fieldName) { + if (!rootElement.has(fieldName)) { + return null; + } + if (rootElement.has(fieldName) && rootElement.get(fieldName).isJsonNull()) { + return null; + } + if (rootElement.get(fieldName).isJsonArray()) { + if (!isValidJsonArray(rootElement, fieldName)) { + return null; + } + return getArrayValues(rootElement, fieldName); + } else { + String field = getStringValue(rootElement, fieldName); + return Arrays.asList(cleanField(field)); + } + } + + private String cleanField(String value) { + if (value != null && !value.isEmpty() && value.charAt(0) == '"' && value.charAt(value.length() - 1) == '"') { + value = value.substring(1, value.length() - 1); + } + return value; + } + + private void settingRelevantDate(final JsonObject rootElement, + final Publication publication, + final String jsonKey, + final String dictionaryKey, + final boolean addToDateOfAcceptance) { + + final String pubDate = getPublicationDate(rootElement, "publication_date"); + if (StringUtils.isNotBlank(pubDate)) { + if (addToDateOfAcceptance) { + publication.setDateofacceptance(mapStringField(pubDate, null)); + } + Qualifier q = mapQualifier(dictionaryKey, dictionaryKey, "dnet:dataCite_date", "dnet:dataCite_date"); + publication + .setRelevantdate( + Arrays + .asList(pubDate) + .stream() + .map(r -> { + return mapStructuredProperty(r, q, null); + }) + .filter(s -> s != null) + .collect(Collectors.toList())); + } + } + + private String getPublicationDate(final JsonObject rootElement, + final String jsonKey) { + + JsonObject pubDateJson = null; + try { + pubDateJson = rootElement.getAsJsonObject(jsonKey); + } catch (Exception e) { + return null; + } + if (pubDateJson == null) { + return null; + } + final String year = getStringValue(pubDateJson, "year"); + final String month = getStringValue(pubDateJson, "month"); + final String day = getStringValue(pubDateJson, "day"); + + if (StringUtils.isBlank(year)) { + return null; + } + String pubDate = "".concat(year); + if (StringUtils.isNotBlank(month)) { + pubDate = pubDate.concat("-" + month); + if (StringUtils.isNotBlank(day)) { + pubDate = pubDate.concat("-" + day); + } else { + pubDate += "-01"; + } + } else { + pubDate += "-01-01"; + } + if (isValidDate(pubDate)) { + return pubDate; + } + return null; + } + + protected boolean isValid(final JsonObject rootElement/* , final Reporter context */) { + + final String type = getStringValue(rootElement, "type"); + if (!typologiesMapping.containsKey(type)) { + logger.error("unknowntype_" + type); + if (errorsInvalidType != null) { + errorsInvalidType.add(1); + } + return false; + } + + if (!isValidJsonArray(rootElement, "titles")) { + if (errorsInvalidTitle != null) { + errorsInvalidTitle.add(1); + } + return false; + } + return true; + } + + private boolean isValidJsonArray(final JsonObject rootElement, final String fieldName) { + if (!rootElement.has(fieldName)) { + return false; + } + final JsonElement jsonElement = rootElement.get(fieldName); + if (jsonElement.isJsonNull()) { + return false; + } + if (jsonElement.isJsonArray()) { + final JsonArray jsonArray = jsonElement.getAsJsonArray(); + if (jsonArray.isJsonNull()) { + return false; + } + if (jsonArray.get(0).isJsonNull()) { + return false; + } + } + return true; + } + + private Qualifier mapQualifier(String classId, String className, String schemeId, String schemeName) { + final Qualifier qualifier = new Qualifier(); + qualifier.setClassid(classId); + qualifier.setClassname(className); + qualifier.setSchemeid(schemeId); + qualifier.setSchemename(schemeName); + return qualifier; + } + + private ExternalReference convertExtRef(String extId, String classId, String className, String schemeId, + String schemeName) { + ExternalReference ex = new ExternalReference(); + ex.setRefidentifier(extId); + ex.setQualifier(mapQualifier(classId, className, schemeId, schemeName)); + return ex; + } + + private StructuredProperty mapStructuredProperty(String value, Qualifier qualifier, DataInfo dataInfo) { + if (value == null | StringUtils.isBlank(value)) { + return null; + } + + final StructuredProperty structuredProperty = new StructuredProperty(); + structuredProperty.setValue(value); + structuredProperty.setQualifier(qualifier); + structuredProperty.setDataInfo(dataInfo); + return structuredProperty; + } + + private Field mapStringField(String value, DataInfo dataInfo) { + if (value == null || StringUtils.isBlank(value)) { + return null; + } + + final Field stringField = new Field<>(); + stringField.setValue(value); + stringField.setDataInfo(dataInfo); + return stringField; + } + + private KeyValue createCollectedFrom() { + KeyValue cf = new KeyValue(); + cf.setValue(ORCID); + cf.setKey("10|" + OPENAIRE_PREFIX + SEPARATOR + "806360c771262b4d6770e7cdf04b5c5a"); + return cf; + } + + private KeyValue createHostedBy() { + KeyValue hb = new KeyValue(); + hb.setValue("Unknown Repository"); + hb.setKey("10|" + OPENAIRE_PREFIX + SEPARATOR + "55045bd2a65019fd8e6741a755395c8c"); + return hb; + } + + private StructuredProperty mapAuthorId(String orcidId) { + final StructuredProperty sp = new StructuredProperty(); + sp.setValue(orcidId); + final Qualifier q = new Qualifier(); + q.setClassid(ORCID.toLowerCase()); + q.setClassname(ORCID.toLowerCase()); + q.setSchemeid(ModelConstants.DNET_PID_TYPES); + q.setSchemename(ModelConstants.DNET_PID_TYPES); + sp.setQualifier(q); + final DataInfo dataInfo = new DataInfo(); + dataInfo.setDeletedbyinference(false); + dataInfo.setInferred(false); + dataInfo.setTrust("0.9"); + dataInfo + .setProvenanceaction( + mapQualifier( + "sysimport:crosswalk:entityregistry", + "Harvested", + "dnet:provenanceActions", + "dnet:provenanceActions")); + sp.setDataInfo(dataInfo); + return sp; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/similarity/AuthorMatcher.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/similarity/AuthorMatcher.java new file mode 100644 index 000000000..c0f617868 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/similarity/AuthorMatcher.java @@ -0,0 +1,217 @@ + +package eu.dnetlib.doiboost.orcidnodoi.similarity; + +import java.io.IOException; +import java.text.Normalizer; +import java.util.*; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.text.similarity.JaroWinklerSimilarity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.ximpleware.NavException; +import com.ximpleware.ParseException; +import com.ximpleware.XPathEvalException; +import com.ximpleware.XPathParseException; + +import eu.dnetlib.dhp.parser.utility.VtdException; +import eu.dnetlib.dhp.schema.orcid.AuthorData; +import eu.dnetlib.doiboost.orcidnodoi.model.Contributor; +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; + +/** + * This class is used for searching from a list of publication contributors a + * specific author making a similarity check on both name and surname of the + * author with the credit name of each contributor of the list; as soon as + * the match is found (if exist) author informations are used to enrich the + * matched contribuotr inside contributors list + */ + +public class AuthorMatcher { + + private static final Logger logger = LoggerFactory.getLogger(AuthorMatcher.class); + public static final Double threshold = 0.8; + + public static void match(AuthorData author, List contributors) + throws IOException, XPathEvalException, XPathParseException, NavException, VtdException, ParseException { + + int matchCounter = 0; + List matchCounters = Arrays.asList(matchCounter); + Contributor contributor = null; + contributors + .stream() + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .forEach(c -> { + if (simpleMatch(c.getCreditName(), author.getName()) || + simpleMatch(c.getCreditName(), author.getSurname()) || + simpleMatchOnOtherNames(c.getCreditName(), author.getOtherNames())) { + matchCounters.set(0, matchCounters.get(0) + 1); + c.setSimpleMatch(true); + } + }); + if (matchCounters.get(0) == 1) { + updateAuthorsSimpleMatch(contributors, author); + } else if (matchCounters.get(0) == 0) { + Optional optCon = contributors + .stream() + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .map(c -> { + c.setScore(bestMatch(author.getName(), author.getSurname(), c.getCreditName())); + return c; + }) + .filter(c -> c.getScore() >= threshold) + .max(Comparator.comparing(c -> c.getScore())); + Contributor bestMatchContributor = null; + if (optCon.isPresent()) { + bestMatchContributor = optCon.get(); + bestMatchContributor.setBestMatch(true); + updateAuthorsSimilarityMatch(contributors, author); + } + } else if (matchCounters.get(0) > 1) { + Optional optCon = contributors + .stream() + .filter(c -> c.isSimpleMatch()) + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .map(c -> { + c.setScore(bestMatch(author.getName(), author.getSurname(), c.getCreditName())); + return c; + }) + .filter(c -> c.getScore() >= threshold) + .max(Comparator.comparing(c -> c.getScore())); + Contributor bestMatchContributor = null; + if (optCon.isPresent()) { + bestMatchContributor = optCon.get(); + bestMatchContributor.setBestMatch(true); + updateAuthorsSimilarityMatch(contributors, author); + } + } + + } + + public static boolean simpleMatchOnOtherNames(String name, List otherNames) { + if (otherNames == null || (otherNames != null && otherNames.isEmpty())) { + return false; + } + return otherNames.stream().filter(o -> simpleMatch(name, o)).count() > 0; + } + + public static boolean simpleMatch(String name, String searchValue) { + if (searchValue == null) { + return false; + } + return normalize(name).contains(normalize(searchValue)); + } + + public static Double bestMatch(String authorSurname, String authorName, String contributor) { + String[] contributorSplitted = contributor.split(" "); + if (contributorSplitted.length == 0) { + return 0.0; + } + final String contributorName = contributorSplitted[contributorSplitted.length - 1]; + String contributorSurname = ""; + if (contributorSplitted.length > 1) { + StringJoiner joiner = new StringJoiner(" "); + for (int i = 0; i < contributorSplitted.length - 1; i++) { + joiner.add(contributorSplitted[i]); + } + contributorSurname = joiner.toString(); + } + String authorNameNrm = normalize(authorName); + String authorSurnameNrm = normalize(authorSurname); + String contributorNameNrm = normalize(contributorName); + String contributorSurnameNrm = normalize(contributorSurname); + Double sm1 = similarity(authorNameNrm, authorSurnameNrm, contributorNameNrm, contributorSurnameNrm); + Double sm2 = similarity(authorNameNrm, authorSurnameNrm, contributorSurnameNrm, contributorNameNrm); + if (sm1.compareTo(sm2) >= 0) { + return sm1; + } + return sm2; + } + + public static Double similarity(String nameA, String surnameA, String nameB, String surnameB) { + Double score = similarityJaroWinkler(nameA, surnameA, nameB, surnameB); + return score; + } + + private static Double similarityJaroWinkler(String nameA, String surnameA, String nameB, String surnameB) { + return new JaroWinklerSimilarity().apply(normalize(parse(nameA, surnameA)), normalize(parse(nameB, surnameB))); + } + + public static String normalize(final String s) { + if (s == null) { + return new String(""); + } + return nfd(s) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim(); + } + + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } + + private static String parse(String name, String surname) { + return surname + " " + name; + } + + public static void updateAuthorsSimpleMatch(List contributors, AuthorData author) { + contributors.forEach(c -> { + if (c.isSimpleMatch()) { + c.setName(author.getName()); + c.setSurname(author.getSurname()); + c.setOid(author.getOid()); + } + }); + updateRanks(contributors); + } + + public static void updateAuthorsSimilarityMatch(List contributors, AuthorData author) { + contributors + .stream() + .filter(c -> c.isBestMatch()) + .forEach(c -> { + c.setName(author.getName()); + c.setSurname(author.getSurname()); + c.setOid(author.getOid()); + }); + updateRanks(contributors); + } + + private static void updateRanks(List contributors) { + boolean seqFound = false; + if (contributors + .stream() + .filter( + c -> c.getRole() != null && c.getSequence() != null && + c.getRole().equals("author") && (c.getSequence().equals("first") || + c.getSequence().equals("additional"))) + .count() > 0) { + seqFound = true; + } + if (!seqFound) { + List seqIds = Arrays.asList(0); + contributors.forEach(c -> { + int currentSeq = seqIds.get(0) + 1; + seqIds.set(0, currentSeq); + c.setSequence(Integer.toString(seqIds.get(0))); + }); + } + } + + private static String toJson(WorkDataNoDoi work) { + GsonBuilder builder = new GsonBuilder(); + Gson gson = builder.create(); + return gson.toJson(work); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/DumpToActionsUtility.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/DumpToActionsUtility.java new file mode 100644 index 000000000..8096c4e8e --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/DumpToActionsUtility.java @@ -0,0 +1,113 @@ + +package eu.dnetlib.doiboost.orcidnodoi.util; + +import java.text.SimpleDateFormat; +import java.util.*; + +import org.apache.commons.lang3.StringUtils; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; + +/** + * Utility class + */ + +public class DumpToActionsUtility { + + private static final SimpleDateFormat ISO8601FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.US); + + public static String getStringValue(final JsonObject root, final String key) { + if (root.has(key) && !root.get(key).isJsonNull()) + return root.get(key).getAsString(); + return new String(""); + } + + public static List getArrayValues(final JsonObject root, final String key) { + if (root.has(key) && root.get(key).isJsonArray()) { + final JsonArray asJsonArray = root.get(key).getAsJsonArray(); + final List result = new ArrayList<>(); + + asJsonArray.forEach(it -> { + if (StringUtils.isNotBlank(it.getAsString())) { + result.add(it.getAsString()); + } + }); + return result; + } + return new ArrayList<>(); + } + + public static List getArrayObjects(final JsonObject root, final String key) { + if (root.has(key) && root.get(key).isJsonArray()) { + final JsonArray asJsonArray = root.get(key).getAsJsonArray(); + final List result = new ArrayList<>(); + asJsonArray.forEach(it -> { + if (it.getAsJsonObject() != null) { + result.add(it.getAsJsonObject()); + } + }); + return result; + } + return new ArrayList<>(); + } + + public static boolean isValidDate(final String date) { + return date.matches("\\d{4}-\\d{2}-\\d{2}"); + } + + public static String now_ISO8601() { // NOPMD + String result; + synchronized (ISO8601FORMAT) { + result = ISO8601FORMAT.format(new Date()); + } + // convert YYYYMMDDTHH:mm:ss+HH00 into YYYYMMDDTHH:mm:ss+HH:00 + // - note the added colon for the Timezone + return result.substring(0, result.length() - 2) + ":" + result.substring(result.length() - 2); + } + + public static String getDefaultResulttype(final String cobjcategory) { + switch (cobjcategory) { + case "0029": + return "software"; + case "0021": + case "0024": + case "0025": + case "0030": + return "dataset"; + case "0000": + case "0010": + case "0018": + case "0020": + case "0022": + case "0023": + case "0026": + case "0027": + case "0028": + case "0037": + return "other"; + case "0001": + case "0002": + case "0004": + case "0005": + case "0006": + case "0007": + case "0008": + case "0009": + case "0011": + case "0012": + case "0013": + case "0014": + case "0015": + case "0016": + case "0017": + case "0019": + case "0031": + case "0032": + return "publication"; + default: + return "publication"; + } + } + +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/Pair.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/Pair.java new file mode 100644 index 000000000..8883d00f5 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/util/Pair.java @@ -0,0 +1,32 @@ + +package eu.dnetlib.doiboost.orcidnodoi.util; + +public class Pair { + + private K k; + + private V v; + + public Pair(K k, V v) { + this.k = k; + this.v = v; + } + + public K getKey() { + return k; + } + + public V getValue() { + return v; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Pair) { + Pair tmp = (Pair) obj; + return k.equals(tmp.getKey()) && v.equals(tmp.getValue()); + } else + return false; + } + +} diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/xml/XMLRecordParserNoDoi.java b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/xml/XMLRecordParserNoDoi.java new file mode 100644 index 000000000..f4b093402 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcidnodoi/xml/XMLRecordParserNoDoi.java @@ -0,0 +1,217 @@ + +package eu.dnetlib.doiboost.orcidnodoi.xml; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.ximpleware.*; + +import eu.dnetlib.dhp.parser.utility.VtdException; +import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; +import eu.dnetlib.doiboost.orcidnodoi.model.Contributor; +import eu.dnetlib.doiboost.orcidnodoi.model.ExternalId; +import eu.dnetlib.doiboost.orcidnodoi.model.PublicationDate; +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; + +/** + * This class is used for parsing xml data with vtd parser + */ + +public class XMLRecordParserNoDoi { + + private static final Logger logger = LoggerFactory.getLogger(XMLRecordParserNoDoi.class); + + private static final String NS_COMMON_URL = "http://www.orcid.org/ns/common"; + private static final String NS_COMMON = "common"; + private static final String NS_PERSON_URL = "http://www.orcid.org/ns/person"; + private static final String NS_PERSON = "person"; + private static final String NS_DETAILS_URL = "http://www.orcid.org/ns/personal-details"; + private static final String NS_DETAILS = "personal-details"; + private static final String NS_OTHER_URL = "http://www.orcid.org/ns/other-name"; + private static final String NS_OTHER = "other-name"; + private static final String NS_RECORD_URL = "http://www.orcid.org/ns/record"; + private static final String NS_RECORD = "record"; + private static final String NS_ERROR_URL = "http://www.orcid.org/ns/error"; + + private static final String NS_WORK = "work"; + private static final String NS_WORK_URL = "http://www.orcid.org/ns/work"; + + private static final String NS_ERROR = "error"; + + public static WorkDataNoDoi VTDParseWorkData(byte[] bytes) + throws VtdException, EncodingException, EOFException, EntityException, ParseException, XPathParseException, + NavException, XPathEvalException { + final VTDGen vg = new VTDGen(); + vg.setDoc(bytes); + vg.parse(true); + final VTDNav vn = vg.getNav(); + final AutoPilot ap = new AutoPilot(vn); + ap.declareXPathNameSpace(NS_COMMON, NS_COMMON_URL); + ap.declareXPathNameSpace(NS_WORK, NS_WORK_URL); + ap.declareXPathNameSpace(NS_ERROR, NS_ERROR_URL); + + WorkDataNoDoi workData = new WorkDataNoDoi(); + final List errors = VtdUtilityParser.getTextValue(ap, vn, "//error:response-code"); + if (!errors.isEmpty()) { + workData.setErrorCode(errors.get(0)); + return workData; + } + + List workNodes = VtdUtilityParser + .getTextValuesWithAttributes(ap, vn, "//work:work", Arrays.asList("path", "put-code")); + if (!workNodes.isEmpty()) { + final String oid = (workNodes.get(0).getAttributes().get("path")).split("/")[1]; + workData.setOid(oid); + final String id = (workNodes.get(0).getAttributes().get("put-code")); + workData.setId(id); + } else { + return null; + } + + final List titles = VtdUtilityParser + .getTextValue( + ap, vn, "//common:title"); + if (!titles.isEmpty()) { + workData.setTitles(titles); + } + + final List sourceNames = VtdUtilityParser + .getTextValue( + ap, vn, "//common:source-name"); + if (!sourceNames.isEmpty()) { + workData.setSourceName(sourceNames.get(0)); + } + + final List types = VtdUtilityParser + .getTextValue( + ap, vn, "//work:type"); + if (!types.isEmpty()) { + workData.setType(types.get(0)); + } + + final List urls = VtdUtilityParser + .getTextValue( + ap, vn, "//common:url"); + if (!urls.isEmpty()) { + workData.setUrls(urls); + } + + workData.setPublicationDates(getPublicationDates(vg, vn, ap)); + workData.setExtIds(getExternalIds(vg, vn, ap)); + workData.setContributors(getContributors(vg, vn, ap)); + return workData; + + } + + private static List getPublicationDates(VTDGen vg, VTDNav vn, AutoPilot ap) + throws XPathParseException, NavException, XPathEvalException { + List publicationDates = new ArrayList(); + int yearIndex = 0; + ap.selectXPath("//common:publication-date/common:year"); + while (ap.evalXPath() != -1) { + PublicationDate publicationDate = new PublicationDate(); + int t = vn.getText(); + if (t >= 0) { + publicationDate.setYear(vn.toNormalizedString(t)); + publicationDates.add(yearIndex, publicationDate); + yearIndex++; + } + } + int monthIndex = 0; + ap.selectXPath("//common:publication-date/common:month"); + while (ap.evalXPath() != -1) { + int t = vn.getText(); + if (t >= 0) { + publicationDates.get(monthIndex).setMonth(vn.toNormalizedString(t)); + monthIndex++; + } + } + int dayIndex = 0; + ap.selectXPath("//common:publication-date/common:day"); + while (ap.evalXPath() != -1) { + int t = vn.getText(); + if (t >= 0) { + publicationDates.get(dayIndex).setDay(vn.toNormalizedString(t)); + dayIndex++; + } + } + return publicationDates; + } + + private static List getExternalIds(VTDGen vg, VTDNav vn, AutoPilot ap) + throws XPathParseException, NavException, XPathEvalException { + List extIds = new ArrayList(); + int typeIndex = 0; + ap.selectXPath("//common:external-id/common:external-id-type"); + while (ap.evalXPath() != -1) { + ExternalId extId = new ExternalId(); + int t = vn.getText(); + if (t >= 0) { + extId.setType(vn.toNormalizedString(t)); + extIds.add(typeIndex, extId); + typeIndex++; + } + } + int valueIndex = 0; + ap.selectXPath("//common:external-id/common:external-id-value"); + while (ap.evalXPath() != -1) { + int t = vn.getText(); + if (t >= 0) { + extIds.get(valueIndex).setValue(vn.toNormalizedString(t)); + valueIndex++; + } + } + int relationshipIndex = 0; + ap.selectXPath("//common:external-id/common:external-id-relationship"); + while (ap.evalXPath() != -1) { + int t = vn.getText(); + if (t >= 0) { + extIds.get(relationshipIndex).setRelationShip(vn.toNormalizedString(t)); + relationshipIndex++; + } + } + if (typeIndex == valueIndex) { + return extIds; + } + return new ArrayList(); + } + + private static List getContributors(VTDGen vg, VTDNav vn, AutoPilot ap) + throws XPathParseException, NavException, XPathEvalException { + List contributors = new ArrayList(); + ap.selectXPath("//work:contributors/work:contributor"); + while (ap.evalXPath() != -1) { + Contributor contributor = new Contributor(); + if (vn.toElement(VTDNav.FIRST_CHILD, "work:credit-name")) { + int val = vn.getText(); + if (val != -1) { + contributor.setCreditName(vn.toNormalizedString(val)); + } + vn.toElement(VTDNav.PARENT); + } + if (vn.toElement(VTDNav.FIRST_CHILD, "work:contributor-attributes")) { + if (vn.toElement(VTDNav.FIRST_CHILD, "work:contributor-sequence")) { + int val = vn.getText(); + if (val != -1) { + contributor.setSequence(vn.toNormalizedString(val)); + } + vn.toElement(VTDNav.PARENT); + } + if (vn.toElement(VTDNav.FIRST_CHILD, "work:contributor-role")) { + int val = vn.getText(); + if (val != -1) { + contributor.setRole(vn.toNormalizedString(val)); + } + vn.toElement(VTDNav.PARENT); + } + vn.toElement(VTDNav.PARENT); + } + contributors.add(contributor); + } + return contributors; + } +} diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_dois_data.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_dois_data.json index 131c30125..b2f0fdeda 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_dois_data.json +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_dois_data.json @@ -1,6 +1,6 @@ [ {"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true}, - {"paramName":"d", "paramLongName":"hdfsOrcidDefaultPath", "paramDescription": "the default work path", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true}, {"paramName":"f", "paramLongName":"activitiesFileNameTarGz", "paramDescription": "the name of the activities orcid file", "paramRequired": true}, {"paramName":"o", "paramLongName":"outputAuthorsDOIsPath", "paramDescription": "the relative folder of the sequencial file to write", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml index be4a45afe..a9cc9ea3c 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml @@ -16,10 +16,10 @@ sparkExecutorCores number of cores used by single executor - - timestamp - Timestamp for incremental Harvesting - + + + + @@ -30,29 +30,29 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - + + + + + + + + - - - ${jobTracker} - ${nameNode} - eu.dnetlib.doiboost.crossref.CrossrefImporter - -t${workingPath}/input/crossref/index_dump_1 - -n${nameNode} - -ts${timestamp} - - - - + + + + + + + + + + + + @@ -68,7 +68,7 @@ --driver-memory=${sparkDriverMemory} ${sparkExtraOPT} - --sourcePath${workingPath}/input/crossref/index_dump,${workingPath}/input/crossref/index_dump_1,${workingPath}/crossref/index_dump + --sourcePath${workingPath}/input/crossref/crossref_ds --targetPath${workingPath}/input/crossref --masteryarn-cluster @@ -78,26 +78,26 @@ - - - yarn-cluster - cluster - ExtractCrossrefToOAF - eu.dnetlib.doiboost.crossref.CrossrefDataset - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - ${sparkExtraOPT} - - --sourcePath/data/doiboost/crossref/cr_dataset - --targetPath/data/doiboost/crossref/crossrefDataset - --masteryarn-cluster - - - - + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/download_orcid_data.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/download_orcid_data.json index 444e487f7..8c69b168b 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/download_orcid_data.json +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/download_orcid_data.json @@ -1,6 +1,6 @@ [ {"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true}, - {"paramName":"d", "paramLongName":"hdfsOrcidDefaultPath", "paramDescription": "the default work path", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true}, {"paramName":"f", "paramLongName":"lambdaFileName", "paramDescription": "the name of the lambda file", "paramRequired": true}, {"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the relative folder of the sequencial file to write", "paramRequired": true}, {"paramName":"t", "paramLongName":"token", "paramDescription": "token to grant access", "paramRequired": true} diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_enriched_orcid_works_parameters.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_enriched_orcid_works_parameters.json new file mode 100644 index 000000000..c3a8f92ec --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_enriched_orcid_works_parameters.json @@ -0,0 +1,7 @@ +[ + {"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true}, + {"paramName":"f", "paramLongName":"activitiesFileNameTarGz", "paramDescription": "the name of the activities orcid file", "paramRequired": true}, + {"paramName":"ow", "paramLongName":"outputWorksPath", "paramDescription": "the relative folder of the sequencial file to write", "paramRequired": true}, + {"paramName":"oew", "paramLongName":"outputEnrichedWorksPath", "paramDescription": "the relative folder of the sequencial file to write the data", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_data.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_from_summaries.json similarity index 76% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_data.json rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_from_summaries.json index bf992b508..6f213e415 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/create_orcid_authors_data.json +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_from_summaries.json @@ -1,6 +1,6 @@ [ {"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true}, - {"paramName":"d", "paramLongName":"hdfsOrcidDefaultPath", "paramDescription": "the default work path", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true}, {"paramName":"f", "paramLongName":"summariesFileNameTarGz", "paramDescription": "the name of the summaries orcid file", "paramRequired": true}, {"paramName":"o", "paramLongName":"outputAuthorsPath", "paramDescription": "the relative folder of the sequencial file to write", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json deleted file mode 100644 index 35bfe1b41..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_authors_parameters.json +++ /dev/null @@ -1,4 +0,0 @@ -[{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path", "paramRequired": true}, - {"paramName":"t", "paramLongName":"token", "paramDescription": "token to grant access", "paramRequired": true}, - {"paramName":"o", "paramLongName":"outputAuthorsPath", "paramDescription": "the relative folder of the sequencial file to write the authors data", "paramRequired": true} -] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_works-no-doi_from_activities.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_works-no-doi_from_activities.json new file mode 100644 index 000000000..c3a8f92ec --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/gen_orcid_works-no-doi_from_activities.json @@ -0,0 +1,7 @@ +[ + {"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true}, + {"paramName":"f", "paramLongName":"activitiesFileNameTarGz", "paramDescription": "the name of the activities orcid file", "paramRequired": true}, + {"paramName":"ow", "paramLongName":"outputWorksPath", "paramDescription": "the relative folder of the sequencial file to write", "paramRequired": true}, + {"paramName":"oew", "paramLongName":"outputEnrichedWorksPath", "paramDescription": "the relative folder of the sequencial file to write the data", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/config-default.xml deleted file mode 100644 index fe14bb8cb..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/config-default.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - jobTracker - hadoop-rm3.garr-pa1.d4science.org:8032 - - - nameNode - hdfs://hadoop-rm1.garr-pa1.d4science.org:8020 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - - oozie.launcher.mapreduce.user.classpath.first - true - - - hive_metastore_uris - thrift://hadoop-edge2.garr-pa1.d4science.org:9083 - - - spark2YarnHistoryServerAddress - http://hadoop-edge1.garr-pa1.d4science.org:18089/ - - - spark2EventLogDir - /user/spark/spark2ApplicationHistory - - - spark2ExtraListeners - "com.cloudera.spark.lineage.NavigatorAppListener" - - - spark2SqlQueryExecutionListeners - "com.cloudera.spark.lineage.NavigatorQueryListener" - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/config-default.xml new file mode 100644 index 000000000..05fe6d014 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/config-default.xml @@ -0,0 +1,31 @@ + + + oozie.action.sharelib.for.java + spark2 + + + oozie.launcher.mapreduce.user.classpath.first + true + + + oozie.launcher.mapreduce.map.java.opts + -Xmx2g + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/workflow.xml new file mode 100644 index 000000000..ea4d33296 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_activities/oozie_app/workflow.xml @@ -0,0 +1,542 @@ + + + + workingPath + the working dir base path + + + shell_cmd_0 + wget -O /tmp/ORCID_2020_10_activites_0.tar.gz https://orcid.figshare.com/ndownloader/files/25002232 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_0.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_0.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_0.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 0 + + + shell_cmd_1 + wget -O /tmp/ORCID_2020_10_activites_1.tar.gz https://orcid.figshare.com/ndownloader/files/25002088 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_1.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_1.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_1.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 1 + + + shell_cmd_2 + wget -O /tmp/ORCID_2020_10_activites_2.tar.gz https://orcid.figshare.com/ndownloader/files/25000596 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_2.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_2.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_2.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 2 + + + shell_cmd_3 + wget -O /tmp/ORCID_2020_10_activites_3.tar.gz https://orcid.figshare.com/ndownloader/files/25015150 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_3.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_3.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_3.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 3 + + + shell_cmd_4 + wget -O /tmp/ORCID_2020_10_activites_4.tar.gz https://orcid.figshare.com/ndownloader/files/25033643 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_4.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_4.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_4.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 4 + + + shell_cmd_5 + wget -O /tmp/ORCID_2020_10_activites_5.tar.gz https://orcid.figshare.com/ndownloader/files/25005483 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_5.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_5.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_5.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 5 + + + shell_cmd_6 + wget -O /tmp/ORCID_2020_10_activites_6.tar.gz https://orcid.figshare.com/ndownloader/files/25005425 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_6.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_6.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_6.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 6 + + + shell_cmd_7 + wget -O /tmp/ORCID_2020_10_activites_7.tar.gz https://orcid.figshare.com/ndownloader/files/25012016 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_7.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_7.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_7.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 7 + + + shell_cmd_8 + wget -O /tmp/ORCID_2020_10_activites_8.tar.gz https://orcid.figshare.com/ndownloader/files/25012079 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_8.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_8.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_8.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 8 + + + shell_cmd_9 + wget -O /tmp/ORCID_2020_10_activites_9.tar.gz https://orcid.figshare.com/ndownloader/files/25010727 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_9.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_9.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_9.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file 9 + + + shell_cmd_X + wget -O /tmp/ORCID_2020_10_activites_X.tar.gz https://orcid.figshare.com/ndownloader/files/25011025 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_activites_X.tar.gz /data/orcid_activities_2020/ORCID_2020_10_activites_X.tar.gz ; rm -f /tmp/ORCID_2020_10_activites_X.tar.gz + + the shell command that downloads and puts to hdfs orcid activity file X + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + + + + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_0.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_0} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_0.tar.gz + -owno_doi_works/works_0.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_1.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_1} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_1.tar.gz + -owno_doi_works/works_1.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_2.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_2} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_2.tar.gz + -owno_doi_works/works_2.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_3.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_3} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_3.tar.gz + -owno_doi_works/works_3.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_4.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_4} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_4.tar.gz + -owno_doi_works/works_4.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_5.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_5} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_5.tar.gz + -owno_doi_works/works_5.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_6.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_6} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_6.tar.gz + -owno_doi_works/works_6.seq + -oewno_doi_enriched_works/ + + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_7.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_7} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_7.tar.gz + -owno_doi_works/works_7.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_8.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_8} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_8.tar.gz + -owno_doi_works/works_8.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_9.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_9} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_9.tar.gz + -owno_doi_works/works_9.seq + -oewno_doi_enriched_works/ + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_activites_X.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_X} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcidnodoi.GenOrcidAuthorWork + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_X.tar.gz + -owno_doi_works/works_X.seq + -oewno_doi_enriched_works/ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_download/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_download/oozie_app/workflow.xml deleted file mode 100644 index 1f9adeb4d..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_download/oozie_app/workflow.xml +++ /dev/null @@ -1,45 +0,0 @@ - - - - workingPathOrcid - the working dir base path - - - token - access token - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - eu.dnetlib.doiboost.orcid.OrcidDownloader - -d${workingPathOrcid}/ - -n${nameNode} - -flast_modified.csv - -odownload/ - -t${token} - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_activities/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_activities/oozie_app/workflow.xml new file mode 100644 index 000000000..6f629c754 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_activities/oozie_app/workflow.xml @@ -0,0 +1,232 @@ + + + + workingPath + the working dir base path + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.java + ${oozieActionShareLibForSpark2} + + + oozie.launcher.mapreduce.user.classpath.first + true + + + oozie.launcher.mapreduce.map.java.opts + -Xmx2g + + + oozie.use.system.libpath + true + + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_0.tar.gz + -owxml/works/xml_works_0.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_1.tar.gz + -owxml/works/xml_works_1.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_2.tar.gz + -owxml/works/xml_works_2.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_3.tar.gz + -owxml/works/xml_works_3.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_4.tar.gz + -owxml/works/xml_works_4.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_5.tar.gz + -owxml/works/xml_works_5.seq + -oew--- + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_6.tar.gz + -owxml/works/xml_works_6.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_7.tar.gz + -owxml/works/xml_works_7.seq + -oew--- + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_8.tar.gz + -owxml/works/xml_works_8.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_9.tar.gz + -owxml/works/xml_works_9.seq + -oew--- + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.ExtractXMLActivitiesData + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_activites_X.tar.gz + -owxml/works/xml_works_X.seq + -oew--- + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/config-default.xml new file mode 100644 index 000000000..191654378 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/config-default.xml @@ -0,0 +1,26 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + oozie.launcher.mapreduce.user.classpath.first + true + + + oozie.launcher.mapreduce.map.java.opts + -Xmx8g + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/workflow.xml similarity index 55% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/workflow.xml rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/workflow.xml index 7a8d04187..68d468ab3 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_extract_xml_summaries/oozie_app/workflow.xml @@ -1,41 +1,40 @@ - + workingPath the working dir base path - + - - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - - + + - + - - - - + + ${jobTracker} ${nameNode} - eu.dnetlib.doiboost.orcid.OrcidDSManager - -d${workingPath}/ + eu.dnetlib.doiboost.orcid.ExtractXMLSummariesData + -w${workingPath}/ -n${nameNode} - -fORCID_2019_summaries.tar.gz - -ooutput/ + -fORCID_2020_10_summaries.tar.gz + -oxml/authors/ + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/config-default.xml deleted file mode 100644 index a720e7592..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/config-default.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - jobTracker - hadoop-rm3.garr-pa1.d4science.org:8032 - - - nameNode - hdfs://hadoop-rm1.garr-pa1.d4science.org:8020 - - - queueName - default - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/workflow.xml deleted file mode 100644 index 7ebc5f0a0..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_gen_authors/oozie_app/workflow.xml +++ /dev/null @@ -1,83 +0,0 @@ - - - - workingPath - the working dir base path - - - token - access token - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - outputPath - the working dir base path - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - yarn - cluster - Split_Lambda_File - eu.dnetlib.doiboost.orcid.SparkPartitionLambdaFile - dhp-doiboost-1.2.1-SNAPSHOT.jar - --num-executors 24 --conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2" --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} - - -w${workingPath}/ - -oauthors/ - -t${token} - - - - - - - - ${jobTracker} - ${nameNode} - yarn - cluster - Gen_Orcid_Authors - eu.dnetlib.doiboost.orcid.SparkOrcidGenerateAuthors - dhp-doiboost-1.2.1-SNAPSHOT.jar - --num-executors 20 --conf spark.yarn.jars="hdfs://hadoop-rm1.garr-pa1.d4science.org:8020/user/oozie/share/lib/lib_20180405103059/spark2" --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} - - -w${workingPath}/ - -oauthors/ - -t${token} - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/config-default.xml new file mode 100644 index 000000000..191654378 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/config-default.xml @@ -0,0 +1,26 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + oozie.launcher.mapreduce.user.classpath.first + true + + + oozie.launcher.mapreduce.map.java.opts + -Xmx8g + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/workflow.xml new file mode 100644 index 000000000..8517f35ee --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_summaries/oozie_app/workflow.xml @@ -0,0 +1,68 @@ + + + + workingPath + the working dir base path + + + shell_cmd_0 + wget -O /tmp/ORCID_2020_10_summaries.tar.gz https://orcid.figshare.com/ndownloader/files/25032905 ; hdfs dfs -copyFromLocal /tmp/ORCID_2020_10_summaries.tar.gz /data/orcid_activities_2020/ORCID_2020_10_summaries.tar.gz ; rm -f /tmp/ORCID_2020_10_summaries.tar.gz + + the shell command that downloads and puts to hdfs orcid summaries + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + ${fs:exists(concat(workingPath,'/ORCID_2020_10_summaries.tar.gz'))} + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd_0} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.OrcidDSManager + -w${workingPath}/ + -n${nameNode} + -fORCID_2020_10_summaries.tar.gz + -oauthors/ + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_download/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_updates_download/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_download/oozie_app/config-default.xml rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_updates_download/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_updates_download/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_updates_download/oozie_app/workflow.xml new file mode 100644 index 000000000..b9383558c --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_updates_download/oozie_app/workflow.xml @@ -0,0 +1,168 @@ + + + + workingPath + the working dir base path + + + token + access token + + + shell_cmd + wget -O /tmp/last_modified.csv.tar http://74804fb637bd8e2fba5b-e0a029c2f87486cddec3b416996a6057.r3.cf1.rackcdn.com/last_modified.csv.tar ; hdfs dfs -copyFromLocal /tmp/last_modified.csv.tar /data/orcid_activities_2020/last_modified.csv.tar ; rm -f /tmp/last_modified.csv.tar + + the shell command that downloads the lambda file from orcid containing last orcid update informations + + + sparkDriverMemory + 7G + memory for driver process + + + sparkExecutorMemory + 2G + memory for individual executor + + + sparkExecutorCores + 1 + number of cores used by single executor + + + spark2MaxExecutors + 10 + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + bash + -c + ${shell_cmd} + + + + + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.orcid.OrcidDownloader + -w${workingPath}/ + -n${nameNode} + -flast_modified.csv.tar + -odownloads/ + -t${token} + + + + + + + + yarn-cluster + cluster + GenLastModifiedSeq + eu.dnetlib.doiboost.orcid.SparkGenLastModifiedSeq + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + + -w${workingPath}/ + -n${nameNode} + -flast_modified.csv.tar + -olast_modified.seq + -t- + + + + + + + + yarn-cluster + cluster + DownloadOrcidAuthors + eu.dnetlib.doiboost.orcid.SparkDownloadOrcidAuthors + dhp-doiboost-${projectVersion}.jar + + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + + -w${workingPath}/ + -n${nameNode} + -flast_modified.seq + -odownloads/updated_authors + -t${token} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/mappings/typologies.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/mappings/typologies.json new file mode 100644 index 000000000..cb696f279 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/mappings/typologies.json @@ -0,0 +1,41 @@ +{ + "reference-entry": {"cobj":"0013", "value": "Part of book or chapter of book"}, + "report": {"cobj":"0017", "value": "Report"}, + "dataset": {"cobj":"0021", "value": "Dataset"}, + "journal-article": {"cobj":"0001", "value": "Article"}, + "reference-book": {"cobj":"0002", "value": "Book"}, + "other": {"cobj":"0020", "value": "Other ORP type"}, + "proceedings-article": {"cobj":"0004", "value": "Conference object"}, + "standard": {"cobj":"0038", "value": "Other literature type"}, + "book-part": {"cobj":"0002", "value": "Book"}, + "monograph": {"cobj":"0002", "value": "Book"}, + "report-series": {"cobj":"0017", "value": "Report"}, + "book": {"cobj":"0002", "value": "Book"}, + "book-chapter": {"cobj":"0013", "value": "Part of book or chapter of book"}, + "peer-review": {"cobj":"0015", "value": "Review"}, + "book-section": {"cobj":"0013", "value": "Part of book or chapter of book"}, + "book-review": {"cobj":"0015", "value": "Review"}, + "conference-abstract": {"cobj":"0004", "value": "Conference object"}, + "conference-paper": {"cobj":"0004", "value": "Conference object"}, + "conference-poster": {"cobj":"0004", "value": "Conference object"}, + "data-set": {"cobj":"0021", "value": "Dataset"}, + "dictionary-entry": {"cobj":"0038", "value": "Other literature type"}, + "disclosure": {"cobj":"0038", "value": "Other literature type"}, + "dissertation": {"cobj":"0006", "value": "Doctoral thesis"}, + "edited-book": {"cobj":"0002", "value": "Book"}, + "encyclopedia-entry": {"cobj":"0038", "value": "Other literature type"}, + "lecture-speech": {"cobj":"0010", "value": "Lecture"}, + "license": {"cobj":"0038", "value": "Other literature type"}, + "magazine-article": {"cobj":"0005", "value": "Contribution for newspaper or weekly magazine"}, + "manual": {"cobj":"0038", "value": "Other literature type"}, + "newsletter-article": {"cobj":"0012", "value": "Newsletter"}, + "newspaper-article": {"cobj":"0005", "value": "Contribution for newspaper or weekly magazine"}, + "patent": {"cobj":"0019", "value": "Patent"}, + "research-technique": {"cobj":"0020", "value": "Other ORP type"}, + "research-tool": {"cobj":"0020", "value": "Other ORP type"}, + "standards-and-policy": {"cobj":"0038", "value": "Other literature type"}, + "supervised-student-publication": {"cobj":"0001", "value": "Article"}, + "technical-standard": {"cobj":"0038", "value": "Other literature type"}, + "website": {"cobj":"0020", "value": "Other ORP type"}, + "working-paper": {"cobj":"0014", "value": "Research"} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/oozie_app/workflow.xml new file mode 100644 index 000000000..6cec48a6d --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcidnodoi/oozie_app/workflow.xml @@ -0,0 +1,95 @@ + + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + workingPath + the working dir base path + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + yarn-cluster + cluster + GenOrcidNoDoiDataset + eu.dnetlib.doiboost.orcidnodoi.SparkGenEnrichedOrcidWorks + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + + -w${workingPath}/ + -n${nameNode} + -f- + -owno_doi_works/ + -oewno_doi_dataset + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala deleted file mode 100644 index 61c1f5111..000000000 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/dhp/doiboost/QueryTest.scala +++ /dev/null @@ -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() - - - } - -} diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala index a3bb2a4f4..4568e23a5 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/crossref/CrossrefMappingTest.scala @@ -158,7 +158,7 @@ class CrossrefMappingTest { rels.foreach(s => logger.info(s.getTarget)) - assertEquals(rels.size, 3 ) + assertEquals(rels.size, 6 ) } diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/OrcidClientTest.java b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/OrcidClientTest.java index 75f857ca4..66a7badb7 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/OrcidClientTest.java +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/OrcidClientTest.java @@ -3,23 +3,34 @@ package eu.dnetlib.doiboost.orcid; import static org.junit.jupiter.api.Assertions.assertTrue; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; +import java.io.*; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.time.Duration; +import java.time.LocalDateTime; +import java.time.temporal.TemporalUnit; import java.util.Arrays; import java.util.Date; import java.util.List; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; import org.apache.commons.io.IOUtils; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; +import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull; import org.junit.jupiter.api.Test; +import org.mortbay.log.Log; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import jdk.nashorn.internal.ir.annotations.Ignore; public class OrcidClientTest { final String orcidId = "0000-0001-7291-3210"; @@ -32,16 +43,64 @@ public class OrcidClientTest { String lastUpdate = "2019-09-30 00:00:00"; String shortDate = "2020-05-06 16:06:11"; -// curl -i -H "Accept: application/vnd.orcid+xml" +// curl -i -H "Accept: application/vnd.orcid+xml" // -H 'Authorization: Bearer 78fdb232-7105-4086-8570-e153f4198e3d' // 'https://api.orcid.org/v3.0/0000-0001-7291-3210/record' - public String testDownloadRecord(String orcidId) throws Exception { + @Test + private void multipleDownloadTest() throws Exception { + int toDownload = 10; + long start = System.currentTimeMillis(); + OrcidDownloader downloader = new OrcidDownloader(); + TarArchiveInputStream input = new TarArchiveInputStream( + new GzipCompressorInputStream(new FileInputStream("/tmp/last_modified.csv.tar"))); + TarArchiveEntry entry = input.getNextTarEntry(); + BufferedReader br = null; + StringBuilder sb = new StringBuilder(); + int rowNum = 0; + int entryNum = 0; + int modified = 0; + while (entry != null) { + br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput + String line; + while ((line = br.readLine()) != null) { + String[] values = line.toString().split(","); + List recordInfo = Arrays.asList(values); + String orcidId = recordInfo.get(0); + if (downloader.isModified(orcidId, recordInfo.get(3))) { + slowedDownDownload(orcidId); + modified++; + } + rowNum++; + if (modified > toDownload) { + break; + } + } + entryNum++; + entry = input.getNextTarEntry(); + } + long end = System.currentTimeMillis(); + logToFile("start test: " + new Date(start).toString()); + logToFile("end test: " + new Date(end).toString()); + } + + @Test + private void downloadTest(String orcid) throws Exception { + String record = testDownloadRecord(orcid); + String filename = "/tmp/downloaded_".concat(orcid).concat(".xml"); + File f = new File(filename); + OutputStream outStream = new FileOutputStream(f); + IOUtils.write(record.getBytes(), outStream); + } + + private String testDownloadRecord(String orcidId) throws Exception { try (CloseableHttpClient client = HttpClients.createDefault()) { HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record"); httpGet.addHeader("Accept", "application/vnd.orcid+xml"); httpGet.addHeader("Authorization", "Bearer 78fdb232-7105-4086-8570-e153f4198e3d"); + logToFile("start connection: " + new Date(System.currentTimeMillis()).toString()); CloseableHttpResponse response = client.execute(httpGet); + logToFile("end connection: " + new Date(System.currentTimeMillis()).toString()); if (response.getStatusLine().getStatusCode() != 200) { System.out .println("Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode()); @@ -53,8 +112,8 @@ public class OrcidClientTest { return new String(""); } -// @Test - public void testLambdaFileParser() throws Exception { + // @Test + private void testLambdaFileParser() throws Exception { try (BufferedReader br = new BufferedReader( new InputStreamReader(this.getClass().getResourceAsStream("last_modified.csv")))) { String line; @@ -99,8 +158,8 @@ public class OrcidClientTest { } } -// @Test - public void getRecordDatestamp() throws ParseException { + // @Test + private void getRecordDatestamp() throws ParseException { Date toRetrieveDateDt = new SimpleDateFormat(DATE_FORMAT).parse(toRetrieveDate); Date toNotRetrieveDateDt = new SimpleDateFormat(DATE_FORMAT).parse(toNotRetrieveDate); Date lastUpdateDt = new SimpleDateFormat(DATE_FORMAT).parse(lastUpdate); @@ -108,7 +167,7 @@ public class OrcidClientTest { assertTrue(!toNotRetrieveDateDt.after(lastUpdateDt)); } - public void testDate(String value) throws ParseException { + private void testDate(String value) throws ParseException { System.out.println(value.toString()); if (value.length() != 19) { value = value.substring(0, 19); @@ -117,20 +176,126 @@ public class OrcidClientTest { System.out.println(valueDt.toString()); } -// @Test - public void testModifiedDate() throws ParseException { + // @Test + @Ignore + private void testModifiedDate() throws ParseException { testDate(toRetrieveDate); testDate(toNotRetrieveDate); testDate(shortDate); } -// @Test - public void testReadBase64CompressedRecord() throws Exception { + @Test + private void testReadBase64CompressedRecord() throws Exception { final String base64CompressedRecord = IOUtils - .toString(getClass().getResourceAsStream("0000-0001-6645-509X.compressed.base64")); + .toString(getClass().getResourceAsStream("0000-0003-3028-6161.compressed.base64")); final String recordFromSeqFile = ArgumentApplicationParser.decompressValue(base64CompressedRecord); - System.out.println(recordFromSeqFile); - final String downloadedRecord = testDownloadRecord("0000-0001-6645-509X"); + logToFile("\n\ndownloaded \n\n" + recordFromSeqFile); + final String downloadedRecord = testDownloadRecord("0000-0003-3028-6161"); assertTrue(recordFromSeqFile.equals(downloadedRecord)); } + + @Test + private void lambdaFileReaderTest() throws Exception { + TarArchiveInputStream input = new TarArchiveInputStream( + new GzipCompressorInputStream(new FileInputStream("/develop/last_modified.csv.tar"))); + TarArchiveEntry entry = input.getNextTarEntry(); + BufferedReader br = null; + StringBuilder sb = new StringBuilder(); + int rowNum = 0; + int entryNum = 0; + while (entry != null) { + br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput + String line; + while ((line = br.readLine()) != null) { + String[] values = line.toString().split(","); + List recordInfo = Arrays.asList(values); + assertTrue(recordInfo.size() == 4); + + rowNum++; + if (rowNum == 1) { + assertTrue(recordInfo.get(3).equals("last_modified")); + } else if (rowNum == 2) { + assertTrue(recordInfo.get(0).equals("0000-0002-0499-7333")); + } + } + entryNum++; + assertTrue(entryNum == 1); + entry = input.getNextTarEntry(); + } + } + + @Test + private void lambdaFileCounterTest() throws Exception { + final String lastUpdate = "2020-09-29 00:00:00"; + OrcidDownloader downloader = new OrcidDownloader(); + TarArchiveInputStream input = new TarArchiveInputStream( + new GzipCompressorInputStream(new FileInputStream("/tmp/last_modified.csv.tar"))); + TarArchiveEntry entry = input.getNextTarEntry(); + BufferedReader br = null; + StringBuilder sb = new StringBuilder(); + int rowNum = 0; + int entryNum = 0; + int modified = 0; + while (entry != null) { + br = new BufferedReader(new InputStreamReader(input)); // Read directly from tarInput + String line; + while ((line = br.readLine()) != null) { + String[] values = line.toString().split(","); + List recordInfo = Arrays.asList(values); + String orcidId = recordInfo.get(0); + if (downloader.isModified(orcidId, recordInfo.get(3))) { + modified++; + } + rowNum++; + } + entryNum++; + entry = input.getNextTarEntry(); + } + logToFile("rowNum: " + rowNum); + logToFile("modified: " + modified); + } + + private void logToFile(String log) + throws IOException { + log = log.concat("\n"); + Path path = Paths.get("/tmp/orcid_log.txt"); + Files.write(path, log.getBytes(), StandardOpenOption.APPEND); + } + + @Test + private void slowedDownDownloadTest() throws Exception { + String orcid = "0000-0001-5496-1243"; + String record = slowedDownDownload(orcid); + String filename = "/tmp/downloaded_".concat(orcid).concat(".xml"); + File f = new File(filename); + OutputStream outStream = new FileOutputStream(f); + IOUtils.write(record.getBytes(), outStream); + } + + private String slowedDownDownload(String orcidId) throws Exception { + try (CloseableHttpClient client = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet("https://api.orcid.org/v3.0/" + orcidId + "/record"); + httpGet.addHeader("Accept", "application/vnd.orcid+xml"); + httpGet.addHeader("Authorization", "Bearer 78fdb232-7105-4086-8570-e153f4198e3d"); + long start = System.currentTimeMillis(); + CloseableHttpResponse response = client.execute(httpGet); + long endReq = System.currentTimeMillis(); + long reqSessionDuration = endReq - start; + logToFile("req time (millisec): " + reqSessionDuration); + if (reqSessionDuration < 1000) { + logToFile("wait ...."); + Thread.sleep(1000 - reqSessionDuration); + } + long end = System.currentTimeMillis(); + long total = end - start; + logToFile("total time (millisec): " + total); + if (response.getStatusLine().getStatusCode() != 200) { + logToFile("Downloading " + orcidId + " status code: " + response.getStatusLine().getStatusCode()); + } + return IOUtils.toString(response.getEntity().getContent()); + } catch (Throwable e) { + e.printStackTrace(); + } + return new String(""); + } } diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParserTest.java b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParserTest.java index d5da4eec0..b7be5e5cd 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParserTest.java +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/xml/XMLRecordParserTest.java @@ -2,17 +2,19 @@ package eu.dnetlib.doiboost.orcid.xml; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Test; -import eu.dnetlib.doiboost.orcid.model.AuthorData; +import eu.dnetlib.dhp.schema.orcid.AuthorData; import eu.dnetlib.doiboost.orcid.model.WorkData; +import eu.dnetlib.doiboost.orcidnodoi.json.JsonWriter; public class XMLRecordParserTest { @Test - public void testOrcidAuthorDataXMLParser() throws Exception { + private void testOrcidAuthorDataXMLParser() throws Exception { String xml = IOUtils.toString(this.getClass().getResourceAsStream("summary_0000-0001-6828-479X.xml")); @@ -27,7 +29,7 @@ public class XMLRecordParserTest { } @Test - public void testOrcidXMLErrorRecordParser() throws Exception { + private void testOrcidXMLErrorRecordParser() throws Exception { String xml = IOUtils.toString(this.getClass().getResourceAsStream("summary_error.xml")); @@ -40,11 +42,11 @@ public class XMLRecordParserTest { } @Test - public void testOrcidWorkDataXMLParser() throws Exception { + private void testOrcidWorkDataXMLParser() throws Exception { String xml = IOUtils .toString( - this.getClass().getResourceAsStream("activity_work_0000-0002-5982-8983.xml")); + this.getClass().getResourceAsStream("activity_work_0000-0003-2760-1191.xml")); XMLRecordParser p = new XMLRecordParser(); @@ -55,4 +57,21 @@ public class XMLRecordParserTest { assertNotNull(workData.getDoi()); System.out.println("doi: " + workData.getDoi()); } + + @Test + public void testOrcidOtherNamesXMLParser() throws Exception { + + String xml = IOUtils + .toString( + this.getClass().getResourceAsStream("summary_0000-0001-5109-1000_othername.xml")); + + XMLRecordParser p = new XMLRecordParser(); + + AuthorData authorData = XMLRecordParser.VTDParseAuthorData(xml.getBytes()); + assertNotNull(authorData); + assertNotNull(authorData.getOtherNames()); + assertTrue(authorData.getOtherNames().get(0).equals("Andrew C. Porteus")); + String jsonData = JsonWriter.create(authorData); + assertNotNull(jsonData); + } } diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/PublicationToOafTest.java b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/PublicationToOafTest.java new file mode 100644 index 000000000..01e26dcb4 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/PublicationToOafTest.java @@ -0,0 +1,78 @@ + +package eu.dnetlib.doiboost.orcidnodoi; + +import static org.junit.jupiter.api.Assertions.*; + +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.JsonElement; +import com.google.gson.JsonParser; + +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.doiboost.orcidnodoi.oaf.PublicationToOaf; +import jdk.nashorn.internal.ir.annotations.Ignore; + +public class PublicationToOafTest { + + private static final Logger logger = LoggerFactory.getLogger(PublicationToOafTest.class); + + @Test + @Ignore + private void convertOafPublicationTest() throws Exception { + String jsonPublication = IOUtils + .toString( + PublicationToOafTest.class.getResourceAsStream("publication.json")); + JsonElement j = new JsonParser().parse(jsonPublication); + logger.info("json publication loaded: " + j.toString()); + PublicationToOaf publicationToOaf = new PublicationToOaf(); + Publication oafPublication = (Publication) publicationToOaf + .generatePublicationActionsFromDump(j.getAsJsonObject()); + assertNotNull(oafPublication.getId()); + assertNotNull(oafPublication.getOriginalId()); + assertEquals(oafPublication.getOriginalId().get(0), "60153327"); + logger.info("oafPublication.getId(): " + oafPublication.getId()); + assertEquals( + oafPublication.getTitle().get(0).getValue(), + "Evaluation of a percutaneous optical fibre glucose sensor (FiberSense) across the glycemic range with rapid glucoseexcursions using the glucose clamp"); + assertNotNull(oafPublication.getLastupdatetimestamp()); + assertNotNull(oafPublication.getDateofcollection()); + assertNotNull(oafPublication.getDateoftransformation()); + assertTrue(oafPublication.getAuthor().size() == 7); + oafPublication.getAuthor().forEach(a -> { + assertNotNull(a.getFullname()); + assertNotNull(a.getRank()); + logger.info("a.getFullname(): " + a.getFullname()); + if (a.getName() != null) { + logger.info("a.getName(): " + a.getName()); + } + if (a.getSurname() != null) { + logger.info("a.getSurname(): " + a.getSurname()); + } + logger.info("a.getRank(): " + a.getRank()); + if (a.getPid() != null) { + logger.info("a.getPid(): " + a.getPid().get(0).getValue()); + } + + }); + assertNotNull(oafPublication.getCollectedfrom()); + if (oafPublication.getSource() != null) { + logger.info((oafPublication.getSource().get(0).getValue())); + } + if (oafPublication.getExternalReference() != null) { + oafPublication.getExternalReference().forEach(e -> { + assertNotNull(e.getRefidentifier()); + assertEquals(e.getQualifier().getSchemeid(), "dnet:pid_types"); + }); + } + assertNotNull(oafPublication.getInstance()); + oafPublication.getInstance().forEach(i -> { + assertNotNull(i.getInstancetype().getClassid()); + logger.info("i.getInstancetype().getClassid(): " + i.getInstancetype().getClassid()); + assertNotNull(i.getInstancetype().getClassname()); + logger.info("i.getInstancetype().getClassname(): " + i.getInstancetype().getClassname()); + }); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/xml/OrcidNoDoiTest.java b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/xml/OrcidNoDoiTest.java new file mode 100644 index 000000000..1f77197ab --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcidnodoi/xml/OrcidNoDoiTest.java @@ -0,0 +1,348 @@ + +package eu.dnetlib.doiboost.orcidnodoi.xml; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.util.*; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.ximpleware.NavException; +import com.ximpleware.ParseException; +import com.ximpleware.XPathEvalException; +import com.ximpleware.XPathParseException; + +import eu.dnetlib.dhp.parser.utility.VtdException; +import eu.dnetlib.dhp.schema.orcid.AuthorData; +import eu.dnetlib.doiboost.orcidnodoi.model.Contributor; +import eu.dnetlib.doiboost.orcidnodoi.model.WorkDataNoDoi; +import eu.dnetlib.doiboost.orcidnodoi.similarity.AuthorMatcher; + +public class OrcidNoDoiTest { + + private static final Logger logger = LoggerFactory.getLogger(OrcidNoDoiTest.class); + + static String nameA = "Khairy"; + static String surnameA = "Abdel Dayem"; + static String orcidIdA = "0000-0003-2760-1191"; + + @Test + public void readPublicationFieldsTest() + throws IOException, XPathEvalException, XPathParseException, NavException, VtdException, ParseException { + logger.info("running loadPublicationFieldsTest ...."); + String xml = IOUtils + .toString( + OrcidNoDoiTest.class.getResourceAsStream("activity_work_0000-0002-2536-4498.xml")); + + if (xml == null) { + logger.info("Resource not found"); + } + XMLRecordParserNoDoi p = new XMLRecordParserNoDoi(); + if (p == null) { + logger.info("XMLRecordParserNoDoi null"); + } + WorkDataNoDoi workData = null; + try { + workData = p.VTDParseWorkData(xml.getBytes()); + } catch (Exception e) { + logger.error("parsing xml", e); + } + assertNotNull(workData); + assertNotNull(workData.getOid()); + logger.info("oid: " + workData.getOid()); + assertNotNull(workData.getTitles()); + logger.info("titles: "); + workData.getTitles().forEach(t -> { + logger.info(t); + }); + logger.info("source: " + workData.getSourceName()); + logger.info("type: " + workData.getType()); + logger.info("urls: "); + workData.getUrls().forEach(u -> { + logger.info(u); + }); + logger.info("publication date: "); + workData.getPublicationDates().forEach(d -> { + logger.info(d.getYear() + " - " + d.getMonth() + " - " + d.getDay()); + }); + logger.info("external id: "); + workData.getExtIds().removeIf(e -> e.getRelationShip() != null && !e.getRelationShip().equals("self")); + workData.getExtIds().forEach(e -> { + logger.info(e.getType() + " - " + e.getValue() + " - " + e.getRelationShip()); + }); + logger.info("contributors: "); + workData.getContributors().forEach(c -> { + logger + .info( + c.getName() + " - " + c.getRole() + " - " + c.getSequence()); + }); + + } + + @Test + public void authorDoubleMatchTest() throws Exception { + logger.info("running authorSimpleMatchTest ...."); + String orcidWork = "activity_work_0000-0003-2760-1191-similarity.xml"; + AuthorData author = new AuthorData(); + author.setName(nameA); + author.setSurname(surnameA); + author.setOid(orcidIdA); + String xml = IOUtils + .toString( + OrcidNoDoiTest.class.getResourceAsStream(orcidWork)); + + if (xml == null) { + logger.info("Resource not found"); + } + XMLRecordParserNoDoi p = new XMLRecordParserNoDoi(); + if (p == null) { + logger.info("XMLRecordParserNoDoi null"); + } + WorkDataNoDoi workData = null; + try { + workData = p.VTDParseWorkData(xml.getBytes()); + } catch (Exception e) { + logger.error("parsing xml", e); + } + assertNotNull(workData); + + Contributor a = workData.getContributors().get(0); + assertTrue(a.getCreditName().equals("Abdel-Dayem K")); + + AuthorMatcher.match(author, workData.getContributors()); + + assertTrue(workData.getContributors().size() == 6); + } + + @Test + public void readContributorsTest() + throws IOException, XPathEvalException, XPathParseException, NavException, VtdException, ParseException { + logger.info("running loadPublicationFieldsTest ...."); + String xml = IOUtils + .toString( + OrcidNoDoiTest.class.getResourceAsStream("activity_work_0000-0003-2760-1191_contributors.xml")); + + if (xml == null) { + logger.info("Resource not found"); + } + XMLRecordParserNoDoi p = new XMLRecordParserNoDoi(); + if (p == null) { + logger.info("XMLRecordParserNoDoi null"); + } + WorkDataNoDoi workData = null; + try { + workData = p.VTDParseWorkData(xml.getBytes()); + } catch (Exception e) { + logger.error("parsing xml", e); + } + assertNotNull(workData.getContributors()); + assertTrue(workData.getContributors().size() == 5); + assertTrue(StringUtils.isBlank(workData.getContributors().get(0).getCreditName())); + assertTrue(workData.getContributors().get(0).getSequence().equals("seq0")); + assertTrue(workData.getContributors().get(0).getRole().equals("role0")); + assertTrue(workData.getContributors().get(1).getCreditName().equals("creditname1")); + assertTrue(StringUtils.isBlank(workData.getContributors().get(1).getSequence())); + assertTrue(StringUtils.isBlank(workData.getContributors().get(1).getRole())); + assertTrue(workData.getContributors().get(2).getCreditName().equals("creditname2")); + assertTrue(workData.getContributors().get(2).getSequence().equals("seq2")); + assertTrue(StringUtils.isBlank(workData.getContributors().get(2).getRole())); + assertTrue(workData.getContributors().get(3).getCreditName().equals("creditname3")); + assertTrue(StringUtils.isBlank(workData.getContributors().get(3).getSequence())); + assertTrue(workData.getContributors().get(3).getRole().equals("role3")); + assertTrue(StringUtils.isBlank(workData.getContributors().get(4).getCreditName())); + assertTrue(workData.getContributors().get(4).getSequence().equals("seq4")); + assertTrue(workData.getContributors().get(4).getRole().equals("role4")); + } + + @Test + public void authorSimpleMatchTest() throws Exception { + String orcidWork = "activity_work_0000-0002-5982-8983.xml"; + AuthorData author = new AuthorData(); + author.setName("Parkhouse"); + author.setSurname("H."); + author.setOid("0000-0002-5982-8983"); + String xml = IOUtils + .toString( + OrcidNoDoiTest.class.getResourceAsStream(orcidWork)); + + if (xml == null) { + logger.info("Resource not found"); + } + XMLRecordParserNoDoi p = new XMLRecordParserNoDoi(); + if (p == null) { + logger.info("XMLRecordParserNoDoi null"); + } + WorkDataNoDoi workData = null; + try { + workData = p.VTDParseWorkData(xml.getBytes()); + } catch (Exception e) { + logger.error("parsing xml", e); + } + assertNotNull(workData); + + Contributor a = workData.getContributors().get(0); + assertTrue(a.getCreditName().equals("Parkhouse, H.")); + + AuthorMatcher.match(author, workData.getContributors()); + + assertTrue(workData.getContributors().size() == 2); + Contributor c = workData.getContributors().get(0); + assertTrue(c.getOid().equals("0000-0002-5982-8983")); + assertTrue(c.getName().equals("Parkhouse")); + assertTrue(c.getSurname().equals("H.")); + assertTrue(c.getCreditName().equals("Parkhouse, H.")); + } + + @Test + public void match() { + + AuthorData author = new AuthorData(); + author.setName("Joe"); + author.setSurname("Dodge"); + author.setOid("0000-1111-2222-3333"); + Contributor contributor = new Contributor(); + contributor.setCreditName("Joe Dodge"); + List contributors = Arrays.asList(contributor); + AuthorMatcher am = new AuthorMatcher(); + int matchCounter = 0; + List matchCounters = Arrays.asList(matchCounter); + contributors + .stream() + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .forEach(c -> { + if (am.simpleMatch(c.getCreditName(), author.getName()) || + am.simpleMatch(c.getCreditName(), author.getSurname()) || + am.simpleMatchOnOtherNames(c.getCreditName(), author.getOtherNames())) { + matchCounters.set(0, matchCounters.get(0) + 1); + c.setSimpleMatch(true); + } + }); + + assertTrue(matchCounters.get(0) == 1); + am.updateAuthorsSimpleMatch(contributors, author); + assertTrue(contributors.get(0).getName().equals("Joe")); + assertTrue(contributors.get(0).getSurname().equals("Dodge")); + assertTrue(contributors.get(0).getCreditName().equals("Joe Dodge")); + assertTrue(contributors.get(0).getOid().equals("0000-1111-2222-3333")); + + AuthorData authorX = new AuthorData(); + authorX.setName(nameA); + authorX.setSurname(surnameA); + authorX.setOid(orcidIdA); + Contributor contributorA = new Contributor(); + contributorA.setCreditName("Abdel-Dayem Khai"); + Contributor contributorB = new Contributor(); + contributorB.setCreditName("Abdel-Dayem Fake"); + List contributorList = new ArrayList<>(); + contributorList.add(contributorA); + contributorList.add(contributorB); + int matchCounter2 = 0; + List matchCounters2 = Arrays.asList(matchCounter2); + contributorList + .stream() + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .forEach(c -> { + if (am.simpleMatch(c.getCreditName(), authorX.getName()) || + am.simpleMatch(c.getCreditName(), authorX.getSurname()) || + am.simpleMatchOnOtherNames(c.getCreditName(), author.getOtherNames())) { + int currentCounter = matchCounters2.get(0); + currentCounter += 1; + matchCounters2.set(0, currentCounter); + c.setSimpleMatch(true); + } + }); + + assertTrue(matchCounters2.get(0) == 2); + assertTrue(contributorList.get(0).isSimpleMatch()); + assertTrue(contributorList.get(1).isSimpleMatch()); + + Optional optCon = contributorList + .stream() + .filter(c -> c.isSimpleMatch()) + .filter(c -> !StringUtils.isBlank(c.getCreditName())) + .map(c -> { + c.setScore(am.bestMatch(authorX.getName(), authorX.getSurname(), c.getCreditName())); + return c; + }) + .filter(c -> c.getScore() >= AuthorMatcher.threshold) + .max(Comparator.comparing(c -> c.getScore())); + assertTrue(optCon.isPresent()); + + final Contributor bestMatchContributor = optCon.get(); + bestMatchContributor.setBestMatch(true); + assertTrue(bestMatchContributor.getCreditName().equals("Abdel-Dayem Khai")); + assertTrue(contributorList.get(0).isBestMatch()); + assertTrue(!contributorList.get(1).isBestMatch()); + am.updateAuthorsSimilarityMatch(contributorList, authorX); + assertTrue(contributorList.get(0).getName().equals(nameA)); + assertTrue(contributorList.get(0).getSurname().equals(surnameA)); + assertTrue(contributorList.get(0).getCreditName().equals("Abdel-Dayem Khai")); + assertTrue(contributorList.get(0).getOid().equals(orcidIdA)); + assertTrue(StringUtils.isBlank(contributorList.get(1).getOid())); + } + + @Test + public void authorBestMatchTest() throws Exception { + String name = "Khairy"; + String surname = "Abdel Dayem"; + String orcidWork = "activity_work_0000-0003-2760-1191.xml"; + AuthorData author = new AuthorData(); + author.setName(name); + author.setSurname(surname); + author.setOid(orcidIdA); + String xml = IOUtils + .toString( + OrcidNoDoiTest.class.getResourceAsStream(orcidWork)); + + if (xml == null) { + logger.info("Resource not found"); + } + XMLRecordParserNoDoi p = new XMLRecordParserNoDoi(); + if (p == null) { + logger.info("XMLRecordParserNoDoi null"); + } + WorkDataNoDoi workData = null; + try { + workData = p.VTDParseWorkData(xml.getBytes()); + } catch (Exception e) { + logger.error("parsing xml", e); + } + AuthorMatcher.match(author, workData.getContributors()); + assertTrue(workData.getContributors().size() == 5); + List c = workData.getContributors(); + assertTrue(c.get(0).getName().equals(name)); + assertTrue(c.get(0).getSurname().equals(surname)); + assertTrue(c.get(0).getCreditName().equals("Khair Abde Daye")); + assertTrue(c.get(0).getOid().equals(orcidIdA)); + } + + @Test + public void otherNamesMatchTest() + throws VtdException, ParseException, IOException, XPathEvalException, NavException, XPathParseException { + + AuthorData author = new AuthorData(); + author.setName("Joe"); + author.setSurname("Dodge"); + author.setOid("0000-1111-2222-3333"); + String otherName1 = new String("Joe Dr. Dodge"); + String otherName2 = new String("XY"); + List others = Lists.newArrayList(); + others.add(otherName1); + others.add(otherName2); + author.setOtherNames(others); + Contributor contributor = new Contributor(); + contributor.setCreditName("XY"); + List contributors = Arrays.asList(contributor); + AuthorMatcher.match(author, contributors); + assertTrue(contributors.get(0).getName().equals("Joe")); + assertTrue(contributors.get(0).getSurname().equals("Dodge")); + assertTrue(contributors.get(0).getOid().equals("0000-1111-2222-3333")); + } +} diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0001-6645-509X.compressed.base64 b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0001-6645-509X.compressed.base64 deleted file mode 100644 index 1b088e061..000000000 --- a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0001-6645-509X.compressed.base64 +++ /dev/null @@ -1 +0,0 @@ -H4sIAAAAAAAAAO1a227bOBB9z1cIepd18SW24aho0wTbAgEWjRdY9I2RaJtbSdSSkhP165eURIm6kHa2SbCLNkBiWDxzhhxyZg7tbN49xZFxhIQinFyZ7sQxDZgEOETJ/sr8Y3trLU2DZiAJQYQTeGUWkJrv/IsNgQEm4bp6MVKQHa5M22E/Fvt1rcViNrfmzupP02AOErpGSQZJAqIr85Bl6dq2Hx8fJ5gEKGR/93ZCbYEQFjDMA5CV01KZNBBhEyKaoSTQW0mgxg6mbCUgg6HGrMEIK5wdILESEEO1VYsRVjGMH1i8DyhVW7WYJhqEYKKJBB8W2ADHsS4A1bhAV1uoRlfjAp2yaWG2S1YIM4AiqrbrIwXDN1g8ah3WgGblMbPWrJwPN9in6gxZKIRJhnYI6mI2BAueXZ5UGaCyrQFNVAjcQcISB+oC0oKEHQhDAqnGpga0WXRE7ABaKaZIf8j7SMHAIvtNbcVHBfLA0gSTQg2uAe0+pREuYhZK3WYJjLD6OwcRC/2pTO/AhC2F5IgCTfLVgO7ZPXVim71hFYLFEOm2tMW02UQhIAFP+pxojm0X186QvSfwiOCjbpoNSNg95JFmV/lof36MgOKc6KI3gJr+hcF+NlX9WJdgKXmqURmRE+RzdsroW+qRLrGxJYsBDe8uvs6qBAzMDphmfuO2AZePq4XY2pVspISVM1zyJCMiHIAI+jDZ2COPa4dayk2dUSL1JEdiJCCwTAErhtkBh/5d2SiskonAcGOrgEMqmj/EiPK+b4Wsq/me464sZ2l53tadrmeLtXc58ZbLry1n32IQ8QjQzIqZeGBBDAWrx7Ztbrnu1puu59P11JksPfdrE/sRm5FlRwDFMPQzkkNpjfXTIZ4Jmoqv7A49s96gxjolKAak0LN0QfU+j+7kpiowdR3SiCZRieSTVplyIWEcEUUPKEIZK85p/hChwKzJxgRYSyJvVXk+2k0abv187rWb1EGP8o1u/QlW3dZLi24lxHqPjjAp1RT1twgkRb4Z6IwO6ATfDsQoKkqs/xmBETIZ0e6GLW2H9LgVe5I2pLqNlmCmLTF120Ovq2gZe9AOa3lEK0Gl5ag0lWxZ6xAhWPSLEqJFJqhFnVB/WnuB6c59qNbG5J5+XSN44aTZ0+qlftg2eEkPWDSPecprY9Aqg2fUyZnlTLfObD2brZ3pZHm5OLNOStOUbjfaWMi47la3XM39Sh/VBqXkaWTfiWPXwFRMte7W0giMiqMvjbVkA7CKtb2yafkkmIpJ0ndaKhmn4uroZi1bF6niG2jCs2pRi1bx1kpdyyYwKg5+edESlABFP3zplOxPbk9wnnaHX9u9zC9VPjpEKZDjQAXYyooU+iFGzfwGg8+iO4Ioh77rTFzXWdnvr69v7u8nPCYTb7X0PNcZ9VNZPctRgknMjv53GBoZAQlF5Q2Wiz2zcQ8Cdu7oafct1/PmwDp1c1FiISyvSc9dOud4llMCoyrZWTHyKYx2o7Qd1PjJGTEbOYkjqJGjuOFJWqZy22XzzApwyG6qly67kCxWjnkqy+0WOSaWWe9LI1BYKAnhE1PNpj4lelqZp+XUmjpbz1szYTt3JjP38hyt3Od9raSXfVR19/TBqHBWEPHjr8192Wr8gl+RSJuzWi5nlrtyp+P3fJ2H3t1/yNS9++uoTn4eMGpsPztAvZCWd4Rrgillt/Q+XfcCoXGsAJXZkqEsOmOLK9g9K1CR9ZFdnBN+kzdu2WnNCTTuQEbQk3HNMp3VvlIXGnflZwfGDhPjI6y+FDC+wBQyJnbHMm7Ze0iMO3yElba7JTg2biIYZATzzzXSA4jwnoDYuEd7lvK0WZRmyhv71KLOb2oK9Hnn5YWam4ryVRqcytlbNznVPF690akcv1SzK/nPangq5An99W8jpIxKXSP4Gf2LlRI+CUAyFERQZJry+DZFuOyb1eeJ6pYjWxRM95fNrJlf+UQfpPPcVOsRS6nKxKebmxvjfXl+60V1x0fUyEBn9LS7rRfvP6rt64/GVlt3vnYXa8ebLJz5T6jt53ObB8OeLl2m2WZvJurP8fviav4cpz+BjF+4znzqzd3TMr5FvryMP5GBPyjjXyC/ZR+/ZPwvGd+Rzh8IQIl1jWOWVkyDf+L/PLMDATSuDyBJYGTdQ67DuYq/ZxUwg/vC+AAoq4fsyXuWtwVF1MA74+bIA/GFlwc2+BHSIgkOBCfoe1kvjC1OuYRPD4WBSi78DRq/szGu+H/p+ddqaiovb9bYVBN4veam8vj/l+6q0PwnNbu7OkOzy3bslxf3ZWNWPThpF4LC91or/va17gefq3e83v0GQZQdAkCgcZPsUQIhQcn+DW4NnbHyqwjxxaP2S0b/YmN3/tnSv/gH9+klwrUpAAA= \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0003-3028-6161.compressed.base64 b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0003-3028-6161.compressed.base64 new file mode 100644 index 000000000..8dc3d32ad --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/0000-0003-3028-6161.compressed.base64 @@ -0,0 +1 @@ +H4sIAAAAAAAAAO1dW5fbthF+z6/A2XPal5biRaREqmvlrG+JG6/t4900bd+4JLSCQxIqSa1X+fUFeIUkAiIlkpFs5jRxTQ1mgAEw881gQF7/+Ox74AmGEcLBiyt1pFwBGDjYRcHji6tf799K5hWIYjtwbQ8H8MXVBkZXP85/uA6hg0N3lv4BVna8fHElK+Qfifw7lsaKZkoTdaJeASIgiGYoiGEY2N6Lq2Ucr2ay/PXr1xEOHeSS/z7KQSTnFHkL6K4dO066xWtSkORtXBTFKHDErRiioh1ckZHYMXQFzQqavBWOlzCUAtuH/FYlTd7Kh/4D0fcSrfitSppCG2GIQ4Em6M85rYN9X6SA9PecOp1CPnX6e069It3CZJYkF8Y28iJ+u13KnMPvcPNVKDAjKEbuk9aCkdOfC9rndA1JyIVBjBYIinS2T5zzWayDdAfw2mYEhVZCuIAh2ThQpJCSKG9nu24II0GbjKDcRU+ILEBphSMkXuS7lDkHotnf+a3orznlkmwTHG74xBlBOU8rD298okrRZOU0eav/rW2PqP7QTt8iy9tGMHxCjmDzZQTba/fQii3mhlgIokMkmtKSptxNEbRDh276dShYttt0ZQ/J30P4hOBXUTcLorzdw9oTzCr9dbd/hEGE16FIe3ukV/MfAPnnOrUfs4SY2TzpryzFOkRzyj0i7EvWFV7iWmZa7LGh3mUuapUQ7DVb4iieF2IL4uRxOhBZOJJrZsOyO5yRxFJ42LE9OIfBtVzxOBMoZHmd7ah86zGC8l+cECZbQPJhvMTu/DZxFFLCKYTutcwj3GcVrR98FFG/L7nEq801RdUlxZK08b2mzDR9NlZHlmX9t+S522JP454dxZJPwANRoptz1RRJVSV1eq+NZwZhrIx0TflvofuKNhXD9mzkQ3ceh2vIjDF7uk9PAE3KL/EOO812fhS0XoXIt8ONmMs2UTbPlTN5nRqYzA4JQFNuiWpqWDUlZSqpk3vVnCnaTLNGxsSqqeGsSxSggCcUoQfkoZgY/dX6wUPOVdbJKmBXMmE7mKw7pmsTSdEl1Ugm35ypxshUpmXXtqgr+VUPWMxVNGBm0CU0mT2iJxgkKC2avwwJ2sV0F4uoDjBc2D7yNgnt/PWacIwr+LFE5YzIzJQwj0sgyeDOSLSIGLIrmeG07Xp2PJaQ4w7pFtdk+adgTcgjxWtsywzj5GBIPKgcELEMMsCYI0th+5xmu+/7SLAKSorHVUHP2SNtb+ImYwCrdSyR+I74fVUxjYkyuRLs+9ojlQtmJLpaefZGQoELn4nl2NGByFaINcC3FV3rluWfIqH93/dpJMdDRD9ES9XUbItqoJQyKOZAkwzL1CTTMsfVeInHfQs/VXHZxk88Ngfx1F5DuZFCdtSX2L87B6/WEZDAGy+iiDfc5bltJavY2cSkhAkUwiF6RPQP5/g5qQ1ea03GYTDb/mQ00QdXh4naM08JcgcnJN7fUfKBLZULZ+yNFG9WxaK4WRNkG4J3rwtOe5S1eD7Z3hrO9SmZBFXVp4pSyS+lqsWQ+MY5E1RFSXdHhJBE5V/t0JXtpOevUxgwIuQ/pk/evX7BdOOvtr/6x8oO4wDSX24/mPcfbz7fVfaOiqzVtxB6SVxAc0vzCHqLSnZbVNt+psr8VzkaFtHU9a9FlMTi5OxhGWozkbkUrX0KvoWIoYzRj49Y1Jrwku0mk2cUIgeWbhsYlbyKTKcYgxRUTZAHO1zdmmnaSB2bDZAHOzOLBcERaeD5GOL1qqGjPrErnEUfyRVkha5K3ZarqcBI+tTSLGMP1ahigJQzlPPmFQhLbHB3oREbmVsUwChjvS406kPrrAwRRNqnO+SO2RYtu2SW9YlumWXV2DUnjeVGWqnCShx3fBgoHXLErEAXUo9EM7gpx1dL6BP7FW4KLrsUQnYh9qAUo9iD80/L0pEzj8VLLSaiBEuSpd2Q0JVupXJKkycH25F/6dIwi2bpg4PtXHsz14xSLfbmkPoPDKawbIFoF1YN2TxqyKp2zJDVJkMWD6VMND/aAfojMamHO5Esul8DlBxqxhuAF+C3DfZRUG5F/rpkGWWphnqb3iGi5u/t0PYRLO0yfVireQgf6eB++0+5BdIn9YTjdUC24PzXEhzmjw4bnIPDLGYXRbb/gB7Xia+pNyn12rOwUdfVSbmCajVpKj1x9amt+/zuw08/fXz/ukoiQ3ZYi02Vw5w9iEivZQFO2UXm9YFYm5htC5uY5H8j3TD+dMymVWSiGmA2rWXMtq+XEzFbC1pnZQyYbcBsR2C2l7azhB4OI+pl7xxEDzMBCsC+hA4RnaL3ieiUc0B0ynGITjGOQnTm+SG6e/hsR8COwc2aJk86R3Y7YhoDu/t/f/fATtXGY2VAdiVZ68hui67MBXIyikUp1oHj2oLs2JxiwxTqbkfbzCke0RWW/0F8WiiLUW8FQlXHmmFYk8Z5xYKnXHI4FaYeUE+LqcUjlc/KGGDqdwRTPy03EXKIlw9ccEPAJA6w30KakejKxU6MQ9sDn7OCFsI/wg4xMrAPwKpax6E3/Rj0pqgdANaT8dvH3z17iX27c+D2AYe+fQJw+/jLmQG3+vBC5IzaABglMqhGGFmNvBBe5DS8c8/dMnYhsz1iHtfyeoWQH0PG48TUYgtZsXQ8Xls17kJu25Q8fnv127Vq+0pqHt+sql7ILafh8aAXDYQMEoJG9XMWrQlVjHuFwoyZYY0svUn9HNO5o7Kgp4ln+bMo80DeoKQ8bDv3imNcjBpUzojYpbUxqjJSVd2StZGiqKqlGF9g1KSURiQhoGbcQ39AF8QEgkQouWZCK7Kv6sstmTQTntfzUGxIlJYgw9pCKytzRNJOrdApeFc0q/ITtVbdNd2Pya6tCGeMiaqa1tgSBi+0rVxSVtWc1igdZZW2m28X78BT8+2n729WRsNApkYpKduqVjmpKlmmqUuqpdYsJ2UlbMU0VZx6jmka1NqyQcurEEdRCBdtRzTJbhEED2x/UrI77NkheLW0w0di4z5DFz+Dtx7+Cl7aMdmYmxm4AfckEoI++LTEMYYedEhI5NBEu0MimZvVKsS2syz6Jwpd5EP9q++KONT1pr4ll8Rj2a5b4knpzTXxOtCde+JJbMtFJfxruSke5QGA1Fg1XHVkW5quwy9k51PpJFRGjgfz3cRdooXNTNxcGli1ny8oLW8tMel48qGkpuCf6d+S85UlBG92TMwddhCMN9l4t5tW4Io9xCCIOQ+UKBwFyLfvfOiTkTYdnzcgh8htFZBrUkQWuWQaim7qk4nFrKMO4XhNqceC8X7hcXuT20m0pVjyp3/dvRqR5T4dmePJZKKbvYVcRPjqKXIOC7+MqW4jEiLrnRgefXw4EiopO4iExKbv1EjodMPKyjiDSKjxxTpWwomX61hWf2YkdMJFO5Zl08t2bNvv4Qju2MuIXVzA2+pYy+FsEqM+YS+2kQNCGK2IfYYUWAZ2TDrreRuAHWcdhih4BBEFmDhw1wTx0b+uvQVyIfBRAAlpCc6HkLYL388T1a//5/Xizwk1Ob05cgkch+t5LNvE9jwZPeF7nviLyy9wb/qmL09LrviSVfBiX3eHLvnWSUgQ25m9SEha2SvqujvPSNSpP20hI0FNC3j35s0boOvxEmx5lbsVdBBZYVEMXhUK+DugNhHQdpebpzD1885TdBHKKupE/jKKsOfbMfUoxkixRophHGPVjvFmjeRfRkDb3oR3lZjSLd0YK5apTPtMTB2QehmT22K2wqidraguN20/W2G2d7e9BfPKyhiyFUO2YshWNNbrhWcrwC8wIlYWxRA44SaKh6REDyiOJ613JMfryJCa6BLd8WT0hPB44r/51ESpuzZSE/3XSpQvj+g0M5HWa70hqnncgFubegfiFZK7KlkpF/Sol7jUHMSk5iuad/lfcg6CTalPRtOpQZxHL5HpbjpdLPwyAtT2prq7shhlbFhT09T7mOOaUi9jclvMPmi1sw/V72RpP/vAmr72sw9NDSsrY8g+DNmHIfvQWK8Xnn0YaiXarJU4CdjxRPUL7ni9GBISXQI+noyeQB9P/DefkCh1d6G1Ej1lJMoiCPA5/SojcSHxEqb1E5zSiYipnbjcZMW0hXeEX2IEq1uqppCtofZSJlFT6mVEsO1NbtuZqMxGus8j9sba7qmGTkDFSNG0Pia+sfxTLtpXMvwOLtGrE0U3FKvG1ZGSsoN0iJFcc58mxnU607XRpP43mFi+1abboKZb05OX1hozXR0Z4/7SIUN4nTSWG2mlKv/R7dvAmgfQb+2HMMNMFPO8cu8hiZ2JEcIusMPQ3kQAP8EQePRyPXkCbYKLCPJ4XNK9RncssXrZhXoX5m8PAgscArhYIPpa2xhEyakOTA98CIrMvpk9BNz9+y+e/N59GK8jx/kxHrfLjfCS90G7iAR2cRLoZU/k7DRYXiEk3ymWNlU03VR1RZko2rhRONf/+bLRTzRX73wZtBSzlb0QH9botQ9r9L4utpqdXmzt7zNIw2HNbuPhsOYw67NBk9/LYc2ANc8Ja353yHI44ukyI8qT0VNWlCf+cgOA2jWnue4us+b0rGKCSznG2csFTvXJeR/j2OEzeuoi1083y5cHJ4XwOIgphtfMqazJ6niijhZrzxut3MXflrHvtZ/wl2AYEvixZ9nq2SnaNgkS56bCvGueed6Ajw+jyH6E81d2EOAYFH0E5TdhQDJWkMzF7CT9bUlsZDD3lPd9HHNwenEGZ2YJBpvqMl0EtyrBXvQGg97L6+5Y0f4B0cO52NHnYmbtczGz83MxazZWR4bZ3ldyWvCFrIzhXGw4F6Nkb+zQ24Abj+y97JsSeAFIJ+kHGglSXDNo8mfsQ6IeO0IR/X7jz+uAlpfGdDAognYEwS3ZjUN9aaeuhyeoT/fD68O3cKTF6c2Ri+nYaIDHtNeIgNeJE6OCLbYnRgb7vM4wOkg6eVqEwJuKb2HHnZ5DEkyc/RCR1enEDNY4x+RQmQTu6+XtLxH28GPyznbmQ8yXmhWyWvhY56XFsuOpKmdTNVrhANJPFZjj8eQYU38Momgkf4hqj4xqVUX8vdcyqk0ou41qVbLh9JFptvii8NP3MitjiGqHqJaSJfEsMVyv8cqm1x7T8w8YYhKtktD1Fsb2A/ZQ5NNA9pY8pef19BLlTbTxVzH2iRFwwOeJrIFb5JQffB0i2o4cEE9a706I15EBae8U6xLN/0V7K9T9eWJttQbWZtv56YeVy5o2n/9h5RYw+qf3+A58/PDmYsG4qZ35N027PaL1wq1Y1NBkS1anujac0NbKwTRV33BAKxzkpQW1qqXLdA3cKupUnfb3napMrC8QO4SwR4ewNa4ElJQdh7DqTJuOpnqLVwJO94CsjCGEHULYpJAPPpNlv8jeu5Acy5LpCOj+osHs78gN4AY8evgBxyGycbTxHBiSaNdNo11vE2Gp+mcS89IS9Q3wh9i2Oz/EE9KXL+LJ/xYiWU5vzvaUtruggNeHb/aQtpsAIenjcEbb4Rktd94u5Ii2Ttqo3SPa92iFXPAZRkSes+whH7T1G2WRTfHW8/L/lgKus0sbs/SP+Q//BxvQAv4zvAAA \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/activity_work_0000-0003-2760-1191.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/activity_work_0000-0003-2760-1191.xml new file mode 100644 index 000000000..485f4f8e8 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/activity_work_0000-0003-2760-1191.xml @@ -0,0 +1,106 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + Abdel-Dayem K + + first + author + + + + Eweda II + + first + author + + + + El-Sherbiny A + + first + author + + + + Dimitry MO + + first + author + + + + Nammas W + + first + author + + + + diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/record_8888-8888-8888-8880.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/record_8888-8888-8888-8880.xml new file mode 100644 index 000000000..7abc2f35a --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/record_8888-8888-8888-8880.xml @@ -0,0 +1,770 @@ + + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + zh_CN + + + API + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + true + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + 2001-12-31T12:00:00 + true + true + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + give-names + family-name + credit-name + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + other-name-1 + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + biography + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + url-name-1 + http://url.com/ + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + user1@email.com + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + US + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + keyword1 + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + type-1 + value-1 + http://url.com/1 + self + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + distinction:department-name + distinction:role-title + + 1948 + 02 + 02 + + + 1948 + 02 + 02 + + + distinction-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-distinction + GRID + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + education:department-name + education:role-title + + 1948 + 02 + 02 + + + 2019 + 01 + 01 + + + education-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-education + GRID + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + employment:department-name + employment:role-title + + 1948 + 02 + 02 + + + 2025 + + + employment-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-employment + GRID + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + grant_number + external-id-value-1 + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + + common:title + common:translated-title + + + + grant_number + external-id-value-1 + http://tempuri.org + self + + + grant + + 1948 + 02 + 02 + + + 1948 + 02 + 02 + + + common:name + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-funding + FUNDREF + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + invited-position:department-name + invited-position:role-title + + 2019 + 01 + 01 + + + 2025 + 01 + 01 + + + invited-position-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-invited-position + GRID + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + membership:department-name + membership:role-title + + 1948 + 02 + 02 + + + membership-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-membership + RINGGOLD + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + something + external-id-value + http://orcid.org + self + + + + 2001-12-31T12:00:00 + + + something + external-id-value + + http://orcid.org + self + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/client/APP-9999999999999901 + APP-9999999999999901 + orcid.org + + + + reviewer + + + something + external-id-value + http://orcid.org + self + + + http://orcid.org + review + + 1948 + 02 + 02 + + orcid-generated:12345 + + common:name + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-peer-review + RINGGOLD + + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + qualification:department-name + qualification:role-title + + 1948 + 02 + 02 + + + 2025 + 12 + + + qualification-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-qualification + RINGGOLD + + + + + + + 2017-01-18T15:06:05.147-06:00 + + 2017-01-18T15:03:56.856-06:00 + + + proposal_id + 123456 + self + + + + + 2015-06-25T16:01:12.718Z + 2017-09-08T13:31:19.987Z + + + + https://orcid.org/0000-0000-0000-0000 + 0000-0000-0000-0000 + orcid.org + + XSEDE ORCID integration + + + + + Giant Laser Award + + + + XSEDE + + city + region + US + + + XX + grid + + + + + + proposal_id + 123456 + self + + + + 1999 + 02 + 02 + + + 2012 + 02 + 02 + + http://xsede.org/GiantLaserAward + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + + agr + external-id-value + + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + service:department-name + service:role-title + + 1948 + 02 + 02 + + + service-org + + common:city + common:region + AF + + + common:disambiguated-organization-identifier-service + RINGGOLD + + + + + + + 2001-12-31T12:00:00 + + 2001-12-31T12:00:00 + + + agr + external-id-value + http://orcid.org + part-of + + + + 2001-12-31T12:00:00 + 2001-12-31T12:00:00 + + + https://orcid.org/client/8888-8888-8888-8880 + 8888-8888-8888-8880 + orcid.org + + + + + common:title + + common:translated-title + + + + agr + external-id-value + http://tempuri.org + self + + + artistic-performance + + 1948 + 02 + 02 + + Procedia Computer Science + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/summary_0000-0001-5109-1000_othername.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/summary_0000-0001-5109-1000_othername.xml new file mode 100644 index 000000000..43bc96b8c --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/summary_0000-0001-5109-1000_othername.xml @@ -0,0 +1,196 @@ + + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + + en + + + Member-referred + 2019-05-01T13:04:57.507Z + 2019-05-01T13:59:54.268Z + true + true + true + + + 2019-05-01T13:45:47.727Z + + 2019-05-01T13:04:57.507Z + 2019-05-01T13:04:57.740Z + Andrew + Porteus + + + 2019-05-01T13:44:57.072Z + + 2019-05-01T13:44:57.072Z + 2019-05-01T13:44:57.072Z + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + Andrew Porteus + + Andrew C. Porteus + + + + 2019-05-01T13:59:54.263Z + 2019-05-01T13:59:54.263Z + Retired Librarian + + + 2019-05-01T13:45:47.727Z + + 2019-05-01T13:45:47.727Z + 2019-05-01T13:45:47.727Z + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + Andrew Porteus + + Niagara Falls Poetry Project + http://niagarapoetry.ca + + + + + 2019-05-01T13:45:09.764Z + + 2019-05-01T13:45:09.764Z + 2019-05-01T13:45:09.764Z + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + Andrew Porteus + + CA + + + + + + + 2019-05-01T13:57:45.787Z + + + 2019-05-01T13:15:26.102Z + + 2019-05-01T13:15:26.102Z + + + 2019-05-01T13:15:26.102Z + 2019-05-01T13:15:26.102Z + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + Andrew Porteus + + Library Technician Diploma + + 1976 + 09 + + + 1978 + 05 + + + Niagara College + + Welland + ON + CA + + + 125147 + RINGGOLD + + + + + + + + + + + + 2019-05-01T13:19:49.021Z + + 2019-05-01T13:19:49.021Z + + + 2019-05-01T13:19:49.021Z + 2019-05-01T13:19:49.021Z + + + https://orcid.org/0000-0001-5109-1000 + 0000-0001-5109-1000 + orcid.org + + Andrew Porteus + + Communication, Film & Popular Culture + Master's Candidate + + 2018 + 09 + + + Brock University + + Saint Catharines + ON + CA + + + 7497 + RINGGOLD + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/publication.json b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/publication.json new file mode 100644 index 000000000..579e12f2e --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/publication.json @@ -0,0 +1 @@ +{"oid":"0000-0002-4147-3387","id":"60153327","sourceName":"The Chinese University of Hong Kong","type":"conference-paper","titles":["Evaluation of a percutaneous optical fibre glucose sensor (FiberSense) across the glycemic range with rapid glucoseexcursions using the glucose clamp"],"extIds":[{"type":"wosuid","value":"000425015800225","relationShip":"self"},{"type":"other-id","value":"441f521e-ab19-448d-ba32-83157b348ada","relationShip":"self"}],"publicationDates":[],"contributors":[{"sequence":"1","oid":"0000-0002-4147-3387","name":"Elaine","surname":"Chow","creditName":"Elaine Chow"},{"sequence":"2","creditName":"Victor Tsui"},{"sequence":"3","creditName":"Achim Müller"},{"sequence":"4","creditName":"Vincy Lee"},{"sequence":"5","creditName":"Lucia Krivánekova"},{"sequence":"6","creditName":"Roland Krivánek"},{"sequence":"7","creditName":"Juliana CN Chan"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0002-2536-4498.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0002-2536-4498.xml new file mode 100644 index 000000000..43d3b2351 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0002-2536-4498.xml @@ -0,0 +1,72 @@ + + + 2019-10-22T03:18:13.755Z + 2020-06-17T11:07:13.703Z + + + https://orcid.org/client/0000-0001-8607-8906 + 0000-0001-8607-8906 + orcid.org + + INSPIRE-HEP + + + Measurement of the $t\bar{t}$ production cross-section and lepton differential distributions in $e\mu$ dilepton events from $pp$ collisions at $\sqrt{s}=13$ TeV with the ATLAS detector + + + + other-id + 1759875 + 1759875 + http://inspirehep.net/record/1759875 + self + + + doi + 10.1140/epjc/s10052-020-7907-9 + 10.1140/epjc/s10052-020-7907-9 + http://dx.doi.org/10.1140/epjc/s10052-020-7907-9 + self + + + arxiv + 1910.08819 + arXiv:1910.08819 + http://arxiv.org/abs/1910.08819 + self + + + http://inspirehep.net/record/1759875 + journal-article + + 2020 + 06 + 12 + + Eur.Phys.J.C + diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/activity_work_0000-0002-5982-8983.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0002-5982-8983.xml similarity index 100% rename from dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcid/xml/activity_work_0000-0002-5982-8983.xml rename to dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0002-5982-8983.xml diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191-similarity.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191-similarity.xml new file mode 100644 index 000000000..650d5a4cb --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191-similarity.xml @@ -0,0 +1,113 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + Abdel-Dayem K + + first + author + + + + Abdel-Dayem Fake + + first + author + + + + Eweda II + + first + author + + + + El-Sherbiny A + + first + author + + + + Dimitry MO + + first + author + + + + Nammas W + + first + author + + + + diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191.xml new file mode 100644 index 000000000..83752b145 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191.xml @@ -0,0 +1,106 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + Khair Abde Daye + + first + author + + + + Eweda II + + first + author + + + + El-Sherbiny A + + first + author + + + + Dimitry MO + + first + author + + + + Nammas W + + first + author + + + + diff --git a/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191_contributors.xml b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191_contributors.xml new file mode 100644 index 000000000..26e64aeda --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/test/resources/eu/dnetlib/doiboost/orcidnodoi/xml/activity_work_0000-0003-2760-1191_contributors.xml @@ -0,0 +1,101 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + + seq0 + role0 + + + + creditname1 + + + creditname2 + + seq2 + + + + + creditname3 + + + role3 + + + + + + seq4 + role4 + + + + diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 1cc41c395..692605b03 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -44,8 +44,6 @@ public class PropagationConstant { public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "authorpid:result"; public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of authors pid to result through semantic relations"; - public static final String PROPAGATION_AUTHOR_PID = "ORCID"; - public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String cfHbforResultQuery = "select distinct r.id, inst.collectedfrom.key cf, inst.hostedby.key hb " diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index 869831ba2..dd8342980 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; @@ -102,7 +103,8 @@ public class PrepareResultOrcidAssociationStep1 { + " FROM result " + " LATERAL VIEW EXPLODE (author) a AS MyT " + " LATERAL VIEW EXPLODE (MyT.pid) p AS MyP " - + " WHERE lower(MyP.qualifier.classid) = 'orcid') tmp " + + " WHERE lower(MyP.qualifier.classid) = '" + ModelConstants.ORCID + "' or " + + " lower(MyP.qalifier.classid) = '" + ModelConstants.ORCID_PENDING + "') tmp " + " GROUP BY id) r_t " + " JOIN (" + " SELECT source, target " diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java index 3fc127064..9f08fe580 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java @@ -23,6 +23,7 @@ import com.google.common.collect.Lists; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.PacePerson; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; @@ -176,7 +177,7 @@ public class SparkOrcidToResultFromSemRelJob { if (toaddpid) { StructuredProperty p = new StructuredProperty(); p.setValue(autoritative_author.getOrcid()); - p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID)); + p.setQualifier(getQualifier(ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME)); p .setDataInfo( getDataInfo( @@ -201,7 +202,8 @@ public class SparkOrcidToResultFromSemRelJob { return false; } for (StructuredProperty pid : pids.get()) { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + if (ModelConstants.ORCID_PENDING.equals(pid.getQualifier().getClassid().toLowerCase()) || + ModelConstants.ORCID.equals(pid.getQualifier().getClassid().toLowerCase())) { return true; } } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index edd2e7ba7..238375197 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -19,8 +19,11 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.cloudera.org.codehaus.jackson.map.jsontype.impl.ClassNameIdResolver; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.PropagationConstant; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Dataset; public class OrcidPropagationJobTest { @@ -166,7 +169,9 @@ public class OrcidPropagationJobTest { propagatedAuthors .filter( "id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " - + "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'") + + "and name = 'Vajinder' and surname = 'Kumar' and pidType = '" + + + ModelConstants.ORCID_PENDING + "'") .count()); Assertions.assertEquals(1, propagatedAuthors.filter("pid = '0000-0002-8825-3517'").count()); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java index 3d8ce6dcb..2a6fd3a1d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctions.java @@ -17,7 +17,7 @@ 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 NEWLINES = "(?:\\n|\\r)"; + public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)"; public static final Set PID_BLACKLIST = new HashSet<>(); @@ -111,7 +111,7 @@ public class CleaningFunctions { .filter(sp -> StringUtils.isNotBlank(sp.getValue())) .filter(sp -> Objects.nonNull(sp.getQualifier())) .filter(sp -> StringUtils.isNotBlank(sp.getQualifier().getClassid())) - .map(CleaningFunctions::removeNewLines) + .map(CleaningFunctions::cleanValue) .collect(Collectors.toList())); } if (Objects.nonNull(r.getTitle())) { @@ -122,7 +122,7 @@ public class CleaningFunctions { .stream() .filter(Objects::nonNull) .filter(sp -> StringUtils.isNotBlank(sp.getValue())) - .map(CleaningFunctions::removeNewLines) + .map(CleaningFunctions::cleanValue) .collect(Collectors.toList())); } if (Objects.nonNull(r.getDescription())) { @@ -133,7 +133,7 @@ public class CleaningFunctions { .stream() .filter(Objects::nonNull) .filter(sp -> StringUtils.isNotBlank(sp.getValue())) - .map(CleaningFunctions::removeNewLines) + .map(CleaningFunctions::cleanValue) .collect(Collectors.toList())); } if (Objects.nonNull(r.getPid())) { @@ -189,6 +189,16 @@ public class CleaningFunctions { author.setRank(i++); } } + + final Set collectedFrom = Optional + .ofNullable(r.getCollectedfrom()) + .map( + c -> c + .stream() + .map(KeyValue::getKey) + .collect(Collectors.toCollection(HashSet::new))) + .orElse(new HashSet<>()); + for (Author a : r.getAuthor()) { if (Objects.isNull(a.getPid())) { a.setPid(Lists.newArrayList()); @@ -201,13 +211,29 @@ public class CleaningFunctions { .filter(p -> Objects.nonNull(p.getQualifier())) .filter(p -> StringUtils.isNotBlank(p.getValue())) .map(p -> { + // hack to distinguish orcid from orcid_pending + String pidProvenance = Optional + .ofNullable(p.getDataInfo()) + .map( + d -> Optional + .ofNullable(d.getProvenanceaction()) + .map(Qualifier::getClassid) + .orElse("")) + .orElse(""); + if (pidProvenance.equals(ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY)) { + p.getQualifier().setClassid(ModelConstants.ORCID); + } else { + p.getQualifier().setClassid(ModelConstants.ORCID_PENDING); + } p.setValue(p.getValue().trim().replaceAll(ORCID_PREFIX_REGEX, "")); return p; }) .collect( Collectors .toMap( - StructuredProperty::getValue, Function.identity(), (p1, p2) -> p1, + p -> p.getQualifier().getClassid() + p.getValue(), + Function.identity(), + (p1, p2) -> p1, LinkedHashMap::new)) .values() .stream() @@ -230,13 +256,13 @@ public class CleaningFunctions { return value; } - protected static StructuredProperty removeNewLines(StructuredProperty s) { - s.setValue(s.getValue().replaceAll(NEWLINES, " ")); + protected static StructuredProperty cleanValue(StructuredProperty s) { + s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " ")); return s; } - protected static Field removeNewLines(Field s) { - s.setValue(s.getValue().replaceAll(NEWLINES, " ")); + protected static Field cleanValue(Field s) { + s.setValue(s.getValue().replaceAll(CLEANING_REGEX, " ")); return s; } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/community/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/community/oozie_app/workflow.xml index 161fd2dec..fcef2547a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/community/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/dump/community/oozie_app/workflow.xml @@ -26,8 +26,8 @@ the metadata associated to the deposition - newDeposition - true if it is a brand new depositon. false for new version of an old deposition + depositionType + one among {new, update, version} conceptRecordId @@ -419,6 +419,7 @@ --metadata${metadata} --communityMapPath${workingDir}/communityMap --conceptRecordId${conceptRecordId} + --depositionId${depositionId} --depositionType${depositionType} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 2d4cccdfb..46cb1a535 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -358,6 +358,23 @@ public class MappersTest { System.out.println(p.getTitle().get(0).getValue()); } + @Test + void testBologna() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf-bologna.xml")); + final List list = new OafToOafMapper(vocs, false).processMdRecord(xml); + + System.out.println("***************"); + System.out.println(new ObjectMapper().writeValueAsString(list)); + System.out.println("***************"); + + final Publication p = (Publication) list.get(0); + assertValidId(p.getId()); + assertValidId(p.getCollectedfrom().get(0).getKey()); + System.out.println(p.getTitle().get(0).getValue()); + assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); + System.out.println(p.getTitle().get(0).getValue()); + } + private void assertValidId(final String id) { assertEquals(49, id.length()); assertEquals('|', id.charAt(2)); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json index 5c903cd0e..e746d236e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json @@ -49,6 +49,28 @@ "schemename": "dnet:pid_types" }, "value": "https://orcid.org/0000-0001-9613-6639" + }, + { + "dataInfo": { + "deletedbyinference": false, + "inferenceprovenance": "", + "inferred": false, + "invisible": false, + "provenanceaction": { + "classid": "sysimport:crosswalk:entityregistry", + "classname": "sysimport:crosswalk:entityregistry", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + }, + "trust": "0.9" + }, + "qualifier": { + "classid": "orcid", + "classname": "ORCID12", + "schemeid": "dnet:pid_types", + "schemename": "dnet:pid_types" + }, + "value": "0000-0001-9613-6639" } ], "rank": 1, diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt index 93cc00eca..67c070d1d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt @@ -1031,6 +1031,7 @@ dnet:pid_types @=@ dnet:pid_types @=@ jprn @=@ JPRN Identifier dnet:pid_types @=@ dnet:pid_types @=@ mag_id @=@ Microsoft Academic Graph Identifier dnet:pid_types @=@ dnet:pid_types @=@ oai @=@ Open Archives Initiative dnet:pid_types @=@ dnet:pid_types @=@ orcid @=@ Open Researcher and Contributor ID +dnet:pid_types @=@ dnet:pid_types @=@ orcid_pending @=@ Open Researcher and Contributor ID dnet:pid_types @=@ dnet:pid_types @=@ PANGAEA @=@ PANGAEA dnet:pid_types @=@ dnet:pid_types @=@ epo_nr_epodoc @=@ Patent application number in EPODOC format dnet:pid_types @=@ dnet:pid_types @=@ UNKNOWN @=@ UNKNOWN diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf-bologna.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf-bologna.xml new file mode 100644 index 000000000..296a2fe94 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf-bologna.xml @@ -0,0 +1,115 @@ + + +
+ od_________3::4f7038e665fdd3800297735f087a530c + oai:amsacta.unibo.it:6382 + + + + + + 2020-05-21T05:26:15.93Z + 2020-08-01T11:06:26.977Z + od_________3 +
+ + CONSOLE Project - Deliverable 5.1 - "Guidelines for Community of Practice (CoP) + management at local level" + Blanco-Velázquez, Francisco José + Runge, Tania + Anaya-Romero, María + 2020-05-11 + The Community of Practice (CoP) is foreseen to play a key role in boosting + innovation in the effective and long-lasting delivery of agri-environmental-climate + public goods (AECPGs). The CONSOLE CoP will be organized around practitioners + experienced in the provision of AECPGs and those interested in it and will be nourished + throughout the project lifetime. In line with the definition of Wenger1 the CoP is + defined as a group of people (the community) who share a common interest and who learn + how to perform better through regular interaction and exchange of experiences. The idea + is to set up a pan-European CoP with national and/or local (regional) sub-groups managed + by the CONSOLE partners with the aim of developing improved and novel contract solutions + in collaboration with its members. This document sets out: (a) the purpose and + objectives of the CoP in CONSOLE, (b) the setting up and management of the CoP at + European, national and local level, (c) the process for motivating individuals to + participate. The CONSOLE CoP is intended to facilitate knowledge exchange and mutual + learning, mainly through virtual contacts. Participation in the CoP is based on sharing + and reciprocity principle. A core objective of these guidelines is to ensure a sound + management and facilitation of the CoP by all CONSOLE partners in view of optimizing the + input from CoP members in the project activities. Members within a national or local + CONSOLE CoP are: 1) CONSOLE partners; 2) practitioners, mainly farmers and foresters, + who test and implement practically the contractual models, and 3) experts, that may have + punctual interventions in the CoP. A vibrant CoP with active involvement of its members + is crucial for the assessment and testing of improved and novel voluntary measures for + the delivery of AECPGs. For each of the CONSOLE countries one national contact person is + nominated to take over the role as national focal point for the CoP activities in his + country and to serve as facilitator of the CoP. These facilitators are responsible to + ensure participation along the various project tasks foreseen within several WPs and to + overcome potential language barriers. The national contact person may be supported by + other CONSOLE partners from his country for local activities. At local level the CoP + benefits from existing contacts of CONSOLE partners to practitioners, including the + experts interviewed for the case studies analysis within WP2. The forming/development of + a CoP requires promoting exchanges taking on board the interests and needs of the actors + involved and to animate them to share their expertise. Collaborative learning within the + CoP supported by dedicated training activities will be crucial to ensure the intended + major transition towards smarter AECPGs-related practices in Europe. These guidelines + focus on the identification of the various tasks where CoP participation is foreseen at + local (regional) level and to provide support for the identification of potential + members. In the deliverable D5.2 “Guidelines for testing the solutions catalogue by CoP + and partners” further details about the involvement of the CoP will be + provided. + application/pdf + http://amsacta.unibo.it/6382/ + eng + info:eu-repo/grantAgreement/EC/H2020/817949/EU/CONtract SOLutions for Effective + and lasting delivery of agri-environmental-climate public goods by EU agriculture and + forestry/CONSOLE + info:eu-repo/semantics/altIdentifier/doi/10.6092/unibo/amsacta/6382 + info:eu-repo/semantics/reference/url/https://console-project.eu/ + Blanco-Velázquez, Francisco José ; Runge, Tania ; Anaya-Romero, María (2020) + CONSOLE Project - Deliverable 5.1 - "Guidelines for Community of Practice (CoP) + management at local level". p. 21. DOI 10.6092/unibo/amsacta/6382 + <http://doi.org/10.6092/unibo/amsacta/6382>. + AGR/01 Economia ed estimo rurale + SECS-S/01 Statistica + info:eu-repo/semantics/book + info:eu-repo/semantics/publishedVersion + 0002 + 2020-05-11 + corda__h2020::817949 + + opendoar____::3 + OPEN + + + 10.6092/unibo/amsacta/6382 + + + + + + http%3A%2F%2Famsacta.unibo.it%2Fcgi%2Fopenaire3 + oai:amsacta.unibo.it:6382 + 2020-05-13T09:27:00Z + http://www.openarchives.org/OAI/2.0/oai_dc/ + + + + false + false + 0.9 + + + + +
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/textgrid.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/textgrid.xml index d6970ab3e..eddbc1ec4 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/textgrid.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/textgrid.xml @@ -1,23 +1,24 @@ - + + xmlns:dri="http://www.driver-repository.eu/namespace/dri" + xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> r3f52792889d::000051aa1f61d77d2c0b340091f8024e textgrid:q9cv.0 2020-11-17T09:34:11.128+01:00 r3f52792889d textgrid:q9cv.0 2012-01-21T13:35:20Z - 2020-11-17T09:46:21.551+01:00 + 2020-11-17T19:08:56.703+01:00 + xmlns:dri="http://www.driver-repository.eu/namespace/dri" + xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> hdl:11858/00-1734-0000-0003-7664-F @@ -46,8 +47,8 @@ 2012-01-21T13:35:20Z - - textgrid:q9cv.0 + + textgrid:q9cv.0 http://hdl.handle.net/hdl:11858/00-1734-0000-0003-7664-F @@ -83,7 +84,7 @@ hdl:11858/00-1734-0000-0003-7664-F 0021 0002 - 2012-01-01 + 2012-01-21 OPEN http://creativecommons.org/licenses/by/3.0/de/legalcode und @@ -91,7 +92,8 @@ + xmlns:dri="http://www.driver-repository.eu/namespace/dri" + xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> https%3A%2F%2Fdev.textgridlab.org%2F1.0%2Ftgoaipmh%2Foai diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java index 95bea74a2..e19432f29 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/MakeTar.java @@ -12,6 +12,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.MakeTarArchive; public class MakeTar implements Serializable { @@ -41,71 +42,71 @@ public class MakeTar implements Serializable { 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 { - - RemoteIterator dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath)); - - while (dir_iterator.hasNext()) { - LocatedFileStatus fileStatus = dir_iterator.next(); - - Path p = fileStatus.getPath(); - String p_string = p.toString(); - String entity = p_string.substring(p_string.lastIndexOf("/") + 1); - - write(fileSystem, p_string, outputPath + "/" + entity + ".tar", entity); - } - - } - - private static void write(FileSystem fileSystem, String inputPath, String outputPath, String dir_name) - throws IOException { - - Path hdfsWritePath = new Path(outputPath); - FSDataOutputStream fsDataOutputStream = null; - if (fileSystem.exists(hdfsWritePath)) { - fileSystem.delete(hdfsWritePath, true); - - } - fsDataOutputStream = fileSystem.create(hdfsWritePath); - - TarArchiveOutputStream ar = new TarArchiveOutputStream(fsDataOutputStream.getWrappedStream()); - - RemoteIterator fileStatusListIterator = fileSystem - .listFiles( - new Path(inputPath), true); - - while (fileStatusListIterator.hasNext()) { - LocatedFileStatus fileStatus = fileStatusListIterator.next(); - - Path p = fileStatus.getPath(); - String p_string = p.toString(); - if (!p_string.endsWith("_SUCCESS")) { - String name = p_string.substring(p_string.lastIndexOf("/") + 1); - TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name + ".json.gz"); - entry.setSize(fileStatus.getLen()); - ar.putArchiveEntry(entry); - - InputStream is = fileSystem.open(fileStatus.getPath()); - - BufferedInputStream bis = new BufferedInputStream(is); - - int count; - byte data[] = new byte[1024]; - while ((count = bis.read(data, 0, data.length)) != -1) { - ar.write(data, 0, count); - } - bis.close(); - ar.closeArchiveEntry(); - - } - - } - - ar.close(); - } +// public static void makeTArArchive(FileSystem fileSystem, String inputPath, String outputPath) throws IOException { +// +// RemoteIterator dir_iterator = fileSystem.listLocatedStatus(new Path(inputPath)); +// +// while (dir_iterator.hasNext()) { +// LocatedFileStatus fileStatus = dir_iterator.next(); +// +// Path p = fileStatus.getPath(); +// String p_string = p.toString(); +// String entity = p_string.substring(p_string.lastIndexOf("/") + 1); +// +// write(fileSystem, p_string, outputPath + "/" + entity + ".tar", entity); +// } +// +// } +// +// private static void write(FileSystem fileSystem, String inputPath, String outputPath, String dir_name) +// throws IOException { +// +// Path hdfsWritePath = new Path(outputPath); +// FSDataOutputStream fsDataOutputStream = null; +// if (fileSystem.exists(hdfsWritePath)) { +// fileSystem.delete(hdfsWritePath, true); +// +// } +// fsDataOutputStream = fileSystem.create(hdfsWritePath); +// +// TarArchiveOutputStream ar = new TarArchiveOutputStream(fsDataOutputStream.getWrappedStream()); +// +// RemoteIterator fileStatusListIterator = fileSystem +// .listFiles( +// new Path(inputPath), true); +// +// while (fileStatusListIterator.hasNext()) { +// LocatedFileStatus fileStatus = fileStatusListIterator.next(); +// +// Path p = fileStatus.getPath(); +// String p_string = p.toString(); +// if (!p_string.endsWith("_SUCCESS")) { +// String name = p_string.substring(p_string.lastIndexOf("/") + 1); +// TarArchiveEntry entry = new TarArchiveEntry(dir_name + "/" + name + ".json.gz"); +// entry.setSize(fileStatus.getLen()); +// ar.putArchiveEntry(entry); +// +// InputStream is = fileSystem.open(fileStatus.getPath()); +// +// BufferedInputStream bis = new BufferedInputStream(is); +// +// int count; +// byte data[] = new byte[1024]; +// while ((count = bis.read(data, 0, data.length)) != -1) { +// ar.write(data, 0, count); +// } +// bis.close(); +// ar.closeArchiveEntry(); +// +// } +// +// } +// +// ar.close(); +// } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java index 1dcbf6ccc..2e2b7bc26 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/zenodo/SendToZenodoHDFS.java @@ -73,7 +73,7 @@ public class SendToZenodoHDFS implements Serializable { } zenodoApiClient.sendMretadata(metadata); - zenodoApiClient.publish(); +// zenodoApiClient.publish(); } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml index 6d7056503..fd8c773c9 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/resources/eu/dnetlib/sx/zenodo/oozie_app/workflow.xml @@ -8,10 +8,10 @@ targetPath the target path
- - metadata - the metadata - + + + + @@ -34,20 +34,20 @@
- - - eu.dnetlib.dhp.export.zenodo.SendToZenodoHDFS - --hdfsPath/user/dnet.scholexplorer/scholix/provision/scholix.tar/scholix-2020-10-16.tar - --nameNode${nameNode} - --accessTokenb6ddrY6b77WxcDEevn9gqVE5sL5sDNjdUijt75W3o7cQo5vpFFI48dMiu8Gv - --connectionUrlhttps://zenodo.org/api/deposit/depositions - --metadata${metadata} - --conceptRecordId1200252 - --newDepositionfalse - - - - + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java index 992ab26e8..e84f97836 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/XmlRecordFactoryTest.java @@ -5,23 +5,31 @@ import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.io.StringReader; +import java.util.List; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.DocumentException; import org.dom4j.io.SAXReader; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.mockito.Mock; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; //TODO to enable it we need to update the joined_entity.json test file -@Disabled +//@Disabled public class XmlRecordFactoryTest { private static final String otherDsTypeId = "scholarcomminfra,infospace,pubsrepository::mock,entityregistry,entityregistry::projects,entityregistry::repositories,websource"; @@ -35,6 +43,27 @@ public class XmlRecordFactoryTest { JoinedEntity je = new ObjectMapper().readValue(json, JoinedEntity.class); assertNotNull(je); + Document doc = buildXml(je); + //// TODO specific test assertion on doc + } + + @Test + void testBologna() throws IOException, DocumentException { + final String json = IOUtils.toString(getClass().getResourceAsStream("oaf-bologna.json")); + Publication oaf = new ObjectMapper().readValue(json, Publication.class); + assertNotNull(oaf); + JoinedEntity je = new JoinedEntity(); + je.setEntity(oaf); + assertNotNull(je); + + Document doc = buildXml(je); + // TODO specific test assertion on doc + + System.out.println(doc.asXML()); + + } + + private Document buildXml(JoinedEntity je) throws DocumentException { ContextMapper contextMapper = new ContextMapper(); XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, XmlConverterJob.schemaLocation, @@ -49,5 +78,7 @@ public class XmlRecordFactoryTest { assertNotNull(doc); // TODO add assertions based of values extracted from the XML record + + return doc; } } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/oaf-bologna.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/oaf-bologna.json new file mode 100644 index 000000000..3bb7d5b68 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/oaf-bologna.json @@ -0,0 +1,379 @@ + + { + "collectedfrom": [ + { + "key": "10|opendoar____::eccbc87e4b5ce2fe28308fd9f2a7baf3", + "value": "AMS Acta", + "dataInfo": null + } + ], + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + }, + "lastupdatetimestamp": 1606898557407, + "id": "50|od_________3::4f7038e665fdd3800297735f087a530c", + "originalId": [ + "oai:amsacta.unibo.it:6382" + ], + "pid": [ + { + "value": "10.6092/unibo/amsacta/6382", + "qualifier": { + "classid": "doi", + "classname": "doi", + "schemeid": "dnet:pid_types", + "schemename": "dnet:pid_types" + }, + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "dateofcollection": "", + "dateoftransformation": "2020-08-01T11:06:26.977Z", + "extraInfo": [], + "oaiprovenance": { + "originDescription": { + "harvestDate": "2020-05-21T05:26:15.93Z", + "altered": true, + "baseURL": "http%3A%2F%2Famsacta.unibo.it%2Fcgi%2Fopenaire3", + "identifier": "oai:amsacta.unibo.it:6382", + "datestamp": "2020-05-13T09:27:00Z", + "metadataNamespace": "http://www.openarchives.org/OAI/2.0/oai_dc/" + } + }, + "measures": null, + "author": [ + { + "fullname": "Blanco-Velázquez, Francisco José", + "name": "Francisco José", + "surname": "Blanco-Velázquez", + "rank": 1, + "pid": [], + "affiliation": null + }, + { + "fullname": "Runge, Tania", + "name": "Tania", + "surname": "Runge", + "rank": 2, + "pid": [], + "affiliation": null + }, + { + "fullname": "Anaya-Romero, María", + "name": "María", + "surname": "Anaya-Romero", + "rank": 3, + "pid": [], + "affiliation": null + } + ], + "resulttype": { + "classid": "publication", + "classname": "publication", + "schemeid": "dnet:result_typologies", + "schemename": "dnet:result_typologies" + }, + "language": { + "classid": "eng", + "classname": "English", + "schemeid": "dnet:languages", + "schemename": "dnet:languages" + }, + "country": [], + "subject": [ + { + "value": "AGR/01 Economia ed estimo rurale", + "qualifier": { + "classid": "keyword", + "classname": "keyword", + "schemeid": "dnet:result_subject", + "schemename": "dnet:result_subject" + }, + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + }, + { + "value": "SECS-S/01 Statistica", + "qualifier": { + "classid": "keyword", + "classname": "keyword", + "schemeid": "dnet:result_subject", + "schemename": "dnet:result_subject" + }, + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "title": [ + { + "value": "CONSOLE Project - Deliverable 5.1 - \"Guidelines for Community of Practice (CoP)\n management at local level\"", + "qualifier": { + "classid": "main title", + "classname": "main title", + "schemeid": "dnet:dataCite_title", + "schemename": "dnet:dataCite_title" + }, + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "relevantdate": [], + "description": [ + { + "value": "The Community of Practice (CoP) is foreseen to play a key role in boosting\n innovation in the effective and long-lasting delivery of agri-environmental-climate\n public goods (AECPGs). The CONSOLE CoP will be organized around practitioners\n experienced in the provision of AECPGs and those interested in it and will be nourished\n throughout the project lifetime. In line with the definition of Wenger1 the CoP is\n defined as a group of people (the community) who share a common interest and who learn\n how to perform better through regular interaction and exchange of experiences. The idea\n is to set up a pan-European CoP with national and/or local (regional) sub-groups managed\n by the CONSOLE partners with the aim of developing improved and novel contract solutions\n in collaboration with its members. This document sets out: (a) the purpose and\n objectives of the CoP in CONSOLE, (b) the setting up and management of the CoP at\n European, national and local level, (c) the process for motivating individuals to\n participate. The CONSOLE CoP is intended to facilitate knowledge exchange and mutual\n learning, mainly through virtual contacts. Participation in the CoP is based on sharing\n and reciprocity principle. A core objective of these guidelines is to ensure a sound\n management and facilitation of the CoP by all CONSOLE partners in view of optimizing the\n input from CoP members in the project activities. Members within a national or local\n CONSOLE CoP are: 1) CONSOLE partners; 2) practitioners, mainly farmers and foresters,\n who test and implement practically the contractual models, and 3) experts, that may have\n punctual interventions in the CoP. A vibrant CoP with active involvement of its members\n is crucial for the assessment and testing of improved and novel voluntary measures for\n the delivery of AECPGs. For each of the CONSOLE countries one national contact person is\n nominated to take over the role as national focal point for the CoP activities in his\n country and to serve as facilitator of the CoP. These facilitators are responsible to\n ensure participation along the various project tasks foreseen within several WPs and to\n overcome potential language barriers. The national contact person may be supported by\n other CONSOLE partners from his country for local activities. At local level the CoP\n benefits from existing contacts of CONSOLE partners to practitioners, including the\n experts interviewed for the case studies analysis within WP2. The forming/development of\n a CoP requires promoting exchanges taking on board the interests and needs of the actors\n involved and to animate them to share their expertise. Collaborative learning within the\n CoP supported by dedicated training activities will be crucial to ensure the intended\n major transition towards smarter AECPGs-related practices in Europe. These guidelines\n focus on the identification of the various tasks where CoP participation is foreseen at\n local (regional) level and to provide support for the identification of potential\n members. In the deliverable D5.2 “Guidelines for testing the solutions catalogue by CoP\n and partners” further details about the involvement of the CoP will be\n provided.", + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "dateofacceptance": { + "value": "2020-05-11", + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + }, + "publisher": null, + "embargoenddate": null, + "source": [ + { + "value": "Blanco-Velázquez, Francisco José ; Runge, Tania ; Anaya-Romero, María (2020)\n CONSOLE Project - Deliverable 5.1 - \"Guidelines for Community of Practice (CoP)\n management at local level\". p. 21. DOI 10.6092/unibo/amsacta/6382\n .", + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "fulltext": [], + "format": [ + { + "value": "application/pdf", + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + } + ], + "contributor": [], + "resourcetype": null, + "coverage": [], + "bestaccessright": { + "classid": "OPEN", + "classname": "Open Access", + "schemeid": "dnet:access_modes", + "schemename": "dnet:access_modes" + }, + "context": [], + "externalReference": [], + "instance": [ + { + "license": null, + "accessright": { + "classid": "OPEN", + "classname": "Open Access", + "schemeid": "dnet:access_modes", + "schemename": "dnet:access_modes" + }, + "instancetype": { + "classid": "0002", + "classname": "Book", + "schemeid": "dnet:publication_resource", + "schemename": "dnet:publication_resource" + }, + "hostedby": { + "key": "10|opendoar____::eccbc87e4b5ce2fe28308fd9f2a7baf3", + "value": "AMS Acta", + "dataInfo": null + }, + "url": [ + "http://amsacta.unibo.it/6382/" + ], + "distributionlocation": "", + "collectedfrom": { + "key": "10|opendoar____::eccbc87e4b5ce2fe28308fd9f2a7baf3", + "value": "AMS Acta", + "dataInfo": null + }, + "dateofacceptance": { + "value": "2020-05-11", + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + } + }, + "processingchargeamount": null, + "processingchargecurrency": null, + "refereed": { + "classid": "UNKNOWN", + "classname": "Unknown", + "schemeid": "dnet:review_levels", + "schemename": "dnet:review_levels" + } + } + ], + "journal": null + }, + { + "collectedfrom": [ + { + "key": "10|opendoar____::eccbc87e4b5ce2fe28308fd9f2a7baf3", + "value": "AMS Acta", + "dataInfo": null + } + ], + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + }, + "lastupdatetimestamp": 1606898557407, + "relType": "resultProject", + "subRelType": "outcome", + "relClass": "isProducedBy", + "source": "50|od_________3::4f7038e665fdd3800297735f087a530c", + "target": "40|corda__h2020::79a0e16c122c9a18eb60e4a5e64b620d", + "validated": null, + "validationDate": null, + "properties": [] + }, + { + "collectedfrom": [ + { + "key": "10|opendoar____::eccbc87e4b5ce2fe28308fd9f2a7baf3", + "value": "AMS Acta", + "dataInfo": null + } + ], + "dataInfo": { + "invisible": false, + "inferred": false, + "deletedbyinference": false, + "trust": "0.9", + "inferenceprovenance": "", + "provenanceaction": { + "classid": "sysimport:crosswalk:repository", + "classname": "sysimport:crosswalk:repository", + "schemeid": "dnet:provenanceActions", + "schemename": "dnet:provenanceActions" + } + }, + "lastupdatetimestamp": 1606898557407, + "relType": "resultProject", + "subRelType": "outcome", + "relClass": "produces", + "source": "40|corda__h2020::79a0e16c122c9a18eb60e4a5e64b620d", + "target": "50|od_________3::4f7038e665fdd3800297735f087a530c", + "validated": null, + "validationDate": null, + "properties": [] + } + diff --git a/pom.xml b/pom.xml index d0757f145..d06bdbe20 100644 --- a/pom.xml +++ b/pom.xml @@ -478,6 +478,18 @@ ${jsonschemagenerator.version} + + org.apache.commons + commons-text + ${common.text.version} + + + + org.apache.httpcomponents + httpclient + ${org.apache.httpcomponents.version} + + @@ -706,5 +718,8 @@ 4.13.0 1.8 4.1.2 + 1.8 + 4.3.4 + 4.0.1