mergin with branch beta

This commit is contained in:
Miriam Baglioni 2021-12-21 13:57:19 +01:00
commit e24a7f3496
41 changed files with 3134 additions and 338 deletions

View File

@ -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(

View File

@ -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)
}

View File

@ -0,0 +1,19 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

View File

@ -0,0 +1,62 @@
<workflow-app name="Retrieve Scholix Update" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path of scholix graph</description>
</property>
<property>
<name>datacitePath</name>
<description>the datacite native path</description>
</property>
<property>
<name>workingSupportPath</name>
<description>the working Support path</description>
</property>
<property>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>updateDS</name>
<value>false</value>
<description>The transformation Rule to apply</description>
</property>
</parameters>
<start to="RetrieveDeltaDatacite"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="RetrieveDeltaDatacite">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>New Update from Datacite to Scholix</name>
<class>eu.dnetlib.dhp.sx.graph.SparkRetrieveDataciteDelta</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--datacitePath</arg><arg>${datacitePath}</arg>
<arg>--master</arg><arg>yarn</arg>
<arg>--workingSupportPath</arg><arg>${workingSupportPath}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--updateDS</arg><arg>${updateDS}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -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
}
]

View File

@ -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()
}
}

View File

@ -48,6 +48,12 @@
<groupId>io.github.classgraph</groupId>
<artifactId>classgraph</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-aggregation</artifactId>
<version>1.2.4-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
</dependencies>

View File

@ -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<String> valueSet;
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
public ArrayList<String> getValueSet() {
return valueSet;
}
public void setValueSet(ArrayList<String> valueSet) {
this.valueSet = valueSet;
}
}

View File

@ -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<Relation> getOrganizationRelationPair(String orgId,
String resultId,
String classID,
String className
) {
ArrayList<Relation> 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,

View File

@ -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))

View File

@ -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<String> organizationSet;
public String getResultId() {
return resultId;
}
public void setResultId(String resultId) {
this.resultId = resultId;
}
public ArrayList<String> getOrganizationSet() {
return organizationSet;
}
public void setOrganizationSet(ArrayList<String> organizationSet) {
this.organizationSet = organizationSet;
}
}

View File

@ -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<DatasourceOrganization> dsOrg = readPath(spark, datasourceorganization, DatasourceOrganization.class);
Dataset<ResultOrganizationSet> potentialUpdates = getPotentialRelations(spark, inputPath, clazz, dsOrg);
Dataset<KeyValueSet> potentialUpdates = getPotentialRelations(spark, inputPath, clazz, dsOrg);
Dataset<ResultOrganizationSet> alreadyLinked = readPath(spark, alreadyLinkedPath, ResultOrganizationSet.class);
Dataset<KeyValueSet> 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<Tuple2<ResultOrganizationSet, ResultOrganizationSet>, Relation> createRelationFn() {
private static FlatMapFunction<Tuple2<KeyValueSet, KeyValueSet>, Relation> createRelationFn() {
return value -> {
List<Relation> newRelations = new ArrayList<>();
ResultOrganizationSet potentialUpdate = value._1();
Optional<ResultOrganizationSet> alreadyLinked = Optional.ofNullable(value._2());
List<String> organizations = potentialUpdate.getOrganizationSet();
KeyValueSet potentialUpdate = value._1();
Optional<KeyValueSet> alreadyLinked = Optional.ofNullable(value._2());
List<String> 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 <R extends Result> Dataset<ResultOrganizationSet> getPotentialRelations(
private static <R extends Result> Dataset<KeyValueSet> getPotentialRelations(
SparkSession spark,
String inputPath,
Class<R> resultClazz,
@ -179,7 +165,7 @@ public class SparkResultToOrganizationFromIstRepoJob {
return spark
.sql(RESULT_ORGANIZATIONSET_QUERY)
.as(Encoders.bean(ResultOrganizationSet.class));
.as(Encoders.bean(KeyValueSet.class));
}
}

View File

@ -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;
}
}

View File

@ -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> 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<Relation>) r -> !r.getDataInfo().getDeletedbyinference() &&
r.getRelClass().equals(ModelConstants.HAS_AUTHOR_INSTITUTION))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(relationPath);
Dataset<String> 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<String> 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<Tuple2<String, String>, String>) value -> {
if (Optional.ofNullable(value._2()).isPresent()) {
return null;
}
return value._1();
}, Encoders.STRING())
.filter(Objects::nonNull)
.write()
.mode(SaveMode.Overwrite)
.json(currentIterationPath);
}
}

