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 42e19628c..37f5573d0 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 @@ -230,6 +230,15 @@ public class GraphCleaningFunctions extends CleaningFunctions { .map(GraphCleaningFunctions::cleanValue) .collect(Collectors.toList())); } + if (Objects.nonNull(r.getFormat())) { + r + .setFormat( + r + .getFormat() + .stream() + .map(GraphCleaningFunctions::cleanValue) + .collect(Collectors.toList())); + } if (Objects.nonNull(r.getDescription())) { r .setDescription( diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala b/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala similarity index 83% rename from dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala rename to dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala index bf81a26d4..f35af0905 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala +++ b/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala @@ -12,7 +12,7 @@ import org.json4s.jackson.JsonMethods.parse import scala.collection.JavaConverters._ import scala.io.Source -object ScholixUtils { +object ScholixUtils extends Serializable { val DNET_IDENTIFIER_SCHEMA: String = "DNET Identifier" @@ -24,7 +24,7 @@ object ScholixUtils { case class RelatedEntities(id: String, relatedDataset: Long, relatedPublication: Long) {} val relations: Map[String, RelationVocabulary] = { - val input = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/relations.json")).mkString + val input = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/scholexplorer/relation/relations.json")).mkString implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats lazy val json: json4s.JValue = parse(input) @@ -53,8 +53,6 @@ object ScholixUtils { else { summary.getDate.get(0) } - - } def inverseRelationShip(rel: ScholixRelationship): ScholixRelationship = { @@ -64,11 +62,15 @@ object ScholixUtils { } + def generateScholixResourceFromResult(r:Result) :ScholixResource = { + generateScholixResourceFromSummary(ScholixUtils.resultToSummary(r)) + } + + val statsAggregator: Aggregator[(String, String, Long), RelatedEntities, RelatedEntities] = new Aggregator[(String, String, Long), RelatedEntities, RelatedEntities] with Serializable { override def zero: RelatedEntities = null override def reduce(b: RelatedEntities, a: (String, String, Long)): RelatedEntities = { - val id = a._1 val relatedDataset = if ("dataset".equalsIgnoreCase(a._2)) a._3 else 0 val relatedPublication = if ("publication".equalsIgnoreCase(a._2)) a._3 else 0 @@ -142,6 +144,14 @@ object ScholixUtils { } + def extractCollectedFrom(summary: ScholixResource): List[ScholixEntityId] = { + if (summary.getCollectedFrom != null && !summary.getCollectedFrom.isEmpty) { + val l: List[ScholixEntityId] = summary.getCollectedFrom.asScala.map { + d => new ScholixEntityId(d.getProvider.getName, d.getProvider.getIdentifiers) + }(collection.breakOut) + l + } else List() + } def extractCollectedFrom(summary: ScholixSummary): List[ScholixEntityId] = { if (summary.getDatasources != null && !summary.getDatasources.isEmpty) { @@ -160,7 +170,7 @@ object ScholixUtils { c => new ScholixEntityId(c.getValue, List(new ScholixIdentifier(c.getKey, DNET_IDENTIFIER_SCHEMA, null)).asJava) - }(collection breakOut) + }.toList l } else List() } @@ -179,6 +189,19 @@ object ScholixUtils { } + def generateCompleteScholix(scholix: Scholix, target: ScholixResource): Scholix = { + val s = new Scholix + s.setPublicationDate(scholix.getPublicationDate) + s.setPublisher(scholix.getPublisher) + s.setLinkprovider(scholix.getLinkprovider) + s.setRelationship(scholix.getRelationship) + s.setSource(scholix.getSource) + s.setTarget(target) + s.setIdentifier(DHPUtils.md5(s"${s.getSource.getIdentifier}::${s.getRelationship.getName}::${s.getTarget.getIdentifier}")) + s + } + + def generateScholixResourceFromSummary(summaryObject: ScholixSummary): ScholixResource = { val r = new ScholixResource r.setIdentifier(summaryObject.getLocalIdentifier) @@ -222,6 +245,38 @@ object ScholixUtils { } + + def scholixFromSource(relation: Relation, source: ScholixResource):Scholix = { + if (relation == null || source == null) + return null + val s = new Scholix + var l: List[ScholixEntityId] = extractCollectedFrom(relation) + if (l.isEmpty) + l = extractCollectedFrom(source) + if (l.isEmpty) + return null + s.setLinkprovider(l.asJava) + var d = extractRelationDate(relation) + if (d == null) + d = source.getPublicationDate + + s.setPublicationDate(d) + + + if (source.getPublisher != null && !source.getPublisher.isEmpty) { + s.setPublisher(source.getPublisher) + } + + val semanticRelation = relations.getOrElse(relation.getRelClass.toLowerCase, null) + if (semanticRelation == null) + return null + s.setRelationship(new ScholixRelationship(semanticRelation.original, "datacite", semanticRelation.inverse)) + s.setSource(source) + + s + } + + def scholixFromSource(relation: Relation, source: ScholixSummary): Scholix = { if (relation == null || source == null) @@ -303,7 +358,7 @@ object ScholixUtils { s.setSubType(r.getInstance().get(0).getInstancetype.getClassname) if (r.getTitle != null && r.getTitle.asScala.nonEmpty) { - val titles: List[String] = r.getTitle.asScala.map(t => t.getValue)(collection breakOut) + val titles: List[String] = r.getTitle.asScala.map(t => t.getValue).toList if (titles.nonEmpty) s.setTitle(titles.asJava) else @@ -311,12 +366,12 @@ object ScholixUtils { } if (r.getAuthor != null && !r.getAuthor.isEmpty) { - val authors: List[String] = r.getAuthor.asScala.map(a => a.getFullname)(collection breakOut) - if (authors nonEmpty) + val authors: List[String] = r.getAuthor.asScala.map(a => a.getFullname).toList + if (authors.nonEmpty) s.setAuthor(authors.asJava) } if (r.getInstance() != null) { - val dt: List[String] = r.getInstance().asScala.filter(i => i.getDateofacceptance != null).map(i => i.getDateofacceptance.getValue)(collection.breakOut) + val dt: List[String] = r.getInstance().asScala.filter(i => i.getDateofacceptance != null).map(i => i.getDateofacceptance.getValue).toList if (dt.nonEmpty) s.setDate(dt.distinct.asJava) } @@ -327,7 +382,7 @@ object ScholixUtils { } if (r.getSubject != null && !r.getSubject.isEmpty) { - val subjects: List[SchemeValue] = r.getSubject.asScala.map(s => new SchemeValue(s.getQualifier.getClassname, s.getValue))(collection breakOut) + val subjects: List[SchemeValue] = r.getSubject.asScala.map(s => new SchemeValue(s.getQualifier.getClassname, s.getValue)).toList if (subjects.nonEmpty) s.setSubject(subjects.asJava) } @@ -336,7 +391,7 @@ object ScholixUtils { s.setPublisher(List(r.getPublisher.getValue).asJava) if (r.getCollectedfrom != null && !r.getCollectedfrom.isEmpty) { - val cf: List[CollectedFromType] = r.getCollectedfrom.asScala.map(c => new CollectedFromType(c.getValue, c.getKey, "complete"))(collection breakOut) + val cf: List[CollectedFromType] = r.getCollectedfrom.asScala.map(c => new CollectedFromType(c.getValue, c.getKey, "complete")).toList if (cf.nonEmpty) s.setDatasources(cf.distinct.asJava) } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/config-default.xml new file mode 100644 index 000000000..bdd48b0ab --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/config-default.xml @@ -0,0 +1,19 @@ + + + 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-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/workflow.xml new file mode 100644 index 000000000..751b124cf --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/datacite/oozie_app/workflow.xml @@ -0,0 +1,62 @@ + + + + sourcePath + the source path of scholix graph + + + datacitePath + the datacite native path + + + workingSupportPath + the working Support path + + + isLookupUrl + The IS lookUp service endopoint + + + updateDS + false + The transformation Rule to apply + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + New Update from Datacite to Scholix + eu.dnetlib.dhp.sx.graph.SparkRetrieveDataciteDelta + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=6000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --datacitePath${datacitePath} + --masteryarn + --workingSupportPath${workingSupportPath} + --isLookupUrl${isLookupUrl} + --updateDS${updateDS} + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/retrieve_datacite_delta_params.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/retrieve_datacite_delta_params.json new file mode 100644 index 000000000..78777ffff --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/sx/graph/retrieve_datacite_delta_params.json @@ -0,0 +1,41 @@ +[ + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the source mdstore path", + "paramRequired": true + }, + + { + "paramName": "d", + "paramLongName": "datacitePath", + "paramDescription": "the datacite native path", + "paramRequired": true + }, + + { + "paramName": "w", + "paramLongName": "workingSupportPath", + "paramDescription": "the working Support path", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "isLookupUrl", + "paramDescription": "the isLookup URL", + "paramRequired": true + }, + { + "paramName": "m", + "paramLongName": "master", + "paramDescription": "the master name", + "paramRequired": true + }, + { + "paramName": "u", + "paramLongName": "updateDS", + "paramDescription": "Need to regenerate all support Dataset", + "paramRequired": false + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkRetrieveDataciteDelta.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkRetrieveDataciteDelta.scala new file mode 100644 index 000000000..45a6cfc89 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkRetrieveDataciteDelta.scala @@ -0,0 +1,305 @@ +package eu.dnetlib.dhp.sx.graph + +import eu.dnetlib.dhp.application.AbstractScalaApplication +import eu.dnetlib.dhp.collection.CollectionUtils.fixRelations +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup +import eu.dnetlib.dhp.datacite.{DataciteToOAFTransformation, DataciteType} +import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result} +import eu.dnetlib.dhp.schema.sx.scholix.{Scholix, ScholixResource} +import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary +import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils +import eu.dnetlib.dhp.utils.{DHPUtils, ISLookupClientFactory} +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.sql.functions.max +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +import scala.collection.JavaConverters._ +import java.text.SimpleDateFormat + +class SparkRetrieveDataciteDelta (propertyPath:String, args:Array[String], log:Logger) extends AbstractScalaApplication(propertyPath, args, log:Logger) { + + val ISO_DATE_PATTERN = "yyyy-MM-dd'T'HH:mm:ssZ" + val simpleFormatter = new SimpleDateFormat(ISO_DATE_PATTERN) + + val SCHOLIX_RESOURCE_PATH_NAME = "scholixResource" + val DATACITE_OAF_PATH_NAME = "dataciteOAFUpdate" + val PID_MAP_PATH_NAME = "pidMap" + val RESOLVED_REL_PATH_NAME ="resolvedRelation" + val SCHOLIX_PATH_NAME = "scholix" + + + def scholixResourcePath(workingPath:String) = s"$workingPath/$SCHOLIX_RESOURCE_PATH_NAME" + def dataciteOAFPath(workingPath:String) = s"$workingPath/$DATACITE_OAF_PATH_NAME" + def pidMapPath(workingPath:String) = s"$workingPath/$PID_MAP_PATH_NAME" + def resolvedRelationPath(workingPath:String) = s"$workingPath/$RESOLVED_REL_PATH_NAME" + def scholixPath(workingPath:String) = s"$workingPath/$SCHOLIX_PATH_NAME" + + + /** + * Utility to parse Date in ISO8601 to epochMillis + * @param inputDate The String represents an input date in ISO8601 + * @return The relative epochMillis of parsed date + */ + def ISO8601toEpochMillis(inputDate:String):Long = { + simpleFormatter.parse(inputDate).getTime + } + + + /** + * This method tries to retrieve the last collection date from all datacite + * records in HDFS. + * This method should be called before indexing scholexplorer to retrieve + * the delta of Datacite record to download, since from the generation of + * raw graph to the generation of Scholexplorer sometimes it takes 20 days + * @param spark + * @param entitiesPath + * @return the last collection date from the current scholexplorer Graph of the datacite records + */ + def retrieveLastCollectedFrom(spark:SparkSession, entitiesPath:String):Long = { + log.info("Retrieve last entities collected From") + + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val resultEncoder:Encoder[Result] = Encoders.kryo[Result] + import spark.implicits._ + + val entitiesDS = spark.read.load(s"$entitiesPath/*").as[Oaf].filter(o =>o.isInstanceOf[Result]).map(r => r.asInstanceOf[Result]) + + val date = entitiesDS.filter(r => r.getDateofcollection!= null).map(_.getDateofcollection).select(max("value")).first.getString(0) + + ISO8601toEpochMillis(date) / 1000 + } + + + /** + * The method of update Datacite relationships on Scholexplorer + * needs some utilities data structures + * One is the scholixResource DS that stores all the nodes in the Scholix Graph + * in format ScholixResource + * @param summaryPath the path of the summary in Scholix + * @param workingPath the working path + * @param spark the spark session + */ + def generateScholixResource(summaryPath:String, workingPath: String, spark:SparkSession) :Unit = { + implicit val summaryEncoder:Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary] + implicit val scholixResourceEncoder:Encoder[ScholixResource] = Encoders.kryo[ScholixResource] + + log.info("Convert All summary to ScholixResource") + spark.read.load(summaryPath).as[ScholixSummary] + .map(ScholixUtils.generateScholixResourceFromSummary)(scholixResourceEncoder) + .filter(r => r.getIdentifier!= null && r.getIdentifier.size>0) + .write.mode(SaveMode.Overwrite).save(s"${scholixResourcePath(workingPath)}_native") + } + + /** + * This method convert the new Datacite Resource into Scholix Resource + * Needed to fill the source and the type of Scholix Relationships + * @param workingPath the Working Path + * @param spark The spark Session + */ + def addMissingScholixResource(workingPath:String, spark:SparkSession ) :Unit = { + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val scholixResourceEncoder:Encoder[ScholixResource] = Encoders.kryo[ScholixResource] + implicit val resultEncoder:Encoder[Result] = Encoders.kryo[Result] + import spark.implicits._ + + spark.read.load(dataciteOAFPath(workingPath)).as[Oaf] + .filter(_.isInstanceOf[Result]) + .map(_.asInstanceOf[Result]) + .map(ScholixUtils.generateScholixResourceFromResult) + .filter(r => r.getIdentifier!= null && r.getIdentifier.size>0) + .write.mode(SaveMode.Overwrite).save(s"${scholixResourcePath(workingPath)}_update") + + val update = spark.read.load(s"${scholixResourcePath(workingPath)}_update").as[ScholixResource] + val native = spark.read.load(s"${scholixResourcePath(workingPath)}_native").as[ScholixResource] + val graph = update.union(native) + .groupByKey(_.getDnetIdentifier) + .reduceGroups((a,b) => if (a!= null && a.getDnetIdentifier!= null) a else b) + .map(_._2) + graph.write.mode(SaveMode.Overwrite).save(s"${scholixResourcePath(workingPath)}_graph") + } + + + /** + * This method get and Transform only datacite records with + * timestamp greater than timestamp + * @param datacitePath the datacite input Path + * @param timestamp the timestamp + * @param workingPath the working path where save the generated Dataset + * @param spark SparkSession + * @param vocabularies Vocabularies needed for transformation + */ + + def getDataciteUpdate(datacitePath:String, timestamp:Long, workingPath:String, spark:SparkSession,vocabularies: VocabularyGroup): Long = { + import spark.implicits._ + val ds = spark.read.load(datacitePath).as[DataciteType] + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + val total = ds.filter(_.timestamp>=timestamp).count() + if (total >0) { + ds.filter(_.timestamp >= timestamp) + .flatMap(d => DataciteToOAFTransformation.generateOAF(d.json, d.timestamp, d.timestamp, vocabularies, exportLinks = true)) + .flatMap(i => fixRelations(i)).filter(i => i != null) + .write.mode(SaveMode.Overwrite).save(dataciteOAFPath(workingPath)) + } + total + } + + /** + * After added the new ScholixResource, we need to update the scholix Pid Map + * to intersected with the new Datacite Relations + + * @param workingPath The working Path starting from save the new Map + * @param spark the spark session + */ + def generatePidMap(workingPath:String, spark:SparkSession ) :Unit = { + implicit val scholixResourceEncoder:Encoder[ScholixResource] = Encoders.kryo[ScholixResource] + import spark.implicits._ + spark.read.load(s"${scholixResourcePath(workingPath)}_graph").as[ScholixResource] + .flatMap(r=> + r.getIdentifier.asScala + .map(i =>DHPUtils.generateUnresolvedIdentifier(i.getIdentifier, i.getSchema)) + .map(t =>(t, r.getDnetIdentifier)) + )(Encoders.tuple(Encoders.STRING, Encoders.STRING)) + .groupByKey(_._1) + .reduceGroups((a,b) => if (a!= null && a._2!= null) a else b) + .map(_._2)(Encoders.tuple(Encoders.STRING, Encoders.STRING)) + .write.mode(SaveMode.Overwrite).save(pidMapPath(workingPath)) + } + + /** + * This method resolve the datacite relation and filter the resolved + * relation + * @param workingPath the working path + * @param spark the spark session + */ + + def resolveUpdateRelation(workingPath:String, spark:SparkSession) :Unit = { + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val relationEncoder:Encoder[Relation] = Encoders.kryo[Relation] + import spark.implicits._ + + val pidMap = spark.read.load(pidMapPath(workingPath)).as[(String,String)] + + val unresolvedRelations:Dataset[(String,Relation)] = spark.read.load(dataciteOAFPath(workingPath)).as[Oaf] + .filter(_.isInstanceOf[Relation]) + .map(_.asInstanceOf[Relation]) + .map { r => + if (r.getSource.startsWith("unresolved")) + (r.getSource, r) + else + (r.getTarget, r) + }(Encoders.tuple(Encoders.STRING, relationEncoder)) + + unresolvedRelations + .joinWith(pidMap, unresolvedRelations("_1").equalTo(pidMap("_1"))) + .map(t => { + val r =t._1._2 + val resolvedIdentifier = t._2._2 + if (r.getSource.startsWith("unresolved")) + r.setSource(resolvedIdentifier) + else + r.setTarget(resolvedIdentifier) + r + })(relationEncoder) + .filter(r => !(r.getSource.startsWith("unresolved") || r.getTarget.startsWith("unresolved") )) + .write.mode(SaveMode.Overwrite) + .save(resolvedRelationPath(workingPath)) + } + + + + /** + * This method generate scholix starting from resolved relation + * + * + * @param workingPath + * @param spark + */ + def generateScholixUpdate(workingPath:String, spark:SparkSession) :Unit = { + implicit val oafEncoder:Encoder[Oaf] = Encoders.kryo[Oaf] + implicit val scholixEncoder:Encoder[Scholix] = Encoders.kryo[Scholix] + implicit val scholixResourceEncoder:Encoder[ScholixResource] = Encoders.kryo[ScholixResource] + implicit val relationEncoder:Encoder[Relation] = Encoders.kryo[Relation] + implicit val intermediateEncoder :Encoder[(String,Scholix)] = Encoders.tuple(Encoders.STRING, scholixEncoder) + + + val relations:Dataset[(String, Relation)] = spark.read.load(resolvedRelationPath(workingPath)).as[Relation].map(r =>(r.getSource,r))(Encoders.tuple(Encoders.STRING, relationEncoder)) + + val id_summary:Dataset[(String,ScholixResource)] = spark.read.load(s"${scholixResourcePath(workingPath)}_graph").as[ScholixResource].map(r => (r.getDnetIdentifier,r))(Encoders.tuple(Encoders.STRING, scholixResourceEncoder)) + + id_summary.cache() + + relations.joinWith(id_summary, relations("_1").equalTo(id_summary("_1")),"inner") + .map(t => (t._1._2.getTarget,ScholixUtils.scholixFromSource(t._1._2, t._2._2))) + .write.mode(SaveMode.Overwrite).save(s"$workingPath/scholix_one_verse") + + val source_scholix:Dataset[(String, Scholix)] =spark.read.load(s"$workingPath/scholix_one_verse").as[(String,Scholix)] + + source_scholix.joinWith(id_summary, source_scholix("_1").equalTo(id_summary("_1")),"inner") + .map(t => { + val target:ScholixResource =t._2._2 + val scholix:Scholix = t._1._2 + ScholixUtils.generateCompleteScholix(scholix,target) + })(scholixEncoder).write.mode(SaveMode.Overwrite).save(s"$workingPath/scholix") + } + + + + + + /** + * Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + override def run(): Unit = { + val sourcePath = parser.get("sourcePath") + log.info(s"SourcePath is '$sourcePath'") + + val datacitePath = parser.get("datacitePath") + log.info(s"DatacitePath is '$datacitePath'") + + val workingPath = parser.get("workingSupportPath") + log.info(s"workingPath is '$workingPath'") + + val isLookupUrl: String = parser.get("isLookupUrl") + log.info("isLookupUrl: {}", isLookupUrl) + + val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl) + val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService) + require(vocabularies != null) + + + val updateDS:Boolean = "true".equalsIgnoreCase(parser.get("updateDS")) + log.info(s"updateDS is '$updateDS'") + + var lastCollectionDate = 0L + if (updateDS) { + generateScholixResource(s"$sourcePath/provision/summaries", workingPath, spark) + log.info("Retrieve last entities collected From starting from scholix Graph") + lastCollectionDate = retrieveLastCollectedFrom(spark, s"$sourcePath/entities") + } + else { + val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) + fs.delete(new Path(s"${scholixResourcePath(workingPath)}_native"), true) + fs.rename(new Path(s"${scholixResourcePath(workingPath)}_graph"), new Path(s"${scholixResourcePath(workingPath)}_native")) + lastCollectionDate = retrieveLastCollectedFrom(spark, dataciteOAFPath(workingPath)) + } + + val numRecords = getDataciteUpdate(datacitePath, lastCollectionDate, workingPath, spark, vocabularies) + if (numRecords>0) { + addMissingScholixResource(workingPath,spark) + generatePidMap(workingPath, spark) + resolveUpdateRelation(workingPath,spark) + generateScholixUpdate(workingPath, spark) + } + } +} + + +object SparkRetrieveDataciteDelta { + val log: Logger = LoggerFactory.getLogger(SparkRetrieveDataciteDelta.getClass) + + def main(args: Array[String]): Unit = { + new SparkRetrieveDataciteDelta("/eu/dnetlib/dhp/sx/graph/retrieve_datacite_delta_params.json", args, log).initialize().run() + } +} diff --git a/dhp-workflows/dhp-enrichment/pom.xml b/dhp-workflows/dhp-enrichment/pom.xml index 644ac2140..0b4269acd 100644 --- a/dhp-workflows/dhp-enrichment/pom.xml +++ b/dhp-workflows/dhp-enrichment/pom.xml @@ -48,6 +48,12 @@ io.github.classgraph classgraph + + eu.dnetlib.dhp + dhp-aggregation + 1.2.4-SNAPSHOT + compile + diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/KeyValueSet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/KeyValueSet.java new file mode 100644 index 000000000..57ab716b3 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/KeyValueSet.java @@ -0,0 +1,26 @@ + +package eu.dnetlib.dhp; + +import java.io.Serializable; +import java.util.ArrayList; + +public class KeyValueSet implements Serializable { + private String key; + private ArrayList valueSet; + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public ArrayList getValueSet() { + return valueSet; + } + + public void setValueSet(ArrayList valueSet) { + this.valueSet = valueSet; + } +} 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 392a5ab44..02fdcb09b 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 @@ -1,6 +1,7 @@ package eu.dnetlib.dhp; +import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -55,6 +56,9 @@ public class PropagationConstant { public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID = "result:organization:semrel"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME = "Propagation of affiliation to result through sematic relations"; + public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; @@ -67,6 +71,13 @@ 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 ITERATION_ONE = "ExitAtFirstIteration"; + public static final String ITERATION_TWO = "ExitAtSecondIteration"; + public static final String ITERATION_THREE = "ExitAtThirdIteration"; + public static final String ITERATION_FOUR = "ExitAtFourthIteration"; + public static final String ITERATION_FIVE = "ExitAtFifthIteration"; + public static final String ITERATION_NO_PARENT = "ExitAtNoFirstParentReached"; + 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 " @@ -127,6 +138,39 @@ public class PropagationConstant { return pa; } + public static ArrayList getOrganizationRelationPair(String orgId, + String resultId, + String classID, + String className + + ) { + ArrayList newRelations = new ArrayList(); + newRelations + .add( + getRelation( + orgId, + resultId, + ModelConstants.IS_AUTHOR_INSTITUTION_OF, + ModelConstants.RESULT_ORGANIZATION, + ModelConstants.AFFILIATION, + PROPAGATION_DATA_INFO_TYPE, + classID, + className)); + newRelations + .add( + getRelation( + resultId, + orgId, + ModelConstants.HAS_AUTHOR_INSTITUTION, + ModelConstants.RESULT_ORGANIZATION, + ModelConstants.AFFILIATION, + PROPAGATION_DATA_INFO_TYPE, + classID, + className)); + + return newRelations; + } + public static Relation getRelation( String source, String target, diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 0ef5ca181..50ab997b6 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -22,6 +22,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.KeyValueSet; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Datasource; @@ -124,7 +125,7 @@ public class PrepareResultInstRepoAssociation { private static void prepareAlreadyLinkedAssociation( SparkSession spark, String alreadyLinkedPath) { - String query = "Select source resultId, collect_set(target) organizationSet " + String query = "Select source key, collect_set(target) valueSet " + "from relation " + "where datainfo.deletedbyinference = false " + "and lower(relClass) = '" @@ -134,7 +135,7 @@ public class PrepareResultInstRepoAssociation { spark .sql(query) - .as(Encoders.bean(ResultOrganizationSet.class)) + .as(Encoders.bean(KeyValueSet.class)) // TODO retry to stick with datasets .toJavaRDD() .map(r -> OBJECT_MAPPER.writeValueAsString(r)) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java deleted file mode 100644 index 3bce14cdb..000000000 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java +++ /dev/null @@ -1,26 +0,0 @@ - -package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; - -import java.io.Serializable; -import java.util.ArrayList; - -public class ResultOrganizationSet implements Serializable { - private String resultId; - private ArrayList organizationSet; - - public String getResultId() { - return resultId; - } - - public void setResultId(String resultId) { - this.resultId = resultId; - } - - public ArrayList getOrganizationSet() { - return organizationSet; - } - - public void setOrganizationSet(ArrayList organizationSet) { - this.organizationSet = organizationSet; - } -} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index 63824f1a8..0757ebccd 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -18,6 +18,7 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.KeyValueSet; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; @@ -28,7 +29,7 @@ public class SparkResultToOrganizationFromIstRepoJob { private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob.class); - private static final String RESULT_ORGANIZATIONSET_QUERY = "SELECT id resultId, collect_set(organizationId) organizationSet " + private static final String RESULT_ORGANIZATIONSET_QUERY = "SELECT id key, collect_set(organizationId) valueSet " + "FROM ( SELECT id, organizationId " + "FROM rels " + "JOIN cfhb " @@ -107,14 +108,14 @@ public class SparkResultToOrganizationFromIstRepoJob { Dataset dsOrg = readPath(spark, datasourceorganization, DatasourceOrganization.class); - Dataset potentialUpdates = getPotentialRelations(spark, inputPath, clazz, dsOrg); + Dataset potentialUpdates = getPotentialRelations(spark, inputPath, clazz, dsOrg); - Dataset alreadyLinked = readPath(spark, alreadyLinkedPath, ResultOrganizationSet.class); + Dataset alreadyLinked = readPath(spark, alreadyLinkedPath, KeyValueSet.class); potentialUpdates .joinWith( alreadyLinked, - potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), + potentialUpdates.col("key").equalTo(alreadyLinked.col("key")), "left_outer") .flatMap(createRelationFn(), Encoders.bean(Relation.class)) .write() @@ -123,49 +124,34 @@ public class SparkResultToOrganizationFromIstRepoJob { .json(outputPath); } - private static FlatMapFunction, Relation> createRelationFn() { + private static FlatMapFunction, Relation> createRelationFn() { return value -> { List newRelations = new ArrayList<>(); - ResultOrganizationSet potentialUpdate = value._1(); - Optional alreadyLinked = Optional.ofNullable(value._2()); - List organizations = potentialUpdate.getOrganizationSet(); + KeyValueSet potentialUpdate = value._1(); + Optional alreadyLinked = Optional.ofNullable(value._2()); + List organizations = potentialUpdate.getValueSet(); alreadyLinked .ifPresent( resOrg -> resOrg - .getOrganizationSet() + .getValueSet() .forEach(organizations::remove)); - String resultId = potentialUpdate.getResultId(); + String resultId = potentialUpdate.getKey(); organizations .forEach( - orgId -> { - newRelations - .add( - getRelation( - orgId, - resultId, - ModelConstants.IS_AUTHOR_INSTITUTION_OF, - ModelConstants.RESULT_ORGANIZATION, - ModelConstants.AFFILIATION, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - newRelations - .add( - getRelation( - resultId, - orgId, - ModelConstants.HAS_AUTHOR_INSTITUTION, - ModelConstants.RESULT_ORGANIZATION, - ModelConstants.AFFILIATION, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - }); + orgId -> newRelations + .addAll( + getOrganizationRelationPair( + orgId, + resultId, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)) + + ); return newRelations.iterator(); }; } - private static Dataset getPotentialRelations( + private static Dataset getPotentialRelations( SparkSession spark, String inputPath, Class resultClazz, @@ -179,7 +165,7 @@ public class SparkResultToOrganizationFromIstRepoJob { return spark .sql(RESULT_ORGANIZATIONSET_QUERY) - .as(Encoders.bean(ResultOrganizationSet.class)); + .as(Encoders.bean(KeyValueSet.class)); } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/Leaves.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/Leaves.java new file mode 100644 index 000000000..7984721e8 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/Leaves.java @@ -0,0 +1,16 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import java.io.Serializable; + +public class Leaves implements Serializable { + private String value; + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfo.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfo.java new file mode 100644 index 000000000..23909fd9a --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfo.java @@ -0,0 +1,155 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.io.Serializable; +import java.util.*; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; + +/** + * Searches for all the association between result and organization already existing in the graph + * Creates also the parenthood hierarchy from the organizations + */ + +public class PrepareInfo implements Serializable { + + private static final Logger log = LoggerFactory.getLogger(PrepareInfo.class); + + // associate orgs with all their parent + private static final String ORGANIZATION_ORGANIZATION_QUERY = "SELECT target key, collect_set(source) as valueSet " + + + "FROM relation " + + "WHERE lower(relclass) = '" + ModelConstants.IS_PARENT_OF.toLowerCase() + + "' and datainfo.deletedbyinference = false " + + "GROUP BY target"; + + // associates results with all the orgs they are affiliated to + private static final String RESULT_ORGANIZATION_QUERY = "SELECT source key, collect_set(target) as valueSet " + + "FROM relation " + + "WHERE lower(relclass) = '" + ModelConstants.HAS_AUTHOR_INSTITUTION.toLowerCase() + + "' and datainfo.deletedbyinference = false " + + "GROUP BY source"; + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkResultToOrganizationFromIstRepoJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_preparation_parameter.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String graphPath = parser.get("graphPath"); + log.info("graphPath: {}", graphPath); + + final String leavesPath = parser.get("leavesPath"); + log.info("leavesPath: {}", leavesPath); + + final String childParentPath = parser.get("childParentPath"); + log.info("childParentPath: {}", childParentPath); + + final String resultOrganizationPath = parser.get("resultOrgPath"); + log.info("resultOrganizationPath: {}", resultOrganizationPath); + + final String relationPath = parser.get("relationPath"); + log.info("relationPath: {}", relationPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> prepareInfo( + spark, + graphPath, + childParentPath, + leavesPath, + resultOrganizationPath, + relationPath)); + } + + private static void prepareInfo(SparkSession spark, String inputPath, String childParentOrganizationPath, + String currentIterationPath, String resultOrganizationPath, String relationPath) { + Dataset relation = readPath(spark, inputPath + "/relation", Relation.class); + relation.createOrReplaceTempView("relation"); + + spark + .sql(ORGANIZATION_ORGANIZATION_QUERY) + .as(Encoders.bean(KeyValueSet.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(childParentOrganizationPath); + + spark + .sql(RESULT_ORGANIZATION_QUERY) + .as(Encoders.bean(KeyValueSet.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(resultOrganizationPath); + + relation + .filter( + (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && + r.getRelClass().equals(ModelConstants.HAS_AUTHOR_INSTITUTION)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(relationPath); + + Dataset children = spark + .sql( + "Select distinct target as child from relation where " + + "lower(relclass)='" + ModelConstants.IS_PARENT_OF.toLowerCase() + + "' and datainfo.deletedbyinference = false") + .as(Encoders.STRING()); + + Dataset parent = spark + .sql( + "Select distinct source as parent from relation " + + "where lower(relclass)='" + ModelConstants.IS_PARENT_OF.toLowerCase() + + "' and datainfo.deletedbyinference = false") + .as(Encoders.STRING()); + + // takes from the join the entities having only the left hand side: the leaves. Saves them + children + .joinWith(parent, children.col("child").equalTo(parent.col("parent")), "left") + .map((MapFunction, String>) value -> { + if (Optional.ofNullable(value._2()).isPresent()) { + return null; + } + + return value._1(); + }, Encoders.STRING()) + .filter(Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .json(currentIterationPath); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PropagationCounter.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PropagationCounter.java new file mode 100644 index 000000000..788eff0e3 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PropagationCounter.java @@ -0,0 +1,77 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import java.io.Serializable; + +import org.apache.spark.util.LongAccumulator; + +public class PropagationCounter implements Serializable { + private LongAccumulator iterationOne; + private LongAccumulator iterationTwo; + private LongAccumulator iterationThree; + private LongAccumulator iterationFour; + private LongAccumulator iterationFive; + private LongAccumulator notReachedFirstParent; + + public PropagationCounter() { + } + + public PropagationCounter(LongAccumulator iterationOne, LongAccumulator iterationTwo, + LongAccumulator iterationThree, LongAccumulator iterationFour, LongAccumulator iterationFive, + LongAccumulator notReachedFirstParent) { + this.iterationOne = iterationOne; + this.iterationTwo = iterationTwo; + this.iterationThree = iterationThree; + this.iterationFour = iterationFour; + this.iterationFive = iterationFive; + this.notReachedFirstParent = notReachedFirstParent; + } + + public LongAccumulator getIterationOne() { + return iterationOne; + } + + public void setIterationOne(LongAccumulator iterationOne) { + this.iterationOne = iterationOne; + } + + public LongAccumulator getIterationTwo() { + return iterationTwo; + } + + public void setIterationTwo(LongAccumulator iterationTwo) { + this.iterationTwo = iterationTwo; + } + + public LongAccumulator getIterationThree() { + return iterationThree; + } + + public void setIterationThree(LongAccumulator iterationThree) { + this.iterationThree = iterationThree; + } + + public LongAccumulator getIterationFour() { + return iterationFour; + } + + public void setIterationFour(LongAccumulator iterationFour) { + this.iterationFour = iterationFour; + } + + public LongAccumulator getIterationFive() { + return iterationFive; + } + + public void setIterationFive(LongAccumulator iterationFive) { + this.iterationFive = iterationFive; + } + + public LongAccumulator getNotReachedFirstParent() { + return notReachedFirstParent; + } + + public void setNotReachedFirstParent(LongAccumulator notReachedFirstParent) { + this.notReachedFirstParent = notReachedFirstParent; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkResultToOrganizationFromSemRel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkResultToOrganizationFromSemRel.java new file mode 100644 index 000000000..9ceebf222 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkResultToOrganizationFromSemRel.java @@ -0,0 +1,201 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.io.Serializable; +import java.util.Arrays; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class SparkResultToOrganizationFromSemRel implements Serializable { + private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromSemRel.class); + private static final int MAX_ITERATION = 5; + public static final String NEW_RELATION_PATH = "/newRelation"; + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkResultToOrganizationFromIstRepoJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_propagation_parameter.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String relationPath = parser.get("relationPath"); + log.info("relationPath: {}", relationPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String leavesPath = parser.get("leavesPath"); + log.info("leavesPath: {}", leavesPath); + + final String childParentPath = parser.get("childParentPath"); + log.info("childParentPath: {}", childParentPath); + + final String resultOrganizationPath = parser.get("resultOrgPath"); + log.info("resultOrganizationPath: {}", resultOrganizationPath); + + final String workingPath = parser.get("workingDir"); + log.info("workingPath: {}", workingPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> execPropagation( + spark, + leavesPath, + childParentPath, + resultOrganizationPath, + relationPath, + workingPath, + outputPath)); + } + + public static void execPropagation(SparkSession spark, + String leavesPath, + String childParentPath, + String resultOrganizationPath, + String graphPath, + String workingPath, + String outputPath) { + + final LongAccumulator iterationOne = spark.sparkContext().longAccumulator(ITERATION_ONE); + final LongAccumulator iterationTwo = spark.sparkContext().longAccumulator(ITERATION_TWO); + final LongAccumulator iterationThree = spark.sparkContext().longAccumulator(ITERATION_THREE); + final LongAccumulator iterationFour = spark.sparkContext().longAccumulator(ITERATION_FOUR); + final LongAccumulator iterationFive = spark.sparkContext().longAccumulator(ITERATION_FIVE); + final LongAccumulator notReachedFirstParent = spark.sparkContext().longAccumulator(ITERATION_NO_PARENT); + + final PropagationCounter propagationCounter = new PropagationCounter(iterationOne, + iterationTwo, + iterationThree, + iterationFour, + iterationFive, + notReachedFirstParent); + + doPropagate( + spark, leavesPath, childParentPath, resultOrganizationPath, graphPath, + workingPath, outputPath, propagationCounter); + + } + + private static void doPropagate(SparkSession spark, String leavesPath, String childParentPath, + String resultOrganizationPath, String graphPath, String workingPath, String outputPath, + PropagationCounter propagationCounter) { + int iteration = 0; + long leavesCount; + + do { + iteration++; + StepActions + .execStep( + spark, graphPath, workingPath + NEW_RELATION_PATH, + leavesPath, childParentPath, resultOrganizationPath); + StepActions + .prepareForNextStep( + spark, workingPath + NEW_RELATION_PATH, resultOrganizationPath, leavesPath, + childParentPath, workingPath + "/leaves", workingPath + "/resOrg"); + moveOutput(spark, workingPath, leavesPath, resultOrganizationPath); + leavesCount = readPath(spark, leavesPath, Leaves.class).count(); + } while (leavesCount > 0 && iteration < MAX_ITERATION); + + if (leavesCount == 0) { + switch (String.valueOf(iteration)) { + case "1": + propagationCounter.getIterationOne().add(1); + break; + case "2": + propagationCounter.getIterationTwo().add(1); + break; + case "3": + propagationCounter.getIterationThree().add(1); + break; + case "4": + propagationCounter.getIterationFour().add(1); + break; + case "5": + propagationCounter.getIterationFive().add(1); + break; + default: + break; + } + } else { + propagationCounter.getNotReachedFirstParent().add(1); + } + + addNewRelations(spark, workingPath + NEW_RELATION_PATH, outputPath); + } + + private static void moveOutput(SparkSession spark, String workingPath, String leavesPath, + String resultOrganizationPath) { + readPath(spark, workingPath + "/leaves", Leaves.class) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(leavesPath); + + readPath(spark, workingPath + "/resOrg", KeyValueSet.class) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(resultOrganizationPath); + + } + + private static void addNewRelations(SparkSession spark, String newRelationPath, String outputPath) { + Dataset relation = readPath(spark, newRelationPath, Relation.class); + + relation + .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) (k, it) -> it.next(), Encoders.bean(Relation.class)) + .flatMap( + (FlatMapFunction) r -> Arrays + .asList( + r, getRelation( + r.getTarget(), r.getSource(), ModelConstants.IS_AUTHOR_INSTITUTION_OF, + ModelConstants.RESULT_ORGANIZATION, + ModelConstants.AFFILIATION, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME)) + .iterator() + + , Encoders.bean(Relation.class)) + .write() + + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(outputPath); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActions.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActions.java new file mode 100644 index 000000000..1adbbe60e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActions.java @@ -0,0 +1,204 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.readPath; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.jetbrains.annotations.NotNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; + +public class StepActions implements Serializable { + + public static void execStep(SparkSession spark, + String graphPath, String newRelationPath, + String leavesPath, String chldParentOrgPath, String resultOrgPath) { + + Dataset relationGraph = readPath(spark, graphPath, Relation.class); + // select only the relation source target among those proposed by propagation that are not already existent + getNewRels( + newRelationPath, relationGraph, + getPropagationRelation(spark, leavesPath, chldParentOrgPath, resultOrgPath)); + + } + + public static void prepareForNextStep(SparkSession spark, String selectedRelsPath, String resultOrgPath, + String leavesPath, String chldParentOrgPath, String leavesOutputPath, + String orgOutputPath) { + // use of the parents as new leaves set + changeLeavesSet(spark, leavesPath, chldParentOrgPath, leavesOutputPath); + + // add the new relations obtained from propagation to the keyvalueset result organization + updateResultOrganization( + spark, resultOrgPath, readPath(spark, selectedRelsPath, Relation.class), orgOutputPath); + } + + private static void updateResultOrganization(SparkSession spark, String resultOrgPath, + Dataset selectedRels, String outputPath) { + Dataset resultOrg = readPath(spark, resultOrgPath, KeyValueSet.class); + resultOrg + .joinWith( + selectedRels, resultOrg + .col("key") + .equalTo(selectedRels.col("source")), + "left") + .groupByKey((MapFunction, String>) mf -> mf._1().getKey(), Encoders.STRING()) + .mapGroups((MapGroupsFunction, KeyValueSet>) (key, it) -> { + Tuple2 first = it.next(); + if (!Optional.ofNullable(first._2()).isPresent()) { + return first._1(); + } + KeyValueSet ret = new KeyValueSet(); + ret.setKey(first._1().getKey()); + HashSet hs = new HashSet<>(); + hs.addAll(first._1().getValueSet()); + hs.add(first._2().getTarget()); + it.forEachRemaining(rel -> hs.add(rel._2().getTarget())); + ArrayList orgs = new ArrayList<>(); + orgs.addAll(hs); + ret.setValueSet(orgs); + return ret; + }, Encoders.bean(KeyValueSet.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } + + private static void changeLeavesSet(SparkSession spark, String leavesPath, String chldParentOrgPath, + String leavesOutputPath) { + Dataset childParent = readPath(spark, chldParentOrgPath, KeyValueSet.class); + Dataset leaves = readPath(spark, leavesPath, Leaves.class); + + childParent.createOrReplaceTempView("childParent"); + leaves.createOrReplaceTempView("leaves"); + + spark + .sql( + "SELECT distinct parent as value " + + "FROM leaves " + + "JOIN (SELECT key, parent " + + " FROM childParent " + + " LATERAL VIEW explode(valueSet) kv as parent) tmp " + + "ON value = key ") + .as(Encoders.bean(Leaves.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(leavesOutputPath); + } + + @NotNull + private static void getNewRels(String newRelationPath, Dataset relationDataset, + Dataset newRels) { + // selects new, not already existent relations + // union of new propagation relations to the relation set + // grouping from sourcetarget (we are sure the only relations are those from result to organization by + // construction of the set) + // if at least one relation in the set was not produced by propagation no new relation will be returned + + relationDataset + .union(newRels) + .groupByKey((MapFunction) r -> r.getSource() + r.getTarget(), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> { + + ArrayList relationList = new ArrayList<>(); + relationList.add(it.next()); + it.forEachRemaining(rel -> relationList.add(rel)); + + if (relationList + .stream() + .filter( + rel -> !rel + .getDataInfo() + .getProvenanceaction() + .getClassid() + .equals(PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID)) + .count() > 0) { + return null; + } + + return new ObjectMapper().writeValueAsString(relationList.get(0)); + + }, Encoders.STRING()) + .filter(Objects::nonNull) + .map( + (MapFunction) r -> new ObjectMapper().readValue(r, Relation.class), + Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(newRelationPath); + + } + + // get the possible relations from propagation + private static Dataset getPropagationRelation(SparkSession spark, + String leavesPath, + String chldParentOrgPath, + String resultOrgPath) { + + Dataset childParent = readPath(spark, chldParentOrgPath, KeyValueSet.class); + Dataset resultOrg = readPath(spark, resultOrgPath, KeyValueSet.class); + Dataset leaves = readPath(spark, leavesPath, Leaves.class); + + childParent.createOrReplaceTempView("childParent"); + resultOrg.createOrReplaceTempView("resultOrg"); + leaves.createOrReplaceTempView("leaves"); + + Dataset resultParent = spark + .sql( + "SELECT resId as key, " + + "collect_set(parent) valueSet " + + "FROM (SELECT key as child, parent " + + " FROM childParent " + + " LATERAL VIEW explode(valueSet) ks as parent) as cp " + + "JOIN leaves " + + "ON leaves.value = cp.child " + + "JOIN (" + + "SELECT key as resId, org " + + "FROM resultOrg " + + "LATERAL VIEW explode (valueSet) ks as org ) as ro " + + "ON leaves.value = ro.org " + + "GROUP BY resId") + .as(Encoders.bean(KeyValueSet.class)); + + // create new relations from result to organization for each result linked to a leaf + return resultParent + .flatMap( + (FlatMapFunction) v -> v + .getValueSet() + .stream() + .map( + orgId -> getRelation( + v.getKey(), + orgId, + ModelConstants.HAS_AUTHOR_INSTITUTION, + ModelConstants.RESULT_ORGANIZATION, + ModelConstants.AFFILIATION, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME)) + .collect(Collectors.toList()) + .iterator(), + Encoders.bean(Relation.class)); + + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_preparation_parameter.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_preparation_parameter.json new file mode 100644 index 000000000..c79bfe05d --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_preparation_parameter.json @@ -0,0 +1,44 @@ +[ + { + "paramName":"gp", + "paramLongName":"graphPath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"lp", + "paramLongName":"leavesPath", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, + { + "paramName":"cp", + "paramLongName":"childParentPath", + "paramDescription": "path where to store/find association from datasource and organization", + "paramRequired": true + }, + { + "paramName":"rp", + "paramLongName":"resultOrgPath", + "paramDescription": "path where to store/find already linked results and organizations", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "rep", + "paramLongName": "relationPath", + "paramDescription": "the path where to store the selected subset of relations", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_propagation_parameter.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_propagation_parameter.json new file mode 100644 index 000000000..f73cc221e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/input_propagation_parameter.json @@ -0,0 +1,50 @@ +[ + { + "paramName":"rep", + "paramLongName":"relationPath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"lp", + "paramLongName":"leavesPath", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, + { + "paramName":"cp", + "paramLongName":"childParentPath", + "paramDescription": "path where to store/find association from datasource and organization", + "paramRequired": true + }, + { + "paramName":"rp", + "paramLongName":"resultOrgPath", + "paramDescription": "path where to store/find already linked results and organizations", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "wd", + "paramLongName": "workingDir", + "paramDescription": "true if it is a test running", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/config-default.xml new file mode 100644 index 000000000..2744ea92b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/config-default.xml @@ -0,0 +1,58 @@ + + + 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 + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/workflow.xml new file mode 100644 index 000000000..17502abea --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/oozie_app/workflow.xml @@ -0,0 +1,193 @@ + + + + sourcePath + the source path + + + outputPath + sets the outputPath + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${wf:conf('resumeFrom') eq 'PrepareInfo'} + + + + + + + + + + + + + + + + + + + + + + + + + + + ${nameNode}/${sourcePath}/relation + ${nameNode}/${outputPath}/relation + + + + + + + + ${nameNode}/${sourcePath}/publication + ${nameNode}/${outputPath}/publication + + + + + + + + ${nameNode}/${sourcePath}/dataset + ${nameNode}/${outputPath}/dataset + + + + + + + + ${nameNode}/${sourcePath}/otherresearchproduct + ${nameNode}/${outputPath}/otherresearchproduct + + + + + + + + ${nameNode}/${sourcePath}/software + ${nameNode}/${outputPath}/software + + + + + + + + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + + + + + yarn + cluster + PrepareResultOrganizationAssociation + eu.dnetlib.dhp.resulttoorganizationfromsemrel.PrepareInfo + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --graphPath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --leavesPath${workingDir}/preparedInfo/leavesPath + --childParentPath${workingDir}/preparedInfo/childParentPath + --resultOrgPath${workingDir}/preparedInfo/resultOrgPath + --relationPath${workingDir}/preparedInfo/relation + + + + + + + + yarn + cluster + resultToOrganizationFromSemRel + eu.dnetlib.dhp.resulttoorganizationfromsemrel.SparkResultToOrganizationFromSemRel + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --relationPath${workingDir}/preparedInfo/relation + --outputPath${outputPath}/relation + --leavesPath${workingDir}/preparedInfo/leavesPath + --childParentPath${workingDir}/preparedInfo/childParentPath + --resultOrgPath${workingDir}/preparedInfo/resultOrgPath + --hive_metastore_uris${hive_metastore_uris} + --workingDir${workingDir}/working + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfoJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfoJobTest.java new file mode 100644 index 000000000..2d2668db3 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/PrepareInfoJobTest.java @@ -0,0 +1,579 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.readPath; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; + +import org.apache.commons.io.FileUtils; +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.FilterFunction; +import org.apache.spark.api.java.function.ForeachFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob; +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class PrepareInfoJobTest { + + private static final Logger log = LoggerFactory.getLogger(PrepareInfoJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(PrepareInfoJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(PrepareInfoJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(PrepareInfoJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void childParentTest1() throws Exception { + + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest1") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/childParentOrg/") + .map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(KeyValueSet.class)); + + Assertions.assertEquals(6, verificationDs.count()); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|dedup_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertEquals( + "20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f", + verificationDs + .filter("key = '20|dedup_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .get(0)); + + Assertions + .assertEquals( + 2, verificationDs + .filter("key = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|openaire____::ec653e804967133b9436fdd30d3ff51d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + + verificationDs + .foreach((ForeachFunction) v -> System.out.println(OBJECT_MAPPER.writeValueAsString(v))); + + } + + @Test + public void childParentTest2() throws Exception { + + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest2") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/childParentOrg/") + .map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(KeyValueSet.class)); + + Assertions.assertEquals(5, verificationDs.count()); + + Assertions + .assertEquals( + 0, verificationDs.filter("key = '20|dedup_wf_001::2899e571609779168222fdeb59cb916d'").count()); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertEquals( + "20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f", + verificationDs + .filter("key = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .get(0)); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|openaire____::ec653e804967133b9436fdd30d3ff51d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + + } + + @Test + public void relationTest() throws Exception { + + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + + Assertions.assertEquals(7, verificationDs.count()); + + } + + @Test + public void resultOrganizationTest1() throws Exception { + + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/resultOrganization/") + .map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(KeyValueSet.class)); + + Assertions.assertEquals(5, verificationDs.count()); + + Assertions + .assertEquals( + 2, verificationDs + .filter("key = '50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|pippo_wf_001::2899e571609779168222fdeb59cb916d")); + + Assertions + .assertEquals( + 2, verificationDs + .filter("key = '50|dedup_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|dedup_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0")); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|dedup_wf_001::2899e571609779168222fdeb59cb916d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|pippo_wf_001::2899e571609779168222fdeb59cb916d")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '50|doajarticles::03748bcb5d754c951efec9700e18a56d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|doajarticles::03748bcb5d754c951efec9700e18a56d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '50|openaire____::ec653e804967133b9436fdd30d3ff51d'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|openaire____::ec653e804967133b9436fdd30d3ff51d'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + + Assertions + .assertEquals( + 1, verificationDs + .filter("key = '50|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + verificationDs + .filter("key = '50|doajarticles::1cae0b82b56ccd97c2db1f698def7074'") + .collectAsList() + .get(0) + .getValueSet() + .contains("20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1")); + + verificationDs + .foreach((ForeachFunction) v -> System.out.println(OBJECT_MAPPER.writeValueAsString(v))); + + } + + @Test + public void foundLeavesTest1() throws Exception { + + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/currentIteration/") + .map(item -> OBJECT_MAPPER.readValue(item, String.class)); + + Assertions.assertEquals(0, tmp.count()); + + } + + @Test + public void foundLeavesTest2() throws Exception { + PrepareInfo + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-graphPath", getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest1") + .getPath(), + "-hive_metastore_uris", "", + "-leavesPath", workingDir.toString() + "/currentIteration/", + "-resultOrgPath", workingDir.toString() + "/resultOrganization/", + "-childParentPath", workingDir.toString() + "/childParentOrg/", + "-relationPath", workingDir.toString() + "/relation" + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/currentIteration/") + .map(item -> OBJECT_MAPPER.readValue(item, Leaves.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Leaves.class)); + + Assertions.assertEquals(3, verificationDs.count()); + + Assertions + .assertEquals( + 1, verificationDs + .filter("value = '20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0'") + .count()); + + Assertions + .assertEquals( + 1, verificationDs + .filter("value = '20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1'") + .count()); + + Assertions + .assertEquals( + 1, verificationDs + .filter("value = '20|pippo_wf_001::2899e571609779168222fdeb59cb916d'") + .count()); + + verificationDs.foreach((ForeachFunction) l -> System.out.println(OBJECT_MAPPER.writeValueAsString(l))); + + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkJobTest.java new file mode 100644 index 000000000..7dd575b66 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/SparkJobTest.java @@ -0,0 +1,325 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.isSparkSessionManaged; +import static eu.dnetlib.dhp.PropagationConstant.readPath; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +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.ForeachFunction; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.PropagationConstant; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class SparkJobTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + private static final Logger log = LoggerFactory.getLogger(SparkJobTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(StepActionsTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(PrepareInfoJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(PrepareInfoJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void completeExecution() throws Exception { + + final String graphPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep") + .getPath(); + final String leavesPath = getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/") + .getPath(); + final String childParentPath = getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/") + .getPath(); + + final String resultOrgPath = getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/") + .getPath(); + + readPath(spark, leavesPath, Leaves.class) + .write() + .option("compression", "gzip") + .json(workingDir.toString() + "/leavesInput"); + + readPath(spark, resultOrgPath, KeyValueSet.class) + .write() + .option("compression", "gzip") + .json(workingDir.toString() + "/orgsInput"); + + SparkResultToOrganizationFromSemRel + + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-relationPath", graphPath, + "-hive_metastore_uris", "", + "-outputPath", workingDir.toString() + "/finalrelation", + "-leavesPath", workingDir.toString() + "/leavesInput", + "-resultOrgPath", workingDir.toString() + "/orgsInput", + "-childParentPath", childParentPath, + "-workingDir", workingDir.toString() + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/finalrelation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r))); + + Assertions.assertEquals(18, tmp.count()); + tmp.foreach(r -> Assertions.assertEquals(ModelConstants.AFFILIATION, r.getSubRelType())); + tmp.foreach(r -> Assertions.assertEquals(ModelConstants.RESULT_ORGANIZATION, r.getRelType())); + tmp + .foreach( + r -> Assertions + .assertEquals( + PropagationConstant.PROPAGATION_DATA_INFO_TYPE, r.getDataInfo().getInferenceprovenance())); + tmp + .foreach( + r -> Assertions + .assertEquals( + PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID, + r.getDataInfo().getProvenanceaction().getClassid())); + tmp + .foreach( + r -> Assertions + .assertEquals( + PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME, + r.getDataInfo().getProvenanceaction().getClassname())); + tmp + .foreach( + r -> Assertions + .assertEquals( + "0.85", + r.getDataInfo().getTrust())); + + Assertions.assertEquals(9, tmp.filter(r -> r.getSource().substring(0, 3).equals("50|")).count()); + tmp + .filter(r -> r.getSource().substring(0, 3).equals("50|")) + .foreach(r -> Assertions.assertEquals(ModelConstants.HAS_AUTHOR_INSTITUTION, r.getRelClass())); + Assertions + .assertEquals( + 2, tmp.filter(r -> r.getSource().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")).count()); + Assertions + .assertEquals( + 3, tmp.filter(r -> r.getSource().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")).count()); + Assertions + .assertEquals( + 2, tmp.filter(r -> r.getSource().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")).count()); + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getSource().equals("50|openaire____::ec653e804967133b9436fdd30d3ff51d")).count()); + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getSource().equals("50|doajarticles::03748bcb5d754c951efec9700e18a56d")).count()); + + Assertions.assertEquals(9, tmp.filter(r -> r.getSource().substring(0, 3).equals("20|")).count()); + tmp + .filter(r -> r.getSource().substring(0, 3).equals("20|")) + .foreach(r -> Assertions.assertEquals(ModelConstants.IS_AUTHOR_INSTITUTION_OF, r.getRelClass())); + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getSource().equals("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")).count()); + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getSource().equals("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")).count()); + Assertions + .assertEquals( + 2, tmp.filter(r -> r.getSource().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")).count()); + Assertions + .assertEquals( + 2, tmp.filter(r -> r.getSource().equals("20|openaire____::ec653e804967133b9436fdd30d3ff51d")).count()); + Assertions + .assertEquals( + 3, tmp.filter(r -> r.getSource().equals("20|doajarticles::03748bcb5d754c951efec9700e18a56d")).count()); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .map(r -> r.getTarget()) + .collect() + .contains("20|openaire____::ec653e804967133b9436fdd30d3ff51d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .map(r -> r.getTarget()) + .collect() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|openaire____::ec653e804967133b9436fdd30d3ff51d")) + .map(r -> r.getTarget()) + .collect() + .contains("20|openaire____::ec653e804967133b9436fdd30d3ff51d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("50|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .map(r -> r.getTarget()) + .collect() + .contains("20|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|openaire____::ec653e804967133b9436fdd30d3ff51d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|openaire____::ec653e804967133b9436fdd30d3ff51d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|openaire____::ec653e804967133b9436fdd30d3ff51d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .map(r -> r.getTarget()) + .collect() + .contains("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .map(r -> r.getTarget()) + .collect() + .contains("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|doajarticles::03748bcb5d754c951efec9700e18a56d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .map(r -> r.getTarget()) + .collect() + .contains("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getSource().equals("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .map(r -> r.getTarget()) + .collect() + .contains("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActionsTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActionsTest.java new file mode 100644 index 000000000..5c715f3b9 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfromsemrel/StepActionsTest.java @@ -0,0 +1,411 @@ + +package eu.dnetlib.dhp.resulttoorganizationfromsemrel; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +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.FilterFunction; +import org.apache.spark.api.java.function.ForeachFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.KeyValueSet; +import eu.dnetlib.dhp.PropagationConstant; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class StepActionsTest { + + private static final Logger log = LoggerFactory.getLogger(StepActionsTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(StepActionsTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(PrepareInfoJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(PrepareInfoJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void execStepTest() { + + StepActions + .execStep( + spark, getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/") + .getPath(), + workingDir.toString() + "/newRelationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/") + .getPath()); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/newRelationPath") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(4, tmp.count()); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals("propagation", r.getDataInfo().getInferenceprovenance())); + + verificationDs + .foreach((ForeachFunction) r -> Assertions.assertEquals("0.85", r.getDataInfo().getTrust())); + + verificationDs + .foreach((ForeachFunction) r -> Assertions.assertEquals("50|", r.getSource().substring(0, 3))); + + verificationDs + .foreach((ForeachFunction) r -> Assertions.assertEquals("20|", r.getTarget().substring(0, 3))); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals(ModelConstants.HAS_AUTHOR_INSTITUTION, r.getRelClass())); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals(ModelConstants.RESULT_ORGANIZATION, r.getRelType())); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals(ModelConstants.AFFILIATION, r.getSubRelType())); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals( + PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID, + r.getDataInfo().getProvenanceaction().getClassid())); + + verificationDs + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals( + PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME, + r.getDataInfo().getProvenanceaction().getClassname())); + + verificationDs + .filter( + (FilterFunction) r -> r + .getSource() + .equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074", r.getTarget())); + + verificationDs + .filter( + (FilterFunction) r -> r + .getSource() + .equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .foreach( + (ForeachFunction) r -> Assertions + .assertEquals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f", r.getTarget())); + + Assertions + .assertEquals( + 2, + verificationDs + .filter( + (FilterFunction) r -> r + .getSource() + .equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .count()); + + Assertions + .assertEquals( + 1, + verificationDs + .filter( + (FilterFunction) r -> r + .getSource() + .equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d") && + r.getTarget().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .count()); + + Assertions + .assertEquals( + 1, + verificationDs + .filter( + (FilterFunction) r -> r + .getSource() + .equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d") && + r.getTarget().equals("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .count()); + + tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r))); + } + + @Test + public void prepareForNextStepLeavesTest() { + + StepActions + .prepareForNextStep( + spark, + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relsforiteration1/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/") + .getPath(), + workingDir.toString() + "/tempLeaves", workingDir.toString() + "/tempOrgs"); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/tempLeaves") + .map(item -> OBJECT_MAPPER.readValue(item, Leaves.class)); + + Assertions.assertEquals(3, tmp.count()); + + Assertions + .assertEquals( + 1, tmp.filter(l -> l.getValue().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")).count()); + + Assertions + .assertEquals( + 1, tmp.filter(l -> l.getValue().equals("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")).count()); + + Assertions + .assertEquals( + 1, tmp.filter(l -> l.getValue().equals("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")).count()); + + } + + @Test + public void prepareFonNextStepOrgTest() { + StepActions + .prepareForNextStep( + spark, + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relsforiteration1/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/") + .getPath(), + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/") + .getPath(), + workingDir.toString() + "/tempLeaves", workingDir.toString() + "/tempOrgs"); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/tempOrgs") + .map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class)); + + Assertions.assertEquals(5, tmp.count()); + + Assertions + .assertEquals( + 1, tmp + .filter(kv -> kv.getKey().equals("50|openaire____::ec653e804967133b9436fdd30d3ff51d")) + .collect() + .get(0) + .getValueSet() + .size()); + Assertions + .assertEquals( + "20|doajarticles::1cae0b82b56ccd97c2db1f698def7074", + tmp + .filter(kv -> kv.getKey().equals("50|openaire____::ec653e804967133b9436fdd30d3ff51d")) + .collect() + .get(0) + .getValueSet() + .get(0)); + + Assertions + .assertEquals( + 1, tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .collect() + .get(0) + .getValueSet() + .size()); + Assertions + .assertEquals( + "20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f", + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::03748bcb5d754c951efec9700e18a56d")) + .collect() + .get(0) + .getValueSet() + .get(0)); + + Assertions + .assertEquals( + 4, tmp + .filter(kv -> kv.getKey().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .collect() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .collect() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .collect() + .get(0) + .getValueSet() + .contains("20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .collect() + .get(0) + .getValueSet() + .contains("20|pippo_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d")) + .collect() + .get(0) + .getValueSet() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + + Assertions + .assertEquals( + 2, tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .collect() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .collect() + .get(0) + .getValueSet() + .contains("20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074")) + .collect() + .get(0) + .getValueSet() + .contains("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074")); + + Assertions + .assertEquals( + 3, tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .collect() + .get(0) + .getValueSet() + .size()); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .collect() + .get(0) + .getValueSet() + .contains("20|dedup_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .collect() + .get(0) + .getValueSet() + .contains("20|pippo_wf_001::2899e571609779168222fdeb59cb916d")); + Assertions + .assertTrue( + tmp + .filter(kv -> kv.getKey().equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")) + .collect() + .get(0) + .getValueSet() + .contains("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f")); + + } +} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest1/relation b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest1/relation new file mode 100644 index 000000000..c63a2e0ac --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest1/relation @@ -0,0 +1,7 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::03748bcb5d754c951efec9700e18a56d","subRelType":"provision","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|openaire____::ec653e804967133b9436fdd30d3ff51d","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest2/relation b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest2/relation new file mode 100644 index 000000000..54589de32 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/childparenttest2/relation @@ -0,0 +1,7 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::03748bcb5d754c951efec9700e18a56d","subRelType":"provision","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|openaire____::ec653e804967133b9436fdd30d3ff51d","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/childparent b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/childparent new file mode 100644 index 000000000..7d9ea588b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/childParentOrg/childparent @@ -0,0 +1,6 @@ +{"key":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1","valueSet":["20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"]} +{"key":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d","valueSet":["20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","20|dedup_wf_001::2899e571609779168222fdeb59cb916d"]} +{"key":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074","valueSet":["20|openaire____::ec653e804967133b9436fdd30d3ff51d"]} +{"key":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0","valueSet":["20|dedup_wf_001::2899e571609779168222fdeb59cb916d"]} +{"key":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","valueSet":["20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"]} +{"key":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","valueSet":["20|doajarticles::03748bcb5d754c951efec9700e18a56d"]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/leaves b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/leaves new file mode 100644 index 000000000..3be9cae3b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/currentIteration/leaves @@ -0,0 +1,3 @@ +{"value":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} +{"value":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"value":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relation b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relation new file mode 100644 index 000000000..db7db8fdd --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relation @@ -0,0 +1,14 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::03748bcb5d754c951efec9700e18a56d","subRelType":"provision","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|openaire____::ec653e804967133b9436fdd30d3ff51d","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"isparentof","relType":"datasourceOrganization","source":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::03748bcb5d754c951efec9700e18a56d","subRelType":"provision","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|openaire____::ec653e804967133b9436fdd30d3ff51d","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::1cae0b82b56ccd97c2db1f698def7074","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relsforiteration1/relation b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relsforiteration1/relation new file mode 100644 index 000000000..32b816ef7 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/relsforiteration1/relation @@ -0,0 +1,4 @@ +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.85","inferenceprovenance":"propagation","provenanceaction":{"classid":"result:organization:semrel","classname":"Propagation of affiliation to result through sematic relations","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"resultOrganization","subRelType":"affiliation","relClass":"hasAuthorInstitution","source":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","validated":false,"validationDate":null,"properties":[]} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.85","inferenceprovenance":"propagation","provenanceaction":{"classid":"result:organization:semrel","classname":"Propagation of affiliation to result through sematic relations","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"resultOrganization","subRelType":"affiliation","relClass":"hasAuthorInstitution","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","validated":false,"validationDate":null,"properties":[]} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.85","inferenceprovenance":"propagation","provenanceaction":{"classid":"result:organization:semrel","classname":"Propagation of affiliation to result through sematic relations","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"resultOrganization","subRelType":"affiliation","relClass":"hasAuthorInstitution","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d","validated":false,"validationDate":null,"properties":[]} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.85","inferenceprovenance":"propagation","provenanceaction":{"classid":"result:organization:semrel","classname":"Propagation of affiliation to result through sematic relations","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"relType":"resultOrganization","subRelType":"affiliation","relClass":"hasAuthorInstitution","source":"50|doajarticles::1cae0b82b56ccd97c2db1f698def7074","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074","validated":false,"validationDate":null,"properties":[]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/resultorganization b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/resultorganization new file mode 100644 index 000000000..b4e227227 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/execstep/resultOrganization/resultorganization @@ -0,0 +1,5 @@ +{"key":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","valueSet":["20|pippo_wf_001::2899e571609779168222fdeb59cb916d","20|dedup_wf_001::2899e571609779168222fdeb59cb916d"]} +{"key":"50|doajarticles::1cae0b82b56ccd97c2db1f698def7074","valueSet":["20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"]} +{"key":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","valueSet":["20|pippo_wf_001::2899e571609779168222fdeb59cb916d","20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"]} +{"key":"50|openaire____::ec653e804967133b9436fdd30d3ff51d","valueSet":["20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"]} +{"key":"50|doajarticles::03748bcb5d754c951efec9700e18a56d","valueSet":["20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest/relation b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest/relation new file mode 100644 index 000000000..5aeabb71b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfromsemrel/resultorganizationtest/relation @@ -0,0 +1,7 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|dedup_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|dedup_wf_001::2899e571609779168222fdeb59cb916d","subRelType":"provision","target":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::03748bcb5d754c951efec9700e18a56d","subRelType":"provision","target":"20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|openaire____::ec653e804967133b9436fdd30d3ff51d","subRelType":"provision","target":"20|doajarticles::1cae0b82b56ccd97c2db1f698def7074"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1592688952862,"properties":[],"relClass":"hasAuthorInstitution","relType":"datasourceOrganization","source":"50|doajarticles::1cae0b82b56ccd97c2db1f698def7074","subRelType":"provision","target":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_object_json_params.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_object_json_params.json index 4b15da623..890570a0b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_object_json_params.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/convert_object_json_params.json @@ -1,6 +1,7 @@ [ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true}, - {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the raw graph", "paramRequired": true}, - {"paramName":"o", "paramLongName":"objectType", "paramDescription": "should be scholix or Summary", "paramRequired": true} + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true}, + {"paramName":"su", "paramLongName":"scholixUpdatePath", "paramDescription": "the scholix updated Path", "paramRequired": false}, + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the raw graph", "paramRequired": true}, + {"paramName":"o", "paramLongName":"objectType", "paramDescription": "should be scholix or Summary", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml index 17996c82c..e46e59cc0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/finalGraph/oozie_app/workflow.xml @@ -79,7 +79,7 @@ --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.shuffle.partitions=20000 + --conf spark.sql.shuffle.partitions=30000 --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} @@ -90,68 +90,6 @@ --relationPath${targetPath}/relation - - - - - - - - - - - - - - - - yarn - cluster - Serialize scholix to JSON - eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.shuffle.partitions=6000 - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --masteryarn - --sourcePath${targetPath}/provision/scholix/scholix - --targetPath${targetPath}/index/scholix_json - --objectTypescholix - - - - - - - - - yarn - cluster - Serialize summary to JSON - eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson - dhp-graph-mapper-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.shuffle.partitions=6000 - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --masteryarn - --sourcePath${targetPath}/provision/summaries_filtered - --targetPath${targetPath}/index/summaries_json - --objectTypesummary - diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/relations.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/relations.json deleted file mode 100644 index 98e8daa18..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/relations.json +++ /dev/null @@ -1,158 +0,0 @@ -{ - "cites":{ - "original":"Cites", - "inverse":"IsCitedBy" - }, - "compiles":{ - "original":"Compiles", - "inverse":"IsCompiledBy" - }, - "continues":{ - "original":"Continues", - "inverse":"IsContinuedBy" - }, - "derives":{ - "original":"IsSourceOf", - "inverse":"IsDerivedFrom" - }, - "describes":{ - "original":"Describes", - "inverse":"IsDescribedBy" - }, - "documents":{ - "original":"Documents", - "inverse":"IsDocumentedBy" - }, - "hasmetadata":{ - "original":"HasMetadata", - "inverse":"IsMetadataOf" - }, - "hasassociationwith":{ - "original":"HasAssociationWith", - "inverse":"HasAssociationWith" - }, - "haspart":{ - "original":"HasPart", - "inverse":"IsPartOf" - }, - "hasversion":{ - "original":"HasVersion", - "inverse":"IsVersionOf" - }, - "iscitedby":{ - "original":"IsCitedBy", - "inverse":"Cites" - }, - "iscompiledby":{ - "original":"IsCompiledBy", - "inverse":"Compiles" - }, - "iscontinuedby":{ - "original":"IsContinuedBy", - "inverse":"Continues" - }, - "isderivedfrom":{ - "original":"IsDerivedFrom", - "inverse":"IsSourceOf" - }, - "isdescribedby":{ - "original":"IsDescribedBy", - "inverse":"Describes" - }, - "isdocumentedby":{ - "original":"IsDocumentedBy", - "inverse":"Documents" - }, - "isidenticalto":{ - "original":"IsIdenticalTo", - "inverse":"IsIdenticalTo" - }, - "ismetadatafor":{ - "original":"IsMetadataFor", - "inverse":"IsMetadataOf" - }, - "ismetadataof":{ - "original":"IsMetadataOf", - "inverse":"IsMetadataFor" - }, - "isnewversionof":{ - "original":"IsNewVersionOf", - "inverse":"IsPreviousVersionOf" - }, - "isobsoletedby":{ - "original":"IsObsoletedBy", - "inverse":"Obsoletes" - }, - "isoriginalformof":{ - "original":"IsOriginalFormOf", - "inverse":"IsVariantFormOf" - }, - "ispartof":{ - "original":"IsPartOf", - "inverse":"HasPart" - }, - "ispreviousversionof":{ - "original":"IsPreviousVersionOf", - "inverse":"IsNewVersionOf" - }, - "isreferencedby":{ - "original":"IsReferencedBy", - "inverse":"References" - }, - "isrelatedto":{ - "original":"IsRelatedTo", - "inverse":"IsRelatedTo" - }, - "isrequiredby":{ - "original":"IsRequiredBy", - "inverse":"Requires" - }, - "isreviewedby":{ - "original":"IsReviewedBy", - "inverse":"Reviews" - }, - "issourceof":{ - "original":"IsSourceOf", - "inverse":"IsDerivedFrom" - }, - "issupplementedby":{ - "original":"IsSupplementedBy", - "inverse":"IsSupplementTo" - }, - "issupplementto":{ - "original":"IsSupplementTo", - "inverse":"IsSupplementedBy" - }, - "isvariantformof":{ - "original":"IsVariantFormOf", - "inverse":"IsOriginalFormOf" - }, - "isversionof":{ - "original":"IsVersionOf", - "inverse":"HasVersion" - }, - "obsoletes":{ - "original":"Obsoletes", - "inverse":"IsObsoletedBy" - }, - "references":{ - "original":"References", - "inverse":"IsReferencedBy" - }, - "requires":{ - "original":"Requires", - "inverse":"IsRequiredBy" - }, - "related":{ - "original":"IsRelatedTo", - "inverse":"IsRelatedTo" - }, - "reviews":{ - "original":"Reviews", - "inverse":"IsReviewedBy" - }, - "unknown":{ - "original":"Unknown", - "inverse":"Unknown" - } -} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/config-default.xml new file mode 100644 index 000000000..6fb2a1253 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/config-default.xml @@ -0,0 +1,10 @@ + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/workflow.xml new file mode 100644 index 000000000..2844d7baa --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/graph/serializeGraph/oozie_app/workflow.xml @@ -0,0 +1,83 @@ + + + + scholixUpdatePath + the working dir base path of the scholix updated + + + targetPath + the final graph path + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + yarn + cluster + Serialize scholix to JSON + eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=6000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn + --sourcePath${targetPath}/provision/scholix/scholix + --targetPath${targetPath}/index/scholix_json + --scholixUpdatePath${scholixUpdatePath} + --objectTypescholix + + + + + + + + + yarn + cluster + Serialize summary to JSON + eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.shuffle.partitions=6000 + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --masteryarn + --sourcePath${targetPath}/provision/summaries_filtered + --targetPath${targetPath}/index/summaries_json + --objectTypesummary + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertObjectToJson.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertObjectToJson.scala index cc1b97fd6..0c54de7c8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertObjectToJson.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertObjectToJson.scala @@ -30,6 +30,9 @@ object SparkConvertObjectToJson { log.info(s"targetPath -> $targetPath") val objectType = parser.get("objectType") log.info(s"objectType -> $objectType") + val scholixUpdatePath = parser.get("scholixUpdatePath") + log.info(s"scholixUpdatePath -> $scholixUpdatePath") + implicit val scholixEncoder: Encoder[Scholix] = Encoders.kryo[Scholix] @@ -42,7 +45,8 @@ object SparkConvertObjectToJson { case "scholix" => log.info("Serialize Scholix") val d: Dataset[Scholix] = spark.read.load(sourcePath).as[Scholix] - d.map(s => mapper.writeValueAsString(s))(Encoders.STRING).rdd.repartition(6000).saveAsTextFile(targetPath, classOf[GzipCodec]) + val u :Dataset[Scholix]= spark.read.load(s"$scholixUpdatePath/scholix").as[Scholix] + d.union(u).repartition(8000).map(s => mapper.writeValueAsString(s))(Encoders.STRING).rdd.saveAsTextFile(targetPath, classOf[GzipCodec]) case "summary" => log.info("Serialize Summary") val d: Dataset[ScholixSummary] = spark.read.load(sourcePath).as[ScholixSummary] diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala index 23f039c70..2115df1fd 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkConvertRDDtoDataset.scala @@ -2,12 +2,11 @@ package eu.dnetlib.dhp.sx.graph import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Relation, Software,Dataset => OafDataset} +import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Relation, Result, Software, Dataset => OafDataset} import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession} import org.slf4j.{Logger, LoggerFactory} - object SparkConvertRDDtoDataset { def main(args: Array[String]): Unit = { @@ -32,39 +31,40 @@ object SparkConvertRDDtoDataset { val entityPath = s"$t/entities" val relPath = s"$t/relation" val mapper = new ObjectMapper() - implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset]) - implicit val publicationEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication]) - implicit val relationEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation]) - implicit val orpEncoder: Encoder[OtherResearchProduct] = Encoders.kryo(classOf[OtherResearchProduct]) - implicit val softwareEncoder: Encoder[Software] = Encoders.kryo(classOf[Software]) + implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset]) + implicit val publicationEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication]) + implicit val relationEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation]) + implicit val orpEncoder: Encoder[OtherResearchProduct] = Encoders.kryo(classOf[OtherResearchProduct]) + implicit val softwareEncoder: Encoder[Software] = Encoders.kryo(classOf[Software]) log.info("Converting dataset") - val rddDataset = spark.sparkContext.textFile(s"$sourcePath/dataset").map(s => mapper.readValue(s, classOf[OafDataset])) + val rddDataset =spark.sparkContext.textFile(s"$sourcePath/dataset").map(s => mapper.readValue(s, classOf[OafDataset])).filter(r=> r.getDataInfo!= null && r.getDataInfo.getDeletedbyinference == false) spark.createDataset(rddDataset).as[OafDataset].write.mode(SaveMode.Overwrite).save(s"$entityPath/dataset") log.info("Converting publication") - val rddPublication = spark.sparkContext.textFile(s"$sourcePath/publication").map(s => mapper.readValue(s, classOf[Publication])) + val rddPublication =spark.sparkContext.textFile(s"$sourcePath/publication").map(s => mapper.readValue(s, classOf[Publication])).filter(r=> r.getDataInfo!= null && r.getDataInfo.getDeletedbyinference == false) spark.createDataset(rddPublication).as[Publication].write.mode(SaveMode.Overwrite).save(s"$entityPath/publication") log.info("Converting software") - val rddSoftware = spark.sparkContext.textFile(s"$sourcePath/software").map(s => mapper.readValue(s, classOf[Software])) + val rddSoftware =spark.sparkContext.textFile(s"$sourcePath/software").map(s => mapper.readValue(s, classOf[Software])).filter(r=> r.getDataInfo!= null && r.getDataInfo.getDeletedbyinference == false) spark.createDataset(rddSoftware).as[Software].write.mode(SaveMode.Overwrite).save(s"$entityPath/software") log.info("Converting otherresearchproduct") - val rddOtherResearchProduct = spark.sparkContext.textFile(s"$sourcePath/otherresearchproduct").map(s => mapper.readValue(s, classOf[OtherResearchProduct])) + val rddOtherResearchProduct =spark.sparkContext.textFile(s"$sourcePath/otherresearchproduct").map(s => mapper.readValue(s, classOf[OtherResearchProduct])).filter(r=> r.getDataInfo!= null && r.getDataInfo.getDeletedbyinference == false) spark.createDataset(rddOtherResearchProduct).as[OtherResearchProduct].write.mode(SaveMode.Overwrite).save(s"$entityPath/otherresearchproduct") log.info("Converting Relation") - val relationSemanticFilter = List("cites", "iscitedby", "merges", "ismergedin") + val relationSemanticFilter = List("cites", "iscitedby","merges", "ismergedin", "HasAmongTopNSimilarDocuments","IsAmongTopNSimilarDocuments" ) - val rddRelation = spark.sparkContext.textFile(s"$sourcePath/relation") + val rddRelation =spark.sparkContext.textFile(s"$sourcePath/relation") .map(s => mapper.readValue(s, classOf[Relation])) - .filter(r => r.getSource.startsWith("50") && r.getTarget.startsWith("50")) + .filter(r=> r.getDataInfo!= null && r.getDataInfo.getDeletedbyinference == false) + .filter(r=> r.getSource.startsWith("50") && r.getTarget.startsWith("50")) .filter(r => !relationSemanticFilter.exists(k => k.equalsIgnoreCase(r.getRelClass))) spark.createDataset(rddRelation).as[Relation].write.mode(SaveMode.Overwrite).save(s"$relPath") diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/RetrieveDataciteDeltaTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/RetrieveDataciteDeltaTest.scala new file mode 100644 index 000000000..c277b0aa1 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/RetrieveDataciteDeltaTest.scala @@ -0,0 +1,26 @@ +package eu.dnetlib.dhp.sx.graph +import org.junit.jupiter.api.Test + +import java.text.SimpleDateFormat + + + +class RetrieveDataciteDeltaTest { + + @Test + def testParsingDate(): Unit = { + + + val inputDate = "2021-12-02T11:17:36+0000" + + val t = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ").parse(inputDate).getTime + + + println(t) + + + + } + + +} 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 66827af67..af7c7b7c3 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 @@ -11,31 +11,17 @@ import java.io.Serializable; import java.io.StringReader; import java.io.StringWriter; import java.net.URL; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collector; +import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import javax.xml.transform.OutputKeys; -import javax.xml.transform.Transformer; -import javax.xml.transform.TransformerConfigurationException; -import javax.xml.transform.TransformerException; -import javax.xml.transform.TransformerFactory; +import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; -import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import org.apache.solr.common.util.URLUtil; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; import org.dom4j.DocumentException; @@ -60,6 +46,7 @@ import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; import eu.dnetlib.dhp.oa.provision.model.XmlInstance; import eu.dnetlib.dhp.schema.common.*; import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; public class XmlRecordFactory implements Serializable { @@ -1274,6 +1261,7 @@ public class XmlRecordFactory implements Serializable { private Stream groupInstancesByUrl(List instance) { return instance .stream() + .filter(i -> Objects.nonNull(i.getUrl())) .map(i -> { i .setUrl( @@ -1329,18 +1317,28 @@ public class XmlRecordFactory implements Serializable { instance.getCollectedfrom().add(i.getCollectedfrom()); instance.getHostedby().add(i.getHostedby()); instance.getInstancetype().add(i.getInstancetype()); - instance.getLicense().add(i.getLicense().getValue()); - instance.getDistributionlocation().add(i.getDistributionlocation()); - instance.getPid().addAll(i.getPid()); - instance.getAlternateIdentifier().addAll(i.getAlternateIdentifier()); - instance.getDateofacceptance().add(i.getDateofacceptance().getValue()); + instance.getRefereed().add(i.getRefereed()); instance .setProcessingchargeamount( Optional.ofNullable(i.getProcessingchargeamount()).map(apc -> apc.getValue()).orElse(null)); instance .setProcessingchargecurrency( Optional.ofNullable(i.getProcessingchargecurrency()).map(c -> c.getValue()).orElse(null)); - instance.getRefereed().add(i.getRefereed()); + Optional + .ofNullable(i.getPid()) + .ifPresent(pid -> instance.getPid().addAll(pid)); + Optional + .ofNullable(i.getAlternateIdentifier()) + .ifPresent(altId -> instance.getAlternateIdentifier().addAll(altId)); + Optional + .ofNullable(i.getDateofacceptance()) + .ifPresent(d -> instance.getDateofacceptance().add(d.getValue())); + Optional + .ofNullable(i.getLicense()) + .ifPresent(license -> instance.getLicense().add(license.getValue())); + Optional + .ofNullable(i.getDistributionlocation()) + .ifPresent(dl -> instance.getDistributionlocation().add(dl)); }); if (instance.getHostedby().size() > 1