forked from D-Net/dnet-hadoop
Compare commits
17 Commits
master
...
stable_ids
Author | SHA1 | Date |
---|---|---|
Miriam Baglioni | c26980f1c4 | |
Miriam Baglioni | 4f309e625c | |
Miriam Baglioni | 1ea66e8917 | |
Miriam Baglioni | 6e987fc084 | |
Miriam Baglioni | b0d86d32b0 | |
Miriam Baglioni | abe546e5ba | |
Miriam Baglioni | bf24f588e2 | |
Miriam Baglioni | 96255fa647 | |
Miriam Baglioni | 0e47e94099 | |
Miriam Baglioni | 434aa6380b | |
Miriam Baglioni | e0e80cde22 | |
Miriam Baglioni | 97e0c27db9 | |
Miriam Baglioni | 3ed90420e4 | |
Miriam Baglioni | 7498e63174 | |
Miriam Baglioni | 22ce947335 | |
Miriam Baglioni | f64f5d9e23 | |
Miriam Baglioni | 238d692a0a |
|
@ -98,7 +98,7 @@ public class GraphCleaningFunctions extends CleaningFunctions {
|
|||
|
||||
Result r = (Result) value;
|
||||
|
||||
if (Objects.isNull(r.getTitle()) || r.getTitle().isEmpty()) {
|
||||
if (Objects.nonNull(r.getTitle()) && r.getTitle().isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -4,6 +4,8 @@ import com.fasterxml.jackson.databind.ObjectMapper
|
|||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Instance, KeyValue, Oaf, OtherResearchProduct, Publication, Qualifier, Relation, Result, Software, StructuredProperty, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils}
|
||||
import eu.dnetlib.dhp.schema.oaf.{AccessRight, Author, DataInfo, Instance, KeyValue, Oaf, OtherResearchProduct, Publication, Qualifier, Relation, Result, Software, StructuredProperty, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
|
@ -17,16 +19,13 @@ import java.text.SimpleDateFormat
|
|||
import java.time.LocalDate
|
||||
import java.time.chrono.ThaiBuddhistDate
|
||||
import java.time.format.DateTimeFormatter
|
||||
import java.util.regex.Pattern
|
||||
import java.util.{Date, Locale}
|
||||
import java.util.regex.Pattern
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.io.{Codec, Source}
|
||||
import scala.language.postfixOps
|
||||
|
||||
case class DataciteType(doi: String, timestamp: Long, isActive: Boolean, json: String) {}
|
||||
|
||||
case class RelatedIdentifierType(relationType: String, relatedIdentifier: String, relatedIdentifierType: String) {}
|
||||
|
||||
case class NameIdentifiersType(nameIdentifierScheme: Option[String], schemeUri: Option[String], nameIdentifier: Option[String]) {}
|
||||
|
||||
case class CreatorType(nameType: Option[String], nameIdentifiers: Option[List[NameIdentifiersType]], name: Option[String], familyName: Option[String], givenName: Option[String], affiliation: Option[List[String]]) {}
|
||||
|
@ -45,37 +44,6 @@ case class HostedByMapType(openaire_id: String, datacite_name: String, official_
|
|||
|
||||
object DataciteToOAFTransformation {
|
||||
|
||||
val REL_TYPE_VALUE:String = "resultResult"
|
||||
val DATE_RELATION_KEY = "RelationDate"
|
||||
|
||||
val subRelTypeMapping: Map[String,(String,String)] = Map(
|
||||
"References" ->("IsReferencedBy","relationship"),
|
||||
"IsSupplementTo" ->("IsSupplementedBy","supplement"),
|
||||
"IsPartOf" ->("HasPart","part"),
|
||||
"HasPart" ->("IsPartOf","part"),
|
||||
"IsVersionOf" ->("HasVersion","version"),
|
||||
"HasVersion" ->("IsVersionOf","version"),
|
||||
"IsIdenticalTo" ->("IsIdenticalTo","relationship"),
|
||||
"IsPreviousVersionOf" ->("IsNewVersionOf","version"),
|
||||
"IsContinuedBy" ->("Continues","relationship"),
|
||||
"Continues" ->("IsContinuedBy","relationship"),
|
||||
"IsNewVersionOf" ->("IsPreviousVersionOf","version"),
|
||||
"IsSupplementedBy" ->("IsSupplementTo","supplement"),
|
||||
"IsDocumentedBy" ->("Documents","relationship"),
|
||||
"IsSourceOf" ->("IsDerivedFrom","relationship"),
|
||||
"Cites" ->("IsCitedBy","citation"),
|
||||
"IsCitedBy" ->("Cites","citation"),
|
||||
"IsDerivedFrom" ->("IsSourceOf","relationship"),
|
||||
"IsVariantFormOf" ->("IsDerivedFrom","version"),
|
||||
"IsReferencedBy" ->("References","relationship"),
|
||||
"IsObsoletedBy" ->("IsNewVersionOf","version"),
|
||||
"Reviews" ->("IsReviewedBy","review"),
|
||||
"Documents" ->("IsDocumentedBy","relationship"),
|
||||
"IsCompiledBy" ->("Compiles","relationship"),
|
||||
"Compiles" ->("IsCompiledBy","relationship"),
|
||||
"IsReviewedBy" ->("Reviews","review")
|
||||
)
|
||||
|
||||
implicit val codec: Codec = Codec("UTF-8")
|
||||
codec.onMalformedInput(CodingErrorAction.REPLACE)
|
||||
codec.onUnmappableCharacter(CodingErrorAction.REPLACE)
|
||||
|
@ -206,6 +174,7 @@ object DataciteToOAFTransformation {
|
|||
case _: Throwable => ""
|
||||
}
|
||||
}
|
||||
|
||||
def getTypeQualifier(resourceType: String, resourceTypeGeneral: String, schemaOrg: String, vocabularies: VocabularyGroup): (Qualifier, Qualifier) = {
|
||||
if (resourceType != null && resourceType.nonEmpty) {
|
||||
val typeQualifier = vocabularies.getSynonymAsQualifier(ModelConstants.DNET_PUBLICATION_RESOURCE, resourceType)
|
||||
|
@ -274,7 +243,6 @@ object DataciteToOAFTransformation {
|
|||
* As describe in ticket #6377
|
||||
* when the result come from figshare we need to remove subject
|
||||
* and set Access rights OPEN.
|
||||
*
|
||||
* @param r
|
||||
*/
|
||||
def fix_figshare(r: Result): Unit = {
|
||||
|
@ -291,12 +259,6 @@ object DataciteToOAFTransformation {
|
|||
|
||||
}
|
||||
|
||||
|
||||
def createDNetTargetIdentifier(pid: String, pidType: String, idPrefix: String): String = {
|
||||
val f_part = s"$idPrefix|${pidType.toLowerCase}".padTo(15, '_')
|
||||
s"$f_part::${IdentifierFactory.md5(pid.toLowerCase)}"
|
||||
}
|
||||
|
||||
def generateOAFDate(dt: String, q: Qualifier): StructuredProperty = {
|
||||
OafMapperUtils.structuredProperty(dt, q, null)
|
||||
}
|
||||
|
@ -335,7 +297,7 @@ object DataciteToOAFTransformation {
|
|||
}
|
||||
|
||||
|
||||
def generateOAF(input: String, ts: Long, dateOfCollection: Long, vocabularies: VocabularyGroup, exportLinks: Boolean): List[Oaf] = {
|
||||
def generateOAF(input: String, ts: Long, dateOfCollection: Long, vocabularies: VocabularyGroup): List[Oaf] = {
|
||||
if (filter_json(input))
|
||||
return List()
|
||||
|
||||
|
@ -452,7 +414,6 @@ object DataciteToOAFTransformation {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
result.setRelevantdate(dates.filter(d => d.date.isDefined && d.dateType.isDefined)
|
||||
.map(d => (extract_date(d.date.get), d.dateType.get))
|
||||
.filter(d => d._1.isDefined)
|
||||
|
@ -533,23 +494,10 @@ object DataciteToOAFTransformation {
|
|||
} yield awardUri
|
||||
|
||||
result.setId(IdentifierFactory.createIdentifier(result))
|
||||
var relations: List[Relation] = awardUris.flatMap(a => get_projectRelation(a, result.getId)).filter(r => r != null)
|
||||
|
||||
fix_figshare(result)
|
||||
|
||||
if (result.getId == null)
|
||||
return List()
|
||||
|
||||
if (exportLinks) {
|
||||
val rels: List[RelatedIdentifierType] = for {
|
||||
JObject(relIdentifier) <- json \\ "relatedIdentifiers"
|
||||
JField("relationType", JString(relationType)) <- relIdentifier
|
||||
JField("relatedIdentifierType", JString(relatedIdentifierType)) <- relIdentifier
|
||||
JField("relatedIdentifier", JString(relatedIdentifier)) <- relIdentifier
|
||||
} yield RelatedIdentifierType(relationType, relatedIdentifier, relatedIdentifierType)
|
||||
|
||||
relations = relations ::: generateRelations(rels,result.getId, if (i_date.isDefined && i_date.get.isDefined) i_date.get.get else null)
|
||||
}
|
||||
val relations: List[Relation] = awardUris.flatMap(a => get_projectRelation(a, result.getId)).filter(r => r != null)
|
||||
fix_figshare(result)
|
||||
if (relations != null && relations.nonEmpty) {
|
||||
List(result) ::: relations
|
||||
}
|
||||
|
@ -557,36 +505,6 @@ object DataciteToOAFTransformation {
|
|||
List(result)
|
||||
}
|
||||
|
||||
private def generateRelations(rels: List[RelatedIdentifierType], id:String, date:String):List[Relation] = {
|
||||
rels
|
||||
.filter(r =>
|
||||
subRelTypeMapping.contains(r.relationType) && (
|
||||
r.relatedIdentifierType.equalsIgnoreCase("doi") ||
|
||||
r.relatedIdentifierType.equalsIgnoreCase("pmid") ||
|
||||
r.relatedIdentifierType.equalsIgnoreCase("arxiv"))
|
||||
)
|
||||
.map(r => {
|
||||
val rel = new Relation
|
||||
rel.setCollectedfrom(List(DATACITE_COLLECTED_FROM).asJava)
|
||||
rel.setDataInfo(dataInfo)
|
||||
|
||||
val subRelType = subRelTypeMapping(r.relationType)._2
|
||||
rel.setRelType(REL_TYPE_VALUE)
|
||||
rel.setSubRelType(subRelType)
|
||||
rel.setRelClass(r.relationType)
|
||||
|
||||
val dateProps:KeyValue = OafMapperUtils.keyValue(DATE_RELATION_KEY, date)
|
||||
|
||||
rel.setProperties(List(dateProps).asJava)
|
||||
|
||||
rel.setSource(id)
|
||||
rel.setTarget(s"unresolved::${r.relatedIdentifier}::${r.relatedIdentifierType}")
|
||||
rel.setCollectedfrom(List(DATACITE_COLLECTED_FROM).asJava)
|
||||
rel.getCollectedfrom.asScala.map(c => c.getValue)(collection.breakOut)
|
||||
rel
|
||||
})(collection breakOut)
|
||||
}
|
||||
|
||||
def generateDataInfo(trust: String): DataInfo = {
|
||||
val di = new DataInfo
|
||||
di.setDeletedbyinference(false)
|
||||
|
|
|
@ -1,46 +0,0 @@
|
|||
package eu.dnetlib.dhp.actionmanager.datacite
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
||||
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.io.Source
|
||||
|
||||
object FilterCrossrefEntitiesSpark {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(getClass.getClass)
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf = new SparkConf
|
||||
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/filter_crossref_param.json")).mkString)
|
||||
parser.parseArgument(args)
|
||||
val master = parser.get("master")
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
log.info("sourcePath: {}", sourcePath)
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info("targetPath: {}", targetPath)
|
||||
|
||||
|
||||
|
||||
val spark: SparkSession = SparkSession.builder().config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(master)
|
||||
.getOrCreate()
|
||||
|
||||
|
||||
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||
implicit val resEncoder: Encoder[Result] = Encoders.kryo[Result]
|
||||
|
||||
val d:Dataset[Oaf]= spark.read.load(sourcePath).as[Oaf]
|
||||
|
||||
d.filter(r => r.isInstanceOf[Result]).map(r => r.asInstanceOf[Result]).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -22,7 +22,6 @@ object GenerateDataciteDatasetSpark {
|
|||
val master = parser.get("master")
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
val exportLinks = "true".equalsIgnoreCase(parser.get("exportLinks"))
|
||||
val isLookupUrl: String = parser.get("isLookupUrl")
|
||||
log.info("isLookupUrl: {}", isLookupUrl)
|
||||
|
||||
|
@ -41,7 +40,7 @@ object GenerateDataciteDatasetSpark {
|
|||
|
||||
spark.read.load(sourcePath).as[DataciteType]
|
||||
.filter(d => d.isActive)
|
||||
.flatMap(d => DataciteToOAFTransformation.generateOAF(d.json, d.timestamp, d.timestamp, vocabularies, exportLinks))
|
||||
.flatMap(d => DataciteToOAFTransformation.generateOAF(d.json, d.timestamp, d.timestamp, vocabularies))
|
||||
.filter(d => d != null)
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
}
|
||||
|
|
|
@ -23,12 +23,5 @@
|
|||
"paramLongName": "isLookupUrl",
|
||||
"paramDescription": "the isLookup URL",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "l",
|
||||
"paramLongName": "exportLinks",
|
||||
"paramDescription": "should export also links",
|
||||
"paramRequired": false
|
||||
}
|
||||
|
||||
]
|
|
@ -4,10 +4,6 @@
|
|||
<name>mainPath</name>
|
||||
<description>the working path of Datacite stores</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>oafTargetPath</name>
|
||||
<description>the target path where the OAF records are stored</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>isLookupUrl</name>
|
||||
<description>The IS lookUp service endopoint</description>
|
||||
|
@ -17,26 +13,15 @@
|
|||
<value>100</value>
|
||||
<description>The request block size</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>exportLinks</name>
|
||||
<value>false</value>
|
||||
<description>instructs the transformation phase to produce the links or not</description>
|
||||
</property>
|
||||
|
||||
</parameters>
|
||||
|
||||
<start to="resume_from"/>
|
||||
<start to="ImportDatacite"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<decision name="resume_from">
|
||||
<switch>
|
||||
<case to="TransformDatacite">${wf:conf('resumeFrom') eq 'TransformDatacite'}</case>
|
||||
<default to="ImportDatacite"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<action name="ImportDatacite">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
|
@ -60,11 +45,12 @@
|
|||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
<arg>--blocksize</arg><arg>${blocksize}</arg>
|
||||
</spark>
|
||||
<ok to="TransformDatacite"/>
|
||||
<ok to="TransformJob"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="TransformDatacite">
|
||||
|
||||
<action name="TransformJob">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
|
@ -82,9 +68,8 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${mainPath}/datacite_dump</arg>
|
||||
<arg>--targetPath</arg><arg>${oafTargetPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${mainPath}/datacite_oaf</arg>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--exportLinks</arg><arg>${exportLinks}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -1,84 +0,0 @@
|
|||
<workflow-app name="Generate_Datacite_and_Crossref_dump_for_Scholexplorer" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>datacitePath</name>
|
||||
<description>the path of Datacite spark dataset</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>isLookupUrl</name>
|
||||
<description>The IS lookUp service endopoint</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>crossrefPath</name>
|
||||
<description>the path of Crossref spark dataset</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>targetPath</name>
|
||||
<description>the path of Crossref spark dataset</description>
|
||||
</property>
|
||||
|
||||
</parameters>
|
||||
|
||||
<start to="ImportDatacite"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
|
||||
<action name="ImportDatacite">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ImportDatacite</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.datacite.GenerateDataciteDatasetSpark</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${datacitePath}</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}/datacite_oaf</arg>
|
||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||
<arg>--exportLinks</arg><arg>true</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="FilterCrossrefEntities"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="FilterCrossrefEntities">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>FilterCrossrefEntities</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.datacite.FilterCrossrefEntitiesSpark</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${crossrefPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}/crossref_oaf</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -1,15 +1,12 @@
|
|||
package eu.dnetlib.dhp.actionmanager.datacite
|
||||
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import com.fasterxml.jackson.databind.SerializationFeature
|
||||
|
||||
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||
import org.junit.jupiter.api.extension.ExtendWith
|
||||
import org.junit.jupiter.api.{BeforeEach, Test}
|
||||
import org.mockito.junit.jupiter.MockitoExtension
|
||||
|
||||
import org.codehaus.jackson.map.ObjectMapper
|
||||
import scala.io.Source
|
||||
|
||||
@ExtendWith(Array(classOf[MockitoExtension]))
|
||||
|
@ -28,15 +25,9 @@ class DataciteToOAFTest extends AbstractVocabularyTest{
|
|||
|
||||
|
||||
|
||||
val mapper = new ObjectMapper().enable(SerializationFeature.INDENT_OUTPUT)
|
||||
val res:List[Oaf] =DataciteToOAFTransformation.generateOAF(record, 0L,0L, vocabularies, true )
|
||||
|
||||
res.foreach(r => {
|
||||
println (mapper.writeValueAsString(r))
|
||||
println("----------------------------")
|
||||
|
||||
})
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
val res:List[Oaf] =DataciteToOAFTransformation.generateOAF(record, 0L,0L, vocabularies )
|
||||
println (mapper.defaultPrettyPrintingWriter().writeValueAsString(res.head))
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -6,13 +6,13 @@ import static org.mockito.Mockito.lenient;
|
|||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.mockito.Mock;
|
||||
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.transformation.TransformationFactory;
|
||||
import eu.dnetlib.dhp.transformation.TransformationJobTest;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
|
||||
|
@ -34,22 +34,16 @@ public abstract class AbstractVocabularyTest {
|
|||
|
||||
private static List<String> vocs() throws IOException {
|
||||
return IOUtils
|
||||
.readLines(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
AbstractVocabularyTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/terms.txt")));
|
||||
.readLines(TransformationJobTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/terms.txt"));
|
||||
}
|
||||
|
||||
private static List<String> synonyms() throws IOException {
|
||||
return IOUtils
|
||||
.readLines(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
AbstractVocabularyTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/synonyms.txt")));
|
||||
.readLines(TransformationJobTest.class.getResourceAsStream("/eu/dnetlib/dhp/transform/synonyms.txt"));
|
||||
}
|
||||
|
||||
protected void mockupTrasformationRule(final String trule, final String path) throws Exception {
|
||||
final String trValue = IOUtils.toString(Objects.requireNonNull(this.getClass().getResourceAsStream(path)));
|
||||
final String trValue = IOUtils.toString(this.getClass().getResourceAsStream(path));
|
||||
|
||||
lenient()
|
||||
.when(isLookUpService.quickSearchProfile(String.format(TransformationFactory.TRULE_XQUERY, trule)))
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -128,7 +128,6 @@ dnet:publication_resource @=@ 0002 @=@ scientific book
|
|||
dnet:publication_resource @=@ 0002 @=@ Монография
|
||||
dnet:publication_resource @=@ 0002 @=@ Учебник
|
||||
dnet:publication_resource @=@ 0037 @=@ clinicalTrial
|
||||
dnet:publication_resource @=@ 0037 @=@ Clinical Trial
|
||||
dnet:publication_resource @=@ 0037 @=@ http://purl.org/coar/resource_type/c_cb28
|
||||
dnet:publication_resource @=@ 0022 @=@ collection
|
||||
dnet:publication_resource @=@ 0004 @=@ A4 Artikkeli konferenssijulkaisussa
|
||||
|
|
|
@ -0,0 +1,82 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>dhp-workflows</artifactId>
|
||||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<version>1.2.4-SNAPSHOT</version>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<artifactId>dhp-dedup-scholexplorer</artifactId>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>net.alchim31.maven</groupId>
|
||||
<artifactId>scala-maven-plugin</artifactId>
|
||||
<version>4.0.1</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>scala-compile-first</id>
|
||||
<phase>initialize</phase>
|
||||
<goals>
|
||||
<goal>add-source</goal>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
<execution>
|
||||
<id>scala-test-compile</id>
|
||||
<phase>process-test-resources</phase>
|
||||
<goals>
|
||||
<goal>testCompile</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<scalaVersion>${scala.version}</scalaVersion>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core_2.11</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-sql_2.11</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib.dhp</groupId>
|
||||
<artifactId>dhp-common</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>eu.dnetlib</groupId>
|
||||
<artifactId>dnet-pace-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-graphx_2.11</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
</dependency>
|
||||
|
||||
|
||||
|
||||
</dependencies>
|
||||
|
||||
|
||||
</project>
|
|
@ -0,0 +1,121 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import static java.util.Collections.reverseOrder;
|
||||
import static java.util.Map.Entry.comparingByValue;
|
||||
import static java.util.stream.Collectors.toMap;
|
||||
|
||||
import static org.apache.commons.lang.StringUtils.endsWith;
|
||||
import static org.apache.commons.lang.StringUtils.substringBefore;
|
||||
|
||||
import java.time.Year;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Field;
|
||||
|
||||
public class DatePicker {
|
||||
|
||||
private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}";
|
||||
private static final String DATE_DEFAULT_SUFFIX = "01-01";
|
||||
private static final int YEAR_LB = 1300;
|
||||
private static final int YEAR_UB = Year.now().getValue() + 5;
|
||||
|
||||
public static Field<String> pick(final Collection<String> dateofacceptance) {
|
||||
|
||||
final Map<String, Integer> frequencies = dateofacceptance
|
||||
.parallelStream()
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.collect(Collectors.toConcurrentMap(w -> w, w -> 1, Integer::sum));
|
||||
|
||||
if (frequencies.isEmpty()) {
|
||||
return new Field<>();
|
||||
}
|
||||
|
||||
final Field<String> date = new Field<>();
|
||||
date.setValue(frequencies.keySet().iterator().next());
|
||||
|
||||
// let's sort this map by values first, filtering out invalid dates
|
||||
final Map<String, Integer> sorted = frequencies
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(d -> StringUtils.isNotBlank(d.getKey()))
|
||||
.filter(d -> d.getKey().matches(DATE_PATTERN))
|
||||
.filter(d -> inRange(d.getKey()))
|
||||
.sorted(reverseOrder(comparingByValue()))
|
||||
.collect(
|
||||
toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2, LinkedHashMap::new));
|
||||
|
||||
// shortcut
|
||||
if (sorted.size() == 0) {
|
||||
return date;
|
||||
}
|
||||
|
||||
// voting method (1/3 + 1) wins
|
||||
if (sorted.size() >= 3) {
|
||||
final int acceptThreshold = (sorted.size() / 3) + 1;
|
||||
final List<String> accepted = sorted
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> e.getValue() >= acceptThreshold)
|
||||
.map(e -> e.getKey())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// cannot find strong majority
|
||||
if (accepted.isEmpty()) {
|
||||
final int max = sorted.values().iterator().next();
|
||||
Optional<String> first = sorted
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX))
|
||||
.map(Map.Entry::getKey)
|
||||
.findFirst();
|
||||
if (first.isPresent()) {
|
||||
date.setValue(first.get());
|
||||
return date;
|
||||
}
|
||||
|
||||
date.setValue(sorted.keySet().iterator().next());
|
||||
return date;
|
||||
}
|
||||
|
||||
if (accepted.size() == 1) {
|
||||
date.setValue(accepted.get(0));
|
||||
return date;
|
||||
} else {
|
||||
final Optional<String> first = accepted
|
||||
.stream()
|
||||
.filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX))
|
||||
.findFirst();
|
||||
if (first.isPresent()) {
|
||||
date.setValue(first.get());
|
||||
return date;
|
||||
}
|
||||
|
||||
return date;
|
||||
}
|
||||
|
||||
// 1st non YYYY-01-01 is returned
|
||||
} else {
|
||||
if (sorted.size() == 2) {
|
||||
for (Map.Entry<String, Integer> e : sorted.entrySet()) {
|
||||
if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) {
|
||||
date.setValue(e.getKey());
|
||||
return date;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// none of the dates seems good enough, return the 1st one
|
||||
date.setValue(sorted.keySet().iterator().next());
|
||||
return date;
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean inRange(final String date) {
|
||||
final int year = Integer.parseInt(substringBefore(date, "-"));
|
||||
return year >= YEAR_LB && year <= YEAR_UB;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,327 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class DedupRecordFactory {
|
||||
|
||||
public static JavaRDD<OafEntity> createDedupRecord(
|
||||
final JavaSparkContext sc,
|
||||
final SparkSession spark,
|
||||
final String mergeRelsInputPath,
|
||||
final String entitiesInputPath,
|
||||
final OafEntityType entityType,
|
||||
final DedupConfig dedupConf) {
|
||||
long ts = System.currentTimeMillis();
|
||||
// <id, json_entity>
|
||||
final JavaPairRDD<String, String> inputJsonEntities = spark
|
||||
.read()
|
||||
.load(entitiesInputPath)
|
||||
.as(Encoders.kryo(Oaf.class))
|
||||
.map(
|
||||
(MapFunction<Oaf, String>) p -> new org.codehaus.jackson.map.ObjectMapper().writeValueAsString(p),
|
||||
Encoders.STRING())
|
||||
.javaRDD()
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, String>) it -> new Tuple2<>(
|
||||
MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it));
|
||||
|
||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||
JavaPairRDD<String, String> mergeRels = spark
|
||||
.read()
|
||||
.load(mergeRelsInputPath)
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.where("relClass=='merges'")
|
||||
.javaRDD()
|
||||
.mapToPair(
|
||||
(PairFunction<Relation, String, String>) r -> new Tuple2<String, String>(r.getTarget(), r.getSource()));
|
||||
|
||||
// <dedup_id, json_entity_merged>
|
||||
final JavaPairRDD<String, String> joinResult = mergeRels
|
||||
.join(inputJsonEntities)
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
|
||||
|
||||
JavaPairRDD<String, Iterable<String>> sortedJoinResult = joinResult.groupByKey();
|
||||
|
||||
switch (entityType) {
|
||||
case publication:
|
||||
return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts));
|
||||
case dataset:
|
||||
return sortedJoinResult.map(d -> DedupRecordFactory.datasetMerger(d, ts));
|
||||
case project:
|
||||
return sortedJoinResult.map(p -> DedupRecordFactory.projectMerger(p, ts));
|
||||
case software:
|
||||
return sortedJoinResult.map(s -> DedupRecordFactory.softwareMerger(s, ts));
|
||||
case datasource:
|
||||
return sortedJoinResult.map(d -> DedupRecordFactory.datasourceMerger(d, ts));
|
||||
case organization:
|
||||
return sortedJoinResult.map(o -> DedupRecordFactory.organizationMerger(o, ts));
|
||||
case otherresearchproduct:
|
||||
return sortedJoinResult.map(o -> DedupRecordFactory.otherresearchproductMerger(o, ts));
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static DLIPublication publicationMerger(Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
DLIPublication p = new DLIPublication(); // the result of the merge, to be returned at the end
|
||||
|
||||
p.setId(e._1());
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
final Collection<String> dateofacceptance = Lists.newArrayList();
|
||||
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
pub -> {
|
||||
try {
|
||||
DLIPublication publication = mapper.readValue(pub, DLIPublication.class);
|
||||
|
||||
p.mergeFrom(publication);
|
||||
p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor()));
|
||||
// add to the list if they are not null
|
||||
if (publication.getDateofacceptance() != null)
|
||||
dateofacceptance.add(publication.getDateofacceptance().getValue());
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
p.setDateofacceptance(DatePicker.pick(dateofacceptance));
|
||||
if (p.getDataInfo() == null)
|
||||
p.setDataInfo(new DataInfo());
|
||||
p.getDataInfo().setTrust("0.9");
|
||||
p.setLastupdatetimestamp(ts);
|
||||
return p;
|
||||
}
|
||||
|
||||
private static DLIDataset datasetMerger(Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
DLIDataset d = new DLIDataset(); // the result of the merge, to be returned at the end
|
||||
|
||||
d.setId(e._1());
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
final Collection<String> dateofacceptance = Lists.newArrayList();
|
||||
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
dat -> {
|
||||
try {
|
||||
Dataset dataset = mapper.readValue(dat, Dataset.class);
|
||||
|
||||
d.mergeFrom(dataset);
|
||||
d.setAuthor(DedupUtility.mergeAuthor(d.getAuthor(), dataset.getAuthor()));
|
||||
// add to the list if they are not null
|
||||
if (dataset.getDateofacceptance() != null)
|
||||
dateofacceptance.add(dataset.getDateofacceptance().getValue());
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
d.setDateofacceptance(DatePicker.pick(dateofacceptance));
|
||||
if (d.getDataInfo() == null)
|
||||
d.setDataInfo(new DataInfo());
|
||||
d.getDataInfo().setTrust("0.9");
|
||||
d.setLastupdatetimestamp(ts);
|
||||
return d;
|
||||
}
|
||||
|
||||
private static Project projectMerger(Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
Project p = new Project(); // the result of the merge, to be returned at the end
|
||||
|
||||
p.setId(e._1());
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
proj -> {
|
||||
try {
|
||||
Project project = mapper.readValue(proj, Project.class);
|
||||
|
||||
p.mergeFrom(project);
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
if (p.getDataInfo() == null)
|
||||
p.setDataInfo(new DataInfo());
|
||||
p.getDataInfo().setTrust("0.9");
|
||||
p.setLastupdatetimestamp(ts);
|
||||
return p;
|
||||
}
|
||||
|
||||
private static Software softwareMerger(Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
Software s = new Software(); // the result of the merge, to be returned at the end
|
||||
|
||||
s.setId(e._1());
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
final Collection<String> dateofacceptance = Lists.newArrayList();
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
soft -> {
|
||||
try {
|
||||
Software software = mapper.readValue(soft, Software.class);
|
||||
|
||||
s.mergeFrom(software);
|
||||
s.setAuthor(DedupUtility.mergeAuthor(s.getAuthor(), software.getAuthor()));
|
||||
// add to the list if they are not null
|
||||
if (software.getDateofacceptance() != null)
|
||||
dateofacceptance.add(software.getDateofacceptance().getValue());
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
s.setDateofacceptance(DatePicker.pick(dateofacceptance));
|
||||
if (s.getDataInfo() == null)
|
||||
s.setDataInfo(new DataInfo());
|
||||
s.getDataInfo().setTrust("0.9");
|
||||
s.setLastupdatetimestamp(ts);
|
||||
return s;
|
||||
}
|
||||
|
||||
private static Datasource datasourceMerger(Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
Datasource d = new Datasource(); // the result of the merge, to be returned at the end
|
||||
d.setId(e._1());
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
dat -> {
|
||||
try {
|
||||
Datasource datasource = mapper.readValue(dat, Datasource.class);
|
||||
|
||||
d.mergeFrom(datasource);
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
if (d.getDataInfo() == null)
|
||||
d.setDataInfo(new DataInfo());
|
||||
d.getDataInfo().setTrust("0.9");
|
||||
d.setLastupdatetimestamp(ts);
|
||||
return d;
|
||||
}
|
||||
|
||||
private static Organization organizationMerger(
|
||||
Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
Organization o = new Organization(); // the result of the merge, to be returned at the end
|
||||
|
||||
o.setId(e._1());
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
StringBuilder trust = new StringBuilder("0.0");
|
||||
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
pub -> {
|
||||
try {
|
||||
Organization organization = mapper.readValue(pub, Organization.class);
|
||||
|
||||
final String currentTrust = organization.getDataInfo().getTrust();
|
||||
if (!"1.0".equals(currentTrust)) {
|
||||
trust.setLength(0);
|
||||
trust.append(currentTrust);
|
||||
}
|
||||
o.mergeFrom(organization);
|
||||
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
|
||||
if (o.getDataInfo() == null) {
|
||||
o.setDataInfo(new DataInfo());
|
||||
}
|
||||
if (o.getDataInfo() == null)
|
||||
o.setDataInfo(new DataInfo());
|
||||
o.getDataInfo().setTrust("0.9");
|
||||
o.setLastupdatetimestamp(ts);
|
||||
|
||||
return o;
|
||||
}
|
||||
|
||||
private static OtherResearchProduct otherresearchproductMerger(
|
||||
Tuple2<String, Iterable<String>> e, final long ts) {
|
||||
|
||||
OtherResearchProduct o = new OtherResearchProduct(); // the result of the merge, to be
|
||||
// returned at the end
|
||||
|
||||
o.setId(e._1());
|
||||
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
final Collection<String> dateofacceptance = Lists.newArrayList();
|
||||
|
||||
if (e._2() != null)
|
||||
e
|
||||
._2()
|
||||
.forEach(
|
||||
orp -> {
|
||||
try {
|
||||
OtherResearchProduct otherResearchProduct = mapper
|
||||
.readValue(orp, OtherResearchProduct.class);
|
||||
|
||||
o.mergeFrom(otherResearchProduct);
|
||||
o
|
||||
.setAuthor(
|
||||
DedupUtility.mergeAuthor(o.getAuthor(), otherResearchProduct.getAuthor()));
|
||||
// add to the list if they are not null
|
||||
if (otherResearchProduct.getDateofacceptance() != null)
|
||||
dateofacceptance.add(otherResearchProduct.getDateofacceptance().getValue());
|
||||
} catch (Exception exc) {
|
||||
throw new RuntimeException(exc);
|
||||
}
|
||||
});
|
||||
if (o.getDataInfo() == null)
|
||||
o.setDataInfo(new DataInfo());
|
||||
o.setDateofacceptance(DatePicker.pick(dateofacceptance));
|
||||
o.getDataInfo().setTrust("0.9");
|
||||
o.setLastupdatetimestamp(ts);
|
||||
return o;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,239 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.security.MessageDigest;
|
||||
import java.text.Normalizer;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import com.wcohen.ss.JaroWinkler;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.model.Person;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class DedupUtility {
|
||||
private static final Double THRESHOLD = 0.95;
|
||||
|
||||
public static Map<String, LongAccumulator> constructAccumulator(
|
||||
final DedupConfig dedupConf, final SparkContext context) {
|
||||
|
||||
Map<String, LongAccumulator> accumulators = new HashMap<>();
|
||||
|
||||
String acc1 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "records per hash key = 1");
|
||||
accumulators.put(acc1, context.longAccumulator(acc1));
|
||||
String acc2 = String
|
||||
.format(
|
||||
"%s::%s",
|
||||
dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField());
|
||||
accumulators.put(acc2, context.longAccumulator(acc2));
|
||||
String acc3 = String
|
||||
.format(
|
||||
"%s::%s",
|
||||
dedupConf.getWf().getEntityType(),
|
||||
String
|
||||
.format(
|
||||
"Skipped records for count(%s) >= %s",
|
||||
dedupConf.getWf().getOrderField(), dedupConf.getWf().getGroupMaxSize()));
|
||||
accumulators.put(acc3, context.longAccumulator(acc3));
|
||||
String acc4 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "skip list");
|
||||
accumulators.put(acc4, context.longAccumulator(acc4));
|
||||
String acc5 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)");
|
||||
accumulators.put(acc5, context.longAccumulator(acc5));
|
||||
String acc6 = String
|
||||
.format(
|
||||
"%s::%s", dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold());
|
||||
accumulators.put(acc6, context.longAccumulator(acc6));
|
||||
|
||||
return accumulators;
|
||||
}
|
||||
|
||||
public static JavaRDD<String> loadDataFromHDFS(String path, JavaSparkContext context) {
|
||||
return context.textFile(path);
|
||||
}
|
||||
|
||||
public static void deleteIfExists(String path) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fileSystem = FileSystem.get(conf);
|
||||
if (fileSystem.exists(new Path(path))) {
|
||||
fileSystem.delete(new Path(path), true);
|
||||
}
|
||||
}
|
||||
|
||||
public static DedupConfig loadConfigFromHDFS(String path) throws IOException {
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fileSystem = FileSystem.get(conf);
|
||||
FSDataInputStream inputStream = new FSDataInputStream(fileSystem.open(new Path(path)));
|
||||
|
||||
return DedupConfig.load(IOUtils.toString(inputStream, StandardCharsets.UTF_8.name()));
|
||||
}
|
||||
|
||||
static <T> String readFromClasspath(final String filename, final Class<T> clazz) {
|
||||
final StringWriter sw = new StringWriter();
|
||||
try {
|
||||
IOUtils.copy(clazz.getResourceAsStream(filename), sw);
|
||||
return sw.toString();
|
||||
} catch (final IOException e) {
|
||||
throw new RuntimeException("cannot load resource from classpath: " + filename);
|
||||
}
|
||||
}
|
||||
|
||||
static Set<String> getGroupingKeys(DedupConfig conf, MapDocument doc) {
|
||||
return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf));
|
||||
}
|
||||
|
||||
public static String md5(final String s) {
|
||||
try {
|
||||
final MessageDigest md = MessageDigest.getInstance("MD5");
|
||||
md.update(s.getBytes(StandardCharsets.UTF_8));
|
||||
return new String(Hex.encodeHex(md.digest()));
|
||||
} catch (final Exception e) {
|
||||
System.err.println("Error creating id");
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static List<Author> mergeAuthor(final List<Author> a, final List<Author> b) {
|
||||
int pa = countAuthorsPids(a);
|
||||
int pb = countAuthorsPids(b);
|
||||
List<Author> base, enrich;
|
||||
int sa = authorsSize(a);
|
||||
int sb = authorsSize(b);
|
||||
|
||||
if (pa == pb) {
|
||||
base = sa > sb ? a : b;
|
||||
enrich = sa > sb ? b : a;
|
||||
} else {
|
||||
base = pa > pb ? a : b;
|
||||
enrich = pa > pb ? b : a;
|
||||
}
|
||||
enrichPidFromList(base, enrich);
|
||||
return base;
|
||||
}
|
||||
|
||||
private static void enrichPidFromList(List<Author> base, List<Author> enrich) {
|
||||
if (base == null || enrich == null)
|
||||
return;
|
||||
final Map<String, Author> basePidAuthorMap = base
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null && a.getPid().size() > 0)
|
||||
.flatMap(a -> a.getPid().stream().map(p -> new Tuple2<>(p.toComparableString(), a)))
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1));
|
||||
|
||||
final List<Tuple2<StructuredProperty, Author>> pidToEnrich = enrich
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null && a.getPid().size() > 0)
|
||||
.flatMap(
|
||||
a -> a
|
||||
.getPid()
|
||||
.stream()
|
||||
.filter(p -> !basePidAuthorMap.containsKey(p.toComparableString()))
|
||||
.map(p -> new Tuple2<>(p, a)))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
pidToEnrich
|
||||
.forEach(
|
||||
a -> {
|
||||
Optional<Tuple2<Double, Author>> simAuhtor = base
|
||||
.stream()
|
||||
.map(ba -> new Tuple2<>(sim(ba, a._2()), ba))
|
||||
.max(Comparator.comparing(Tuple2::_1));
|
||||
if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) {
|
||||
Author r = simAuhtor.get()._2();
|
||||
r.getPid().add(a._1());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static String createEntityPath(final String basePath, final String entityType) {
|
||||
return String.format("%s/%s", basePath, entityType);
|
||||
}
|
||||
|
||||
public static String createSimRelPath(final String basePath, final String entityType) {
|
||||
return String.format("%s/%s/simRel", basePath, entityType);
|
||||
}
|
||||
|
||||
public static String createMergeRelPath(final String basePath, final String entityType) {
|
||||
return String.format("%s/%s/mergeRel", basePath, entityType);
|
||||
}
|
||||
|
||||
private static Double sim(Author a, Author b) {
|
||||
|
||||
final Person pa = parse(a);
|
||||
final Person pb = parse(b);
|
||||
|
||||
if (pa.isAccurate() & pb.isAccurate()) {
|
||||
return new JaroWinkler()
|
||||
.score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString()));
|
||||
} else {
|
||||
return new JaroWinkler()
|
||||
.score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname()));
|
||||
}
|
||||
}
|
||||
|
||||
private static String normalize(final String s) {
|
||||
return nfd(s)
|
||||
.toLowerCase()
|
||||
// do not compact the regexes in a single expression, would cause StackOverflowError
|
||||
// in case
|
||||
// of large input strings
|
||||
.replaceAll("(\\W)+", " ")
|
||||
.replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ")
|
||||
.replaceAll("(\\p{Punct})+", " ")
|
||||
.replaceAll("(\\d)+", " ")
|
||||
.replaceAll("(\\n)+", " ")
|
||||
.trim();
|
||||
}
|
||||
|
||||
private static String nfd(final String s) {
|
||||
return Normalizer.normalize(s, Normalizer.Form.NFD);
|
||||
}
|
||||
|
||||
private static Person parse(Author author) {
|
||||
if (StringUtils.isNotBlank(author.getSurname())) {
|
||||
return new Person(author.getSurname() + ", " + author.getName(), false);
|
||||
} else {
|
||||
return new Person(author.getFullname(), false);
|
||||
}
|
||||
}
|
||||
|
||||
private static int countAuthorsPids(List<Author> authors) {
|
||||
if (authors == null)
|
||||
return 0;
|
||||
|
||||
return (int) authors.stream().filter(DedupUtility::hasPid).count();
|
||||
}
|
||||
|
||||
private static int authorsSize(List<Author> authors) {
|
||||
if (authors == null)
|
||||
return 0;
|
||||
return authors.size();
|
||||
}
|
||||
|
||||
private static boolean hasPid(Author a) {
|
||||
if (a == null || a.getPid() == null || a.getPid().size() == 0)
|
||||
return false;
|
||||
return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,182 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.util.BlockProcessor;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Serializable;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class Deduper implements Serializable {
|
||||
|
||||
private static final Log log = LogFactory.getLog(Deduper.class);
|
||||
|
||||
/**
|
||||
* @return the list of relations generated by the deduplication
|
||||
* @param: the spark context
|
||||
* @param: list of JSON entities to be deduped
|
||||
* @param: the dedup configuration
|
||||
*/
|
||||
public static JavaPairRDD<String, String> dedup(
|
||||
JavaSparkContext context, JavaRDD<String> entities, DedupConfig config) {
|
||||
|
||||
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
|
||||
|
||||
// create vertexes of the graph: <ID, MapDocument>
|
||||
JavaPairRDD<String, MapDocument> mapDocs = mapToVertexes(context, entities, config);
|
||||
|
||||
// create blocks for deduplication
|
||||
JavaPairRDD<String, Iterable<MapDocument>> blocks = createBlocks(context, mapDocs, config);
|
||||
|
||||
// create relations by comparing only elements in the same group
|
||||
return computeRelations(context, blocks, config);
|
||||
|
||||
// final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new
|
||||
// Edge<>(it._1().hashCode(),
|
||||
// it._2().hashCode(), "equalTo")).rdd();
|
||||
//
|
||||
// RDD<Tuple2<Object, MapDocument>> vertexes =
|
||||
// mapDocs.mapToPair((PairFunction<Tuple2<String, MapDocument>, Object, MapDocument>) t ->
|
||||
// new
|
||||
// Tuple2<Object, MapDocument>((long) t._1().hashCode(), t._2())).rdd();
|
||||
// accumulators.forEach((name, acc) -> log.info(name + " -> " + acc.value()));
|
||||
//
|
||||
// return GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the list of relations generated by the deduplication
|
||||
* @param: the spark context
|
||||
* @param: list of blocks
|
||||
* @param: the dedup configuration
|
||||
*/
|
||||
public static JavaPairRDD<String, String> computeRelations(
|
||||
JavaSparkContext context,
|
||||
JavaPairRDD<String, Iterable<MapDocument>> blocks,
|
||||
DedupConfig config) {
|
||||
|
||||
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
|
||||
|
||||
return blocks
|
||||
.flatMapToPair(
|
||||
(PairFlatMapFunction<Tuple2<String, Iterable<MapDocument>>, String, String>) it -> {
|
||||
final SparkReporter reporter = new SparkReporter(accumulators);
|
||||
new BlockProcessor(config).process(it._1(), it._2(), reporter);
|
||||
return reporter.getRelations().iterator();
|
||||
})
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, String>, String, Tuple2<String, String>>) item -> new Tuple2<String, Tuple2<String, String>>(
|
||||
item._1() + item._2(), item))
|
||||
.reduceByKey((a, b) -> a)
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the list of blocks based on clustering of dedup configuration
|
||||
* @param: the spark context
|
||||
* @param: list of entities: <id, entity>
|
||||
* @param: the dedup configuration
|
||||
*/
|
||||
public static JavaPairRDD<String, Iterable<MapDocument>> createBlocks(
|
||||
JavaSparkContext context, JavaPairRDD<String, MapDocument> mapDocs, DedupConfig config) {
|
||||
return mapDocs
|
||||
// the reduce is just to be sure that we haven't document with same id
|
||||
.reduceByKey((a, b) -> a)
|
||||
.map(Tuple2::_2)
|
||||
// Clustering: from <id, doc> to List<groupkey,doc>
|
||||
.flatMapToPair(
|
||||
(PairFlatMapFunction<MapDocument, String, MapDocument>) a -> DedupUtility
|
||||
.getGroupingKeys(config, a)
|
||||
.stream()
|
||||
.map(it -> new Tuple2<>(it, a))
|
||||
.collect(Collectors.toList())
|
||||
.iterator())
|
||||
.groupByKey();
|
||||
}
|
||||
|
||||
public static JavaPairRDD<String, List<MapDocument>> createsortedBlocks(
|
||||
JavaSparkContext context, JavaPairRDD<String, MapDocument> mapDocs, DedupConfig config) {
|
||||
final String of = config.getWf().getOrderField();
|
||||
final int maxQueueSize = config.getWf().getGroupMaxSize();
|
||||
return mapDocs
|
||||
// the reduce is just to be sure that we haven't document with same id
|
||||
.reduceByKey((a, b) -> a)
|
||||
.map(Tuple2::_2)
|
||||
// Clustering: from <id, doc> to List<groupkey,doc>
|
||||
.flatMapToPair(
|
||||
(PairFlatMapFunction<MapDocument, String, List<MapDocument>>) a -> DedupUtility
|
||||
.getGroupingKeys(config, a)
|
||||
.stream()
|
||||
.map(
|
||||
it -> {
|
||||
List<MapDocument> tmp = new ArrayList<>();
|
||||
tmp.add(a);
|
||||
return new Tuple2<>(it, tmp);
|
||||
})
|
||||
.collect(Collectors.toList())
|
||||
.iterator())
|
||||
.reduceByKey(
|
||||
(Function2<List<MapDocument>, List<MapDocument>, List<MapDocument>>) (v1, v2) -> {
|
||||
v1.addAll(v2);
|
||||
v1.sort(Comparator.comparing(a -> a.getFieldMap().get(of).stringValue()));
|
||||
if (v1.size() > maxQueueSize)
|
||||
return new ArrayList<>(v1.subList(0, maxQueueSize));
|
||||
return v1;
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the list of vertexes: <id, mapDocument>
|
||||
* @param: the spark context
|
||||
* @param: list of JSON entities
|
||||
* @param: the dedup configuration
|
||||
*/
|
||||
public static JavaPairRDD<String, MapDocument> mapToVertexes(
|
||||
JavaSparkContext context, JavaRDD<String> entities, DedupConfig config) {
|
||||
|
||||
return entities
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, MapDocument>) s -> {
|
||||
MapDocument mapDocument = MapDocumentUtil.asMapDocumentWithJPath(config, s);
|
||||
return new Tuple2<String, MapDocument>(mapDocument.getIdentifier(), mapDocument);
|
||||
});
|
||||
}
|
||||
|
||||
public static JavaPairRDD<String, String> computeRelations2(
|
||||
JavaSparkContext context, JavaPairRDD<String, List<MapDocument>> blocks, DedupConfig config) {
|
||||
Map<String, LongAccumulator> accumulators = DedupUtility.constructAccumulator(config, context.sc());
|
||||
|
||||
return blocks
|
||||
.flatMapToPair(
|
||||
(PairFlatMapFunction<Tuple2<String, List<MapDocument>>, String, String>) it -> {
|
||||
try {
|
||||
final SparkReporter reporter = new SparkReporter(accumulators);
|
||||
new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter);
|
||||
return reporter.getRelations().iterator();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(it._2().get(0).getIdentifier(), e);
|
||||
}
|
||||
})
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, String>, String, Tuple2<String, String>>) item -> new Tuple2<String, Tuple2<String, String>>(
|
||||
item._1() + item._2(), item))
|
||||
.reduceByKey((a, b) -> a)
|
||||
.mapToPair(
|
||||
(PairFunction<Tuple2<String, Tuple2<String, String>>, String, String>) Tuple2::_2);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
public enum OafEntityType {
|
||||
datasource, organization, project, dataset, otherresearchproduct, software, publication
|
||||
}
|
|
@ -0,0 +1,112 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.graphx.Edge;
|
||||
import org.apache.spark.rdd.RDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.hash.Hashing;
|
||||
|
||||
import eu.dnetlib.dedup.graph.ConnectedComponent;
|
||||
import eu.dnetlib.dedup.graph.GraphProcessor;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkCreateConnectedComponent {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkCreateConnectedComponent.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkCreateConnectedComponent.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
final String entity = parser.get("entity");
|
||||
final String targetPath = parser.get("targetPath");
|
||||
|
||||
final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
|
||||
|
||||
final JavaPairRDD<Object, String> vertexes = spark
|
||||
.read()
|
||||
.load(inputPath + "/" + entity)
|
||||
.as(Encoders.kryo(Oaf.class))
|
||||
.map((MapFunction<Oaf, String>) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
|
||||
.javaRDD()
|
||||
.map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s))
|
||||
.mapToPair(
|
||||
(PairFunction<String, Object, String>) s -> new Tuple2<Object, String>(getHashcode(s), s));
|
||||
|
||||
final Dataset<Relation> similarityRelations = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(targetPath, entity))
|
||||
.as(Encoders.bean(Relation.class));
|
||||
final RDD<Edge<String>> edgeRdd = similarityRelations
|
||||
.javaRDD()
|
||||
.map(
|
||||
it -> new Edge<>(
|
||||
getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass()))
|
||||
.rdd();
|
||||
final JavaRDD<ConnectedComponent> cc = GraphProcessor
|
||||
.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations())
|
||||
.toJavaRDD();
|
||||
final Dataset<Relation> mergeRelation = spark
|
||||
.createDataset(
|
||||
cc
|
||||
.filter(k -> k.getDocIds().size() > 1)
|
||||
.flatMap(
|
||||
(FlatMapFunction<ConnectedComponent, Relation>) c -> c
|
||||
.getDocIds()
|
||||
.stream()
|
||||
.flatMap(
|
||||
id -> {
|
||||
List<Relation> tmp = new ArrayList<>();
|
||||
Relation r = new Relation();
|
||||
r.setSource(c.getCcId());
|
||||
r.setTarget(id);
|
||||
r.setRelClass(ModelConstants.MERGES);
|
||||
tmp.add(r);
|
||||
r = new Relation();
|
||||
r.setTarget(c.getCcId());
|
||||
r.setSource(id);
|
||||
r.setRelClass(ModelConstants.IS_MERGED_IN);
|
||||
tmp.add(r);
|
||||
return tmp.stream();
|
||||
})
|
||||
.iterator())
|
||||
.rdd(),
|
||||
Encoders.bean(Relation.class));
|
||||
mergeRelation
|
||||
.write()
|
||||
.mode("overwrite")
|
||||
.save(DedupUtility.createMergeRelPath(targetPath, entity));
|
||||
}
|
||||
|
||||
public static long getHashcode(final String id) {
|
||||
return Hashing.murmur3_128().hashString(id).asLong();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
|
||||
public class SparkCreateDedupRecord {
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkCreateDedupRecord.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/dedup/dedupRecord_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkCreateDedupRecord.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String sourcePath = parser.get("sourcePath");
|
||||
final String entity = parser.get("entity");
|
||||
final String dedupPath = parser.get("dedupPath");
|
||||
final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
|
||||
|
||||
final JavaRDD<OafEntity> dedupRecord = DedupRecordFactory
|
||||
.createDedupRecord(
|
||||
sc,
|
||||
spark,
|
||||
DedupUtility.createMergeRelPath(dedupPath, entity),
|
||||
DedupUtility.createEntityPath(sourcePath, entity),
|
||||
OafEntityType.valueOf(entity),
|
||||
dedupConf);
|
||||
spark
|
||||
.createDataset(dedupRecord.rdd(), Encoders.kryo(OafEntity.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(dedupPath + "/" + entity + "/dedup_records");
|
||||
//
|
||||
//
|
||||
// dedupRecord
|
||||
// .map(
|
||||
// r -> {
|
||||
// ObjectMapper mapper = new ObjectMapper();
|
||||
// return mapper.writeValueAsString(r);
|
||||
// })
|
||||
// .saveAsTextFile(dedupPath + "/" + entity + "/dedup_records");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.pace.config.DedupConfig;
|
||||
import eu.dnetlib.pace.model.MapDocument;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This Spark class creates similarity relations between entities, saving result
|
||||
* <p>
|
||||
* param request: sourcePath entityType target Path
|
||||
*/
|
||||
public class SparkCreateSimRels {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkCreateSimRels.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkCreateSimRels.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
final String entity = parser.get("entity");
|
||||
final String targetPath = parser.get("targetPath");
|
||||
// final DedupConfig dedupConf =
|
||||
// DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")));
|
||||
final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf"));
|
||||
|
||||
JavaPairRDD<String, MapDocument> mapDocument = spark
|
||||
.read()
|
||||
.load(inputPath + "/" + entity)
|
||||
.as(Encoders.kryo(Oaf.class))
|
||||
.map((MapFunction<Oaf, String>) p -> new ObjectMapper().writeValueAsString(p), Encoders.STRING())
|
||||
.javaRDD()
|
||||
.repartition(1000)
|
||||
.mapToPair(
|
||||
s -> {
|
||||
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||
return new Tuple2<>(d.getIdentifier(), d);
|
||||
});
|
||||
|
||||
// create blocks for deduplication
|
||||
JavaPairRDD<String, List<MapDocument>> blocks = Deduper.createsortedBlocks(sc, mapDocument, dedupConf);
|
||||
// JavaPairRDD<String, Iterable<MapDocument>> blocks = Deduper.createBlocks(sc,
|
||||
// mapDocument, dedupConf);
|
||||
|
||||
// create relations by comparing only elements in the same group
|
||||
final JavaPairRDD<String, String> dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf);
|
||||
// final JavaPairRDD<String,String> dedupRels = Deduper.computeRelations(sc, blocks,
|
||||
// dedupConf);
|
||||
|
||||
final JavaRDD<Relation> isSimilarToRDD = dedupRels
|
||||
.map(
|
||||
simRel -> {
|
||||
final Relation r = new Relation();
|
||||
r.setSource(simRel._1());
|
||||
r.setTarget(simRel._2());
|
||||
r.setRelClass("isSimilarTo");
|
||||
return r;
|
||||
});
|
||||
|
||||
spark
|
||||
.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class))
|
||||
.write()
|
||||
.mode("overwrite")
|
||||
.save(DedupUtility.createSimRelPath(targetPath, entity));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
|
||||
package eu.dnetlib.dedup;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import eu.dnetlib.pace.util.Reporter;
|
||||
import scala.Serializable;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkReporter implements Serializable, Reporter {
|
||||
|
||||
final List<Tuple2<String, String>> relations = new ArrayList<>();
|
||||
private static final Log log = LogFactory.getLog(SparkReporter.class);
|
||||
Map<String, LongAccumulator> accumulators;
|
||||
|
||||
public SparkReporter(Map<String, LongAccumulator> accumulators) {
|
||||
this.accumulators = accumulators;
|
||||
}
|
||||
|
||||
public void incrementCounter(
|
||||
String counterGroup,
|
||||
String counterName,
|
||||
long delta,
|
||||
Map<String, LongAccumulator> accumulators) {
|
||||
|
||||
final String accumulatorName = String.format("%s::%s", counterGroup, counterName);
|
||||
if (accumulators.containsKey(accumulatorName)) {
|
||||
accumulators.get(accumulatorName).add(delta);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrementCounter(String counterGroup, String counterName, long delta) {
|
||||
|
||||
incrementCounter(counterGroup, counterName, delta, accumulators);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emit(String type, String from, String to) {
|
||||
relations.add(new Tuple2<>(from, to));
|
||||
}
|
||||
|
||||
public List<Tuple2<String, String>> getRelations() {
|
||||
return relations;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
|
||||
package eu.dnetlib.dedup.graph;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dedup.DedupUtility;
|
||||
import eu.dnetlib.pace.util.PaceException;
|
||||
|
||||
public class ConnectedComponent implements Serializable {
|
||||
|
||||
private Set<String> docIds;
|
||||
private String ccId;
|
||||
|
||||
public ConnectedComponent() {
|
||||
}
|
||||
|
||||
public ConnectedComponent(Set<String> docIds) {
|
||||
this.docIds = docIds;
|
||||
createID();
|
||||
}
|
||||
|
||||
public String createID() {
|
||||
if (docIds.size() > 1) {
|
||||
final String s = getMin();
|
||||
String prefix = s.split("\\|")[0];
|
||||
ccId = prefix + "|dedup_wf_001::" + DedupUtility.md5(s);
|
||||
return ccId;
|
||||
} else {
|
||||
return docIds.iterator().next();
|
||||
}
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
public String getMin() {
|
||||
|
||||
final StringBuilder min = new StringBuilder();
|
||||
docIds
|
||||
.forEach(
|
||||
i -> {
|
||||
if (StringUtils.isBlank(min.toString())) {
|
||||
min.append(i);
|
||||
} else {
|
||||
if (min.toString().compareTo(i) > 0) {
|
||||
min.setLength(0);
|
||||
min.append(i);
|
||||
}
|
||||
}
|
||||
});
|
||||
return min.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
try {
|
||||
return mapper.writeValueAsString(this);
|
||||
} catch (IOException e) {
|
||||
throw new PaceException("Failed to create Json: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Set<String> getDocIds() {
|
||||
return docIds;
|
||||
}
|
||||
|
||||
public void setDocIds(Set<String> docIds) {
|
||||
this.docIds = docIds;
|
||||
}
|
||||
|
||||
public String getCcId() {
|
||||
return ccId;
|
||||
}
|
||||
|
||||
public void setCcId(String ccId) {
|
||||
this.ccId = ccId;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
package eu.dnetlib.dedup.graph
|
||||
|
||||
import org.apache.spark.graphx._
|
||||
import org.apache.spark.rdd.RDD
|
||||
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
object GraphProcessor {
|
||||
|
||||
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int): RDD[ConnectedComponent] = {
|
||||
val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
|
||||
val cc = graph.connectedComponents(maxIterations).vertices
|
||||
|
||||
val joinResult = vertexes.leftOuterJoin(cc).map {
|
||||
case (id, (openaireId, cc)) => {
|
||||
if (cc.isEmpty) {
|
||||
(id, openaireId)
|
||||
}
|
||||
else {
|
||||
(cc.get, openaireId)
|
||||
}
|
||||
}
|
||||
}
|
||||
val connectedComponents = joinResult.groupByKey()
|
||||
.map[ConnectedComponent](cc => asConnectedComponent(cc))
|
||||
connectedComponents
|
||||
}
|
||||
|
||||
|
||||
|
||||
def asConnectedComponent(group: (VertexId, Iterable[String])): ConnectedComponent = {
|
||||
val docs = group._2.toSet[String]
|
||||
val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs));
|
||||
connectedComponent
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
|
||||
package eu.dnetlib.dedup.sx;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
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 eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkPropagateRelationsJob {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkPropagateRelationsJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkUpdateEntityJob.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final String relationPath = parser.get("relationPath");
|
||||
final String mergeRelPath = parser.get("mergeRelPath");
|
||||
final String targetRelPath = parser.get("targetRelPath");
|
||||
|
||||
final Dataset<Relation> merge = spark
|
||||
.read()
|
||||
.load(mergeRelPath)
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.where("relClass == 'merges'");
|
||||
|
||||
final Dataset<Relation> rels = spark
|
||||
.read()
|
||||
.load(relationPath)
|
||||
.as(Encoders.kryo(Relation.class))
|
||||
.map(
|
||||
(MapFunction<Relation, Relation>) r -> r,
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
final Dataset<Relation> firstJoin = rels
|
||||
.joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||
final Relation mergeRelation = r._2();
|
||||
final Relation relation = r._1();
|
||||
if (mergeRelation != null)
|
||||
relation.setSource(mergeRelation.getSource());
|
||||
if (relation.getDataInfo() == null)
|
||||
relation.setDataInfo(OafUtils.generateDataInfo("0.9", false));
|
||||
return relation;
|
||||
},
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
final Dataset<Relation> secondJoin = firstJoin
|
||||
.joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) r -> {
|
||||
final Relation mergeRelation = r._2();
|
||||
final Relation relation = r._1();
|
||||
if (mergeRelation != null)
|
||||
relation.setTarget(mergeRelation.getSource());
|
||||
return relation;
|
||||
},
|
||||
Encoders.kryo(Relation.class));
|
||||
|
||||
secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,102 @@
|
|||
|
||||
package eu.dnetlib.dedup.sx;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class SparkUpdateEntityJob {
|
||||
|
||||
static final String IDJSONPATH = "$.id";
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkUpdateEntityJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkUpdateEntityJob.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String entityPath = parser.get("entityPath");
|
||||
final String mergeRelPath = parser.get("mergeRelPath");
|
||||
final String dedupRecordPath = parser.get("dedupRecordPath");
|
||||
final String entity = parser.get("entity");
|
||||
final String destination = parser.get("targetPath");
|
||||
|
||||
final Dataset<Relation> df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class));
|
||||
final JavaPairRDD<String, String> mergedIds = df
|
||||
.where("relClass == 'merges'")
|
||||
.select(df.col("target"))
|
||||
.distinct()
|
||||
.toJavaRDD()
|
||||
.mapToPair((PairFunction<Row, String, String>) r -> new Tuple2<>(r.getString(0), "d"));
|
||||
final JavaRDD<String> sourceEntity = sc.textFile(entityPath);
|
||||
|
||||
final JavaRDD<String> dedupEntity = sc.textFile(dedupRecordPath);
|
||||
JavaPairRDD<String, String> entitiesWithId = sourceEntity
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, String>) s -> new Tuple2<>(DHPUtils.getJPathString(IDJSONPATH, s), s));
|
||||
Class<? extends Oaf> mainClass;
|
||||
switch (entity) {
|
||||
case "publication":
|
||||
mainClass = DLIPublication.class;
|
||||
break;
|
||||
case "dataset":
|
||||
mainClass = DLIDataset.class;
|
||||
break;
|
||||
case "unknown":
|
||||
mainClass = DLIUnknown.class;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Illegal type " + entity);
|
||||
}
|
||||
JavaRDD<String> map = entitiesWithId
|
||||
.leftOuterJoin(mergedIds)
|
||||
.map(
|
||||
k -> k._2()._2().isPresent()
|
||||
? updateDeletedByInference(k._2()._1(), mainClass)
|
||||
: k._2()._1());
|
||||
map.union(dedupEntity).saveAsTextFile(destination, GzipCodec.class);
|
||||
}
|
||||
|
||||
private static <T extends Oaf> String updateDeletedByInference(
|
||||
final String json, final Class<T> clazz) {
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
try {
|
||||
Oaf entity = mapper.readValue(json, clazz);
|
||||
if (entity.getDataInfo() == null)
|
||||
entity.setDataInfo(new DataInfo());
|
||||
entity.getDataInfo().setDeletedbyinference(true);
|
||||
return mapper.writeValueAsString(entity);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Unable to convert json", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
package eu.dnetlib.dedup.sx
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown, OafUtils}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.LoggerFactory
|
||||
import org.apache.spark.sql.functions.col
|
||||
|
||||
object SparkUpdateEntityWithDedupInfo {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkUpdateEntityWithDedupInfo.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json")))
|
||||
val logger = LoggerFactory.getLogger(SparkUpdateEntityWithDedupInfo.getClass)
|
||||
parser.parseArgument(args)
|
||||
|
||||
val workingPath: String = parser.get("workingPath")
|
||||
logger.info(s"Working dir path = $workingPath")
|
||||
|
||||
implicit val oafEncoder: Encoder[OafEntity] = Encoders.kryo[OafEntity]
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
|
||||
|
||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
|
||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo[DLIDataset]
|
||||
implicit val unkEncoder: Encoder[DLIUnknown] = Encoders.kryo[DLIUnknown]
|
||||
|
||||
|
||||
|
||||
val spark: SparkSession = SparkSession
|
||||
.builder()
|
||||
.appName(SparkUpdateEntityWithDedupInfo.getClass.getSimpleName)
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate()
|
||||
|
||||
|
||||
val entityPath = parser.get("entityPath")
|
||||
val mergeRelPath = parser.get("mergeRelPath")
|
||||
val dedupRecordPath = parser.get("dedupRecordPath")
|
||||
val entity = parser.get("entity")
|
||||
val destination = parser.get("targetPath")
|
||||
|
||||
val mergedIds = spark.read.load(mergeRelPath).as[Relation]
|
||||
.where("relClass == 'merges'")
|
||||
.select(col("target"))
|
||||
|
||||
|
||||
val entities: Dataset[(String, OafEntity)] = spark
|
||||
.read
|
||||
.load(entityPath).as[OafEntity]
|
||||
.map(o => (o.getId, o))(Encoders.tuple(Encoders.STRING, oafEncoder))
|
||||
|
||||
|
||||
val finalDataset:Dataset[OafEntity] = entities.joinWith(mergedIds, entities("_1").equalTo(mergedIds("target")), "left")
|
||||
.map(k => {
|
||||
val e: OafEntity = k._1._2
|
||||
val t = k._2
|
||||
if (t != null && t.getString(0).nonEmpty) {
|
||||
if (e.getDataInfo == null) {
|
||||
e.setDataInfo(OafUtils.generateDataInfo())
|
||||
}
|
||||
e.getDataInfo.setDeletedbyinference(true)
|
||||
}
|
||||
e
|
||||
})
|
||||
|
||||
val dedupRecords :Dataset[OafEntity] = spark.read.load(dedupRecordPath).as[OafEntity]
|
||||
|
||||
finalDataset.union(dedupRecords)
|
||||
.repartition(1200).write
|
||||
.mode(SaveMode.Overwrite).save(destination)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
[
|
||||
{
|
||||
"paramName": "mt",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "s",
|
||||
"paramLongName": "sourcePath",
|
||||
"paramDescription": "the path of the sequential file to read",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "e",
|
||||
"paramLongName": "entity",
|
||||
"paramDescription": "the type of entity to be deduped",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "c",
|
||||
"paramLongName": "dedupConf",
|
||||
"paramDescription": "dedup configuration to be used",
|
||||
"compressed": true,
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "d",
|
||||
"paramLongName": "dedupPath",
|
||||
"paramDescription": "dedup path to load mergeRelation",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,38 @@
|
|||
[
|
||||
{
|
||||
"paramName": "mt",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "ep",
|
||||
"paramLongName": "entityPath",
|
||||
"paramDescription": "the input entity path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "mr",
|
||||
"paramLongName": "mergeRelPath",
|
||||
"paramDescription": "the input path of merge Rel",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "dr",
|
||||
"paramLongName": "dedupRecordPath",
|
||||
"paramDescription": "the inputPath of dedup record",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "e",
|
||||
"paramLongName": "entity",
|
||||
"paramDescription": "the type of entity",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "t",
|
||||
"paramLongName": "targetPath",
|
||||
"paramDescription": "the targetPath",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,33 @@
|
|||
[
|
||||
{
|
||||
"paramName": "mt",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "s",
|
||||
"paramLongName": "sourcePath",
|
||||
"paramDescription": "the path of the sequential file to read",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "e",
|
||||
"paramLongName": "entity",
|
||||
"paramDescription": "the type of entity to be deduped",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "c",
|
||||
"paramLongName": "dedupConf",
|
||||
"paramDescription": "dedup configuration to be used",
|
||||
"compressed": true,
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "t",
|
||||
"paramLongName": "targetPath",
|
||||
"paramDescription": "target path to save dedup result",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,26 @@
|
|||
[
|
||||
{
|
||||
"paramName": "mt",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "ep",
|
||||
"paramLongName": "relationPath",
|
||||
"paramDescription": "the input relation path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "mr",
|
||||
"paramLongName": "mergeRelPath",
|
||||
"paramDescription": "the input path of merge Rel",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "t",
|
||||
"paramLongName": "targetRelPath",
|
||||
"paramDescription": "the output Rel Path",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -15,5 +15,4 @@
|
|||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
|
||||
</configuration>
|
|
@ -0,0 +1,182 @@
|
|||
<workflow-app name="Dedup Entities" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the source path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>entity</name>
|
||||
<description>the entity that should be processed</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dedupConf</name>
|
||||
<description>the dedup Configuration</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>targetPath</name>
|
||||
<description>the target path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
<start to="DeleteWorkingPath"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="DeleteWorkingPath">
|
||||
<fs>
|
||||
<delete path='${targetPath}/${entity}'/>
|
||||
<mkdir path="${targetPath}"/>
|
||||
<mkdir path="${targetPath}/${entity}"/>
|
||||
</fs>
|
||||
<ok to="CreateSimRels"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="CreateSimRels">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create Similarity Relations</name>
|
||||
<class>eu.dnetlib.dedup.SparkCreateSimRels</class>
|
||||
<jar>dhp-dedup-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
||||
<arg>--entity</arg><arg>${entity}</arg>
|
||||
<arg>--dedupConf</arg><arg>${dedupConf}</arg>
|
||||
</spark>
|
||||
<ok to="CreateConnectedComponents"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="CreateConnectedComponents">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create Connected Components</name>
|
||||
<class>eu.dnetlib.dedup.SparkCreateConnectedComponent</class>
|
||||
<jar>dhp-dedup-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
||||
<arg>--entity</arg><arg>${entity}</arg>
|
||||
<arg>--dedupConf</arg><arg>${dedupConf}</arg>
|
||||
</spark>
|
||||
<ok to="CreateDedupRecord"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="CreateDedupRecord">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create Dedup Record</name>
|
||||
<class>eu.dnetlib.dedup.SparkCreateDedupRecord</class>
|
||||
<jar>dhp-dedup-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--dedupPath</arg><arg>${targetPath}</arg>
|
||||
<arg>--entity</arg><arg>${entity}</arg>
|
||||
<arg>--dedupConf</arg><arg>${dedupConf}</arg>
|
||||
</spark>
|
||||
<ok to="fixRelation"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="fixRelation">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Propagate Dedup Relations</name>
|
||||
<class>eu.dnetlib.dedup.sx.SparkPropagateRelationsJob</class>
|
||||
<jar>dhp-dedup-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
<arg>--mergeRelPath</arg><arg>${targetPath}/${entity}/mergeRel</arg>
|
||||
<arg>--relationPath</arg><arg>${sourcePath}/relation</arg>
|
||||
<arg>--targetRelPath</arg><arg>${targetPath}/${entity}/updated_relation</arg>
|
||||
</spark>
|
||||
<ok to="updateDeletedByInferenceEntity"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="updateDeletedByInferenceEntity">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Update ${entity} and add DedupRecord</name>
|
||||
<class>eu.dnetlib.dedup.sx.SparkUpdateEntityWithDedupInfo</class>
|
||||
<jar>dhp-dedup-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory ${sparkExecutorMemory}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>-mt</arg><arg>yarn-cluster</arg>
|
||||
<arg>--entityPath</arg><arg>${sourcePath}/${entity}</arg>
|
||||
<arg>--mergeRelPath</arg><arg>${targetPath}/${entity}/mergeRel</arg>
|
||||
<arg>--entity</arg><arg>${entity}</arg>
|
||||
<arg>--dedupRecordPath</arg><arg>${targetPath}/${entity}/dedup_records</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}/${entity}/updated_record</arg>
|
||||
</spark>
|
||||
<ok to="replaceEntity"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="replaceEntity">
|
||||
<fs>
|
||||
<delete path='${sourcePath}/${entity}'/>
|
||||
<delete path='${sourcePath}/relation'/>
|
||||
<move source="${targetPath}/${entity}/updated_relation" target="${sourcePath}/relation" />
|
||||
<move source="${targetPath}/${entity}/updated_record" target="${sourcePath}/${entity}" />
|
||||
</fs>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,378 @@
|
|||
{
|
||||
"wf": {
|
||||
"threshold": "0.99",
|
||||
"dedupRun": "001",
|
||||
"entityType": "result",
|
||||
"subEntityType": "resulttype",
|
||||
"subEntityValue": "publication",
|
||||
"orderField": "title",
|
||||
"queueMaxSize": "2000",
|
||||
"groupMaxSize": "100",
|
||||
"maxChildren": "100",
|
||||
"slidingWindowSize": "200",
|
||||
"rootBuilder": [
|
||||
],
|
||||
"includeChildren": "true",
|
||||
"maxIterations": 20,
|
||||
"idPath": "$.id"
|
||||
},
|
||||
"pace": {
|
||||
"clustering": [
|
||||
{
|
||||
"name": "ngrampairs",
|
||||
"fields": [
|
||||
"title"
|
||||
],
|
||||
"params": {
|
||||
"max": "1",
|
||||
"ngramLen": "3"
|
||||
}
|
||||
},
|
||||
{
|
||||
"name": "suffixprefix",
|
||||
"fields": [
|
||||
"title"
|
||||
],
|
||||
"params": {
|
||||
"max": "1",
|
||||
"len": "3"
|
||||
}
|
||||
}
|
||||
],
|
||||
"decisionTree": {
|
||||
"start": {
|
||||
"fields": [
|
||||
{
|
||||
"field": "pid",
|
||||
"comparator": "jsonListMatch",
|
||||
"weight": 1.0,
|
||||
"countIfUndefined": "false",
|
||||
"params": {
|
||||
"jpath_value": "$.value",
|
||||
"jpath_classid": "$.qualifier.classid"
|
||||
}
|
||||
}
|
||||
],
|
||||
"threshold": 0.5,
|
||||
"aggregation": "AVG",
|
||||
"positive": "MATCH",
|
||||
"negative": "layer2",
|
||||
"undefined": "layer2",
|
||||
"ignoreUndefined": "true"
|
||||
},
|
||||
"layer2": {
|
||||
"fields": [
|
||||
{
|
||||
"field": "title",
|
||||
"comparator": "titleVersionMatch",
|
||||
"weight": 1.0,
|
||||
"countIfUndefined": "false",
|
||||
"params": {}
|
||||
},
|
||||
{
|
||||
"field": "authors",
|
||||
"comparator": "sizeMatch",
|
||||
"weight": 1.0,
|
||||
"countIfUndefined": "false",
|
||||
"params": {}
|
||||
}
|
||||
],
|
||||
"threshold": 1.0,
|
||||
"aggregation": "AND",
|
||||
"positive": "layer3",
|
||||
"negative": "NO_MATCH",
|
||||
"undefined": "layer3",
|
||||
"ignoreUndefined": "false"
|
||||
},
|
||||
"layer3": {
|
||||
"fields": [
|
||||
{
|
||||
"field": "title",
|
||||
"comparator": "levensteinTitle",
|
||||
"weight": 1.0,
|
||||
"countIfUndefined": "true",
|
||||
"params": {}
|
||||
}
|
||||
],
|
||||
"threshold": 0.99,
|
||||
"aggregation": "AVG",
|
||||
"positive": "MATCH",
|
||||
"negative": "NO_MATCH",
|
||||
"undefined": "NO_MATCH",
|
||||
"ignoreUndefined": "true"
|
||||
}
|
||||
},
|
||||
"model": [
|
||||
{
|
||||
"name": "pid",
|
||||
"type": "JSON",
|
||||
"path": "$.pid",
|
||||
"overrideMatch": "true"
|
||||
},
|
||||
{
|
||||
"name": "title",
|
||||
"type": "String",
|
||||
"path": "$.title[*].value",
|
||||
"length": 250,
|
||||
"size": 5
|
||||
},
|
||||
{
|
||||
"name": "authors",
|
||||
"type": "List",
|
||||
"path": "$.author[*].fullname",
|
||||
"size": 200
|
||||
},
|
||||
{
|
||||
"name": "resulttype",
|
||||
"type": "String",
|
||||
"path": "$.resulttype.classid"
|
||||
}
|
||||
],
|
||||
"blacklists": {
|
||||
"title": [
|
||||
"^Inside Front Cover$",
|
||||
"^CORR Insights$",
|
||||
"^Index des notions$",
|
||||
"^Department of Error.$",
|
||||
"^Untitled Item$",
|
||||
"^Department of Error$",
|
||||
"^Tome II : 1598 à 1605$",
|
||||
"^(à l’exception de roi, prince, royauté, pouvoir, image… qui sont omniprésents)$",
|
||||
"^Museen und Ausstellungsinstitute in Nürnberg$",
|
||||
"^Text/Conference Paper$",
|
||||
"^Table des illustrations$",
|
||||
"^An Intimate Insight on Psychopathy and a Novel Hermeneutic Psychological Science$",
|
||||
"^Index des noms$",
|
||||
"^Reply by Authors.$",
|
||||
"^Titelblatt - Inhalt$",
|
||||
"^Index des œuvres,$",
|
||||
"(?i)^Poster presentations$",
|
||||
"^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$",
|
||||
"^Problems with perinatal pathology\\.?$",
|
||||
"(?i)^Cases? of Puerperal Convulsions$",
|
||||
"(?i)^Operative Gyna?ecology$",
|
||||
"(?i)^Mind the gap\\!?\\:?$",
|
||||
"^Chronic fatigue syndrome\\.?$",
|
||||
"^Cartas? ao editor Letters? to the Editor$",
|
||||
"^Note from the Editor$",
|
||||
"^Anesthesia Abstract$",
|
||||
"^Annual report$",
|
||||
"(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$",
|
||||
"(?i)^Graph and Table of Infectious Diseases?$",
|
||||
"^Presentation$",
|
||||
"(?i)^Reviews and Information on Publications$",
|
||||
"(?i)^PUBLIC HEALTH SERVICES?$",
|
||||
"(?i)^COMBINED TEXT-?BOOK OF OBSTETRICS AND GYN(Æ|ae)COLOGY$",
|
||||
"(?i)^Adrese autora$",
|
||||
"(?i)^Systematic Part .*\\. Catalogus Fossilium Austriae, Band 2: Echinoidea neogenica$",
|
||||
"(?i)^Acknowledgement to Referees$",
|
||||
"(?i)^Behçet's disease\\.?$",
|
||||
"(?i)^Isolation and identification of restriction endonuclease.*$",
|
||||
"(?i)^CEREBROVASCULAR DISEASES?.?$",
|
||||
"(?i)^Screening for abdominal aortic aneurysms?\\.?$",
|
||||
"^Event management$",
|
||||
"(?i)^Breakfast and Crohn's disease.*\\.?$",
|
||||
"^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$",
|
||||
"(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$",
|
||||
"^Gushi hakubutsugaku$",
|
||||
"^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$",
|
||||
"^Intestinal spirocha?etosis$",
|
||||
"^Treatment of Rodent Ulcer$",
|
||||
"(?i)^\\W*Cloud Computing\\W*$",
|
||||
"^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$",
|
||||
"^Free Communications, Poster Presentations: Session [A-F]$",
|
||||
"^“The Historical Aspects? of Quackery\\.?”$",
|
||||
"^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$",
|
||||
"^P(er|re)-Mile Premiums for Auto Insurance\\.?$",
|
||||
"(?i)^Case Report$",
|
||||
"^Boletín Informativo$",
|
||||
"(?i)^Glioblastoma Multiforme$",
|
||||
"(?i)^Nuevos táxones animales descritos en la península Ibérica y Macaronesia desde 1994 \\(.*\\)$",
|
||||
"^Zaměstnanecké výhody$",
|
||||
"(?i)^The Economics of Terrorism and Counter-Terrorism: A Survey \\(Part .*\\)$",
|
||||
"(?i)^Carotid body tumours?\\.?$",
|
||||
"(?i)^\\[Españoles en Francia : La condición Emigrante.*\\]$",
|
||||
"^Avant-propos$",
|
||||
"(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Head(s)? and Capital(s)?$",
|
||||
"(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Bases?$",
|
||||
"(?i)^PUBLIC HEALTH VERSUS THE STATE$",
|
||||
"^Viñetas de Cortázar$",
|
||||
"(?i)^Search for heavy neutrinos and W(\\[|_|\\(|_\\{|-)?R(\\]|\\)|\\})? bosons with right-handed couplings in a left-right symmetric model in pp collisions at.*TeV(\\.)?$",
|
||||
"(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$",
|
||||
"(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$",
|
||||
"(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$",
|
||||
"(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$",
|
||||
"^Aus der AGMB$",
|
||||
"^Znanstveno-stručni prilozi$",
|
||||
"(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$",
|
||||
"(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$",
|
||||
"(?i)^Hodnocení finanční situace podniku a návrhy na její zlepšení$",
|
||||
"^Finanční analýza podniku$",
|
||||
"^Financial analysis( of business)?$",
|
||||
"(?i)^Textbook of Gyn(a)?(Æ)?(e)?cology$",
|
||||
"^Jikken nihon shūshinsho$",
|
||||
"(?i)^CORONER('|s)(s|') INQUESTS$",
|
||||
"(?i)^(Μελέτη παραγόντων )?risk management( για ανάπτυξη και εφαρμογή ενός πληροφοριακού συστήματος| και ανάπτυξη συστήματος)?$",
|
||||
"(?i)^Consultants' contract(s)?$",
|
||||
"(?i)^Upute autorima$",
|
||||
"(?i)^Bijdrage tot de Kennis van den Godsdienst der Dajaks van Lan(d|f)ak en Tajan$",
|
||||
"^Joshi shin kokubun$",
|
||||
"^Kōtō shōgaku dokuhon nōson'yō$",
|
||||
"^Jinjō shōgaku shōka$",
|
||||
"^Shōgaku shūjichō$",
|
||||
"^Nihon joshi dokuhon$",
|
||||
"^Joshi shin dokuhon$",
|
||||
"^Chūtō kanbun dokuhon$",
|
||||
"^Wabun dokuhon$",
|
||||
"(?i)^(Analysis of economy selected village or town|Rozbor hospodaření vybrané obce či města)$",
|
||||
"(?i)^cardiac rehabilitation$",
|
||||
"(?i)^Analytical summary$",
|
||||
"^Thesaurus resolutionum Sacrae Congregationis Concilii$",
|
||||
"(?i)^Sumario analítico(\\s{1})?(Analitic summary)?$",
|
||||
"^Prikazi i osvrti$",
|
||||
"^Rodinný dům s provozovnou$",
|
||||
"^Family house with an establishment$",
|
||||
"^Shinsei chūtō shin kokugun$",
|
||||
"^Pulmonary alveolar proteinosis(\\.?)$",
|
||||
"^Shinshū kanbun$",
|
||||
"^Viñeta(s?) de Rodríguez$",
|
||||
"(?i)^RUBRIKA UREDNIKA$",
|
||||
"^A Matching Model of the Academic Publication Market$",
|
||||
"^Yōgaku kōyō$",
|
||||
"^Internetový marketing$",
|
||||
"^Internet marketing$",
|
||||
"^Chūtō kokugo dokuhon$",
|
||||
"^Kokugo dokuhon$",
|
||||
"^Antibiotic Cover for Dental Extraction(s?)$",
|
||||
"^Strategie podniku$",
|
||||
"^Strategy of an Enterprise$",
|
||||
"(?i)^respiratory disease(s?)(\\.?)$",
|
||||
"^Award(s?) for Gallantry in Civil Defence$",
|
||||
"^Podniková kultura$",
|
||||
"^Corporate Culture$",
|
||||
"^Severe hyponatraemia in hospital inpatient(s?)(\\.?)$",
|
||||
"^Pracovní motivace$",
|
||||
"^Work Motivation$",
|
||||
"^Kaitei kōtō jogaku dokuhon$",
|
||||
"^Konsolidovaná účetní závěrka$",
|
||||
"^Consolidated Financial Statements$",
|
||||
"(?i)^intracranial tumour(s?)$",
|
||||
"^Climate Change Mitigation Options and Directed Technical Change: A Decentralized Equilibrium Analysis$",
|
||||
"^\\[CERVECERIAS MAHOU(\\.|\\:) INTERIOR\\] \\[Material gráfico\\]$",
|
||||
"^Housing Market Dynamics(\\:|\\.) On the Contribution of Income Shocks and Credit Constraint(s?)$",
|
||||
"^\\[Funciones auxiliares de la música en Radio París,.*\\]$",
|
||||
"^Úroveň motivačního procesu jako způsobu vedení lidí$",
|
||||
"^The level of motivation process as a leadership$",
|
||||
"^Pay-beds in N(\\.?)H(\\.?)S(\\.?) Hospitals$",
|
||||
"(?i)^news and events$",
|
||||
"(?i)^NOVOSTI I DOGAĐAJI$",
|
||||
"^Sansū no gakushū$",
|
||||
"^Posouzení informačního systému firmy a návrh změn$",
|
||||
"^Information System Assessment and Proposal for ICT Modification$",
|
||||
"^Stresové zatížení pracovníků ve vybrané profesi$",
|
||||
"^Stress load in a specific job$",
|
||||
"^Sunday: Poster Sessions, Pt.*$",
|
||||
"^Monday: Poster Sessions, Pt.*$",
|
||||
"^Wednesday: Poster Sessions, Pt.*",
|
||||
"^Tuesday: Poster Sessions, Pt.*$",
|
||||
"^Analýza reklamy$",
|
||||
"^Analysis of advertising$",
|
||||
"^Shōgaku shūshinsho$",
|
||||
"^Shōgaku sansū$",
|
||||
"^Shintei joshi kokubun$",
|
||||
"^Taishō joshi kokubun dokuhon$",
|
||||
"^Joshi kokubun$",
|
||||
"^Účetní uzávěrka a účetní závěrka v ČR$",
|
||||
"(?i)^The \"?Causes\"? of Cancer$",
|
||||
"^Normas para la publicación de artículos$",
|
||||
"^Editor('|s)(s|') [Rr]eply$",
|
||||
"^Editor(’|s)(s|’) letter$",
|
||||
"^Redaktoriaus žodis$",
|
||||
"^DISCUSSION ON THE PRECEDING PAPER$",
|
||||
"^Kōtō shōgaku shūshinsho jidōyō$",
|
||||
"^Shōgaku nihon rekishi$",
|
||||
"^(Theory of the flow of action currents in isolated myelinated nerve fibers).*$",
|
||||
"^Préface$",
|
||||
"^Occupational [Hh]ealth [Ss]ervices.$",
|
||||
"^In Memoriam Professor Toshiyuki TAKESHIMA$",
|
||||
"^Účetní závěrka ve vybraném podniku.*$",
|
||||
"^Financial statements in selected company$",
|
||||
"^Abdominal [Aa]ortic [Aa]neurysms.*$",
|
||||
"^Pseudomyxoma peritonei$",
|
||||
"^Kazalo autora$",
|
||||
"(?i)^uvodna riječ$",
|
||||
"^Motivace jako způsob vedení lidí$",
|
||||
"^Motivation as a leadership$",
|
||||
"^Polyfunkční dům$",
|
||||
"^Multi\\-funkcional building$",
|
||||
"^Podnikatelský plán$",
|
||||
"(?i)^Podnikatelský záměr$",
|
||||
"(?i)^Business Plan$",
|
||||
"^Oceňování nemovitostí$",
|
||||
"^Marketingová komunikace$",
|
||||
"^Marketing communication$",
|
||||
"^Sumario Analítico$",
|
||||
"^Riječ uredništva$",
|
||||
"^Savjetovanja i priredbe$",
|
||||
"^Índice$",
|
||||
"^(Starobosanski nadpisi).*$",
|
||||
"^Vzdělávání pracovníků v organizaci$",
|
||||
"^Staff training in organization$",
|
||||
"^(Life Histories of North American Geometridae).*$",
|
||||
"^Strategická analýza podniku$",
|
||||
"^Strategic Analysis of an Enterprise$",
|
||||
"^Sadržaj$",
|
||||
"^Upute suradnicima$",
|
||||
"^Rodinný dům$",
|
||||
"(?i)^Fami(l)?ly house$",
|
||||
"^Upute autorima$",
|
||||
"^Strategic Analysis$",
|
||||
"^Finanční analýza vybraného podniku$",
|
||||
"^Finanční analýza$",
|
||||
"^Riječ urednika$",
|
||||
"(?i)^Content(s?)$",
|
||||
"(?i)^Inhalt$",
|
||||
"^Jinjō shōgaku shūshinsho jidōyō$",
|
||||
"(?i)^Index$",
|
||||
"^Chūgaku kokubun kyōkasho$",
|
||||
"^Retrato de una mujer$",
|
||||
"^Retrato de un hombre$",
|
||||
"^Kōtō shōgaku dokuhon$",
|
||||
"^Shotōka kokugo$",
|
||||
"^Shōgaku dokuhon$",
|
||||
"^Jinjō shōgaku kokugo dokuhon$",
|
||||
"^Shinsei kokugo dokuhon$",
|
||||
"^Teikoku dokuhon$",
|
||||
"^Instructions to Authors$",
|
||||
"^KİTAP TAHLİLİ$",
|
||||
"^PRZEGLĄD PIŚMIENNICTWA$",
|
||||
"(?i)^Presentación$",
|
||||
"^İçindekiler$",
|
||||
"(?i)^Tabl?e of contents$",
|
||||
"^(CODICE DEL BEATO DE LOS REYES FERNANDO I Y SANCHA).*$",
|
||||
"^(\\[MADRID\\. BIBL\\. NAC\\. N.*KING FERDINAND I.*FROM SAN ISIDORO DE LEON\\. FACUNDUS SCRIPSIT DATED.*\\]).*",
|
||||
"^Editorial( Board)?$",
|
||||
"(?i)^Editorial \\(English\\)$",
|
||||
"^Editörden$",
|
||||
"^(Corpus Oral Dialectal \\(COD\\)\\.).*$",
|
||||
"^(Kiri Karl Morgensternile).*$",
|
||||
"^(\\[Eksliibris Aleksandr).*\\]$",
|
||||
"^(\\[Eksliibris Aleksandr).*$",
|
||||
"^(Eksliibris Aleksandr).*$",
|
||||
"^(Kiri A\\. de Vignolles).*$",
|
||||
"^(2 kirja Karl Morgensternile).*$",
|
||||
"^(Pirita kloostri idaosa arheoloogilised).*$",
|
||||
"^(Kiri tundmatule).*$",
|
||||
"^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$",
|
||||
"^(Eksliibris Nikolai Birukovile).*$",
|
||||
"^(Eksliibris Nikolai Issakovile).*$",
|
||||
"^(WHP Cruise Summary Information of section).*$",
|
||||
"^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$",
|
||||
"^(Measurement of the spin\\-dependent structure function).*",
|
||||
"(?i)^.*authors['’′]? reply\\.?$",
|
||||
"(?i)^.*authors['’′]? response\\.?$"
|
||||
]
|
||||
},
|
||||
"synonyms": {}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
package eu.dnetlib.doiboost;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class AuthorAssoc implements Serializable {
|
||||
private Double score ;
|
||||
private List<Author> to_be_enriched;
|
||||
private Author with_enricheing_content;
|
||||
|
||||
public Double getScore() {
|
||||
return score;
|
||||
}
|
||||
|
||||
public void setScore(Double score) {
|
||||
this.score = score;
|
||||
}
|
||||
|
||||
public List<Author> getTo_be_enriched() {
|
||||
return to_be_enriched;
|
||||
}
|
||||
|
||||
public void setTo_be_enriched(List<Author> to_be_enriched) {
|
||||
this.to_be_enriched = to_be_enriched;
|
||||
}
|
||||
|
||||
public Author getWith_enricheing_content() {
|
||||
return with_enricheing_content;
|
||||
}
|
||||
|
||||
public void setWith_enricheing_content(Author with_enricheing_content) {
|
||||
this.with_enricheing_content = with_enricheing_content;
|
||||
}
|
||||
|
||||
public static AuthorAssoc newInstance(Author a){
|
||||
AuthorAssoc ret = new AuthorAssoc();
|
||||
ret.score = 0.0;
|
||||
ret.to_be_enriched = new ArrayList<>();
|
||||
ret.with_enricheing_content = a;
|
||||
|
||||
return ret;
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,236 @@
|
|||
|
||||
package eu.dnetlib.doiboost;
|
||||
|
||||
import java.text.Normalizer;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
|
||||
import com.wcohen.ss.JaroWinkler;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This is a version of the AuthorMerger specific for DoiBoost.
|
||||
* Here we suppose a match must exist for the authors. We compare via JaroWrinkler similarity measure each author in the list
|
||||
* that should be enriched with each author in the enriching list. For each enriching author we select the best match that is
|
||||
* the author with the highest similarity score.
|
||||
* The association is done from the enriching author to the enriched because in this way only one match per enriching author can be found
|
||||
* One enriching author can have the same maximum similarity score with more than one
|
||||
*
|
||||
*
|
||||
*
|
||||
*
|
||||
* The idea is to enrich the most similar authors having at least one
|
||||
* word of the name in
|
||||
* common
|
||||
* Quello che faccio e’ abbastanza semplice: ho una struttura dati che mantine l’informazione di associazione fra il record che puo’ possibilmente arricchire e quello che deve essere arricchito.
|
||||
* 6:22
|
||||
* Questa struttura ha la lista di autori che possono essere arricchiti, l’autore che arricchisce e lo score di similarita fra l’autore che arricchisce e gli autori arricchiti. E’ il valore di una mappa che per chiave la il fullname dell’autore che arricchisce
|
||||
* 6:23
|
||||
* per ogni autore che puo’ essere arricchito verifico se la entri nella mappa di quello che arricchisce e’ associata ad un autore con score di similarita’ piu’ basso. Se cosi’ e’ modifico l’associazione nella mappa per l’autore che arricchisce, sostituendo l’autore arricchito a cui era associato prima con quello nuovo che ha score piu’ alto. Se lo score e’ lo stesso, aggiungo il nuovo autore da arricchire alla lista degli autori associata all’autore che arricchisce
|
||||
* 6:25
|
||||
* Alla fine caso facile: ogni entry e’ associata ad un unico autore da arricchire => verifico che almeno una delle parole che sono nei due nomi sia in comune fra i due insiemi Se e’ cosi’, aggiungo i pid mancanti all’autore da arricchire dell’autore che arricchisce
|
||||
* 6:26
|
||||
* caso brutto: ci sono piu’ autori da arricchire con la stessa similarita: arricchisco quello che ha il maggior numero di parole del fullname uguali a quelle dell’autore che arricchisce. In caso di parita’ non si arricchisce
|
||||
* 6:28
|
||||
* ricordiamoci che si parte dal presupposto che un match debba esistere visto che abbiamo lo stesso doi
|
||||
* 6:29
|
||||
* di conseguenza l’autore che ha lo score di similarita’ piu’ alto fra quelli presenti ed anche una parola in comune del nome dovrebbe essere sufficiente per poterlo arricchire.
|
||||
* 6:30
|
||||
* I casi di omonimia che potrebbero portare problemi con i rank degli autori non si mappano
|
||||
*/
|
||||
|
||||
public class DoiBoostAuthorMerger {
|
||||
|
||||
|
||||
public static List<Author> merge(List<List<Author>> authors, Boolean crossref) {
|
||||
|
||||
Iterator<List<Author>> it = authors.iterator();
|
||||
List<Author> author = it.next();
|
||||
|
||||
while (it.hasNext()){
|
||||
List<Author> autList = it.next();
|
||||
Tuple2<List<Author>, Boolean> tmp = mergeAuthor(author, autList, crossref);
|
||||
author = tmp._1();
|
||||
crossref = tmp._2();
|
||||
}
|
||||
|
||||
return author;
|
||||
|
||||
}
|
||||
|
||||
public static Tuple2<List<Author>, Boolean> mergeAuthor(final List<Author> baseAuthor, final List<Author> otherAuthor,
|
||||
final Boolean crossref) {
|
||||
|
||||
if(baseAuthor == null || baseAuthor.size() == 0)
|
||||
return new Tuple2<>(otherAuthor, false);
|
||||
if(otherAuthor == null || otherAuthor.size() == 0)
|
||||
return new Tuple2<>(baseAuthor, crossref);
|
||||
|
||||
if(crossref) {
|
||||
enrichPidFromList(baseAuthor, otherAuthor);
|
||||
return new Tuple2<>(baseAuthor, true);
|
||||
}
|
||||
else
|
||||
if (baseAuthor.size() > otherAuthor.size()){
|
||||
enrichPidFromList(baseAuthor, otherAuthor);
|
||||
return new Tuple2<>(baseAuthor, false);
|
||||
}else{
|
||||
enrichPidFromList(otherAuthor, baseAuthor);
|
||||
return new Tuple2<>(otherAuthor, false);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
private static void enrichPidFromList(List<Author> base, List<Author> enrich) {
|
||||
|
||||
//search authors having identifiers in the enrich list
|
||||
final List<Author> authorsWithPids = enrich
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null && a.getPid().size() > 0)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Map<String, AuthorAssoc> assocMap = authorsWithPids
|
||||
.stream()
|
||||
.map(
|
||||
a -> new Tuple2<>(DHPUtils.md5(a.getFullname()), AuthorAssoc.newInstance(a)))
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1));
|
||||
|
||||
|
||||
//for each author in the base list, we search the best enriched match
|
||||
base.stream()
|
||||
.map(a -> new Tuple2<>(a, authorsWithPids.stream()
|
||||
.map(e -> new Tuple2<>(e, sim(a, e))).collect(Collectors.toList())))
|
||||
.forEach(t2 -> {
|
||||
|
||||
for (Tuple2<Author, Double> t : t2._2()) {
|
||||
String mapEntry = DHPUtils.md5(t._1().getFullname());
|
||||
AuthorAssoc aa = assocMap.get(mapEntry);
|
||||
if(aa.getScore() < t._2() && aa.getScore() < 0.9){
|
||||
aa.setScore(t._2());
|
||||
aa.setTo_be_enriched(new ArrayList<>());
|
||||
aa.getTo_be_enriched().add(t2._1());
|
||||
}else if(t._2() > 0.9){
|
||||
aa.getTo_be_enriched().add(t2._1());
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
assocMap.keySet().forEach(k -> enrichAuthor(assocMap.get(k)));
|
||||
|
||||
|
||||
}
|
||||
|
||||
private static long getCommonWords(List<String> fullEnrich, List<String> fullEnriching){
|
||||
return fullEnrich.stream().filter( w -> fullEnriching.contains(w)).count();
|
||||
}
|
||||
|
||||
|
||||
private static void enrichAuthor(Author enrich, Author enriching){
|
||||
//verify if some of the words in the fullname are contained in the other
|
||||
//get normalized fullname
|
||||
|
||||
long commonWords = getCommonWords(normalize(enrich.getFullname()),
|
||||
normalize(enriching.getFullname()));
|
||||
if(commonWords > 0 ){
|
||||
if(enrich.getPid() == null){
|
||||
enrich.setPid(new ArrayList<>());
|
||||
}
|
||||
Set<String> aPids = enrich.getPid().stream().map(p -> pidToComparableString(p)).collect(Collectors.toSet());
|
||||
enriching.getPid().forEach(p -> {
|
||||
if (!aPids.contains(pidToComparableString(p))){
|
||||
enrich.getPid().add(p);
|
||||
}
|
||||
});
|
||||
if (enrich.getAffiliation() == null){
|
||||
if (enriching.getAffiliation() != null){
|
||||
enrich.setAffiliation(enriching.getAffiliation());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
//Verify the number of words in common. The one that has more, wins. If the number of words in common are the same we
|
||||
//enrich no author
|
||||
private static void enrichAuthor(AuthorAssoc authorAssoc) {
|
||||
if (authorAssoc.getTo_be_enriched().size() == 1){
|
||||
enrichAuthor(authorAssoc.getTo_be_enriched().get(0), authorAssoc.getWith_enricheing_content());
|
||||
}else{
|
||||
long common = 0;
|
||||
List<Author> selected = new ArrayList<>() ;
|
||||
for(Author a : authorAssoc.getTo_be_enriched()){
|
||||
long current_common = getCommonWords(normalize(a.getFullname()),
|
||||
normalize(authorAssoc.getWith_enricheing_content().getFullname()));
|
||||
if (current_common > common){
|
||||
common = current_common;
|
||||
selected = new ArrayList<>();
|
||||
selected.add(a);
|
||||
}else if(current_common == common){
|
||||
selected.add(a);
|
||||
}
|
||||
}
|
||||
if (selected.size() == 1){
|
||||
enrichAuthor(selected.get(0), authorAssoc.getWith_enricheing_content());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
public static String pidToComparableString(StructuredProperty pid) {
|
||||
return (pid.getQualifier() != null
|
||||
? pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() : ""
|
||||
: "")
|
||||
+ (pid.getValue() != null ? pid.getValue().toLowerCase() : "");
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
private static Double sim(Author a, Author b) {
|
||||
return new JaroWinkler()
|
||||
.score(normalizeString(a.getFullname()), normalizeString(b.getFullname()));
|
||||
|
||||
}
|
||||
|
||||
private static String normalizeString(String fullname) {
|
||||
return String.join(" ", normalize(fullname));
|
||||
}
|
||||
|
||||
|
||||
private static List<String> normalize(final String s) {
|
||||
String[] normalized = nfd(s)
|
||||
.replaceAll("[^\\p{ASCII}]", "")
|
||||
.toLowerCase()
|
||||
// do not compact the regexes in a single expression, would cause StackOverflowError
|
||||
// in case
|
||||
// of large input strings
|
||||
.replaceAll("(\\W)+", " ")
|
||||
.replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ")
|
||||
.replaceAll("(\\p{Punct})+", " ")
|
||||
.replaceAll("(\\d)+", " ")
|
||||
.replaceAll("(\\n)+", " ")
|
||||
.trim()
|
||||
.split(" ");
|
||||
|
||||
Arrays.sort(normalized);
|
||||
|
||||
return Arrays.asList(normalized);
|
||||
|
||||
|
||||
}
|
||||
|
||||
private static String nfd(final String s) {
|
||||
return Normalizer.normalize(s, Normalizer.Form.NFD);
|
||||
}
|
||||
}
|
|
@ -6,7 +6,7 @@ import eu.dnetlib.dhp.utils.DHPUtils
|
|||
import org.apache.commons.lang3.StringUtils
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils
|
||||
import org.json4s
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.jackson.JsonMethods.parse
|
||||
|
@ -121,11 +121,11 @@ object DoiBoostMappingUtil {
|
|||
|
||||
|
||||
def getOpenAccessQualifier():AccessRight = {
|
||||
OafMapperUtils.accessRight("OPEN","Open Access", ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES)
|
||||
OafUtils.createAccessRight("OPEN","Open Access", ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES)
|
||||
}
|
||||
|
||||
def getRestrictedQualifier():AccessRight = {
|
||||
OafMapperUtils.accessRight("RESTRICTED","Restricted",ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES)
|
||||
OafUtils.createAccessRight("RESTRICTED","Restricted",ModelConstants.DNET_ACCESS_MODES, ModelConstants.DNET_ACCESS_MODES)
|
||||
}
|
||||
|
||||
|
||||
|
@ -153,7 +153,7 @@ object DoiBoostMappingUtil {
|
|||
if (item.openAccess)
|
||||
i.setAccessright(getOpenAccessQualifier())
|
||||
val ar = getOpenAccessQualifier()
|
||||
publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
publication.setBestaccessright(OafUtils.createQualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
}
|
||||
else {
|
||||
hb = ModelConstants.UNKNOWN_REPOSITORY
|
||||
|
@ -165,11 +165,11 @@ object DoiBoostMappingUtil {
|
|||
if (ar.nonEmpty) {
|
||||
if(ar.contains(ModelConstants.ACCESS_RIGHT_OPEN)){
|
||||
val ar = getOpenAccessQualifier()
|
||||
publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
publication.setBestaccessright(OafUtils.createQualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
}
|
||||
else {
|
||||
val ar = getRestrictedQualifier()
|
||||
publication.setBestaccessright(OafMapperUtils.qualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
publication.setBestaccessright(OafUtils.createQualifier(ar.getClassid, ar.getClassname, ar.getSchemeid, ar.getSchemename))
|
||||
}
|
||||
}
|
||||
publication
|
||||
|
@ -257,7 +257,7 @@ object DoiBoostMappingUtil {
|
|||
di.setInferred(false)
|
||||
di.setInvisible(false)
|
||||
di.setTrust(trust)
|
||||
di.setProvenanceaction(OafMapperUtils.qualifier(ModelConstants.SYSIMPORT_ACTIONSET,ModelConstants.SYSIMPORT_ACTIONSET, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS))
|
||||
di.setProvenanceaction(OafUtils.createQualifier(ModelConstants.SYSIMPORT_ACTIONSET, ModelConstants.DNET_PROVENANCE_ACTIONS))
|
||||
di
|
||||
}
|
||||
|
||||
|
@ -265,7 +265,7 @@ object DoiBoostMappingUtil {
|
|||
|
||||
def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String): StructuredProperty = {
|
||||
val sp = new StructuredProperty
|
||||
sp.setQualifier(OafMapperUtils.qualifier(classId,className, schemeId, schemeName))
|
||||
sp.setQualifier(OafUtils.createQualifier(classId,className, schemeId, schemeName))
|
||||
sp.setValue(value)
|
||||
sp
|
||||
|
||||
|
@ -275,7 +275,7 @@ object DoiBoostMappingUtil {
|
|||
|
||||
def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String, dataInfo: DataInfo): StructuredProperty = {
|
||||
val sp = new StructuredProperty
|
||||
sp.setQualifier(OafMapperUtils.qualifier(classId,className, schemeId, schemeName))
|
||||
sp.setQualifier(OafUtils.createQualifier(classId,className, schemeId, schemeName))
|
||||
sp.setValue(value)
|
||||
sp.setDataInfo(dataInfo)
|
||||
sp
|
||||
|
@ -284,7 +284,7 @@ object DoiBoostMappingUtil {
|
|||
|
||||
def createSP(value: String, classId: String, schemeId: String): StructuredProperty = {
|
||||
val sp = new StructuredProperty
|
||||
sp.setQualifier(OafMapperUtils.qualifier(classId,classId, schemeId, schemeId))
|
||||
sp.setQualifier(OafUtils.createQualifier(classId, schemeId))
|
||||
sp.setValue(value)
|
||||
sp
|
||||
|
||||
|
@ -294,7 +294,7 @@ object DoiBoostMappingUtil {
|
|||
|
||||
def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = {
|
||||
val sp = new StructuredProperty
|
||||
sp.setQualifier(OafMapperUtils.qualifier(classId,classId, schemeId, schemeId))
|
||||
sp.setQualifier(OafUtils.createQualifier(classId, schemeId))
|
||||
sp.setValue(value)
|
||||
sp.setDataInfo(dataInfo)
|
||||
sp
|
||||
|
|
|
@ -3,7 +3,7 @@ package eu.dnetlib.doiboost
|
|||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Organization, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Organization, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.doiboost.mag.ConversionUtil
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
|
@ -25,6 +25,7 @@ object SparkGenerateDoiBoost {
|
|||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/generate_doiboost_params.json")))
|
||||
parser.parseArgument(args)
|
||||
var crossref : Boolean = true
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
|
@ -104,7 +105,10 @@ object SparkGenerateDoiBoost {
|
|||
val otherPub = item._2._2
|
||||
if (otherPub != null) {
|
||||
crossrefPub.mergeFrom(otherPub)
|
||||
crossrefPub.setAuthor(AuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor))
|
||||
val mergeRes : (java.util.List[Author], java.lang.Boolean) = DoiBoostAuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor, crossref)
|
||||
crossrefPub.setAuthor(mergeRes._1)
|
||||
crossref = mergeRes._2
|
||||
|
||||
}
|
||||
}
|
||||
crossrefPub
|
||||
|
|
|
@ -2,19 +2,20 @@ package eu.dnetlib.doiboost.crossref
|
|||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf._
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils}
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
|
||||
import org.apache.commons.lang.StringUtils
|
||||
import org.json4s
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JValue, _}
|
||||
import org.json4s.JsonAST._
|
||||
import org.json4s.jackson.JsonMethods._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.matching.Regex
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils
|
||||
import java.util
|
||||
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil
|
||||
|
@ -182,12 +183,12 @@ case object Crossref2Oaf {
|
|||
|
||||
if(has_review != JNothing) {
|
||||
instance.setRefereed(
|
||||
OafMapperUtils.qualifier("0001", "peerReviewed", ModelConstants.DNET_REVIEW_LEVELS, ModelConstants.DNET_REVIEW_LEVELS))
|
||||
OafUtils.createQualifier("0001", "peerReviewed", ModelConstants.DNET_REVIEW_LEVELS, ModelConstants.DNET_REVIEW_LEVELS))
|
||||
}
|
||||
|
||||
instance.setAccessright(getRestrictedQualifier())
|
||||
instance.setInstancetype(OafMapperUtils.qualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
result.setResourcetype(OafMapperUtils.qualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
instance.setInstancetype(OafUtils.createQualifier(cobjCategory.substring(0, 4), cobjCategory.substring(5), ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
result.setResourcetype(OafUtils.createQualifier(cobjCategory.substring(0, 4),ModelConstants.DNET_DATA_CITE_RESOURCE))
|
||||
|
||||
instance.setCollectedfrom(createCrossrefCollectedFrom())
|
||||
if (StringUtils.isNotBlank(issuedDate)) {
|
||||
|
|
|
@ -21,7 +21,7 @@ object SparkMapDumpIntoOAF {
|
|||
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_crossref_dump_to_oaf_params.json")))
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
|
|
|
@ -1,19 +1,52 @@
|
|||
package eu.dnetlib.doiboost.orcid
|
||||
|
||||
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication
|
||||
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkConvertORCIDToOAF {
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkConvertORCIDToOAF.getClass)
|
||||
|
||||
def fixORCIDItem(item :ORCIDItem):ORCIDItem = {
|
||||
new ORCIDItem(item.doi, item.authors.groupBy(_.oid).map(_._2.head).toList)
|
||||
|
||||
def run(spark:SparkSession, workingPath:String, targetPath:String) :Unit = {
|
||||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
}
|
||||
|
||||
|
||||
def run(spark:SparkSession,sourcePath:String,workingPath:String, targetPath:String):Unit = {
|
||||
import spark.implicits._
|
||||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
|
||||
val inputRDD:RDD[OrcidAuthor] = spark.sparkContext.textFile(s"$sourcePath/authors").map(s => ORCIDToOAF.convertORCIDAuthor(s)).filter(s => s!= null).filter(s => ORCIDToOAF.authorValid(s))
|
||||
|
||||
spark.createDataset(inputRDD).as[OrcidAuthor].write.mode(SaveMode.Overwrite).save(s"$workingPath/author")
|
||||
|
||||
val res = spark.sparkContext.textFile(s"$sourcePath/works").flatMap(s => ORCIDToOAF.extractDOIWorks(s)).filter(s => s!= null)
|
||||
|
||||
spark.createDataset(res).as[OrcidWork].write.mode(SaveMode.Overwrite).save(s"$workingPath/works")
|
||||
|
||||
val authors :Dataset[OrcidAuthor] = spark.read.load(s"$workingPath/author").as[OrcidAuthor]
|
||||
|
||||
val works :Dataset[OrcidWork] = spark.read.load(s"$workingPath/works").as[OrcidWork]
|
||||
|
||||
works.joinWith(authors, authors("oid").equalTo(works("oid")))
|
||||
.map(i =>{
|
||||
val doi = i._1.doi
|
||||
var author = i._2
|
||||
(doi, author)
|
||||
}).groupBy(col("_1").alias("doi"))
|
||||
.agg(collect_list(col("_2")).alias("authors")).as[ORCIDItem]
|
||||
.map(s => fixORCIDItem(s))
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/orcidworksWithAuthor")
|
||||
|
||||
val dataset: Dataset[ORCIDItem] =spark.read.load(s"$workingPath/orcidworksWithAuthor").as[ORCIDItem]
|
||||
|
||||
logger.info("Converting ORCID to OAF")
|
||||
|
@ -22,7 +55,7 @@ object SparkConvertORCIDToOAF {
|
|||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_orcid_to_oaf_params.json")))
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
|
@ -32,10 +65,10 @@ object SparkConvertORCIDToOAF {
|
|||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
val workingPath = parser.get("workingPath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
|
||||
run(spark,workingPath, targetPath)
|
||||
run(spark, sourcePath, workingPath, targetPath)
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -1,70 +0,0 @@
|
|||
package eu.dnetlib.doiboost.orcid
|
||||
|
||||
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication
|
||||
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkPreprocessORCID {
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkConvertORCIDToOAF.getClass)
|
||||
|
||||
def fixORCIDItem(item :ORCIDItem):ORCIDItem = {
|
||||
ORCIDItem(item.doi, item.authors.groupBy(_.oid).map(_._2.head).toList)
|
||||
|
||||
}
|
||||
|
||||
|
||||
def run(spark:SparkSession,sourcePath:String,workingPath:String):Unit = {
|
||||
import spark.implicits._
|
||||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
|
||||
val inputRDD:RDD[OrcidAuthor] = spark.sparkContext.textFile(s"$sourcePath/authors").map(s => ORCIDToOAF.convertORCIDAuthor(s)).filter(s => s!= null).filter(s => ORCIDToOAF.authorValid(s))
|
||||
|
||||
spark.createDataset(inputRDD).as[OrcidAuthor].write.mode(SaveMode.Overwrite).save(s"$workingPath/author")
|
||||
|
||||
val res = spark.sparkContext.textFile(s"$sourcePath/works").flatMap(s => ORCIDToOAF.extractDOIWorks(s)).filter(s => s!= null)
|
||||
|
||||
spark.createDataset(res).as[OrcidWork].write.mode(SaveMode.Overwrite).save(s"$workingPath/works")
|
||||
|
||||
val authors :Dataset[OrcidAuthor] = spark.read.load(s"$workingPath/author").as[OrcidAuthor]
|
||||
|
||||
val works :Dataset[OrcidWork] = spark.read.load(s"$workingPath/works").as[OrcidWork]
|
||||
|
||||
works.joinWith(authors, authors("oid").equalTo(works("oid")))
|
||||
.map(i =>{
|
||||
val doi = i._1.doi
|
||||
val author = i._2
|
||||
(doi, author)
|
||||
}).groupBy(col("_1").alias("doi"))
|
||||
.agg(collect_list(col("_2")).alias("authors")).as[ORCIDItem]
|
||||
.map(s => fixORCIDItem(s))
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/orcidworksWithAuthor")
|
||||
}
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkConvertORCIDToOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/preprocess_orcid_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
val workingPath = parser.get("workingPath")
|
||||
|
||||
run(spark, sourcePath, workingPath)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -4,7 +4,6 @@ package eu.dnetlib.doiboost.orcidnodoi;
|
|||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
@ -33,7 +32,10 @@ import com.google.gson.JsonParser;
|
|||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
import eu.dnetlib.dhp.schema.orcid.*;
|
||||
import eu.dnetlib.dhp.schema.orcid.AuthorData;
|
||||
import eu.dnetlib.dhp.schema.orcid.AuthorSummary;
|
||||
import eu.dnetlib.dhp.schema.orcid.Work;
|
||||
import eu.dnetlib.dhp.schema.orcid.WorkDetail;
|
||||
import eu.dnetlib.doiboost.orcid.json.JsonHelper;
|
||||
import eu.dnetlib.doiboost.orcid.util.HDFSUtil;
|
||||
import eu.dnetlib.doiboost.orcidnodoi.oaf.PublicationToOaf;
|
||||
|
@ -109,10 +111,6 @@ public class SparkGenEnrichedOrcidWorks {
|
|||
Encoders.bean(WorkDetail.class));
|
||||
logger.info("Works data loaded: " + workDataset.count());
|
||||
|
||||
final LongAccumulator warnNotFoundContributors = spark
|
||||
.sparkContext()
|
||||
.longAccumulator("warnNotFoundContributors");
|
||||
|
||||
JavaRDD<Tuple2<String, String>> enrichedWorksRDD = workDataset
|
||||
.joinWith(
|
||||
authorDataset,
|
||||
|
@ -121,21 +119,7 @@ public class SparkGenEnrichedOrcidWorks {
|
|||
(MapFunction<Tuple2<WorkDetail, AuthorData>, Tuple2<String, String>>) value -> {
|
||||
WorkDetail w = value._1;
|
||||
AuthorData a = value._2;
|
||||
if (w.getContributors() == null
|
||||
|| (w.getContributors() != null && w.getContributors().size() == 0)) {
|
||||
Contributor c = new Contributor();
|
||||
c.setName(a.getName());
|
||||
c.setSurname(a.getSurname());
|
||||
c.setCreditName(a.getCreditName());
|
||||
c.setOid(a.getOid());
|
||||
List<Contributor> contributors = Arrays.asList(c);
|
||||
w.setContributors(contributors);
|
||||
if (warnNotFoundContributors != null) {
|
||||
warnNotFoundContributors.add(1);
|
||||
}
|
||||
} else {
|
||||
AuthorMatcher.match(a, w.getContributors());
|
||||
}
|
||||
AuthorMatcher.match(a, w.getContributors());
|
||||
return new Tuple2<>(a.getOid(), JsonHelper.createOidWork(w));
|
||||
},
|
||||
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
|
||||
|
@ -188,7 +172,7 @@ public class SparkGenEnrichedOrcidWorks {
|
|||
OBJECT_MAPPER.writeValueAsString(new AtomicAction<>(Publication.class, p))))
|
||||
.mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2())))
|
||||
.saveAsNewAPIHadoopFile(
|
||||
outputEnrichedWorksPath,
|
||||
workingPath.concat(outputEnrichedWorksPath),
|
||||
Text.class,
|
||||
Text.class,
|
||||
SequenceFileOutputFormat.class,
|
||||
|
@ -196,7 +180,6 @@ public class SparkGenEnrichedOrcidWorks {
|
|||
|
||||
logger.info("parsedPublications: " + parsedPublications.value().toString());
|
||||
logger.info("enrichedPublications: " + enrichedPublications.value().toString());
|
||||
logger.info("warnNotFoundContributors: " + warnNotFoundContributors.value().toString());
|
||||
logger.info("errorsGeneric: " + errorsGeneric.value().toString());
|
||||
logger.info("errorsInvalidTitle: " + errorsInvalidTitle.value().toString());
|
||||
logger.info("errorsNotFoundAuthors: " + errorsNotFoundAuthors.value().toString());
|
||||
|
|
|
@ -18,7 +18,7 @@ object SparkMapUnpayWallToOAF {
|
|||
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_uw_to_oaf_params.json")))
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkMapDumpIntoOAF.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/convert_map_to_oaf_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
|
|
|
@ -1,6 +0,0 @@
|
|||
[
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true},
|
||||
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source path ", "paramRequired": false},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
|
||||
]
|
|
@ -1,6 +1,7 @@
|
|||
[
|
||||
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the Orcid Input file", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": false},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the working dir path", "paramRequired": true},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
|
||||
]
|
|
@ -1,6 +0,0 @@
|
|||
[
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": false},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
|
||||
]
|
|
@ -1,6 +0,0 @@
|
|||
[
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the path of the OAF Orcid transformed", "paramRequired": true},
|
||||
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source path ", "paramRequired": false},
|
||||
{"paramName":"m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true}
|
||||
|
||||
]
|
|
@ -0,0 +1,101 @@
|
|||
<workflow-app name="import Crossref from index into HDFS" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>timestamp</name>
|
||||
<description>Timestamp for incremental Harvesting</description>
|
||||
</property>
|
||||
|
||||
</parameters>
|
||||
|
||||
<start to="ImportCrossRef"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="ImportCrossRef">
|
||||
<java>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>
|
||||
<arg>-t</arg><arg>${workingPath}/input/crossref/index_update</arg>
|
||||
<arg>-n</arg><arg>${nameNode}</arg>
|
||||
<arg>-ts</arg><arg>${timestamp}</arg>
|
||||
</java>
|
||||
<ok to="GenerateDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="GenerateDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ExtractCrossrefToOAF</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>/data/doiboost/input/crossref</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="RenameDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="RenameDataset">
|
||||
<fs>
|
||||
<delete path='${workingPath}/input/crossref/crossref_ds'/>
|
||||
<move source="${workingPath}/input/crossref/crossref_ds_updated"
|
||||
target="${workingPath}/input/crossref/crossref_ds"/>
|
||||
</fs>
|
||||
<ok to="ConvertCrossrefToOAF"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="ConvertCrossrefToOAF">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ConvertCrossrefToOAF</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingPath}/input/crossref/crossref_ds</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/process/</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,38 @@
|
|||
<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>
|
||||
</configuration>
|
|
@ -0,0 +1,96 @@
|
|||
<workflow-app name="Create DOIBoostActionSet" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>hostedByMapPath</name>
|
||||
<description>the Hosted By Map Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>affiliationPath</name>
|
||||
<description>the Affliation Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>paperAffiliationPath</name>
|
||||
<description>the paperAffiliation Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingDirPath</name>
|
||||
<description>the Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
|
||||
|
||||
<start to="CreateDOIBoost"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
|
||||
|
||||
<action name="CreateDOIBoost">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create DOIBoost Infospace</name>
|
||||
<class>eu.dnetlib.doiboost.SparkGenerateDoiBoost</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--hostedByMapPath</arg><arg>${hostedByMapPath}</arg>
|
||||
<arg>--affiliationPath</arg><arg>${affiliationPath}</arg>
|
||||
<arg>--paperAffiliationPath</arg><arg>${paperAffiliationPath}</arg>
|
||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="GenerateActionSet"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="GenerateActionSet">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Generate DOIBoost ActionSet</name>
|
||||
<class>eu.dnetlib.doiboost.SparkGenerateDOIBoostActionSet</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--dbPublicationPath</arg><arg>${workingDirPath}/doiBoostPublicationFiltered</arg>
|
||||
<arg>--dbDatasetPath</arg><arg>${workingDirPath}/crossrefDataset</arg>
|
||||
<arg>--crossRefRelation</arg><arg>${workingDirPath}/crossrefRelation</arg>
|
||||
<arg>--dbaffiliationRelationPath</arg><arg>${workingDirPath}/doiBoostPublicationAffiliation</arg>
|
||||
<arg>-do</arg><arg>${workingDirPath}/doiBoostOrganization</arg>
|
||||
<arg>--targetPath</arg><arg>${workingDirPath}/actionDataSet</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,42 @@
|
|||
<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>oozie.wf.rerun.failnodes</name>
|
||||
<value>false</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>
|
||||
</configuration>
|
|
@ -0,0 +1,92 @@
|
|||
<workflow-app name="import MAG into HDFS" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>targetPath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="ResetWorkingPath"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="ResetWorkingPath">
|
||||
<fs>
|
||||
<delete path='${workingPath}'/>
|
||||
<mkdir path='${workingPath}'/>
|
||||
</fs>
|
||||
<ok to="ConvertMagToDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertMagToDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="PreprocessMag"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<action name="PreprocessMag">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to OAF Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkPreProcessMAG</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingPath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/process</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -1,4 +1,4 @@
|
|||
<workflow-app name="Generate DOIBoost ActionSet for PROD" xmlns="uri:oozie:workflow:0.5">
|
||||
<workflow-app name="Generate DOIBoost ActionSet" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
|
@ -17,6 +17,8 @@
|
|||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
|
||||
|
||||
<!-- Itersection Parameters -->
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
|
@ -38,8 +40,29 @@
|
|||
<name>inputPathCrossref</name>
|
||||
<description>the Crossref input path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>crossrefDumpPath</name>
|
||||
<description>the Crossref dump path</description>
|
||||
</property>
|
||||
<!-- <property>-->
|
||||
<!-- <name>crossrefTimestamp</name>-->
|
||||
<!-- <description>Timestamp for the Crossref incremental Harvesting</description>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>esServer</name>-->
|
||||
<!-- <description>elasticsearch server url for the Crossref Harvesting</description>-->
|
||||
<!-- </property>-->
|
||||
<!-- <property>-->
|
||||
<!-- <name>esIndex</name>-->
|
||||
<!-- <description>elasticsearch index name for the Crossref Harvesting</description>-->
|
||||
<!-- </property>-->
|
||||
|
||||
<!-- MAG Parameters -->
|
||||
<property>
|
||||
<name>MAGDumpPath</name>
|
||||
<description>the MAG dump working path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>inputPathMAG</name>
|
||||
<description>the MAG working path</description>
|
||||
|
@ -53,6 +76,11 @@
|
|||
</property>
|
||||
|
||||
<!-- ORCID Parameters -->
|
||||
<property>
|
||||
<name>inputPathOrcid</name>
|
||||
<description>the ORCID input path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>workingPathOrcid</name>
|
||||
<description>the ORCID working path</description>
|
||||
|
@ -75,12 +103,15 @@
|
|||
|
||||
<decision name="resume_from">
|
||||
<switch>
|
||||
<case to="ConvertCrossrefToOAF">${wf:conf('resumeFrom') eq 'ConvertCrossrefToOAF'}</case>
|
||||
<case to="ResetMagWorkingPath">${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'}</case>
|
||||
<case to="ProcessMAG">${wf:conf('resumeFrom') eq 'PreprocessMag'}</case>
|
||||
<case to="ProcessUW">${wf:conf('resumeFrom') eq 'PreprocessUW'}</case>
|
||||
<case to="ProcessORCID">${wf:conf('resumeFrom') eq 'ProcessORCID'}</case>
|
||||
<case to="ProcessORCID">${wf:conf('resumeFrom') eq 'PreprocessORCID'}</case>
|
||||
<case to="CreateDOIBoost">${wf:conf('resumeFrom') eq 'CreateDOIBoost'}</case>
|
||||
<case to="GenerateActionSet">${wf:conf('resumeFrom') eq 'GenerateActionSet'}</case>
|
||||
<default to="ConvertCrossrefToOAF"/>
|
||||
<case to="GenerateCrossrefDataset">${wf:conf('resumeFrom') eq 'GenerateCrossrefDataset'}</case>
|
||||
<default to="ProcessORCID"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
|
@ -88,6 +119,170 @@
|
|||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<!-- <action name="ImportCrossRef">-->
|
||||
<!-- <java>-->
|
||||
<!-- <main-class>eu.dnetlib.doiboost.crossref.CrossrefImporter</main-class>-->
|
||||
<!-- <arg>--targetPath</arg><arg>${inputPathCrossref}/index_update</arg>-->
|
||||
<!-- <arg>--namenode</arg><arg>${nameNode}</arg>-->
|
||||
<!-- <arg>--esServer</arg><arg>${esServer}</arg>-->
|
||||
<!-- <arg>--esIndex</arg><arg>${esIndex}</arg>-->
|
||||
<!-- <arg>--timestamp</arg><arg>${crossrefTimestamp}</arg>-->
|
||||
<!-- </java>-->
|
||||
<!-- <ok to="GenerateCrossrefDataset"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
<action name="ImportCrossRef">
|
||||
<java>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<main-class>eu.dnetlib.doiboost.crossref.ExtractCrossrefRecords</main-class>
|
||||
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||
<arg>--crossrefFileNameTarGz</arg><arg>${crossrefDumpPath}/crossref.tar.gz</arg>
|
||||
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${crossrefDumpPath}/files/</arg>
|
||||
</java>
|
||||
<ok to="UnpackCrossrefEntries"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="UnpackCrossrefEntries">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>SparkUnpackCrossrefEntries</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/files</arg>
|
||||
<arg>--targetPath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||
|
||||
</spark>
|
||||
<ok to="GenerateCrossrefDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="GenerateCrossrefDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>SparkGenerateCrossrefDataset</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=7G
|
||||
--executor-cores=2
|
||||
--driver-memory=7G
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||
|
||||
</spark>
|
||||
<ok to="removeFiles"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="removeFiles">
|
||||
<fs>
|
||||
<!-- <delete path="${crossrefDumpPath}/files"/>-->
|
||||
<delete path="${crossrefDumpPath}/crossref_unpack/"/>
|
||||
</fs>
|
||||
<ok to="ResetMagWorkingPath"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<!-- CROSSREF SECTION -->
|
||||
|
||||
<!-- <action name="GenerateCrossrefDataset">-->
|
||||
<!-- <spark xmlns="uri:oozie:spark-action:0.2">-->
|
||||
<!-- <master>yarn-cluster</master>-->
|
||||
<!-- <mode>cluster</mode>-->
|
||||
<!-- <name>GenerateCrossrefDataset</name>-->
|
||||
<!-- <class>eu.dnetlib.doiboost.crossref.CrossrefDataset</class>-->
|
||||
<!-- <jar>dhp-doiboost-${projectVersion}.jar</jar>-->
|
||||
<!-- <spark-opts>-->
|
||||
<!-- --executor-memory=${sparkExecutorMemory}-->
|
||||
<!-- --executor-cores=${sparkExecutorCores}-->
|
||||
<!-- --driver-memory=${sparkDriverMemory}-->
|
||||
<!-- --conf spark.sql.shuffle.partitions=3840-->
|
||||
<!-- --conf spark.extraListeners=${spark2ExtraListeners}-->
|
||||
<!-- --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}-->
|
||||
<!-- --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}-->
|
||||
<!-- --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}-->
|
||||
<!-- </spark-opts>-->
|
||||
<!-- <arg>--workingPath</arg><arg>${inputPathCrossref}</arg>-->
|
||||
<!-- <arg>--master</arg><arg>yarn-cluster</arg>-->
|
||||
<!-- </spark>-->
|
||||
<!-- <ok to="RenameDataset"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
<!-- <action name="RenameDataset">-->
|
||||
<!-- <fs>-->
|
||||
<!-- <delete path="${inputPathCrossref}/crossref_ds"/>-->
|
||||
<!-- <move source="${inputPathCrossref}/crossref_ds_updated"-->
|
||||
<!-- target="${inputPathCrossref}/crossref_ds"/>-->
|
||||
<!-- </fs>-->
|
||||
<!-- <ok to="ResetMagWorkingPath"/>-->
|
||||
<!-- <error to="Kill"/>-->
|
||||
<!-- </action>-->
|
||||
|
||||
|
||||
|
||||
<!-- MAG SECTION -->
|
||||
<action name="ResetMagWorkingPath">
|
||||
<fs>
|
||||
<delete path="${inputPathMAG}/dataset"/>
|
||||
<delete path="${inputPathMAG}/process"/>
|
||||
</fs>
|
||||
<ok to="ConvertMagToDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertMagToDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${MAGDumpPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="ConvertCrossrefToOAF"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="ConvertCrossrefToOAF">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
|
@ -124,14 +319,14 @@
|
|||
--executor-memory=${sparkExecutorIntersectionMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.sql.shuffle.partitions=7680
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--workingPath</arg><arg>${inputPathMAG}/process_p</arg>
|
||||
<arg>--workingPath</arg><arg>${inputPathMAG}/process</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
|
@ -185,6 +380,7 @@
|
|||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathOrcid}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPathOrcid}</arg>
|
||||
<arg>--targetPath</arg><arg>${workingPath}/orcidPublication</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
|
@ -34,7 +34,7 @@
|
|||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert ORCID to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.orcid.SparkPreprocessORCID</class>
|
||||
<class>eu.dnetlib.doiboost.orcid.SparkConvertORCIDToOAF</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
|
|
|
@ -7,14 +7,9 @@
|
|||
</property>
|
||||
<property>
|
||||
<name>outputPath</name>
|
||||
<value>/data/orcid_activities_2020/no_doi_dataset_prod/</value>
|
||||
<description>path where to store the action set</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>processOutputPath</name>
|
||||
<value>/data/orcid_activities_2020/process_no_doi_dataset_prod</value>
|
||||
<description>temporary path where to store the action set</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>spark2GenNoDoiDatasetMaxExecutors</name>
|
||||
<value>40</value>
|
||||
|
@ -71,7 +66,7 @@
|
|||
|
||||
<action name="ResetWorkingPath">
|
||||
<fs>
|
||||
<delete path='${processOutputPath}'/>
|
||||
<delete path='${workingPath}/no_doi_dataset'/>
|
||||
</fs>
|
||||
<ok to="GenOrcidNoDoiDataset"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -97,7 +92,7 @@
|
|||
<arg>--workingPath</arg><arg>${workingPath}/</arg>
|
||||
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||
<arg>--orcidDataFolder</arg><arg>last_orcid_dataset</arg>
|
||||
<arg>--outputEnrichedWorksPath</arg><arg>${processOutputPath}</arg>
|
||||
<arg>--outputEnrichedWorksPath</arg><arg>no_doi_dataset</arg>
|
||||
</spark>
|
||||
<ok to="importOrcidNoDoi"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -105,7 +100,7 @@
|
|||
|
||||
<action name="importOrcidNoDoi">
|
||||
<distcp xmlns="uri:oozie:distcp-action:0.2">
|
||||
<arg>${processOutputPath}/*</arg>
|
||||
<arg>${workingPath}/no_doi_dataset/*</arg>
|
||||
<arg>${outputPath}</arg>
|
||||
</distcp>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -1,216 +0,0 @@
|
|||
<workflow-app name="Generate DOIBoost ActionSet" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
|
||||
<!-- Crossref Parameters -->
|
||||
<property>
|
||||
<name>inputPathCrossref</name>
|
||||
<description>the Crossref input path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>crossrefDumpPath</name>
|
||||
<description>the Crossref dump path</description>
|
||||
</property>
|
||||
|
||||
<!-- MAG Parameters -->
|
||||
<property>
|
||||
<name>MAGDumpPath</name>
|
||||
<description>the MAG dump working path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>inputPathMAG</name>
|
||||
<description>the MAG working path</description>
|
||||
</property>
|
||||
|
||||
|
||||
<!-- ORCID Parameters -->
|
||||
<property>
|
||||
<name>inputPathOrcid</name>
|
||||
<description>the ORCID input path</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>workingPathOrcid</name>
|
||||
<description>the ORCID working path</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"/>
|
||||
|
||||
<decision name="resume_from">
|
||||
<switch>
|
||||
<case to="UnpackCrossrefEntries">${wf:conf('resumeFrom') eq 'UnpackCrossrefEntries'}</case>
|
||||
<case to="GenerateCrossrefDataset">${wf:conf('resumeFrom') eq 'GenerateCrossrefDataset'}</case>
|
||||
<case to="ResetMagWorkingPath">${wf:conf('resumeFrom') eq 'ResetMagWorkingPath'}</case>
|
||||
<case to="ConvertMagToDataset">${wf:conf('resumeFrom') eq 'ConvertMagToDataset'}</case>
|
||||
<case to="PreProcessORCID">${wf:conf('resumeFrom') eq 'PreProcessORCID'}</case>
|
||||
<default to="ImportCrossRef"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="ImportCrossRef">
|
||||
<java>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<main-class>eu.dnetlib.doiboost.crossref.ExtractCrossrefRecords</main-class>
|
||||
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||
<arg>--crossrefFileNameTarGz</arg><arg>${crossrefDumpPath}/crossref.tar.gz</arg>
|
||||
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${crossrefDumpPath}/files/</arg>
|
||||
</java>
|
||||
<ok to="UnpackCrossrefEntries"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="UnpackCrossrefEntries">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>SparkUnpackCrossrefEntries</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/files</arg>
|
||||
<arg>--targetPath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||
|
||||
</spark>
|
||||
<ok to="GenerateCrossrefDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="GenerateCrossrefDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>SparkGenerateCrossrefDataset</name>
|
||||
<class>eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=7G
|
||||
--executor-cores=2
|
||||
--driver-memory=7G
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||
|
||||
</spark>
|
||||
<ok to="removeFiles"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="removeFiles">
|
||||
<fs>
|
||||
<!-- <delete path="${crossrefDumpPath}/files"/>-->
|
||||
<delete path="${crossrefDumpPath}/crossref_unpack/"/>
|
||||
</fs>
|
||||
<ok to="ResetMagWorkingPath"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<!-- MAG SECTION -->
|
||||
<action name="ResetMagWorkingPath">
|
||||
<fs>
|
||||
<delete path="${inputPathMAG}/dataset"/>
|
||||
<delete path="${inputPathMAG}/process"/>
|
||||
</fs>
|
||||
<ok to="ConvertMagToDataset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="ConvertMagToDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert Mag to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.mag.SparkImportMagIntoDataset</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${MAGDumpPath}</arg>
|
||||
<arg>--targetPath</arg><arg>${inputPathMAG}/dataset</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="PreProcessORCID"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<!-- ORCID SECTION -->
|
||||
<action name="PreProcessORCID">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert ORCID to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.orcid.SparkPreprocessORCID</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${inputPathOrcid}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPathOrcid}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,38 @@
|
|||
<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>
|
||||
</configuration>
|
|
@ -0,0 +1,55 @@
|
|||
<workflow-app name="import UnpayWall into HDFS" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>targetPath</name>
|
||||
<description>the working dir base path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="PreprocessUW"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="PreprocessUW">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Convert UnpayWall to Dataset</name>
|
||||
<class>eu.dnetlib.doiboost.uw.SparkMapUnpayWallToOAF</class>
|
||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}/uw_extracted</arg>
|
||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,406 @@
|
|||
|
||||
package eu.dnetlib.dhp.doiboost;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.neethi.Assertion;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.doiboost.DoiBoostAuthorMerger;
|
||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class DoiBoostAuthorMergerTest {
|
||||
|
||||
private String publicationsBasePath;
|
||||
|
||||
private List<List<Author>> authors;
|
||||
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
|
||||
publicationsBasePath = Paths
|
||||
.get(DoiBoostAuthorMergerTest.class.getResource("/eu/dnetlib/dhp/doiboost").toURI())
|
||||
.toFile()
|
||||
.getAbsolutePath();
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mergeTestOrcid() {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/matching_authors_first.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (List<Author> authors1 : authors) {
|
||||
System.out.println("List " + (authors.indexOf(authors1) + 1));
|
||||
for (Author author : authors1) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
}
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(10, merge.size());
|
||||
|
||||
Assertions.assertEquals(3, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
merge
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null)
|
||||
.forEach(
|
||||
a -> Assertions
|
||||
.assertTrue(
|
||||
a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))));
|
||||
merge.stream().filter(a -> a.getPid() != null).forEach(a -> {
|
||||
try {
|
||||
System.out.println(new ObjectMapper().writeValueAsString(a));
|
||||
} catch (JsonProcessingException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
public <T> List<Tuple2<String, T>> readSample(String path, Class<T> clazz) {
|
||||
List<Tuple2<String, T>> res = new ArrayList<>();
|
||||
BufferedReader reader;
|
||||
try {
|
||||
reader = new BufferedReader(new FileReader(path));
|
||||
String line = reader.readLine();
|
||||
while (line != null) {
|
||||
res
|
||||
.add(
|
||||
new Tuple2<>(
|
||||
MapDocumentUtil.getJPathString("$.id", line),
|
||||
new ObjectMapper().readValue(line, clazz)));
|
||||
// read next line
|
||||
line = reader.readLine();
|
||||
}
|
||||
reader.close();
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
public String authorToString(Author a) {
|
||||
|
||||
String print = "Fullname = ";
|
||||
print += a.getFullname() + " pid = [";
|
||||
if (a.getPid() != null)
|
||||
for (StructuredProperty sp : a.getPid()) {
|
||||
print += sp.toComparableString() + " ";
|
||||
}
|
||||
print += "]";
|
||||
return print;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mergeTestMAG() {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/matching_authors_second", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (List<Author> authors1 : authors) {
|
||||
System.out.println("List " + (authors.indexOf(authors1) + 1));
|
||||
for (Author author : authors1) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
}
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(10, merge.size());
|
||||
|
||||
Assertions.assertEquals(10, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
merge
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null)
|
||||
.forEach(
|
||||
a -> Assertions
|
||||
.assertTrue(
|
||||
a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals("URL"))));
|
||||
merge.stream().filter(a -> a.getPid() != null).forEach(a -> {
|
||||
try {
|
||||
System.out.println(new ObjectMapper().writeValueAsString(a));
|
||||
} catch (JsonProcessingException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mergeTestCrossrefEmpty1() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/empty_crossref_authors_first.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(3, merge.size());
|
||||
|
||||
Assertions.assertEquals(3, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
merge
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null)
|
||||
.forEach(
|
||||
a -> Assertions
|
||||
.assertTrue(
|
||||
a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))));
|
||||
merge.stream().filter(a -> a.getPid() != null).forEach(a -> {
|
||||
try {
|
||||
System.out.println(new ObjectMapper().writeValueAsString(a));
|
||||
} catch (JsonProcessingException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
|
||||
System.out.println(new ObjectMapper().writeValueAsString(merge));
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mergeTestCrossrefEmpty2() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/empty_crossref_authors_second.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, false);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(10, merge.size());
|
||||
|
||||
Assertions.assertEquals(10, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
merge
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null)
|
||||
.forEach(
|
||||
a -> Assertions
|
||||
.assertTrue(
|
||||
a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals("URL"))));
|
||||
merge.stream().filter(a -> a.getPid() != null).forEach(a -> {
|
||||
try {
|
||||
System.out.println(new ObjectMapper().writeValueAsString(a));
|
||||
} catch (JsonProcessingException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
|
||||
Assertions.assertTrue(3 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mergeTestCrossrefEmpty3() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/empty_crossref_author_third.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(10, merge.size());
|
||||
|
||||
Assertions.assertEquals(10, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
merge
|
||||
.stream()
|
||||
.filter(a -> a.getPid() != null)
|
||||
.forEach(
|
||||
a -> Assertions
|
||||
.assertTrue(
|
||||
a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals("URL"))));
|
||||
|
||||
Assertions.assertTrue(3 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mergeTestCrossrefEmpty4() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/empty_crossref_author_fourth.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
Assertions.assertEquals(3, merge.size());
|
||||
|
||||
Assertions.assertEquals(3, merge.stream().filter(a -> a.getPid() != null).count());
|
||||
|
||||
|
||||
Assertions.assertTrue(3 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldMergeTest1() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/should_appear_author1.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
Assertions.assertTrue(6 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
|
||||
Assertions.assertTrue(34 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING))).count());
|
||||
|
||||
merge.stream().filter(a -> a.getRank() == 26)
|
||||
.forEach(a ->
|
||||
Assertions.assertTrue(a.getPid()
|
||||
.stream()
|
||||
.anyMatch(pid -> pid.getValue().equals("0000-0002-2445-5275")
|
||||
&& pid.getQualifier().getClassid().equals(ModelConstants.ORCID)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldMergeTest2() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/should_appear_author2.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
|
||||
|
||||
Assertions.assertTrue(5 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
|
||||
Assertions.assertTrue(34 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
.filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING))).count());
|
||||
|
||||
merge.stream().filter(a -> a.getFullname().equals("da luz geraldo eduardo"))
|
||||
.forEach(a ->
|
||||
Assertions.assertTrue(a.getPid()
|
||||
.stream()
|
||||
.anyMatch(pid -> pid.getValue().equals("http://orcid.org/0000-0003-2434-0387")
|
||||
&& pid.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING)
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotMergeTest1() throws JsonProcessingException {
|
||||
|
||||
authors = readSample(publicationsBasePath + "/should_appear_author3.json", Publication.class)
|
||||
.stream()
|
||||
.map(p -> p._2().getAuthor())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
|
||||
List<Author> merge = DoiBoostAuthorMerger.merge(authors, true);
|
||||
|
||||
System.out.println("Merge ");
|
||||
for (Author author : merge) {
|
||||
System.out.println(authorToString(author));
|
||||
}
|
||||
|
||||
// Assertions.assertTrue(5 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
// .filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID))).count());
|
||||
//
|
||||
// Assertions.assertTrue(34 == merge.stream().filter(a -> a.getPid() !=null)
|
||||
// .filter(a -> a.getPid().stream().anyMatch(p -> p.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING))).count());
|
||||
//
|
||||
// merge.stream().filter(a -> a.getFullname().equals("da luz geraldo eduardo"))
|
||||
// .forEach(a ->
|
||||
// Assertions.assertTrue(a.getPid()
|
||||
// .stream()
|
||||
// .anyMatch(pid -> pid.getValue().equals("http://orcid.org/0000-0003-2434-0387")
|
||||
// && pid.getQualifier().getClassid().equals(ModelConstants.ORCID_PENDING)
|
||||
// )
|
||||
// )
|
||||
// );
|
||||
|
||||
|
||||
}
|
||||
}
|
|
@ -1,15 +1,22 @@
|
|||
package eu.dnetlib.doiboost.mag
|
||||
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, SparkSession}
|
||||
import org.codehaus.jackson.map.ObjectMapper
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.json4s.DefaultFormats
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import java.sql.Timestamp
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication
|
||||
import org.apache.htrace.fasterxml.jackson.databind.SerializationFeature
|
||||
import org.apache.spark.{SparkConf, SparkContext}
|
||||
import org.apache.spark.api.java.function.MapFunction
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.apache.spark.sql.functions._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.io.Source
|
||||
import scala.reflect.ClassTag
|
||||
import scala.util.matching.Regex
|
||||
|
||||
|
||||
|
||||
|
@ -58,19 +65,14 @@ class MAGMappingTest {
|
|||
@Test
|
||||
def normalizeDoiTest():Unit = {
|
||||
|
||||
|
||||
import org.json4s.jackson.Serialization.write
|
||||
import org.json4s.DefaultFormats
|
||||
|
||||
implicit val formats = DefaultFormats
|
||||
|
||||
val conf = new SparkConf()
|
||||
conf.setMaster("local[*]")
|
||||
conf.set("spark.driver.host", "localhost")
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.appName(getClass.getSimpleName)
|
||||
.config(conf)
|
||||
.getOrCreate()
|
||||
val conf = new SparkConf().setAppName("test").setMaster("local[2]")
|
||||
val sc = new SparkContext(conf)
|
||||
val spark = SparkSession.builder.config(sc.getConf).getOrCreate()
|
||||
val path = getClass.getResource("magPapers.json").getPath
|
||||
|
||||
import org.apache.spark.sql.Encoders
|
||||
|
@ -88,19 +90,14 @@ class MAGMappingTest {
|
|||
@Test
|
||||
def normalizeDoiTest2():Unit = {
|
||||
|
||||
import org.json4s.jackson.Serialization.write
|
||||
import org.json4s.DefaultFormats
|
||||
|
||||
implicit val formats = DefaultFormats
|
||||
|
||||
val conf = new SparkConf()
|
||||
conf.setMaster("local[*]")
|
||||
conf.set("spark.driver.host", "localhost")
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.appName(getClass.getSimpleName)
|
||||
.config(conf)
|
||||
.getOrCreate()
|
||||
val conf = new SparkConf().setAppName("test").setMaster("local[2]")
|
||||
val sc = new SparkContext(conf)
|
||||
val spark = SparkSession.builder.config(sc.getConf).getOrCreate()
|
||||
val path = getClass.getResource("duplicatedMagPapers.json").getPath
|
||||
|
||||
import org.apache.spark.sql.Encoders
|
||||
|
|
|
@ -46,9 +46,7 @@ class MappingORCIDToOAFTest {
|
|||
implicit val mapEncoderPubs: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
import spark.implicits._
|
||||
|
||||
SparkPreprocessORCID.run( spark,sourcePath, workingPath)
|
||||
|
||||
SparkConvertORCIDToOAF.run(spark, workingPath,targetPath)
|
||||
SparkConvertORCIDToOAF.run( spark,sourcePath, workingPath, targetPath)
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
|
||||
|
@ -63,9 +61,7 @@ class MappingORCIDToOAFTest {
|
|||
assertTrue(oA == p.count())
|
||||
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p.first()))
|
||||
|
||||
spark.close()
|
||||
|
||||
|
||||
spark.close()
|
||||
}
|
||||
|
||||
|
||||
|
@ -83,7 +79,7 @@ class MappingORCIDToOAFTest {
|
|||
assert(oaf.getPid.size() == 1)
|
||||
oaf.getPid.toList.foreach(pid => assert(pid.getQualifier.getClassid.equals("doi")))
|
||||
oaf.getPid.toList.foreach(pid => assert(pid.getValue.equals("10.1042/BCJ20160876")))
|
||||
//println(mapper.writeValueAsString(ORCIDToOAF.convertTOOAF(orcid)))
|
||||
//println(mapper.writeValueAsString(oaf))
|
||||
|
||||
|
||||
}
|
||||
|
|
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
|
@ -71,8 +71,6 @@ public abstract class AbstractMdRecordToOafMapper {
|
|||
|
||||
private final boolean shouldHashId;
|
||||
|
||||
private final boolean forceOriginalId;
|
||||
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_4 = "http://datacite.org/schema/kernel-4";
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_4_SLASH = "http://datacite.org/schema/kernel-4/";
|
||||
protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3";
|
||||
|
@ -100,20 +98,11 @@ public abstract class AbstractMdRecordToOafMapper {
|
|||
nsContext.put("datacite", DATACITE_SCHEMA_KERNEL_3);
|
||||
}
|
||||
|
||||
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
||||
final boolean shouldHashId, final boolean forceOriginalId) {
|
||||
this.vocs = vocs;
|
||||
this.invisible = invisible;
|
||||
this.shouldHashId = shouldHashId;
|
||||
this.forceOriginalId = forceOriginalId;
|
||||
}
|
||||
|
||||
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
|
||||
final boolean shouldHashId) {
|
||||
this.vocs = vocs;
|
||||
this.invisible = invisible;
|
||||
this.shouldHashId = shouldHashId;
|
||||
this.forceOriginalId = false;
|
||||
}
|
||||
|
||||
public List<Oaf> processMdRecord(final String xml) {
|
||||
|
@ -201,16 +190,10 @@ public abstract class AbstractMdRecordToOafMapper {
|
|||
final long lastUpdateTimestamp) {
|
||||
|
||||
final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||
|
||||
final Set<String> originalId = Sets.newHashSet(entity.getOriginalId());
|
||||
originalId.add(entity.getId());
|
||||
entity.setOriginalId(Lists.newArrayList(originalId));
|
||||
|
||||
if (!forceOriginalId) {
|
||||
final String id = IdentifierFactory.createIdentifier(entity, shouldHashId);
|
||||
if (!id.equals(entity.getId())) {
|
||||
entity.setId(id);
|
||||
}
|
||||
final String id = IdentifierFactory.createIdentifier(entity, shouldHashId);
|
||||
if (!id.equals(entity.getId())) {
|
||||
entity.getOriginalId().add(entity.getId());
|
||||
entity.setId(id);
|
||||
}
|
||||
|
||||
final List<Oaf> oafs = Lists.newArrayList(entity);
|
||||
|
|
|
@ -163,13 +163,11 @@ public class GenerateEntitiesApplication {
|
|||
|
||||
switch (type.toLowerCase()) {
|
||||
case "oaf-store-cleaned":
|
||||
return new OafToOafMapper(vocs, false, shouldHashId).processMdRecord(s);
|
||||
case "oaf-store-claim":
|
||||
return new OafToOafMapper(vocs, false, shouldHashId, true).processMdRecord(s);
|
||||
return new OafToOafMapper(vocs, false, shouldHashId).processMdRecord(s);
|
||||
case "odf-store-cleaned":
|
||||
return new OdfToOafMapper(vocs, false, shouldHashId).processMdRecord(s);
|
||||
case "odf-store-claim":
|
||||
return new OdfToOafMapper(vocs, false, shouldHashId, true).processMdRecord(s);
|
||||
return new OdfToOafMapper(vocs, false, shouldHashId).processMdRecord(s);
|
||||
case "oaf-store-intersection":
|
||||
return new OafToOafMapper(vocs, true, shouldHashId).processMdRecord(s);
|
||||
case "odf-store-intersection":
|
||||
|
|
|
@ -27,11 +27,6 @@ import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits;
|
|||
|
||||
public class OafToOafMapper extends AbstractMdRecordToOafMapper {
|
||||
|
||||
public OafToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId,
|
||||
final boolean forceOrginalId) {
|
||||
super(vocs, invisible, shouldHashId, forceOrginalId);
|
||||
}
|
||||
|
||||
public OafToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId) {
|
||||
super(vocs, invisible, shouldHashId);
|
||||
}
|
||||
|
|
|
@ -22,11 +22,6 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
|
|||
|
||||
public static final String HTTP_DX_DOI_PREIFX = "http://dx.doi.org/";
|
||||
|
||||
public OdfToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId,
|
||||
final boolean forceOrginalId) {
|
||||
super(vocs, invisible, shouldHashId, forceOrginalId);
|
||||
}
|
||||
|
||||
public OdfToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId) {
|
||||
super(vocs, invisible, shouldHashId);
|
||||
}
|
||||
|
|
|
@ -1,31 +0,0 @@
|
|||
package eu.dnetlib.dhp.oa.sx.graphimport
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.SparkSession
|
||||
|
||||
object SparkDataciteToOAF {
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/datacite_to_df_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val sc = spark.sparkContext
|
||||
|
||||
val inputPath = parser.get("inputPath")
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,176 @@
|
|||
package eu.dnetlib.dhp.sx.ebi
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
|
||||
import org.apache.spark.sql.{Encoder, Encoders}
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
|
||||
|
||||
|
||||
object EBIAggregator {
|
||||
|
||||
def getDatasetAggregator(): Aggregator[(String, OafDataset), OafDataset, OafDataset] = new Aggregator[(String, OafDataset), OafDataset, OafDataset]{
|
||||
|
||||
override def zero: OafDataset = new OafDataset()
|
||||
|
||||
override def reduce(b: OafDataset, a: (String, OafDataset)): OafDataset = {
|
||||
b.mergeFrom(a._2)
|
||||
b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
|
||||
override def merge(wx: OafDataset, wy: OafDataset): OafDataset = {
|
||||
wx.mergeFrom(wy)
|
||||
wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: OafDataset): OafDataset = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[OafDataset] =
|
||||
Encoders.kryo(classOf[OafDataset])
|
||||
|
||||
override def outputEncoder: Encoder[OafDataset] =
|
||||
Encoders.kryo(classOf[OafDataset])
|
||||
}
|
||||
|
||||
def getDLIUnknownAggregator(): Aggregator[(String, DLIUnknown), DLIUnknown, DLIUnknown] = new Aggregator[(String, DLIUnknown), DLIUnknown, DLIUnknown]{
|
||||
|
||||
override def zero: DLIUnknown = new DLIUnknown()
|
||||
|
||||
override def reduce(b: DLIUnknown, a: (String, DLIUnknown)): DLIUnknown = {
|
||||
b.mergeFrom(a._2)
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
override def merge(wx: DLIUnknown, wy: DLIUnknown): DLIUnknown = {
|
||||
wx.mergeFrom(wy)
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: DLIUnknown): DLIUnknown = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[DLIUnknown] =
|
||||
Encoders.kryo(classOf[DLIUnknown])
|
||||
|
||||
override def outputEncoder: Encoder[DLIUnknown] =
|
||||
Encoders.kryo(classOf[DLIUnknown])
|
||||
}
|
||||
|
||||
def getDLIDatasetAggregator(): Aggregator[(String, DLIDataset), DLIDataset, DLIDataset] = new Aggregator[(String, DLIDataset), DLIDataset, DLIDataset]{
|
||||
|
||||
override def zero: DLIDataset = new DLIDataset()
|
||||
|
||||
override def reduce(b: DLIDataset, a: (String, DLIDataset)): DLIDataset = {
|
||||
b.mergeFrom(a._2)
|
||||
b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
override def merge(wx: DLIDataset, wy: DLIDataset): DLIDataset = {
|
||||
wx.mergeFrom(wy)
|
||||
wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: DLIDataset): DLIDataset = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[DLIDataset] =
|
||||
Encoders.kryo(classOf[DLIDataset])
|
||||
|
||||
override def outputEncoder: Encoder[DLIDataset] =
|
||||
Encoders.kryo(classOf[DLIDataset])
|
||||
}
|
||||
|
||||
|
||||
def getDLIPublicationAggregator(): Aggregator[(String, DLIPublication), DLIPublication, DLIPublication] = new Aggregator[(String, DLIPublication), DLIPublication, DLIPublication]{
|
||||
|
||||
override def zero: DLIPublication = new DLIPublication()
|
||||
|
||||
override def reduce(b: DLIPublication, a: (String, DLIPublication)): DLIPublication = {
|
||||
b.mergeFrom(a._2)
|
||||
b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
|
||||
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
|
||||
override def merge(wx: DLIPublication, wy: DLIPublication): DLIPublication = {
|
||||
wx.mergeFrom(wy)
|
||||
wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: DLIPublication): DLIPublication = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[DLIPublication] =
|
||||
Encoders.kryo(classOf[DLIPublication])
|
||||
|
||||
override def outputEncoder: Encoder[DLIPublication] =
|
||||
Encoders.kryo(classOf[DLIPublication])
|
||||
}
|
||||
|
||||
|
||||
def getPublicationAggregator(): Aggregator[(String, Publication), Publication, Publication] = new Aggregator[(String, Publication), Publication, Publication]{
|
||||
|
||||
override def zero: Publication = new Publication()
|
||||
|
||||
override def reduce(b: Publication, a: (String, Publication)): Publication = {
|
||||
b.mergeFrom(a._2)
|
||||
b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
|
||||
if (b.getId == null)
|
||||
b.setId(a._2.getId)
|
||||
b
|
||||
}
|
||||
|
||||
|
||||
override def merge(wx: Publication, wy: Publication): Publication = {
|
||||
wx.mergeFrom(wy)
|
||||
wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
|
||||
if(wx.getId == null && wy.getId.nonEmpty)
|
||||
wx.setId(wy.getId)
|
||||
wx
|
||||
}
|
||||
override def finish(reduction: Publication): Publication = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[Publication] =
|
||||
Encoders.kryo(classOf[Publication])
|
||||
|
||||
override def outputEncoder: Encoder[Publication] =
|
||||
Encoders.kryo(classOf[Publication])
|
||||
}
|
||||
|
||||
def getRelationAggregator(): Aggregator[(String, Relation), Relation, Relation] = new Aggregator[(String, Relation), Relation, Relation]{
|
||||
|
||||
override def zero: Relation = new Relation()
|
||||
|
||||
override def reduce(b: Relation, a: (String, Relation)): Relation = {
|
||||
a._2
|
||||
}
|
||||
|
||||
|
||||
override def merge(a: Relation, b: Relation): Relation = {
|
||||
if(b!= null) b else a
|
||||
}
|
||||
override def finish(reduction: Relation): Relation = reduction
|
||||
|
||||
override def bufferEncoder: Encoder[Relation] =
|
||||
Encoders.kryo(classOf[Relation])
|
||||
|
||||
override def outputEncoder: Encoder[Relation] =
|
||||
Encoders.kryo(classOf[Relation])
|
||||
}
|
||||
}
|
|
@ -0,0 +1,247 @@
|
|||
package eu.dnetlib.dhp.sx.ebi
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Instance, Journal, KeyValue, Oaf, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils.createQualifier
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, OafUtils, ProvenaceInfo}
|
||||
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal}
|
||||
import eu.dnetlib.dhp.utils.DHPUtils
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql._
|
||||
import org.json4s
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||
import org.json4s.jackson.JsonMethods.parse
|
||||
import org.apache.spark.sql.functions._
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object SparkAddLinkUpdates {
|
||||
|
||||
val relationMapper = RelationMapper.load
|
||||
|
||||
|
||||
case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:String, turl:String, title:String, publisher:String) {}
|
||||
|
||||
|
||||
def generatePubmedDLICollectedFrom(): KeyValue = {
|
||||
OafUtils.generateKeyValue("dli_________::europe_pmc__", "Europe PMC")
|
||||
}
|
||||
|
||||
|
||||
|
||||
def journalToOAF(pj:PMJournal): Journal = {
|
||||
val j = new Journal
|
||||
j.setIssnPrinted(pj.getIssn)
|
||||
j.setVol(pj.getVolume)
|
||||
j.setName(pj.getTitle)
|
||||
j.setIss(pj.getIssue)
|
||||
j.setDataInfo(OafUtils.generateDataInfo())
|
||||
j
|
||||
}
|
||||
|
||||
|
||||
def pubmedTOPublication(input:PMArticle):DLIPublication = {
|
||||
|
||||
|
||||
val dnetPublicationId = s"50|${DHPUtils.md5(s"${input.getPmid}::pmid")}"
|
||||
|
||||
val p = new DLIPublication
|
||||
p.setId(dnetPublicationId)
|
||||
p.setDataInfo(OafUtils.generateDataInfo())
|
||||
p.setPid(List(OafUtils.createSP(input.getPmid.toLowerCase.trim, "pmid", ModelConstants.DNET_PID_TYPES)).asJava)
|
||||
p.setCompletionStatus("complete")
|
||||
val pi = new ProvenaceInfo
|
||||
pi.setId("dli_________::europe_pmc__")
|
||||
pi.setName( "Europe PMC")
|
||||
pi.setCompletionStatus("complete")
|
||||
pi.setCollectionMode("collected")
|
||||
p.setDlicollectedfrom(List(pi).asJava)
|
||||
p.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||
|
||||
if (input.getAuthors != null && input.getAuthors.size() >0) {
|
||||
var aths: List[Author] = List()
|
||||
input.getAuthors.asScala.filter(a=> a!= null).foreach(a => {
|
||||
val c = new Author
|
||||
c.setFullname(a.getFullName)
|
||||
c.setName(a.getForeName)
|
||||
c.setSurname(a.getLastName)
|
||||
aths = aths ::: List(c)
|
||||
})
|
||||
if (aths.nonEmpty)
|
||||
p.setAuthor(aths.asJava)
|
||||
}
|
||||
|
||||
|
||||
if (input.getJournal != null)
|
||||
p.setJournal(journalToOAF(input.getJournal))
|
||||
p.setTitle(List(OafUtils.createSP(input.getTitle, "main title", ModelConstants.DNET_DATACITE_TITLE)).asJava)
|
||||
p.setDateofacceptance(OafUtils.asField(input.getDate))
|
||||
val i = new Instance
|
||||
i.setCollectedfrom(generatePubmedDLICollectedFrom())
|
||||
i.setDateofacceptance(p.getDateofacceptance)
|
||||
i.setUrl(List(s"https://pubmed.ncbi.nlm.nih.gov/${input.getPmid}").asJava)
|
||||
i.setInstancetype(createQualifier("0001", "Article", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
p.setInstance(List(i).asJava)
|
||||
p
|
||||
}
|
||||
|
||||
|
||||
def ebiLinksToOaf(input:(String, String)):List[Oaf] = {
|
||||
val pmid :String = input._1
|
||||
val input_json :String = input._2
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: json4s.JValue = parse(input_json)
|
||||
|
||||
|
||||
val targets:List[EBILinks] = for {
|
||||
JObject(link) <- json \\ "Category" \\ "Link"
|
||||
JField("PublicationDate", JString(pubdate)) <- link
|
||||
JField("RelationshipType", JObject(relationshipType)) <- link
|
||||
JField("Name", JString(relname)) <- relationshipType
|
||||
JField("Target", JObject(target)) <- link
|
||||
JField("Identifier", JObject(identifier)) <- target
|
||||
JField("ID", JString(tpid)) <- identifier
|
||||
JField("IDScheme", JString(tpidtype)) <- identifier
|
||||
JField("IDURL", JString(turl)) <- identifier
|
||||
JField("Title", JString(title)) <- target
|
||||
JField("Publisher", JObject(pub)) <- target
|
||||
JField("Name", JString(publisher)) <- pub
|
||||
} yield EBILinks(relname, pubdate, tpid, tpidtype, turl,title, publisher)
|
||||
|
||||
|
||||
|
||||
val dnetPublicationId = s"50|${DHPUtils.md5(s"$pmid::pmid")}"
|
||||
|
||||
targets.flatMap(l => {
|
||||
val relation = new Relation
|
||||
val inverseRelation = new Relation
|
||||
val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}"
|
||||
val relInfo = relationMapper.get(l.relation.toLowerCase)
|
||||
val relationSemantic = relInfo.getOriginal
|
||||
val inverseRelationSemantic = relInfo.getInverse
|
||||
|
||||
relation.setSource(dnetPublicationId)
|
||||
relation.setTarget(targetDnetId)
|
||||
relation.setRelClass("datacite")
|
||||
relation.setRelType(relationSemantic)
|
||||
relation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||
|
||||
inverseRelation.setSource(targetDnetId)
|
||||
inverseRelation.setTarget(dnetPublicationId)
|
||||
inverseRelation.setRelClass("datacite")
|
||||
inverseRelation.setRelType(inverseRelationSemantic)
|
||||
inverseRelation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||
|
||||
|
||||
|
||||
val d = new DLIDataset
|
||||
d.setId(targetDnetId)
|
||||
d.setDataInfo(OafUtils.generateDataInfo())
|
||||
d.setPid(List(OafUtils.createSP(l.tpid.toLowerCase.trim, l.tpidType.toLowerCase.trim, ModelConstants.DNET_PID_TYPES)).asJava)
|
||||
d.setCompletionStatus("complete")
|
||||
val pi = new ProvenaceInfo
|
||||
pi.setId("dli_________::europe_pmc__")
|
||||
pi.setName( "Europe PMC")
|
||||
pi.setCompletionStatus("complete")
|
||||
pi.setCollectionMode("collected")
|
||||
d.setDlicollectedfrom(List(pi).asJava)
|
||||
d.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||
d.setPublisher(OafUtils.asField(l.publisher))
|
||||
d.setTitle(List(OafUtils.createSP(l.title, "main title", ModelConstants.DNET_DATACITE_TITLE)).asJava)
|
||||
d.setDateofacceptance(OafUtils.asField(l.pubdate))
|
||||
val i = new Instance
|
||||
i.setCollectedfrom(generatePubmedDLICollectedFrom())
|
||||
i.setDateofacceptance(d.getDateofacceptance)
|
||||
i.setUrl(List(l.turl).asJava)
|
||||
i.setInstancetype(createQualifier("0021", "Dataset", ModelConstants.DNET_PUBLICATION_RESOURCE, ModelConstants.DNET_PUBLICATION_RESOURCE))
|
||||
d.setInstance(List(i).asJava)
|
||||
List(relation, inverseRelation, d)
|
||||
})
|
||||
}
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val workingPath = parser.get("workingPath")
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
implicit val oafpubEncoder: Encoder[Publication] = Encoders.kryo[Publication]
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
||||
implicit val atEncoder: Encoder[Author] = Encoders.kryo(classOf[Author])
|
||||
implicit val strEncoder:Encoder[String] = Encoders.STRING
|
||||
implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle])
|
||||
implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal])
|
||||
implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor])
|
||||
|
||||
|
||||
val ds:Dataset[(String,String)] = spark.read.load(s"$workingPath/baseline_links_updates").as[(String,String)](Encoders.tuple(Encoders.STRING, Encoders.STRING))
|
||||
|
||||
ds.flatMap(l =>ebiLinksToOaf(l)).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_oaf")
|
||||
|
||||
ds.filter(s => s.isInstanceOf)
|
||||
|
||||
|
||||
|
||||
val oDataset:Dataset[Oaf] = spark.read.load(s"$workingPath/baseline_links_updates_oaf").as[Oaf]
|
||||
|
||||
oDataset.filter(p =>p.isInstanceOf[Relation]).map(p => p.asInstanceOf[Relation]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_relation")
|
||||
oDataset.filter(p =>p.isInstanceOf[DLIDataset]).map(p => p.asInstanceOf[DLIDataset]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_dataset")
|
||||
|
||||
|
||||
val idPublicationSolved:Dataset[String] = spark.read.load(s"$workingPath/baseline_links_updates").where(col("links").isNotNull).select("pmid").as[String]
|
||||
val baseline:Dataset[(String, PMArticle)]= spark.read.load(s"$workingPath/baseline_dataset").as[PMArticle].map(p=> (p.getPmid, p))(Encoders.tuple(strEncoder,PMEncoder))
|
||||
idPublicationSolved.joinWith(baseline, idPublicationSolved("pmid").equalTo(baseline("_1"))).map(k => pubmedTOPublication(k._2._2)).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_publication")
|
||||
|
||||
|
||||
val pmaDatasets = spark.read.load("/user/sandro.labruzzo/scholix/EBI/ebi_garr/baseline_dataset").as[PMArticle]
|
||||
|
||||
pmaDatasets.map(p => pubmedTOPublication(p)).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_publication_all")
|
||||
|
||||
val pubs: Dataset[(String,Publication)] = spark.read.load("/user/sandro.labruzzo/scholix/EBI/publication").as[Publication].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING,oafpubEncoder))
|
||||
val pubdate:Dataset[(String,DLIPublication)] = spark.read.load(s"$workingPath/baseline_publication_all").as[DLIPublication].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING,pubEncoder))
|
||||
|
||||
|
||||
|
||||
pubs.joinWith(pubdate, pubs("_1").equalTo(pubdate("_1"))).map(k => k._2._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_publication_ebi")
|
||||
|
||||
|
||||
|
||||
val dt : Dataset[DLIDataset] = spark.read.load(s"$workingPath/dataset").as[DLIDataset]
|
||||
val update : Dataset[DLIDataset] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_dataset").as[DLIDataset]
|
||||
|
||||
|
||||
dt.union(update).map(d => (d.getId,d))(Encoders.tuple(Encoders.STRING, datEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIDatasetAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset_ebi")
|
||||
|
||||
|
||||
val rel: Dataset[Relation] = spark.read.load(s"$workingPath/relation").as[Relation]
|
||||
val relupdate : Dataset[Relation] = spark.read.load(s"$workingPath/ebi_garr/baseline_links_updates_relation").as[Relation]
|
||||
|
||||
|
||||
rel.union(relupdate)
|
||||
.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite)
|
||||
.save(s"$workingPath/baseline_relation_ebi")
|
||||
|
||||
}
|
||||
}
|
|
@ -1,16 +1,12 @@
|
|||
package eu.dnetlib.dhp.sx.graph.ebi
|
||||
package eu.dnetlib.dhp.sx.ebi
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
||||
import eu.dnetlib.dhp.schema.oaf.Result
|
||||
import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal, PMParser}
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.io.Source
|
||||
import scala.xml.pull.XMLEventReader
|
||||
|
@ -40,37 +36,24 @@ object SparkCreateBaselineDataFrame {
|
|||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkEBILinksToOaf.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json")))
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val isLookupUrl: String = parser.get("isLookupUrl")
|
||||
log.info("isLookupUrl: {}", isLookupUrl)
|
||||
val workingPath = parser.get("workingPath")
|
||||
log.info("workingPath: {}", workingPath)
|
||||
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info("targetPath: {}", targetPath)
|
||||
|
||||
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
|
||||
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
||||
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val sc = spark.sparkContext
|
||||
|
||||
|
||||
val workingPath = parser.get("workingPath")
|
||||
|
||||
implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle])
|
||||
implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal])
|
||||
implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor])
|
||||
implicit val resultEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
|
||||
|
||||
val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline",2000)
|
||||
val ds:Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i =>{
|
||||
val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes()))
|
||||
|
@ -81,13 +64,5 @@ object SparkCreateBaselineDataFrame {
|
|||
ds.map(p => (p.getPmid,p))(Encoders.tuple(Encoders.STRING, PMEncoder)).groupByKey(_._1)
|
||||
.agg(pmArticleAggregator.toColumn)
|
||||
.map(p => p._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset")
|
||||
|
||||
val exported_dataset = spark.read.load(s"$workingPath/baseline_dataset").as[PMArticle]
|
||||
exported_dataset
|
||||
.map(a => PubMedToOaf.convert(a, vocabularies)).as[Result]
|
||||
.filter(p => p!= null)
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
||||
//s"$workingPath/oaf/baseline_oaf"
|
||||
}
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
package eu.dnetlib.dhp.sx.ebi
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication}
|
||||
import eu.dnetlib.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object SparkCreateEBIDataFrame {
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val logger: Logger = LoggerFactory.getLogger(SparkCreateEBIDataFrame.getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
val sc = spark.sparkContext
|
||||
|
||||
|
||||
val workingPath = parser.get("workingPath")
|
||||
val relationMapper = RelationMapper.load
|
||||
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
implicit val datasetEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||
implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo(classOf[DLIPublication])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
|
||||
// logger.info("Extract Publication and relation from publication_xml")
|
||||
// val oafPubsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/publication_xml").map(s =>
|
||||
// {
|
||||
// new ObjectMapper().readValue(s, classOf[String])
|
||||
// }).flatMap(s => {
|
||||
// val d = new PublicationScholexplorerParser
|
||||
// d.parseObject(s, relationMapper).asScala.iterator})
|
||||
//
|
||||
// val mapper = new ObjectMapper()
|
||||
// mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
||||
// spark.createDataset(oafPubsRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/oaf")
|
||||
//
|
||||
// logger.info("Extract Publication and relation from dataset_xml")
|
||||
// val oafDatsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/dataset_xml").map(s =>
|
||||
// {
|
||||
// new ObjectMapper().readValue(s, classOf[String])
|
||||
// }).flatMap(s => {
|
||||
// val d = new DatasetScholexplorerParser
|
||||
// d.parseObject(s, relationMapper).asScala.iterator})
|
||||
|
||||
// spark.createDataset(oafDatsRDD).write.mode(SaveMode.Append).save(s"$workingPath/oaf")
|
||||
val dataset: Dataset[DLIDataset] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[DLIDataset]).map(d => d.asInstanceOf[DLIDataset])
|
||||
val publication: Dataset[DLIPublication] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[DLIPublication]).map(d => d.asInstanceOf[DLIPublication])
|
||||
val relations: Dataset[Relation] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Relation]).map(d => d.asInstanceOf[Relation])
|
||||
publication.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIPublicationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/publication")
|
||||
|
||||
dataset.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, datasetEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getDLIDatasetAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/dataset")
|
||||
|
||||
relations.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
.groupByKey(_._1)(Encoders.STRING)
|
||||
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||
.map(p => p._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$workingPath/relation")
|
||||
|
||||
}
|
||||
}
|
|
@ -1,5 +1,5 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
||||
package eu.dnetlib.dhp.sx.ebi.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
|
@ -16,7 +16,6 @@ public class PMArticle implements Serializable {
|
|||
private String language;
|
||||
private final List<PMSubject> subjects = new ArrayList<>();
|
||||
private final List<PMSubject> publicationTypes = new ArrayList<>();
|
||||
private List<PMAuthor> authors = new ArrayList<>();
|
||||
|
||||
public List<PMSubject> getPublicationTypes() {
|
||||
return publicationTypes;
|
||||
|
@ -36,6 +35,8 @@ public class PMArticle implements Serializable {
|
|||
this.doi = doi;
|
||||
}
|
||||
|
||||
private List<PMAuthor> authors = new ArrayList<>();
|
||||
|
||||
public String getPmid() {
|
||||
return pmid;
|
||||
}
|
|
@ -1,5 +1,5 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
||||
package eu.dnetlib.dhp.sx.ebi.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
||||
package eu.dnetlib.dhp.sx.ebi.model;
|
||||
|
||||
public class PMGrant {
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
||||
package eu.dnetlib.dhp.sx.ebi.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
|
@ -1,4 +1,4 @@
|
|||
package eu.dnetlib.dhp.sx.graph.bio.pubmed
|
||||
package eu.dnetlib.dhp.sx.ebi.model
|
||||
|
||||
import scala.xml.MetaData
|
||||
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
|
|
@ -1,5 +1,5 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
||||
package eu.dnetlib.dhp.sx.ebi.model;
|
||||
|
||||
public class PMSubject {
|
||||
private String value;
|
|
@ -0,0 +1,3 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
case class IdReplace(newId:String, oldId:String) {}
|
|
@ -0,0 +1,153 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.bson.Document;
|
||||
import org.bson.conversions.Bson;
|
||||
|
||||
import com.mongodb.DBObject;
|
||||
import com.mongodb.MongoClient;
|
||||
import com.mongodb.QueryBuilder;
|
||||
import com.mongodb.client.FindIterable;
|
||||
import com.mongodb.client.MongoCollection;
|
||||
import com.mongodb.client.MongoDatabase;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
|
||||
/**
|
||||
* This job is responsible to collect data from mongoDatabase and store in a sequence File on HDFS Mongo database
|
||||
* contains information of each MDSTore in two collections: -metadata That contains info like: ID, format, layout,
|
||||
* interpretation -metadataManager: that contains info : ID, mongoCollectionName from the metadata collection we filter
|
||||
* the ids with Format, layout, and Interpretation from the metadataManager we get the current MONGO collection name
|
||||
* which contains metadata XML see function getCurrentId
|
||||
* <p>
|
||||
* This Job will be called different times in base at the triple we want import, and generates for each triple a
|
||||
* sequence file of XML
|
||||
*/
|
||||
public class ImportDataFromMongo {
|
||||
/**
|
||||
* It requires in input some parameters described on a file
|
||||
* eu/dnetlib/dhp/graph/sx/import_from_mongo_parameters.json
|
||||
* <p>
|
||||
* - the name node - the paht where store HDFS File - the mongo host - the mongo port - the metadata format to
|
||||
* import - the metadata layout to import - the metadata interpretation to import - the mongo database Name
|
||||
* <p>
|
||||
* This params are encoded into args
|
||||
*
|
||||
* @param args
|
||||
* @throws Exception
|
||||
*/
|
||||
public static void main(String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
ImportDataFromMongo.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/graph/argumentparser/import_from_mongo_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final int port = Integer.parseInt(parser.get("dbport"));
|
||||
final String host = parser.get("dbhost");
|
||||
|
||||
final String format = parser.get("format");
|
||||
final String layout = parser.get("layout");
|
||||
final String interpretation = parser.get("interpretation");
|
||||
|
||||
final String dbName = parser.get("dbName");
|
||||
final MongoClient client = new MongoClient(host, port);
|
||||
MongoDatabase database = client.getDatabase(dbName);
|
||||
|
||||
MongoCollection<Document> metadata = database.getCollection("metadata");
|
||||
MongoCollection<Document> metadataManager = database.getCollection("metadataManager");
|
||||
final DBObject query = QueryBuilder
|
||||
.start("format")
|
||||
.is(format)
|
||||
.and("layout")
|
||||
.is(layout)
|
||||
.and("interpretation")
|
||||
.is(interpretation)
|
||||
.get();
|
||||
final List<String> ids = new ArrayList<>();
|
||||
metadata
|
||||
.find((Bson) query)
|
||||
.forEach((Consumer<Document>) document -> ids.add(document.getString("mdId")));
|
||||
List<String> databaseId = ids
|
||||
.stream()
|
||||
.map(it -> getCurrentId(it, metadataManager))
|
||||
.filter(Objects::nonNull)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
final String hdfsuri = parser.get("namenode");
|
||||
// ====== Init HDFS File System Object
|
||||
Configuration conf = new Configuration();
|
||||
// Set FileSystem URI
|
||||
conf.set("fs.defaultFS", hdfsuri);
|
||||
// Because of Maven
|
||||
conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
|
||||
conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
|
||||
|
||||
FileSystem.get(URI.create(hdfsuri), conf);
|
||||
Path hdfswritepath = new Path(parser.get("targetPath"));
|
||||
|
||||
final AtomicInteger counter = new AtomicInteger(0);
|
||||
try (SequenceFile.Writer writer = SequenceFile
|
||||
.createWriter(
|
||||
conf,
|
||||
SequenceFile.Writer.file(hdfswritepath),
|
||||
SequenceFile.Writer.keyClass(IntWritable.class),
|
||||
SequenceFile.Writer.valueClass(Text.class))) {
|
||||
final IntWritable key = new IntWritable(counter.get());
|
||||
final Text value = new Text();
|
||||
databaseId
|
||||
.forEach(
|
||||
id -> {
|
||||
System.out.println("Reading :" + id);
|
||||
MongoCollection<Document> collection = database.getCollection(id);
|
||||
collection
|
||||
.find()
|
||||
.forEach(
|
||||
(Consumer<Document>) document -> {
|
||||
key.set(counter.getAndIncrement());
|
||||
value.set(document.getString("body"));
|
||||
|
||||
if (counter.get() % 10000 == 0) {
|
||||
System.out.println("Added " + counter.get());
|
||||
}
|
||||
try {
|
||||
writer.append(key, value);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the name of mongo collection giving an MdStore ID
|
||||
*
|
||||
* @param mdId The id of the MDStore
|
||||
* @param metadataManager The collection metadataManager on mongo which contains this information
|
||||
* @return
|
||||
*/
|
||||
private static String getCurrentId(
|
||||
final String mdId, final MongoCollection<Document> metadataManager) {
|
||||
FindIterable<Document> result = metadataManager.find((Bson) QueryBuilder.start("mdId").is(mdId).get());
|
||||
final Document item = result.first();
|
||||
return item == null ? null : item.getString("currentId");
|
||||
}
|
||||
}
|
|
@ -1,53 +0,0 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.sx.scholix.Scholix
|
||||
import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
import org.apache.hadoop.io.compress._
|
||||
|
||||
object SparkConvertObjectToJson {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/convert_object_json_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
log.info(s"sourcePath -> $sourcePath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info(s"targetPath -> $targetPath")
|
||||
val objectType = parser.get("objectType")
|
||||
log.info(s"objectType -> $objectType")
|
||||
|
||||
|
||||
implicit val scholixEncoder :Encoder[Scholix]= Encoders.kryo[Scholix]
|
||||
implicit val summaryEncoder :Encoder[ScholixSummary]= Encoders.kryo[ScholixSummary]
|
||||
|
||||
|
||||
val mapper = new ObjectMapper
|
||||
|
||||
objectType.toLowerCase match {
|
||||
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])
|
||||
case "summary" =>
|
||||
log.info("Serialize Summary")
|
||||
val d: Dataset[ScholixSummary] = spark.read.load(sourcePath).as[ScholixSummary]
|
||||
d.map(s => mapper.writeValueAsString(s))(Encoders.STRING).rdd.repartition(1000).saveAsTextFile(targetPath, classOf[GzipCodec])
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,100 +0,0 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, OtherResearchProduct, Publication, Relation, Result, Software, Dataset => OafDataset}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
|
||||
|
||||
object SparkCreateInputGraph {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/extract_entities_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val resultObject = List(
|
||||
("publication", classOf[Publication]),
|
||||
("dataset", classOf[OafDataset]),
|
||||
("software", classOf[Software]),
|
||||
("otherResearchProduct", classOf[OtherResearchProduct])
|
||||
|
||||
)
|
||||
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
implicit val publicationEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication])
|
||||
implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset])
|
||||
implicit val softwareEncoder: Encoder[Software] = Encoders.kryo(classOf[Software])
|
||||
implicit val orpEncoder: Encoder[OtherResearchProduct] = Encoders.kryo(classOf[OtherResearchProduct])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
log.info(s"sourcePath -> $sourcePath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info(s"targetPath -> $targetPath")
|
||||
|
||||
|
||||
val oafDs:Dataset[Oaf] = spark.read.load(s"$sourcePath/*").as[Oaf]
|
||||
|
||||
|
||||
log.info("Extract Publication")
|
||||
oafDs.filter(o => o.isInstanceOf[Publication]).map(p => p.asInstanceOf[Publication]).write.mode(SaveMode.Overwrite).save(s"$targetPath/extracted/publication")
|
||||
|
||||
log.info("Extract dataset")
|
||||
oafDs.filter(o => o.isInstanceOf[OafDataset]).map(p => p.asInstanceOf[OafDataset]).write.mode(SaveMode.Overwrite).save(s"$targetPath/extracted/dataset")
|
||||
|
||||
log.info("Extract software")
|
||||
oafDs.filter(o => o.isInstanceOf[Software]).map(p => p.asInstanceOf[Software]).write.mode(SaveMode.Overwrite).save(s"$targetPath/extracted/software")
|
||||
|
||||
log.info("Extract otherResearchProduct")
|
||||
oafDs.filter(o => o.isInstanceOf[OtherResearchProduct]).map(p => p.asInstanceOf[OtherResearchProduct]).write.mode(SaveMode.Overwrite).save(s"$targetPath/extracted/otherResearchProduct")
|
||||
|
||||
log.info("Extract Relation")
|
||||
oafDs.filter(o => o.isInstanceOf[Relation]).map(p => p.asInstanceOf[Relation]).write.mode(SaveMode.Overwrite).save(s"$targetPath/extracted/relation")
|
||||
|
||||
resultObject.foreach { r =>
|
||||
log.info(s"Make ${r._1} unique")
|
||||
makeDatasetUnique(s"$targetPath/extracted/${r._1}",s"$targetPath/dedup/${r._1}",spark, r._2)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
def extractEntities[T <: Oaf ](oafDs:Dataset[Oaf], targetPath:String, clazz:Class[T], log:Logger) :Unit = {
|
||||
|
||||
implicit val resEncoder: Encoder[T] = Encoders.kryo(clazz)
|
||||
log.info(s"Extract ${clazz.getSimpleName}")
|
||||
oafDs.filter(o => o.isInstanceOf[T]).map(p => p.asInstanceOf[T]).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
}
|
||||
|
||||
|
||||
def makeDatasetUnique[T <: Result ](sourcePath:String, targetPath:String, spark:SparkSession, clazz:Class[T]) :Unit = {
|
||||
import spark.implicits._
|
||||
|
||||
implicit val resEncoder: Encoder[T] = Encoders.kryo(clazz)
|
||||
|
||||
val ds:Dataset[T] = spark.read.load(sourcePath).as[T]
|
||||
|
||||
ds.groupByKey(_.getId).reduceGroups{(x,y) =>
|
||||
x.mergeFrom(y)
|
||||
x
|
||||
}.map(_._2).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,106 +0,0 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import eu.dnetlib.dhp.schema.sx.scholix.Scholix
|
||||
import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary
|
||||
import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils
|
||||
import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils.RelatedEntities
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.functions.count
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkCreateScholix {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/create_scholix_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
val relationPath = parser.get("relationPath")
|
||||
log.info(s"relationPath -> $relationPath")
|
||||
val summaryPath = parser.get("summaryPath")
|
||||
log.info(s"summaryPath -> $summaryPath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info(s"targetPath -> $targetPath")
|
||||
|
||||
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo[Relation]
|
||||
implicit val summaryEncoder: Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
|
||||
implicit val scholixEncoder: Encoder[Scholix] = Encoders.kryo[Scholix]
|
||||
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val relationDS: Dataset[(String, Relation)] = spark.read.load(relationPath).as[Relation]
|
||||
.map(r => (r.getSource, r))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
|
||||
val summaryDS: Dataset[(String, ScholixSummary)] = spark.read.load(summaryPath).as[ScholixSummary]
|
||||
.map(r => (r.getId, r))(Encoders.tuple(Encoders.STRING, summaryEncoder))
|
||||
|
||||
|
||||
relationDS.joinWith(summaryDS, relationDS("_1").equalTo(summaryDS("_1")), "left")
|
||||
.map { input: ((String, Relation), (String, ScholixSummary)) =>
|
||||
val rel: Relation = input._1._2
|
||||
val source: ScholixSummary = input._2._2
|
||||
(rel.getTarget, ScholixUtils.scholixFromSource(rel, source))
|
||||
}(Encoders.tuple(Encoders.STRING, scholixEncoder))
|
||||
.write.mode(SaveMode.Overwrite).save(s"$targetPath/scholix_from_source")
|
||||
|
||||
val scholixSource: Dataset[(String, Scholix)] = spark.read.load(s"$targetPath/scholix_from_source").as[(String, Scholix)](Encoders.tuple(Encoders.STRING, scholixEncoder))
|
||||
|
||||
scholixSource.joinWith(summaryDS, scholixSource("_1").equalTo(summaryDS("_1")), "left")
|
||||
.map { input: ((String, Scholix), (String, ScholixSummary)) =>
|
||||
if (input._2== null) {
|
||||
null
|
||||
} else {
|
||||
val s: Scholix = input._1._2
|
||||
val target: ScholixSummary = input._2._2
|
||||
ScholixUtils.generateCompleteScholix(s, target)
|
||||
}
|
||||
}.filter(s => s!= null).write.mode(SaveMode.Overwrite).save(s"$targetPath/scholix_one_verse")
|
||||
|
||||
|
||||
val scholix_o_v: Dataset[Scholix] = spark.read.load(s"$targetPath/scholix_one_verse").as[Scholix]
|
||||
|
||||
scholix_o_v.flatMap(s => List(s, ScholixUtils.createInverseScholixRelation(s))).as[Scholix]
|
||||
.map(s=> (s.getIdentifier,s))(Encoders.tuple(Encoders.STRING, scholixEncoder))
|
||||
.groupByKey(_._1)
|
||||
.agg(ScholixUtils.scholixAggregator.toColumn)
|
||||
.map(s => s._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$targetPath/scholix")
|
||||
|
||||
val scholix_final:Dataset[Scholix] = spark.read.load(s"$targetPath/scholix").as[Scholix]
|
||||
|
||||
val stats:Dataset[(String,String,Long)]= scholix_final.map(s => (s.getSource.getDnetIdentifier, s.getTarget.getObjectType)).groupBy("_1", "_2").agg(count("_1")).as[(String,String,Long)]
|
||||
|
||||
|
||||
stats
|
||||
.map(s => RelatedEntities(s._1, if ("dataset".equalsIgnoreCase(s._2)) s._3 else 0, if ("publication".equalsIgnoreCase(s._2)) s._3 else 0 ))
|
||||
.groupByKey(_.id)
|
||||
.reduceGroups((a, b) => RelatedEntities(a.id, a.relatedDataset+b.relatedDataset, a.relatedPublication+b.relatedPublication))
|
||||
.map(_._2)
|
||||
.write.mode(SaveMode.Overwrite).save(s"$targetPath/related_entities")
|
||||
|
||||
val relatedEntitiesDS:Dataset[RelatedEntities] = spark.read.load(s"$targetPath/related_entities").as[RelatedEntities].filter(r => r.relatedPublication>0 || r.relatedDataset > 0)
|
||||
|
||||
relatedEntitiesDS.joinWith(summaryDS, relatedEntitiesDS("id").equalTo(summaryDS("_1")), "inner").map{i =>
|
||||
val re = i._1
|
||||
val sum = i._2._2
|
||||
|
||||
sum.setRelatedDatasets(re.relatedDataset)
|
||||
sum.setRelatedPublications(re.relatedPublication)
|
||||
sum
|
||||
}.write.mode(SaveMode.Overwrite).save(s"${summaryPath}_filtered")
|
||||
|
||||
}
|
||||
}
|
|
@ -1,42 +0,0 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.Result
|
||||
import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary
|
||||
import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkCreateSummaryObject {
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/create_summaries_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
log.info(s"sourcePath -> $sourcePath")
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info(s"targetPath -> $targetPath")
|
||||
|
||||
implicit val resultEncoder:Encoder[Result] = Encoders.kryo[Result]
|
||||
|
||||
implicit val summaryEncoder:Encoder[ScholixSummary] = Encoders.kryo[ScholixSummary]
|
||||
|
||||
|
||||
val ds:Dataset[Result] = spark.read.load(s"$sourcePath/*").as[Result]
|
||||
|
||||
ds.repartition(6000).map(r => ScholixUtils.resultToSummary(r)).filter(s => s!= null).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,126 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import net.minidev.json.JSONArray;
|
||||
|
||||
/**
|
||||
* This Job extracts a typology of entity and stores it in a new RDD This job is called different times, for each file
|
||||
* generated by the Job {@link ImportDataFromMongo} and store the new RDD in a path that should be under a folder:
|
||||
* extractedEntities/entity/version1
|
||||
* <p>
|
||||
* at the end of this process we will have : extractedEntities/dataset/version1 extractedEntities/dataset/version2
|
||||
* extractedEntities/dataset/... extractedEntities/publication/version1 extractedEntities/publication/version2
|
||||
* extractedEntities/publication/... extractedEntities/unknown/version1 extractedEntities/unknown/version2
|
||||
* extractedEntities/unknown/... extractedEntities/relation/version1 extractedEntities/relation/version2
|
||||
* extractedEntities/relation/...
|
||||
*/
|
||||
public class SparkExtractEntitiesJob {
|
||||
static final String IDJSONPATH = "$.id";
|
||||
static final String SOURCEJSONPATH = "$.source";
|
||||
static final String TARGETJSONPATH = "$.target";
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkExtractEntitiesJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/graph/argumentparser/input_extract_entities_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkExtractEntitiesJob.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
final String targetPath = parser.get("targetPath");
|
||||
final String tdir = parser.get("targetDir");
|
||||
final JavaRDD<String> inputRDD = sc.textFile(inputPath);
|
||||
|
||||
List<String> entities = Arrays
|
||||
.stream(parser.get("entities").split(","))
|
||||
.map(String::trim)
|
||||
.collect(Collectors.toList());
|
||||
if (entities.stream().anyMatch("dataset"::equalsIgnoreCase)) {
|
||||
// Extract Dataset
|
||||
inputRDD
|
||||
.filter(SparkExtractEntitiesJob::isDataset)
|
||||
.saveAsTextFile(targetPath + "/dataset/" + tdir, GzipCodec.class);
|
||||
}
|
||||
if (entities.stream().anyMatch("unknown"::equalsIgnoreCase)) {
|
||||
// Extract Unknown
|
||||
inputRDD
|
||||
.filter(SparkExtractEntitiesJob::isUnknown)
|
||||
.saveAsTextFile(targetPath + "/unknown/" + tdir, GzipCodec.class);
|
||||
}
|
||||
|
||||
if (entities.stream().anyMatch("relation"::equalsIgnoreCase)) {
|
||||
// Extract Relation
|
||||
inputRDD
|
||||
.filter(SparkExtractEntitiesJob::isRelation)
|
||||
.saveAsTextFile(targetPath + "/relation/" + tdir, GzipCodec.class);
|
||||
}
|
||||
if (entities.stream().anyMatch("publication"::equalsIgnoreCase)) {
|
||||
// Extract Relation
|
||||
inputRDD
|
||||
.filter(SparkExtractEntitiesJob::isPublication)
|
||||
.saveAsTextFile(targetPath + "/publication/" + tdir, GzipCodec.class);
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isDataset(final String json) {
|
||||
final String id = getJPathString(IDJSONPATH, json);
|
||||
if (StringUtils.isBlank(id))
|
||||
return false;
|
||||
return id.startsWith("60|");
|
||||
}
|
||||
|
||||
public static boolean isPublication(final String json) {
|
||||
final String id = getJPathString(IDJSONPATH, json);
|
||||
if (StringUtils.isBlank(id))
|
||||
return false;
|
||||
return id.startsWith("50|");
|
||||
}
|
||||
|
||||
public static boolean isUnknown(final String json) {
|
||||
final String id = getJPathString(IDJSONPATH, json);
|
||||
if (StringUtils.isBlank(id))
|
||||
return false;
|
||||
return id.startsWith("70|");
|
||||
}
|
||||
|
||||
public static boolean isRelation(final String json) {
|
||||
final String source = getJPathString(SOURCEJSONPATH, json);
|
||||
final String target = getJPathString(TARGETJSONPATH, json);
|
||||
return StringUtils.isNotBlank(source) && StringUtils.isNotBlank(target);
|
||||
}
|
||||
|
||||
public static String getJPathString(final String jsonPath, final String json) {
|
||||
try {
|
||||
Object o = JsonPath.read(json, jsonPath);
|
||||
if (o instanceof String)
|
||||
return (String) o;
|
||||
if (o instanceof JSONArray && ((JSONArray) o).size() > 0)
|
||||
return (String) ((JSONArray) o).get(0);
|
||||
return "";
|
||||
} catch (Exception e) {
|
||||
return "";
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,90 +0,0 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Relation, Result}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
object SparkResolveRelation {
|
||||
def main(args: Array[String]): Unit = {
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/resolve_relations_params.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val relationPath = parser.get("relationPath")
|
||||
log.info(s"sourcePath -> $relationPath")
|
||||
val entityPath = parser.get("entityPath")
|
||||
log.info(s"targetPath -> $entityPath")
|
||||
val workingPath = parser.get("workingPath")
|
||||
log.info(s"workingPath -> $workingPath")
|
||||
|
||||
|
||||
implicit val oafEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||
import spark.implicits._
|
||||
val entities:Dataset[Result] = spark.read.load(s"$entityPath/*").as[Result]
|
||||
|
||||
entities.flatMap(e => e.getPid.asScala
|
||||
.map(p =>
|
||||
convertPidToDNETIdentifier(p.getValue, p.getQualifier.getClassid))
|
||||
.filter(s => s!= null)
|
||||
.map(s => (s,e.getId))
|
||||
).groupByKey(_._1)
|
||||
.reduceGroups((x,y) => if (x._2.startsWith("50|doi") || x._2.startsWith("50|pmid")) x else y)
|
||||
.map(s =>s._2)
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(s"$workingPath/resolvedPid")
|
||||
|
||||
val rPid:Dataset[(String,String)] = spark.read.load(s"$workingPath/resolvedPid").as[(String,String)]
|
||||
|
||||
val relationDs:Dataset[(String,Relation)] = spark.read.load(relationPath).as[Relation].map(r => (r.getSource.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
|
||||
relationDs.joinWith(rPid, relationDs("_1").equalTo(rPid("_1")), "left").map{
|
||||
m =>
|
||||
val sourceResolved = m._2
|
||||
val currentRelation = m._1._2
|
||||
if (sourceResolved!=null && sourceResolved._2.nonEmpty)
|
||||
currentRelation.setSource(sourceResolved._2)
|
||||
currentRelation
|
||||
}.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(s"$workingPath/resolvedSource")
|
||||
|
||||
|
||||
val relationSourceResolved:Dataset[(String,Relation)] = spark.read.load(s"$workingPath/resolvedSource").as[Relation].map(r => (r.getTarget.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||
relationSourceResolved.joinWith(rPid, relationSourceResolved("_1").equalTo(rPid("_1")), "left").map{
|
||||
m =>
|
||||
val targetResolved = m._2
|
||||
val currentRelation = m._1._2
|
||||
if (targetResolved!=null && targetResolved._2.nonEmpty)
|
||||
currentRelation.setTarget(targetResolved._2)
|
||||
currentRelation
|
||||
}.filter(r => r.getSource.startsWith("50")&& r.getTarget.startsWith("50"))
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(s"$workingPath/resolvedRelation")
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
def convertPidToDNETIdentifier(pid:String, pidType: String):String = {
|
||||
if (pid==null || pid.isEmpty || pidType== null || pidType.isEmpty)
|
||||
null
|
||||
else
|
||||
s"unresolved::${pid.toLowerCase}::${pidType.toLowerCase}"
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* In some case the identifier generated for the Entity in @{@link SparkExtractEntitiesJob} is different from the
|
||||
* identifier * associated by the aggregator, this means that some relation points to missing identifier To avoid this
|
||||
* problem we store in the model the Id and the OriginalObJIdentifier This jobs extract this pair and creates a Similar
|
||||
* relation that will be used in SparkMergeEntities
|
||||
*/
|
||||
public class SparkSXGeneratePidSimlarity {
|
||||
|
||||
static final String IDJSONPATH = "$.id";
|
||||
static final String OBJIDPATH = "$.originalObjIdentifier";
|
||||
|
||||
public static void generateDataFrame(
|
||||
final SparkSession spark,
|
||||
final JavaSparkContext sc,
|
||||
final String inputPath,
|
||||
final String targetPath) {
|
||||
|
||||
final JavaPairRDD<String, String> datasetSimRel = sc
|
||||
.textFile(inputPath + "/dataset/*")
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, String>) k -> new Tuple2<>(
|
||||
DHPUtils.getJPathString(IDJSONPATH, k),
|
||||
DHPUtils.getJPathString(OBJIDPATH, k)))
|
||||
.filter(
|
||||
t -> !StringUtils
|
||||
.substringAfter(t._1(), "|")
|
||||
.equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::")))
|
||||
.distinct();
|
||||
|
||||
final JavaPairRDD<String, String> publicationSimRel = sc
|
||||
.textFile(inputPath + "/publication/*")
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, String>) k -> new Tuple2<>(
|
||||
DHPUtils.getJPathString(IDJSONPATH, k),
|
||||
DHPUtils.getJPathString(OBJIDPATH, k)))
|
||||
.filter(
|
||||
t -> !StringUtils
|
||||
.substringAfter(t._1(), "|")
|
||||
.equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::")))
|
||||
.distinct();
|
||||
|
||||
JavaRDD<Relation> simRel = datasetSimRel
|
||||
.union(publicationSimRel)
|
||||
.map(
|
||||
s -> {
|
||||
final Relation r = new Relation();
|
||||
r.setSource(s._1());
|
||||
r.setTarget(s._2());
|
||||
r.setRelType("similar");
|
||||
return r;
|
||||
});
|
||||
spark
|
||||
.createDataset(simRel.rdd(), Encoders.bean(Relation.class))
|
||||
.distinct()
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(targetPath + "/pid_simRel");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,256 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
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.MapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.rdd.RDD;
|
||||
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 com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication;
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import net.minidev.json.JSONArray;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This job is responsible of the creation of RAW Graph It is applied to the different entities generated from
|
||||
* {@link SparkExtractEntitiesJob} In case of dataset, publication and Unknown Entities we group all the entities of the
|
||||
* same type by their identifier, and then in the reduce phase we merge all the entities. Merge means: -merge all the
|
||||
* metadata -merge the collected From values
|
||||
* <p>
|
||||
* In case of relation we need to make a different work: -Phase 1: Map reduce jobs Map: Get all Relation and emit a key
|
||||
* constructed by (source, relType, Target) and the relation itself Reduce: Merge all relations Looking at the javadoc
|
||||
* of {@link SparkSXGeneratePidSimlarity} we take the dataset of pid relation and joining by source and target we
|
||||
* replace the wrong identifier in the relation with the correct ones. At the end we replace the new Dataset of Relation
|
||||
*/
|
||||
public class SparkScholexplorerCreateRawGraphJob {
|
||||
|
||||
static final String IDJSONPATH = "$.id";
|
||||
static final String SOURCEJSONPATH = "$.source";
|
||||
static final String TARGETJSONPATH = "$.target";
|
||||
static final String RELJSONPATH = "$.relType";
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkScholexplorerCreateRawGraphJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/graph/argumentparser/merge_entities_scholix_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.config(
|
||||
new SparkConf()
|
||||
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"))
|
||||
.appName(SparkScholexplorerCreateRawGraphJob.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
final String targetPath = parser.get("targetPath");
|
||||
final String entity = parser.get("entity");
|
||||
FileSystem fs = FileSystem.get(sc.sc().hadoopConfiguration());
|
||||
List<Path> subFolder = Arrays
|
||||
.stream(fs.listStatus(new Path(inputPath)))
|
||||
.filter(FileStatus::isDirectory)
|
||||
.map(FileStatus::getPath)
|
||||
.collect(Collectors.toList());
|
||||
List<JavaRDD<String>> inputRdd = new ArrayList<>();
|
||||
subFolder.forEach(p -> inputRdd.add(sc.textFile(p.toUri().getRawPath())));
|
||||
JavaRDD<String> union = sc.emptyRDD();
|
||||
for (JavaRDD<String> item : inputRdd) {
|
||||
union = union.union(item);
|
||||
}
|
||||
switch (entity) {
|
||||
case "dataset":
|
||||
union
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, DLIDataset>) f -> {
|
||||
final String id = getJPathString(IDJSONPATH, f);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return new Tuple2<>(id, mapper.readValue(f, DLIDataset.class));
|
||||
})
|
||||
.reduceByKey(
|
||||
(a, b) -> {
|
||||
a.mergeFrom(b);
|
||||
return a;
|
||||
})
|
||||
.map(
|
||||
item -> {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(item._2());
|
||||
})
|
||||
.saveAsTextFile(targetPath, GzipCodec.class);
|
||||
break;
|
||||
case "publication":
|
||||
union
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, DLIPublication>) f -> {
|
||||
final String id = getJPathString(IDJSONPATH, f);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return new Tuple2<>(id, mapper.readValue(f, DLIPublication.class));
|
||||
})
|
||||
.reduceByKey(
|
||||
(a, b) -> {
|
||||
a.mergeFrom(b);
|
||||
return a;
|
||||
})
|
||||
.map(
|
||||
item -> {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(item._2());
|
||||
})
|
||||
.saveAsTextFile(targetPath, GzipCodec.class);
|
||||
break;
|
||||
case "unknown":
|
||||
union
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, DLIUnknown>) f -> {
|
||||
final String id = getJPathString(IDJSONPATH, f);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return new Tuple2<>(id, mapper.readValue(f, DLIUnknown.class));
|
||||
})
|
||||
.reduceByKey(
|
||||
(a, b) -> {
|
||||
a.mergeFrom(b);
|
||||
return a;
|
||||
})
|
||||
.map(
|
||||
item -> {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(item._2());
|
||||
})
|
||||
.saveAsTextFile(targetPath, GzipCodec.class);
|
||||
break;
|
||||
case "relation":
|
||||
SparkSXGeneratePidSimlarity
|
||||
.generateDataFrame(
|
||||
spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", ""));
|
||||
RDD<Relation> rdd = union
|
||||
.mapToPair(
|
||||
(PairFunction<String, String, Relation>) f -> {
|
||||
final String source = getJPathString(SOURCEJSONPATH, f);
|
||||
final String target = getJPathString(TARGETJSONPATH, f);
|
||||
final String reltype = getJPathString(RELJSONPATH, f);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper
|
||||
.configure(
|
||||
DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
return new Tuple2<>(
|
||||
DHPUtils
|
||||
.md5(
|
||||
String
|
||||
.format(
|
||||
"%s::%s::%s",
|
||||
source.toLowerCase(),
|
||||
reltype.toLowerCase(),
|
||||
target.toLowerCase())),
|
||||
mapper.readValue(f, Relation.class));
|
||||
})
|
||||
.reduceByKey(
|
||||
(a, b) -> {
|
||||
a.mergeFrom(b);
|
||||
return a;
|
||||
})
|
||||
.map(Tuple2::_2)
|
||||
.rdd();
|
||||
|
||||
spark
|
||||
.createDataset(rdd, Encoders.bean(Relation.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(targetPath);
|
||||
Dataset<Relation> rel_ds = spark.read().load(targetPath).as(Encoders.bean(Relation.class));
|
||||
|
||||
System.out.println("LOADING PATH :" + targetPath.replace("/relation", "") + "/pid_simRel");
|
||||
Dataset<Relation> sim_ds = spark
|
||||
.read()
|
||||
.load(targetPath.replace("/relation", "") + "/pid_simRel")
|
||||
.as(Encoders.bean(Relation.class));
|
||||
|
||||
Dataset<Relation> ids = sim_ds
|
||||
.map(
|
||||
(MapFunction<Relation, Relation>) relation -> {
|
||||
final String type = StringUtils.substringBefore(relation.getSource(), "|");
|
||||
relation
|
||||
.setTarget(
|
||||
String
|
||||
.format(
|
||||
"%s|%s",
|
||||
type, StringUtils.substringAfter(relation.getTarget(), "::")));
|
||||
return relation;
|
||||
},
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
final Dataset<Relation> firstJoin = rel_ds
|
||||
.joinWith(ids, ids.col("target").equalTo(rel_ds.col("source")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) s -> {
|
||||
if (s._2() != null) {
|
||||
s._1().setSource(s._2().getSource());
|
||||
}
|
||||
return s._1();
|
||||
},
|
||||
Encoders.bean(Relation.class));
|
||||
|
||||
Dataset<Relation> secondJoin = firstJoin
|
||||
.joinWith(ids, ids.col("target").equalTo(firstJoin.col("target")), "left_outer")
|
||||
.map(
|
||||
(MapFunction<Tuple2<Relation, Relation>, Relation>) s -> {
|
||||
if (s._2() != null) {
|
||||
s._1().setTarget(s._2().getSource());
|
||||
}
|
||||
return s._1();
|
||||
},
|
||||
Encoders.bean(Relation.class));
|
||||
secondJoin.write().mode(SaveMode.Overwrite).save(targetPath + "_fixed");
|
||||
|
||||
FileSystem fileSystem = FileSystem.get(sc.hadoopConfiguration());
|
||||
|
||||
fileSystem.delete(new Path(targetPath), true);
|
||||
fileSystem.rename(new Path(targetPath + "_fixed"), new Path(targetPath));
|
||||
}
|
||||
}
|
||||
|
||||
public static String getJPathString(final String jsonPath, final String json) {
|
||||
try {
|
||||
Object o = JsonPath.read(json, jsonPath);
|
||||
if (o instanceof String)
|
||||
return (String) o;
|
||||
if (o instanceof JSONArray && ((JSONArray) o).size() > 0)
|
||||
return (String) ((JSONArray) o).get(0);
|
||||
return "";
|
||||
} catch (Exception e) {
|
||||
return "";
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
|
||||
package eu.dnetlib.dhp.sx.graph;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.IntWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.sx.graph.parser.DatasetScholexplorerParser;
|
||||
import eu.dnetlib.dhp.sx.graph.parser.PublicationScholexplorerParser;
|
||||
import eu.dnetlib.scholexplorer.relation.RelationMapper;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This Job read a sequential File containing XML stored in the aggregator and generates an RDD of heterogeneous
|
||||
* entities like Dataset, Relation, Publication and Unknown
|
||||
*/
|
||||
public class SparkScholexplorerGraphImporter {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkScholexplorerGraphImporter.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/sx/graph/argumentparser/input_graph_scholix_parameters.json")));
|
||||
|
||||
parser.parseArgument(args);
|
||||
final SparkSession spark = SparkSession
|
||||
.builder()
|
||||
.appName(SparkScholexplorerGraphImporter.class.getSimpleName())
|
||||
.master(parser.get("master"))
|
||||
.getOrCreate();
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
|
||||
RelationMapper relationMapper = RelationMapper.load();
|
||||
|
||||
sc
|
||||
.sequenceFile(inputPath, IntWritable.class, Text.class)
|
||||
.map(Tuple2::_2)
|
||||
.map(Text::toString)
|
||||
.repartition(500)
|
||||
.flatMap(
|
||||
(FlatMapFunction<String, Oaf>) record -> {
|
||||
switch (parser.get("entity")) {
|
||||
case "dataset":
|
||||
final DatasetScholexplorerParser d = new DatasetScholexplorerParser();
|
||||
return d.parseObject(record, relationMapper).iterator();
|
||||
case "publication":
|
||||
final PublicationScholexplorerParser p = new PublicationScholexplorerParser();
|
||||
return p.parseObject(record, relationMapper).iterator();
|
||||
default:
|
||||
throw new IllegalArgumentException("wrong values of entities");
|
||||
}
|
||||
})
|
||||
.map(
|
||||
k -> {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
return mapper.writeValueAsString(k);
|
||||
})
|
||||
.saveAsTextFile(parser.get("targetPath"), GzipCodec.class);
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue