diff --git a/.gitignore b/.gitignore
index 14cd4d345..6fafc7055 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,3 +27,4 @@ spark-warehouse
/**/.factorypath
/**/.scalafmt.conf
/.java-version
+/dhp-shade-package/dependency-reduced-pom.xml
diff --git a/dhp-common/pom.xml b/dhp-common/pom.xml
index 4173ce810..2c7a0ef8c 100644
--- a/dhp-common/pom.xml
+++ b/dhp-common/pom.xml
@@ -128,12 +128,6 @@
eu.dnetlib
cnr-rmi-api
-
-
- log4j
- log4j
-
-
diff --git a/dhp-common/src/main/scala/eu/dnetlib/dhp/application/SparkScalaApplication.scala b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/SparkScalaApplication.scala
index a14c25837..526bbd295 100644
--- a/dhp-common/src/main/scala/eu/dnetlib/dhp/application/SparkScalaApplication.scala
+++ b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/SparkScalaApplication.scala
@@ -65,12 +65,13 @@ abstract class AbstractScalaApplication(
val conf: SparkConf = new SparkConf()
val master = parser.get("master")
log.info(s"Creating Spark session: Master: $master")
- SparkSession
+ val b = SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
- .master(master)
- .getOrCreate()
+ if (master != null)
+ b.master(master)
+ b.getOrCreate()
}
def reportTotalSize(targetPath: String, outputBasePath: String): Unit = {
diff --git a/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala b/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala
index 0ee423569..72a17777e 100644
--- a/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala
+++ b/dhp-common/src/main/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixUtils.scala
@@ -389,7 +389,7 @@ object ScholixUtils extends Serializable {
if (persistentIdentifiers.isEmpty)
return null
s.setLocalIdentifier(persistentIdentifiers.asJava)
- //s.setTypology(r.getResulttype.getClassid)
+// s.setTypology(r.getResulttype.getClassid)
s.setSubType(r.getInstance().get(0).getInstancetype.getClassname)
diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java
deleted file mode 100644
index e69de29bb..000000000
diff --git a/dhp-shade-package/pom.xml b/dhp-shade-package/pom.xml
index 128a57116..d8e17ed46 100644
--- a/dhp-shade-package/pom.xml
+++ b/dhp-shade-package/pom.xml
@@ -31,31 +31,31 @@
dhp-actionmanager
${project.version}
-
- eu.dnetlib.dhp
- dhp-aggregation
- ${project.version}
-
-
- eu.dnetlib.dhp
- dhp-blacklist
- ${project.version}
-
-
- eu.dnetlib.dhp
- dhp-broker-events
- ${project.version}
-
-
- eu.dnetlib.dhp
- dhp-dedup-openaire
- ${project.version}
-
-
- eu.dnetlib.dhp
- dhp-enrichment
- ${project.version}
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
eu.dnetlib.dhp
dhp-graph-mapper
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/create_scholix_dump_params.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/create_scholix_dump_params.json
index fead58ab1..53fe95895 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/create_scholix_dump_params.json
+++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/create_scholix_dump_params.json
@@ -1,5 +1,5 @@
[
- {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
+ {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": false},
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true},
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the scholix dump", "paramRequired": true}
]
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/ScholexplorerUtils.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/ScholexplorerUtils.scala
index 95564d523..d171d96d9 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/ScholexplorerUtils.scala
+++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/ScholexplorerUtils.scala
@@ -1,5 +1,6 @@
package eu.dnetlib.dhp.sx.graph
+import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.{KeyValue, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.sx.scholix.{
Scholix,
@@ -28,6 +29,7 @@ case class RelKeyValue(key: String, value: String) {}
object ScholexplorerUtils {
val OPENAIRE_IDENTIFIER_SCHEMA: String = "OpenAIRE Identifier"
+ val mapper = new ObjectMapper()
case class RelationVocabulary(original: String, inverse: String) {}
@@ -242,7 +244,7 @@ object ScholexplorerUtils {
s
}
- def updateTarget(s: Scholix, t: ScholixResource): Scholix = {
+ def updateTarget(s: Scholix, t: ScholixResource): String = {
s.setTarget(t)
val spublishers: Seq[ScholixEntityId] =
@@ -251,6 +253,6 @@ object ScholexplorerUtils {
if (t.getPublisher != null && !t.getPublisher.isEmpty) t.getPublisher.asScala else List()
val mergedPublishers = spublishers.union(tpublishers).distinct.take(10).toList
s.setPublisher(mergedPublishers.asJava)
- s
+ mapper.writeValueAsString(s)
}
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateScholexplorerDump.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateScholexplorerDump.scala
index 1211dcc78..dd420ab95 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateScholexplorerDump.scala
+++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateScholexplorerDump.scala
@@ -11,7 +11,7 @@ import eu.dnetlib.dhp.schema.oaf.{
Dataset => OafDataset
}
import eu.dnetlib.dhp.schema.sx.scholix.{Scholix, ScholixResource}
-import org.apache.spark.sql.functions.{col, concat, expr, md5}
+import org.apache.spark.sql.functions.{col, concat, expr, first, md5}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
@@ -89,7 +89,13 @@ class SparkCreateScholexplorerDump(propertyPath: String, args: Array[String], lo
.withColumn("cf", expr("transform(collectedfrom, x -> struct(x.key, x.value))"))
.drop("collectedfrom")
.withColumnRenamed("cf", "collectedfrom")
- .distinct()
+ .groupBy(col("id"))
+ .agg(
+ first("source").alias("source"),
+ first("target").alias("target"),
+ first("relClass").alias("relClass"),
+ first("collectedfrom").alias("collectedfrom")
+ )
bidRel.write.mode(SaveMode.Overwrite).save(s"$otuputPath/relation")
@@ -97,7 +103,7 @@ class SparkCreateScholexplorerDump(propertyPath: String, args: Array[String], lo
def generateScholix(outputPath: String, spark: SparkSession): Unit = {
implicit val scholixResourceEncoder: Encoder[ScholixResource] = Encoders.bean(classOf[ScholixResource])
- implicit val scholixEncoder: Encoder[Scholix] = Encoders.bean(classOf[Scholix])
+ implicit val scholixEncoder: Encoder[Scholix] = Encoders.kryo(classOf[Scholix])
import spark.implicits._
val relations = spark.read.load(s"$outputPath/relation").as[RelationInfo]
@@ -106,18 +112,19 @@ class SparkCreateScholexplorerDump(propertyPath: String, args: Array[String], lo
val scholix_one_verse = relations
.joinWith(resource, relations("source") === resource("dnetIdentifier"), "inner")
.map(res => ScholexplorerUtils.generateScholix(res._1, res._2))
+ .map(s => (s.getIdentifier, s))(Encoders.tuple(Encoders.STRING, Encoders.kryo(classOf[Scholix])))
val resourceTarget = relations
.joinWith(resource, relations("target") === resource("dnetIdentifier"), "inner")
.map(res => (res._1.id, res._2))(Encoders.tuple(Encoders.STRING, Encoders.kryo(classOf[ScholixResource])))
scholix_one_verse
- .joinWith(resourceTarget, scholix_one_verse("identifier") === resourceTarget("_1"), "inner")
- .map(k => ScholexplorerUtils.updateTarget(k._1, k._2._2))
+ .joinWith(resourceTarget, scholix_one_verse("_1") === resourceTarget("_1"), "inner")
+ .map(k => ScholexplorerUtils.updateTarget(k._1._2, k._2._2))
.write
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
- .json(s"$outputPath/scholix")
+ .text(s"$outputPath/scholix")
}
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixGenerationTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixGenerationTest.scala
index 0a2872cb4..204fe9794 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixGenerationTest.scala
+++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/sx/graph/scholix/ScholixGenerationTest.scala
@@ -1,17 +1,26 @@
package eu.dnetlib.dhp.sx.graph.scholix
+import eu.dnetlib.dhp.schema.sx.scholix.ScholixResource
import eu.dnetlib.dhp.sx.graph.SparkCreateScholexplorerDump
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.junit.jupiter.api.Test
+import org.objenesis.strategy.StdInstantiatorStrategy
class ScholixGenerationTest {
@Test
def generateScholix(): Unit = {
+
val spark: SparkSession = SparkSession.builder().master("local[*]").getOrCreate()
val app = new SparkCreateScholexplorerDump(null, null, null)
-// app.generateScholixResource("/home/sandro/Downloads/scholix_sample/", "/home/sandro/Downloads/scholix/", spark)
-// app.generateBidirectionalRelations("/home/sandro/Downloads/scholix_sample/", "/home/sandro/Downloads/scholix/", spark)
+// app.generateScholixResource("/home/sandro/Downloads/scholix_sample/", "/home/sandro/Downloads/scholix/", spark)
+// app.generateBidirectionalRelations(
+// "/home/sandro/Downloads/scholix_sample/",
+// "/home/sandro/Downloads/scholix/",
+// spark
+// )
app.generateScholix("/home/sandro/Downloads/scholix/", spark)
+
}
}