View File

@ -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;
}
}

View File

@ -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> relation = readPath(spark, newRelationPath, Relation.class);
relation
.groupByKey((MapFunction<Relation, String>) r -> r.getSource() + r.getTarget(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Relation, Relation>) (k, it) -> it.next(), Encoders.bean(Relation.class))
.flatMap(
(FlatMapFunction<Relation, Relation>) 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);
}
}

View File

@ -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<Relation> 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<Relation> selectedRels, String outputPath) {
Dataset<KeyValueSet> resultOrg = readPath(spark, resultOrgPath, KeyValueSet.class);
resultOrg
.joinWith(
selectedRels, resultOrg
.col("key")
.equalTo(selectedRels.col("source")),
"left")
.groupByKey((MapFunction<Tuple2<KeyValueSet, Relation>, String>) mf -> mf._1().getKey(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, Tuple2<KeyValueSet, Relation>, KeyValueSet>) (key, it) -> {
Tuple2<KeyValueSet, Relation> first = it.next();
if (!Optional.ofNullable(first._2()).isPresent()) {
return first._1();
}
KeyValueSet ret = new KeyValueSet();
ret.setKey(first._1().getKey());
HashSet<String> hs = new HashSet<>();
hs.addAll(first._1().getValueSet());
hs.add(first._2().getTarget());
it.forEachRemaining(rel -> hs.add(rel._2().getTarget()));
ArrayList<String> 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<KeyValueSet> childParent = readPath(spark, chldParentOrgPath, KeyValueSet.class);
Dataset<Leaves> 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<Relation> relationDataset,
Dataset<Relation> 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<Relation, String>) r -> r.getSource() + r.getTarget(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, Relation, String>) (k, it) -> {
ArrayList<Relation> 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<String, Relation>) 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<Relation> getPropagationRelation(SparkSession spark,
String leavesPath,
String chldParentOrgPath,
String resultOrgPath) {
Dataset<KeyValueSet> childParent = readPath(spark, chldParentOrgPath, KeyValueSet.class);
Dataset<KeyValueSet> resultOrg = readPath(spark, resultOrgPath, KeyValueSet.class);
Dataset<Leaves> leaves = readPath(spark, leavesPath, Leaves.class);
childParent.createOrReplaceTempView("childParent");
resultOrg.createOrReplaceTempView("resultOrg");
leaves.createOrReplaceTempView("leaves");
Dataset<KeyValueSet> 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<KeyValueSet, Relation>) 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));
}
}

View File

@ -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
}
]

View File

@ -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
}
]

View File

@ -0,0 +1,58 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</value>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
</property>
<property>
<name>sparkExecutorNumber</name>
<value>4</value>
</property>
<property>
<name>sparkDriverMemory</name>
<value>15G</value>
</property>
<property>
<name>sparkExecutorMemory</name>
<value>6G</value>
</property>
<property>
<name>sparkExecutorCores</name>
<value>1</value>
</property>
<property>
<name>spark2MaxExecutors</name>
<value>50</value>
</property>
</configuration>

View File

