diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index a75cc52e6..e5181b111 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -98,7 +98,7 @@ public class GraphCleaningFunctions extends CleaningFunctions { Result r = (Result) value; - if (Objects.nonNull(r.getTitle()) && r.getTitle().isEmpty()) { + if (Objects.isNull(r.getTitle()) || r.getTitle().isEmpty()) { return false; } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala index 0cdf0accb..045927bed 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/datacite/DataciteToOAFTransformation.scala @@ -532,11 +532,11 @@ object DataciteToOAFTransformation { JField("awardUri", JString(awardUri)) <- fundingReferences } yield awardUri + result.setId(IdentifierFactory.createIdentifier(result)) var relations: List[Relation] = awardUris.flatMap(a => get_projectRelation(a, result.getId)).filter(r => r != null) - fix_figshare(result) - result.setId(IdentifierFactory.createIdentifier(result)) + if (result.getId == null) return List() diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml index 3e0c2bba6..69fc95957 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/datacite/oozie_app/workflow.xml @@ -4,6 +4,10 @@ mainPath the working path of Datacite stores + + oafTargetPath + the target path where the OAF records are stored + isLookupUrl The IS lookUp service endopoint @@ -56,7 +60,7 @@ --masteryarn-cluster --blocksize${blocksize} - + @@ -78,7 +82,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${mainPath}/datacite_dump - --targetPath${mainPath}/datacite_oaf + --targetPath${oafTargetPath} --isLookupUrl${isLookupUrl} --exportLinks${exportLinks} --masteryarn-cluster 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 57acaf404..c65916610 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 @@ -21,7 +21,7 @@ object SparkMapDumpIntoOAF { val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass) val conf: SparkConf = new SparkConf() - val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json"))) + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_crossref_dump_to_oaf_params.json"))) parser.parseArgument(args) val spark: SparkSession = SparkSession diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkConvertORCIDToOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkConvertORCIDToOAF.scala index a359eb3c6..fa4a93e00 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkConvertORCIDToOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkConvertORCIDToOAF.scala @@ -1,52 +1,19 @@ package eu.dnetlib.doiboost.orcid -import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.oa.merge.AuthorMerger import eu.dnetlib.dhp.schema.oaf.Publication -import eu.dnetlib.dhp.schema.orcid.OrcidDOI import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import org.apache.spark.sql._ import org.slf4j.{Logger, LoggerFactory} object SparkConvertORCIDToOAF { val logger: Logger = LoggerFactory.getLogger(SparkConvertORCIDToOAF.getClass) - def fixORCIDItem(item :ORCIDItem):ORCIDItem = { - new ORCIDItem(item.doi, item.authors.groupBy(_.oid).map(_._2.head).toList) - } - - - def run(spark:SparkSession,sourcePath:String,workingPath:String, targetPath:String):Unit = { - import spark.implicits._ + def run(spark:SparkSession, workingPath:String, targetPath:String) :Unit = { implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication] - - val inputRDD:RDD[OrcidAuthor] = spark.sparkContext.textFile(s"$sourcePath/authors").map(s => ORCIDToOAF.convertORCIDAuthor(s)).filter(s => s!= null).filter(s => ORCIDToOAF.authorValid(s)) - - spark.createDataset(inputRDD).as[OrcidAuthor].write.mode(SaveMode.Overwrite).save(s"$workingPath/author") - - val res = spark.sparkContext.textFile(s"$sourcePath/works").flatMap(s => ORCIDToOAF.extractDOIWorks(s)).filter(s => s!= null) - - spark.createDataset(res).as[OrcidWork].write.mode(SaveMode.Overwrite).save(s"$workingPath/works") - - val authors :Dataset[OrcidAuthor] = spark.read.load(s"$workingPath/author").as[OrcidAuthor] - - val works :Dataset[OrcidWork] = spark.read.load(s"$workingPath/works").as[OrcidWork] - - works.joinWith(authors, authors("oid").equalTo(works("oid"))) - .map(i =>{ - val doi = i._1.doi - var author = i._2 - (doi, author) - }).groupBy(col("_1").alias("doi")) - .agg(collect_list(col("_2")).alias("authors")).as[ORCIDItem] - .map(s => fixORCIDItem(s)) - .write.mode(SaveMode.Overwrite).save(s"$workingPath/orcidworksWithAuthor") - + import spark.implicits._ val dataset: Dataset[ORCIDItem] =spark.read.load(s"$workingPath/orcidworksWithAuthor").as[ORCIDItem] logger.info("Converting ORCID to OAF") @@ -55,7 +22,7 @@ object SparkConvertORCIDToOAF { def main(args: Array[String]): Unit = { val conf: SparkConf = new SparkConf() - val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json"))) + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_orcid_to_oaf_params.json"))) parser.parseArgument(args) val spark: SparkSession = SparkSession @@ -65,10 +32,10 @@ object SparkConvertORCIDToOAF { .master(parser.get("master")).getOrCreate() - val sourcePath = parser.get("sourcePath") val workingPath = parser.get("workingPath") val targetPath = parser.get("targetPath") - run(spark, sourcePath, workingPath, targetPath) + + run(spark,workingPath, targetPath) } diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPreprocessORCID.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPreprocessORCID.scala new file mode 100644 index 000000000..31f331912 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/orcid/SparkPreprocessORCID.scala @@ -0,0 +1,70 @@ +package eu.dnetlib.doiboost.orcid + +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.oa.merge.AuthorMerger +import eu.dnetlib.dhp.schema.oaf.Publication +import eu.dnetlib.dhp.schema.orcid.OrcidDOI +import org.apache.commons.io.IOUtils +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +object SparkPreprocessORCID { + val logger: Logger = LoggerFactory.getLogger(SparkConvertORCIDToOAF.getClass) + + def fixORCIDItem(item :ORCIDItem):ORCIDItem = { + ORCIDItem(item.doi, item.authors.groupBy(_.oid).map(_._2.head).toList) + + } + + + def run(spark:SparkSession,sourcePath:String,workingPath:String):Unit = { + import spark.implicits._ + implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication] + + val inputRDD:RDD[OrcidAuthor] = spark.sparkContext.textFile(s"$sourcePath/authors").map(s => ORCIDToOAF.convertORCIDAuthor(s)).filter(s => s!= null).filter(s => ORCIDToOAF.authorValid(s)) + + spark.createDataset(inputRDD).as[OrcidAuthor].write.mode(SaveMode.Overwrite).save(s"$workingPath/author") + + val res = spark.sparkContext.textFile(s"$sourcePath/works").flatMap(s => ORCIDToOAF.extractDOIWorks(s)).filter(s => s!= null) + + spark.createDataset(res).as[OrcidWork].write.mode(SaveMode.Overwrite).save(s"$workingPath/works") + + val authors :Dataset[OrcidAuthor] = spark.read.load(s"$workingPath/author").as[OrcidAuthor] + + val works :Dataset[OrcidWork] = spark.read.load(s"$workingPath/works").as[OrcidWork] + + works.joinWith(authors, authors("oid").equalTo(works("oid"))) + .map(i =>{ + val doi = i._1.doi + val author = i._2 + (doi, author) + }).groupBy(col("_1").alias("doi")) + .agg(collect_list(col("_2")).alias("authors")).as[ORCIDItem] + .map(s => fixORCIDItem(s)) + .write.mode(SaveMode.Overwrite).save(s"$workingPath/orcidworksWithAuthor") + } + + def main(args: Array[String]): Unit = { + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/preprocess_orcid_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + + val sourcePath = parser.get("sourcePath") + val workingPath = parser.get("workingPath") + + run(spark, sourcePath, workingPath) + + } + +} \ No newline at end of file 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 index 9f8727d30..1d47808ef 100644 --- 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 @@ -4,6 +4,7 @@ package eu.dnetlib.doiboost.orcidnodoi; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -32,10 +33,7 @@ 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.dhp.schema.orcid.AuthorSummary; -import eu.dnetlib.dhp.schema.orcid.Work; -import eu.dnetlib.dhp.schema.orcid.WorkDetail; +import eu.dnetlib.dhp.schema.orcid.*; import eu.dnetlib.doiboost.orcid.json.JsonHelper; import eu.dnetlib.doiboost.orcid.util.HDFSUtil; import eu.dnetlib.doiboost.orcidnodoi.oaf.PublicationToOaf; @@ -111,6 +109,10 @@ public class SparkGenEnrichedOrcidWorks { Encoders.bean(WorkDetail.class)); logger.info("Works data loaded: " + workDataset.count()); + final LongAccumulator warnNotFoundContributors = spark + .sparkContext() + .longAccumulator("warnNotFoundContributors"); + JavaRDD> enrichedWorksRDD = workDataset .joinWith( authorDataset, @@ -119,7 +121,21 @@ public class SparkGenEnrichedOrcidWorks { (MapFunction, Tuple2>) value -> { WorkDetail w = value._1; AuthorData a = value._2; - AuthorMatcher.match(a, w.getContributors()); + if (w.getContributors() == null + || (w.getContributors() != null && w.getContributors().size() == 0)) { + Contributor c = new Contributor(); + c.setName(a.getName()); + c.setSurname(a.getSurname()); + c.setCreditName(a.getCreditName()); + c.setOid(a.getOid()); + List contributors = Arrays.asList(c); + w.setContributors(contributors); + if (warnNotFoundContributors != null) { + warnNotFoundContributors.add(1); + } + } else { + AuthorMatcher.match(a, w.getContributors()); + } return new Tuple2<>(a.getOid(), JsonHelper.createOidWork(w)); }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())) @@ -172,7 +188,7 @@ public class SparkGenEnrichedOrcidWorks { OBJECT_MAPPER.writeValueAsString(new AtomicAction<>(Publication.class, p)))) .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2()))) .saveAsNewAPIHadoopFile( - workingPath.concat(outputEnrichedWorksPath), + outputEnrichedWorksPath, Text.class, Text.class, SequenceFileOutputFormat.class, @@ -180,6 +196,7 @@ public class SparkGenEnrichedOrcidWorks { logger.info("parsedPublications: " + parsedPublications.value().toString()); logger.info("enrichedPublications: " + enrichedPublications.value().toString()); + logger.info("warnNotFoundContributors: " + warnNotFoundContributors.value().toString()); logger.info("errorsGeneric: " + errorsGeneric.value().toString()); logger.info("errorsInvalidTitle: " + errorsInvalidTitle.value().toString()); logger.info("errorsNotFoundAuthors: " + errorsNotFoundAuthors.value().toString()); diff --git a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/SparkMapUnpayWallToOAF.scala b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/SparkMapUnpayWallToOAF.scala index a72e4b0d6..4530926f1 100644 --- a/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/SparkMapUnpayWallToOAF.scala +++ b/dhp-workflows/dhp-doiboost/src/main/java/eu/dnetlib/doiboost/uw/SparkMapUnpayWallToOAF.scala @@ -18,7 +18,7 @@ object SparkMapUnpayWallToOAF { val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass) val conf: SparkConf = new SparkConf() - val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json"))) + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_uw_to_oaf_params.json"))) parser.parseArgument(args) val spark: SparkSession = SparkSession diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_crossref_dump_to_oaf_params.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_crossref_dump_to_oaf_params.json new file mode 100644 index 000000000..da324f8c4 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_crossref_dump_to_oaf_params.json @@ -0,0 +1,6 @@ +[ + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source path ", "paramRequired": false}, + {"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true} + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_orcid_to_oaf_params.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_orcid_to_oaf_params.json new file mode 100644 index 000000000..6c9ca5ede --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_orcid_to_oaf_params.json @@ -0,0 +1,6 @@ +[ + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": false}, + {"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true} + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_uw_to_oaf_params.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_uw_to_oaf_params.json new file mode 100644 index 000000000..da324f8c4 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_uw_to_oaf_params.json @@ -0,0 +1,6 @@ +[ + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source path ", "paramRequired": false}, + {"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "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 deleted file mode 100644 index 63c2e9ef2..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/workflow.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - workingPath - the working dir base path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - timestamp - Timestamp for incremental Harvesting - - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - ${jobTracker} - ${nameNode} - eu.dnetlib.doiboost.crossref.CrossrefImporter - -t${workingPath}/input/crossref/index_update - -n${nameNode} - -ts${timestamp} - - - - - - - - yarn-cluster - cluster - ExtractCrossrefToOAF - eu.dnetlib.doiboost.crossref.CrossrefDataset - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --workingPath/data/doiboost/input/crossref - --masteryarn-cluster - - - - - - - - - - - - - - - - - - yarn-cluster - cluster - ConvertCrossrefToOAF - eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --sourcePath${workingPath}/input/crossref/crossref_ds - --targetPath${workingPath}/process/ - --masteryarn-cluster - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/intersection/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/intersection/oozie_app/config-default.xml deleted file mode 100644 index cf617a84c..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/intersection/oozie_app/config-default.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - - hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 - - - spark2YarnHistoryServerAddress - http://iis-cdh5-test-gw.ocean.icm.edu.pl: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/intersection/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/intersection/oozie_app/workflow.xml deleted file mode 100644 index dcde62c9d..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/intersection/oozie_app/workflow.xml +++ /dev/null @@ -1,96 +0,0 @@ - - - - hostedByMapPath - the Hosted By Map Path - - - affiliationPath - the Affliation Path - - - paperAffiliationPath - the paperAffiliation Path - - - workingDirPath - the Working Path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - yarn-cluster - cluster - Create DOIBoost Infospace - eu.dnetlib.doiboost.SparkGenerateDoiBoost - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --hostedByMapPath${hostedByMapPath} - --affiliationPath${affiliationPath} - --paperAffiliationPath${paperAffiliationPath} - --workingDirPath${workingDirPath} - --masteryarn-cluster - - - - - - - - - yarn-cluster - cluster - Generate DOIBoost ActionSet - eu.dnetlib.doiboost.SparkGenerateDOIBoostActionSet - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --dbPublicationPath${workingDirPath}/doiBoostPublicationFiltered - --dbDatasetPath${workingDirPath}/crossrefDataset - --crossRefRelation${workingDirPath}/crossrefRelation - --dbaffiliationRelationPath${workingDirPath}/doiBoostPublicationAffiliation - -do${workingDirPath}/doiBoostOrganization - --targetPath${workingDirPath}/actionDataSet - --masteryarn-cluster - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/mag/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/mag/oozie_app/config-default.xml deleted file mode 100644 index 59e5c059f..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/mag/oozie_app/config-default.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - - oozie.wf.rerun.failnodes - false - - - hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 - - - spark2YarnHistoryServerAddress - http://iis-cdh5-test-gw.ocean.icm.edu.pl: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/mag/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/mag/oozie_app/workflow.xml deleted file mode 100644 index 9d19dddc7..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/mag/oozie_app/workflow.xml +++ /dev/null @@ -1,92 +0,0 @@ - - - - sourcePath - the working dir base path - - - targetPath - the working dir base path - - - workingPath - the working dir base path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - - yarn-cluster - cluster - Convert Mag to Dataset - eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - ${sparkExtraOPT} - - --sourcePath${sourcePath} - --targetPath${workingPath} - --masteryarn-cluster - - - - - - - - - - yarn-cluster - cluster - Convert Mag to OAF Dataset - eu.dnetlib.doiboost.mag.SparkPreProcessMAG - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --sourcePath${workingPath} - --workingPath${workingPath}/process - --targetPath${targetPath} - --masteryarn-cluster - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_oaf/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_oaf/oozie_app/workflow.xml index bffde793b..0670e18de 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_oaf/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/orcid_oaf/oozie_app/workflow.xml @@ -34,7 +34,7 @@ yarn-cluster cluster Convert ORCID to Dataset - eu.dnetlib.doiboost.orcid.SparkConvertORCIDToOAF + eu.dnetlib.doiboost.orcid.SparkPreprocessORCID dhp-doiboost-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} 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 index 365c4d5b4..04ca05af2 100644 --- 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 @@ -7,9 +7,14 @@ outputPath + /data/orcid_activities_2020/no_doi_dataset_prod/ path where to store the action set - + + processOutputPath + /data/orcid_activities_2020/process_no_doi_dataset_prod + temporary path where to store the action set + spark2GenNoDoiDatasetMaxExecutors 40 @@ -66,7 +71,7 @@ - + @@ -92,7 +97,7 @@ --workingPath${workingPath}/ --hdfsServerUri${nameNode} --orcidDataFolderlast_orcid_dataset - --outputEnrichedWorksPathno_doi_dataset + --outputEnrichedWorksPath${processOutputPath} @@ -100,7 +105,7 @@ - ${workingPath}/no_doi_dataset/* + ${processOutputPath}/* ${outputPath} diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/oozie_app/config-default.xml rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml new file mode 100644 index 000000000..03f7b7566 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml @@ -0,0 +1,216 @@ + + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + + + inputPathCrossref + the Crossref input path + + + crossrefDumpPath + the Crossref dump path + + + + + MAGDumpPath + the MAG dump working path + + + + inputPathMAG + the MAG working path + + + + + + inputPathOrcid + the ORCID input path + + + + workingPathOrcid + the ORCID working path + + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + + ${wf:conf('resumeFrom') eq 'UnpackCrossrefEntries'} + ${wf:conf('resumeFrom') eq 'GenerateCrossrefDataset'} + ${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'} + ${wf:conf('resumeFrom') eq 'ConvertMagToDataset'} + ${wf:conf('resumeFrom') eq 'PreProcessORCID'} + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${jobTracker} + ${nameNode} + eu.dnetlib.doiboost.crossref.ExtractCrossrefRecords + --hdfsServerUri${nameNode} + --crossrefFileNameTarGz${crossrefDumpPath}/crossref.tar.gz + --workingPath${crossrefDumpPath} + --outputPath${crossrefDumpPath}/files/ + + + + + + + + yarn-cluster + cluster + SparkUnpackCrossrefEntries + eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn-cluster + --sourcePath${crossrefDumpPath}/files + --targetPath${crossrefDumpPath}/crossref_unpack/ + + + + + + + + + yarn-cluster + cluster + SparkGenerateCrossrefDataset + eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset + dhp-doiboost-${projectVersion}.jar + + --executor-memory=7G + --executor-cores=2 + --driver-memory=7G + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn-cluster + --sourcePath${crossrefDumpPath}/crossref_unpack/ + --targetPath${inputPathCrossref}/crossref_ds + + + + + + + + + + + + + + + + + + + + + + + + + + + + yarn-cluster + cluster + Convert Mag to Dataset + eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${MAGDumpPath} + --targetPath${inputPathMAG}/dataset + --masteryarn-cluster + + + + + + + + + yarn-cluster + cluster + Convert ORCID to Dataset + eu.dnetlib.doiboost.orcid.SparkPreprocessORCID + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${inputPathOrcid} + --workingPath${workingPathOrcid} + --masteryarn-cluster + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess_orcid_params.json similarity index 59% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess_orcid_params.json index 152103570..fdc1e2f20 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess_orcid_params.json @@ -1,7 +1,6 @@ [ {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the Orcid Input file", "paramRequired": true}, {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": false}, - {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the working dir path", "paramRequired": true}, - {"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true} + {"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/oozie_app/config-default.xml rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml similarity index 55% rename from dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/oozie_app/workflow.xml rename to dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml index fa47e142d..f845d97f3 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + sparkDriverMemory @@ -17,8 +17,6 @@ sparkExecutorCores number of cores used by single executor - - workingPath @@ -40,29 +38,8 @@ inputPathCrossref the Crossref input path - - crossrefDumpPath - the Crossref dump path - - - - - - - - - - - - - - - MAGDumpPath - the MAG dump working path - - inputPathMAG the MAG working path @@ -76,11 +53,6 @@ - - inputPathOrcid - the ORCID input path - - workingPathOrcid the ORCID working path @@ -103,15 +75,12 @@ - ${wf:conf('resumeFrom') eq 'ConvertCrossrefToOAF'} - ${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'} ${wf:conf('resumeFrom') eq 'PreprocessMag'} ${wf:conf('resumeFrom') eq 'PreprocessUW'} - ${wf:conf('resumeFrom') eq 'PreprocessORCID'} + ${wf:conf('resumeFrom') eq 'ProcessORCID'} ${wf:conf('resumeFrom') eq 'CreateDOIBoost'} ${wf:conf('resumeFrom') eq 'GenerateActionSet'} - ${wf:conf('resumeFrom') eq 'GenerateCrossrefDataset'} - + @@ -119,170 +88,6 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - eu.dnetlib.doiboost.crossref.ExtractCrossrefRecords - --hdfsServerUri${nameNode} - --crossrefFileNameTarGz${crossrefDumpPath}/crossref.tar.gz - --workingPath${crossrefDumpPath} - --outputPath${crossrefDumpPath}/files/ - - - - - - - - yarn-cluster - cluster - SparkUnpackCrossrefEntries - eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --masteryarn-cluster - --sourcePath${crossrefDumpPath}/files - --targetPath${crossrefDumpPath}/crossref_unpack/ - - - - - - - - - yarn-cluster - cluster - SparkGenerateCrossrefDataset - eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset - dhp-doiboost-${projectVersion}.jar - - --executor-memory=7G - --executor-cores=2 - --driver-memory=7G - --conf spark.sql.shuffle.partitions=3840 - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --masteryarn-cluster - --sourcePath${crossrefDumpPath}/crossref_unpack/ - --targetPath${inputPathCrossref}/crossref_ds - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - yarn-cluster - cluster - Convert Mag to Dataset - eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${MAGDumpPath} - --targetPath${inputPathMAG}/dataset - --masteryarn-cluster - - - - - - yarn-cluster @@ -326,7 +131,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${inputPathMAG}/dataset - --workingPath${inputPathMAG}/process + --workingPath${inputPathMAG}/process_p --targetPath${workingPath} --masteryarn-cluster @@ -380,7 +185,6 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePath${inputPathOrcid} --workingPath${workingPathOrcid} --targetPath${workingPath}/orcidPublication --masteryarn-cluster diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/unpaywall/oozie_app/config-default.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/unpaywall/oozie_app/config-default.xml deleted file mode 100644 index cf617a84c..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/unpaywall/oozie_app/config-default.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - - hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 - - - spark2YarnHistoryServerAddress - http://iis-cdh5-test-gw.ocean.icm.edu.pl: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/unpaywall/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/unpaywall/oozie_app/workflow.xml deleted file mode 100644 index d2a69752e..000000000 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/unpaywall/oozie_app/workflow.xml +++ /dev/null @@ -1,55 +0,0 @@ - - - - sourcePath - the working dir base path - - - targetPath - the working dir base path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - yarn-cluster - cluster - Convert UnpayWall to Dataset - eu.dnetlib.doiboost.uw.SparkMapUnpayWallToOAF - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - ${sparkExtraOPT} - - --sourcePath${sourcePath}/uw_extracted - --targetPath${targetPath} - --masteryarn-cluster - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala index d43e7ed37..9a08b703a 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/mag/MAGMappingTest.scala @@ -1,22 +1,15 @@ package eu.dnetlib.doiboost.mag -import java.sql.Timestamp - -import eu.dnetlib.dhp.schema.oaf.Publication -import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.api.java.function.MapFunction -import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} -import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig} -import org.junit.jupiter.api.Test -import org.slf4j.{Logger, LoggerFactory} +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Dataset, SparkSession} +import org.codehaus.jackson.map.ObjectMapper import org.junit.jupiter.api.Assertions._ -import org.apache.spark.sql.functions._ +import org.junit.jupiter.api.Test +import org.json4s.DefaultFormats +import org.slf4j.{Logger, LoggerFactory} -import scala.collection.JavaConverters._ +import java.sql.Timestamp import scala.io.Source -import scala.reflect.ClassTag -import scala.util.matching.Regex @@ -65,13 +58,13 @@ class MAGMappingTest { @Test def normalizeDoiTest():Unit = { - import org.json4s.jackson.Serialization.write - import org.json4s.DefaultFormats + implicit val formats = DefaultFormats - - val conf = new SparkConf().setAppName("test").setMaster("local[*]").set("spark.driver.host", "localhost") + val conf = new SparkConf() + conf.setMaster("local[*]") + conf.set("spark.driver.host", "localhost") val spark: SparkSession = SparkSession @@ -96,20 +89,21 @@ class MAGMappingTest { @Test def normalizeDoiTest2():Unit = { - import org.json4s.jackson.Serialization.write import org.json4s.DefaultFormats implicit val formats = DefaultFormats - val conf = new SparkConf().setAppName("test").setMaster("local[*]").set("spark.driver.host", "localhost") + val conf = new SparkConf() + conf.setMaster("local[*]") + conf.set("spark.driver.host", "localhost") + val spark: SparkSession = + SparkSession + .builder() + .appName(getClass.getSimpleName) + .config(conf) + .getOrCreate() - val spark: SparkSession = - SparkSession - .builder() - .appName(getClass.getSimpleName) - .config(conf) - .getOrCreate() val path = getClass.getResource("duplicatedMagPapers.json").getPath import org.apache.spark.sql.Encoders diff --git a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/MappingORCIDToOAFTest.scala b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/MappingORCIDToOAFTest.scala index d620c4ef3..b484dc087 100644 --- a/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/MappingORCIDToOAFTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/java/eu/dnetlib/doiboost/orcid/MappingORCIDToOAFTest.scala @@ -46,7 +46,9 @@ class MappingORCIDToOAFTest { implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication] import spark.implicits._ - SparkConvertORCIDToOAF.run( spark,sourcePath, workingPath, targetPath) + SparkPreprocessORCID.run( spark,sourcePath, workingPath) + + SparkConvertORCIDToOAF.run(spark, workingPath,targetPath) val mapper = new ObjectMapper() @@ -62,6 +64,8 @@ class MappingORCIDToOAFTest { println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p.first())) spark.close() + + } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 1e80dfd46..03c3eeb3c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -71,6 +71,8 @@ public abstract class AbstractMdRecordToOafMapper { private final boolean shouldHashId; + private final boolean forceOriginalId; + protected static final String DATACITE_SCHEMA_KERNEL_4 = "http://datacite.org/schema/kernel-4"; protected static final String DATACITE_SCHEMA_KERNEL_4_SLASH = "http://datacite.org/schema/kernel-4/"; protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3"; @@ -98,11 +100,20 @@ public abstract class AbstractMdRecordToOafMapper { nsContext.put("datacite", DATACITE_SCHEMA_KERNEL_3); } + protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible, + final boolean shouldHashId, final boolean forceOriginalId) { + this.vocs = vocs; + this.invisible = invisible; + this.shouldHashId = shouldHashId; + this.forceOriginalId = forceOriginalId; + } + protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId) { this.vocs = vocs; this.invisible = invisible; this.shouldHashId = shouldHashId; + this.forceOriginalId = false; } public List processMdRecord(final String xml) { @@ -190,10 +201,16 @@ public abstract class AbstractMdRecordToOafMapper { final long lastUpdateTimestamp) { final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp); - final String id = IdentifierFactory.createIdentifier(entity, shouldHashId); - if (!id.equals(entity.getId())) { - entity.getOriginalId().add(entity.getId()); - entity.setId(id); + + final Set originalId = Sets.newHashSet(entity.getOriginalId()); + originalId.add(entity.getId()); + entity.setOriginalId(Lists.newArrayList(originalId)); + + if (!forceOriginalId) { + final String id = IdentifierFactory.createIdentifier(entity, shouldHashId); + if (!id.equals(entity.getId())) { + entity.setId(id); + } } final List oafs = Lists.newArrayList(entity); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index fcd6f459a..bbfb7429f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -163,11 +163,13 @@ public class GenerateEntitiesApplication { switch (type.toLowerCase()) { case "oaf-store-cleaned": - case "oaf-store-claim": return new OafToOafMapper(vocs, false, shouldHashId).processMdRecord(s); + case "oaf-store-claim": + return new OafToOafMapper(vocs, false, shouldHashId, true).processMdRecord(s); case "odf-store-cleaned": - case "odf-store-claim": return new OdfToOafMapper(vocs, false, shouldHashId).processMdRecord(s); + case "odf-store-claim": + return new OdfToOafMapper(vocs, false, shouldHashId, true).processMdRecord(s); case "oaf-store-intersection": return new OafToOafMapper(vocs, true, shouldHashId).processMdRecord(s); case "odf-store-intersection": diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 06aeab345..d753cddeb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -27,6 +27,11 @@ import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; public class OafToOafMapper extends AbstractMdRecordToOafMapper { + public OafToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId, + final boolean forceOrginalId) { + super(vocs, invisible, shouldHashId, forceOrginalId); + } + public OafToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId) { super(vocs, invisible, shouldHashId); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index b7400873b..7925a7826 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -22,6 +22,11 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { public static final String HTTP_DX_DOI_PREIFX = "http://dx.doi.org/"; + public OdfToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId, + final boolean forceOrginalId) { + super(vocs, invisible, shouldHashId, forceOrginalId); + } + public OdfToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId) { super(vocs, invisible, shouldHashId); } 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 5b229a625..495dc4a04 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 @@ -567,6 +567,31 @@ public class MappersTest { assertNotNull(d.getInstance().get(0).getUrl()); } + @Test + void testEnermaps() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("enermaps.xml")); + final List list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml); + + System.out.println("***************"); + System.out.println(new ObjectMapper().writeValueAsString(list)); + System.out.println("***************"); + + assertEquals(1, list.size()); + assertTrue(list.get(0) instanceof Dataset); + + final Dataset d = (Dataset) list.get(0); + + assertValidId(d.getId()); + assertValidId(d.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); + assertEquals(1, d.getAuthor().size()); + assertEquals(1, d.getInstance().size()); + assertNotNull(d.getInstance().get(0).getUrl()); + assertNotNull(d.getContext()); + assertTrue(StringUtils.isNotBlank(d.getContext().get(0).getId())); + assertEquals("enermaps::selection::tgs00004", d.getContext().get(0).getId()); + } + @Test void testClaimFromCrossref() throws IOException { final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_crossref.xml")); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/enermaps.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/enermaps.xml new file mode 100644 index 000000000..362b40c85 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/enermaps.xml @@ -0,0 +1,72 @@ + + + + enermaps____::04149ee428d07360314c2cb3ba95d41e + tgs00004 + 2021-07-20T18:43:12.096+02:00 + enermaps____ + + + + https://ec.europa.eu/eurostat/web/products-datasets/-/tgs00004 + + + Statistical Office of the European Union (Eurostat) + + + + + Regional GDP + + + Statistical Office of the European Union (Eurostat) + 2020 + + 2020-10-07 + + + + OPEN + Creative Commons Attribution 4.0 International + + + GDP expressed in PPS (purchasing power standards) eliminates differences in price levels between countries. Calculations on a per inhabitant basis allow for the comparison of economies and regions significantly different in absolute size. GDP per inhabitant in PPS is the key variable for determining the eligibility of NUTS 2 regions in the framework of the European Unions structural policy. + + 0021 + 2020-10-07 + OPEN + Creative Commons Attribution 4.0 International + + + + + + + + + https%3A%2F%2Flab.idiap.ch%2Fenermaps%2Fapi%2Fdatacite + + + + + + + false + false + 0.9 + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 86bbae99e..a985d2371 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -16,6 +16,7 @@ import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import org.apache.commons.lang3.StringUtils; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; @@ -183,6 +184,7 @@ public class XmlRecordFactory implements Serializable { .getOriginalId() .stream() .filter(Objects::nonNull) + .filter(id -> !id.matches("^\\d{2}" + IdentifierFactory.ID_PREFIX_SEPARATOR)) .map(s -> XmlSerializationUtils.asXmlElement("originalId", s)) .collect(Collectors.toList())); } 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 6631cb4da..a5a1563aa 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 @@ -7,6 +7,8 @@ import java.io.IOException; import java.io.StringReader; import java.util.List; +import eu.dnetlib.dhp.oa.provision.utils.ContextDef; +import eu.dnetlib.dhp.schema.oaf.Dataset; import org.apache.commons.io.IOUtils; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -131,4 +133,31 @@ public class XmlRecordFactoryTest { System.out.println(doc.asXML()); assertEquals("", doc.valueOf("//rel/validated")); } + + @Test + public void testEnermapsRecord() throws IOException, DocumentException { + + String contextmap = "" + + ""+ + ""+ + ""; + + ContextMapper contextMapper = ContextMapper.fromXml(contextmap); + XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false, XmlConverterJob.schemaLocation, + otherDsTypeId); + + Dataset d = OBJECT_MAPPER + .readValue(IOUtils.toString(getClass().getResourceAsStream("enermaps.json")), Dataset.class); + + JoinedEntity je = new JoinedEntity<>(d); + + String xml = xmlRecordFactory.build(je); + + assertNotNull(xml); + + Document doc = new SAXReader().read(new StringReader(xml)); + assertNotNull(doc); + System.out.println(doc.asXML()); + assertEquals("enermaps::selection::tgs00004", doc.valueOf("//concept/@id")); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/enermaps.json b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/enermaps.json new file mode 100644 index 000000000..dcd4c2ee1 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/enermaps.json @@ -0,0 +1 @@ +{"collectedfrom":[{"key":"10|enermaps____::d77d5e503ad1439f585ac494268b351b","value":"Enermaps","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1626800904248,"id":"50|enermaps____::04149ee428d07360314c2cb3ba95d41e","originalId":["50|enermaps____::04149ee428d07360314c2cb3ba95d41e","tgs00004"],"pid":[],"dateofcollection":"2021-07-20T18:43:12.096+02:00","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2021-07-20T18:43:12.096+02:00","altered":true,"baseURL":"https%3A%2F%2Flab.idiap.ch%2Fenermaps%2Fapi%2Fdatacite","identifier":"","datestamp":"","metadataNamespace":""}},"measures":null,"author":[{"fullname":"Statistical Office of the European Union (Eurostat)","name":"","surname":"","rank":1,"pid":[],"affiliation":[]}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[],"title":[{"value":"\n Regional GDP\n ","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","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[{"value":"2020-10-07","qualifier":{"classid":"Issued","classname":"Issued","schemeid":"dnet:dataCite_date","schemename":"dnet:dataCite_date"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"description":[{"value":"GDP expressed in PPS (purchasing power standards) eliminates differences in price levels between countries. Calculations on a per inhabitant basis allow for the comparison of economies and regions significantly different in absolute size. GDP per inhabitant in PPS is the key variable for determining the eligibility of NUTS 2 regions in the framework of the European Unions structural policy.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2020-10-07","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Statistical Office of the European Union (Eurostat)","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"coverage":[],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"context":[{"id":"enermaps::selection::tgs00004","dataInfo":[{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}]}],"externalReference":[],"instance":[{"license":{"value":"Creative Commons Attribution 4.0 International","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes","openAccessRoute":null},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"10|openaire____::55045bd2a65019fd8e6741a755395c8c","value":"Unknown Repository","dataInfo":null},"url":["https://ec.europa.eu/eurostat/web/products-datasets/-/tgs00004"],"distributionlocation":null,"collectedfrom":{"key":"10|enermaps____::d77d5e503ad1439f585ac494268b351b","value":"Enermaps","dataInfo":null},"pid":[],"alternateIdentifier":[],"dateofacceptance":{"value":"2020-10-07","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"}}],"storagedate":{"value":"2020-10-07","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk","classname":"sysimport:crosswalk","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]}