dnet-hadoop/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkRetrieveDataciteDelta....

346 lines
13 KiB
Scala

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