@ -0,0 +1,193 @@
<workflow-app name="affiliation_from_instrepo_propagation" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
<property>
<name>outputPath</name>
<description>sets the outputPath</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="resume_from"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<decision name="resume_from">
<switch>
<case to="prepare_info">${wf:conf('resumeFrom') eq 'PrepareInfo'}</case>
<default to="reset_outputpath"/> <!-- first action to be done when downloadDump is to be performed -->
</switch>
</decision>
<action name="reset_outputpath">
<fs>
<delete path="${outputPath}"/>
<mkdir path="${outputPath}"/>
</fs>
<ok to="copy_entities"/>
<error to="Kill"/>
</action>
<fork name="copy_entities">
<path start="copy_relation"/>
<path start="copy_publication"/>
<path start="copy_dataset"/>
<path start="copy_orp"/>
<path start="copy_software"/>
<path start="copy_organization"/>
<path start="copy_projects"/>
<path start="copy_datasources"/>
</fork>
<action name="copy_relation">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/relation</arg>
<arg>${nameNode}/${outputPath}/relation</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_publication">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/publication</arg>
<arg>${nameNode}/${outputPath}/publication</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_dataset">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/dataset</arg>
<arg>${nameNode}/${outputPath}/dataset</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_orp">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/otherresearchproduct</arg>
<arg>${nameNode}/${outputPath}/otherresearchproduct</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_software">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/software</arg>
<arg>${nameNode}/${outputPath}/software</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_organization">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/organization</arg>
<arg>${nameNode}/${outputPath}/organization</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_projects">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/project</arg>
<arg>${nameNode}/${outputPath}/project</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_datasources">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/datasource</arg>
<arg>${nameNode}/${outputPath}/datasource</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<join name="wait" to="prepare_info"/>
<action name="prepare_info">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>PrepareResultOrganizationAssociation</name>
<class>eu.dnetlib.dhp.resulttoorganizationfromsemrel.PrepareInfo</class>
<jar>dhp-enrichment-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--graphPath</arg><arg>${sourcePath}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--leavesPath</arg><arg>${workingDir}/preparedInfo/leavesPath</arg>
<arg>--childParentPath</arg><arg>${workingDir}/preparedInfo/childParentPath</arg>
<arg>--resultOrgPath</arg><arg>${workingDir}/preparedInfo/resultOrgPath</arg>
<arg>--relationPath</arg><arg>${workingDir}/preparedInfo/relation</arg>
</spark>
<ok to="apply_resulttoorganization_propagation"/>
<error to="Kill"/>
</action>
<action name="apply_resulttoorganization_propagation">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>resultToOrganizationFromSemRel</name>
<class>eu.dnetlib.dhp.resulttoorganizationfromsemrel.SparkResultToOrganizationFromSemRel</class>
<jar>dhp-enrichment-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--relationPath</arg><arg>${workingDir}/preparedInfo/relation</arg>
<arg>--outputPath</arg><arg>${outputPath}/relation</arg>
<arg>--leavesPath</arg><arg>${workingDir}/preparedInfo/leavesPath</arg>
<arg>--childParentPath</arg><arg>${workingDir}/preparedInfo/childParentPath</arg>
<arg>--resultOrgPath</arg><arg>${workingDir}/preparedInfo/resultOrgPath</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--workingDir</arg><arg>${workingDir}/working</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -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<KeyValueSet> tmp = sc
.textFile(workingDir.toString() + "/childParentOrg/")
.map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class));
Dataset<KeyValueSet> 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<KeyValueSet>) 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<KeyValueSet> tmp = sc
.textFile(workingDir.toString() + "/childParentOrg/")
.map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class));
Dataset<KeyValueSet> 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<Relation> tmp = sc
.textFile(workingDir.toString() + "/relation")
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class));
Dataset<Relation> 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<KeyValueSet> tmp = sc
.textFile(workingDir.toString() + "/resultOrganization/")
.map(item -> OBJECT_MAPPER.readValue(item, KeyValueSet.class));
Dataset<KeyValueSet> 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<KeyValueSet>) 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<String> 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<Leaves> tmp = sc
.textFile(workingDir.toString() + "/currentIteration/")
.map(item -> OBJECT_MAPPER.readValue(item, Leaves.class));
Dataset<Leaves> 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<Leaves>) l -> System.out.println(OBJECT_MAPPER.writeValueAsString(l)));
}
}

View File

@ -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<Relation> 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"));
}
}

View File

@ -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<Relation> tmp = sc
.textFile(workingDir.toString() + "/newRelationPath")
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class));
Assertions.assertEquals(4, tmp.count());
Dataset<Relation> verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals("propagation", r.getDataInfo().getInferenceprovenance()));
verificationDs
.foreach((ForeachFunction<Relation>) r -> Assertions.assertEquals("0.85", r.getDataInfo().getTrust()));
verificationDs
.foreach((ForeachFunction<Relation>) r -> Assertions.assertEquals("50|", r.getSource().substring(0, 3)));
verificationDs
.foreach((ForeachFunction<Relation>) r -> Assertions.assertEquals("20|", r.getTarget().substring(0, 3)));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals(ModelConstants.HAS_AUTHOR_INSTITUTION, r.getRelClass()));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals(ModelConstants.RESULT_ORGANIZATION, r.getRelType()));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals(ModelConstants.AFFILIATION, r.getSubRelType()));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals(
PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_ID,
r.getDataInfo().getProvenanceaction().getClassid()));
verificationDs
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals(
PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_SEM_REL_CLASS_NAME,
r.getDataInfo().getProvenanceaction().getClassname()));
verificationDs
.filter(
(FilterFunction<Relation>) r -> r
.getSource()
.equals("50|doajarticles::1cae0b82b56ccd97c2db1f698def7074"))
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals("20|doajarticles::1cae0b82b56ccd97c2db1f698def7074", r.getTarget()));
verificationDs
.filter(
(FilterFunction<Relation>) r -> r
.getSource()
.equals("50|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"))
.foreach(
(ForeachFunction<Relation>) r -> Assertions
.assertEquals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f", r.getTarget()));
Assertions
.assertEquals(
2,
verificationDs
.filter(
(FilterFunction<Relation>) r -> r
.getSource()
.equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d"))
.count());
Assertions
.assertEquals(
1,
verificationDs
.filter(
(FilterFunction<Relation>) r -> r
.getSource()
.equals("50|dedup_wf_001::2899e571609779168222fdeb59cb916d") &&
r.getTarget().equals("20|doajarticles::2baa9032dc058d3c8ff780c426b0c19f"))
.count());
Assertions
.assertEquals(
1,
verificationDs
.filter(
(FilterFunction<Relation>) 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<Leaves> 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<KeyValueSet> 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"));
}
}

View File

@ -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"}

View File

@ -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"}

View File

@ -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"]}

View File

@ -0,0 +1,3 @@
{"value":"20|opendoar____::a5fcb8eb25ebd6f7cd219e0fa1e6ddc1"}
{"value":"20|doajarticles::396262ee936f3d3e26ff0e60bea6cae0"}
{"value":"20|pippo_wf_001::2899e571609779168222fdeb59cb916d"}

View File

@ -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"}

View File

@ -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":[]}

View File

@ -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"]}

View File

@ -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"}

View File

@ -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}
]

View File

@ -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 @@
<arg>--relationPath</arg><arg>${targetPath}/relation</arg>
</spark>
<ok to="DropJSONPath"/>
<error to="Kill"/>
</action>
<action name="DropJSONPath">
<fs>
<delete path='${targetPath}/index'/>
<mkdir path='${targetPath}/index/'/>
</fs>
<ok to="SerializeScholix"/>
<error to="Kill"/>
</action>
<action name="SerializeScholix">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Serialize scholix to JSON</name>
<class>eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--sourcePath</arg><arg>${targetPath}/provision/scholix/scholix</arg>
<arg>--targetPath</arg><arg>${targetPath}/index/scholix_json</arg>
<arg>--objectType</arg><arg>scholix</arg>
</spark>
<ok to="SerializeSummary"/>
<error to="Kill"/>
</action>
<action name="SerializeSummary">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Serialize summary to JSON</name>
<class>eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--sourcePath</arg><arg>${targetPath}/provision/summaries_filtered</arg>
<arg>--targetPath</arg><arg>${targetPath}/index/summaries_json</arg>
<arg>--objectType</arg><arg>summary</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>

View File

@ -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"
}
}

View File

@ -0,0 +1,10 @@
<configuration>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

View File

@ -0,0 +1,83 @@
<workflow-app name="Create Scholix final Graph" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>scholixUpdatePath</name>
<description>the working dir base path of the scholix updated</description>
</property>
<property>
<name>targetPath</name>
<description>the final graph path</description>
</property>
</parameters>
<start to="DropJSONPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="DropJSONPath">
<fs>
<delete path='${targetPath}/index'/>
<mkdir path='${targetPath}/index/'/>
</fs>
<ok to="SerializeScholix"/>
<error to="Kill"/>
</action>
<action name="SerializeScholix">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Serialize scholix to JSON</name>
<class>eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--sourcePath</arg><arg>${targetPath}/provision/scholix/scholix</arg>
<arg>--targetPath</arg><arg>${targetPath}/index/scholix_json</arg>
<arg>--scholixUpdatePath</arg><arg>${scholixUpdatePath}</arg>
<arg>--objectType</arg><arg>scholix</arg>
</spark>
<ok to="SerializeSummary"/>
<error to="Kill"/>
</action>
<action name="SerializeSummary">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Serialize summary to JSON</name>
<class>eu.dnetlib.dhp.sx.graph.SparkConvertObjectToJson</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--sourcePath</arg><arg>${targetPath}/provision/summaries_filtered</arg>
<arg>--targetPath</arg><arg>${targetPath}/index/summaries_json</arg>
<arg>--objectType</arg><arg>summary</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -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]

View File

@ -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")

View File

@ -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)
}
}

View File

@ -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<XmlInstance> groupInstancesByUrl(List<Instance> 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