Compare commits

...

10 Commits

32 changed files with 3426 additions and 190 deletions

View File

@ -312,7 +312,8 @@ public class GraphCleaningFunctions extends CleaningFunctions {
}
if (value instanceof Datasource) {
// nothing to evaluate here
final Datasource d = (Datasource) value;
return Objects.nonNull(d.getOfficialname()) && StringUtils.isNotBlank(d.getOfficialname().getValue());
} else if (value instanceof Project) {
final Project p = (Project) value;
return Objects.nonNull(p.getCode()) && StringUtils.isNotBlank(p.getCode().getValue());

View File

@ -0,0 +1,39 @@
/*
* Copyright (c) 2024.
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
package eu.dnetlib.dhp.actionmanager.promote;
/** Encodes the Actionset promotion strategies */
public class PromoteAction {
/** The supported actionset promotion strategies
*
* ENRICH: promotes only records in the actionset matching another record in the
* graph and enriches them applying the given MergeAndGet strategy
* UPSERT: promotes all the records in an actionset, matching records are updated
* using the given MergeAndGet strategy, the non-matching record as inserted as they are.
*/
public enum Strategy {
ENRICH, UPSERT
}
/**
* Returns the string representation of the join type implementing the given PromoteAction.
*
* @param strategy the strategy to be used to promote the Actionset contents
* @return the join type used to implement the promotion strategy
*/
public static String joinTypeForStrategy(PromoteAction.Strategy strategy) {
switch (strategy) {
case ENRICH:
return "left_outer";
case UPSERT:
return "full_outer";
default:
throw new IllegalStateException("unsupported PromoteAction: " + strategy.toString());
}
}
}

View File

@ -67,8 +67,9 @@ public class PromoteActionPayloadForGraphTableJob {
String outputGraphTablePath = parser.get("outputGraphTablePath");
logger.info("outputGraphTablePath: {}", outputGraphTablePath);
MergeAndGet.Strategy strategy = MergeAndGet.Strategy.valueOf(parser.get("mergeAndGetStrategy").toUpperCase());
logger.info("strategy: {}", strategy);
MergeAndGet.Strategy mergeAndGetStrategy = MergeAndGet.Strategy
.valueOf(parser.get("mergeAndGetStrategy").toUpperCase());
logger.info("mergeAndGetStrategy: {}", mergeAndGetStrategy);
Boolean shouldGroupById = Optional
.ofNullable(parser.get("shouldGroupById"))
@ -76,6 +77,12 @@ public class PromoteActionPayloadForGraphTableJob {
.orElse(true);
logger.info("shouldGroupById: {}", shouldGroupById);
PromoteAction.Strategy promoteActionStrategy = Optional
.ofNullable(parser.get("promoteActionStrategy"))
.map(PromoteAction.Strategy::valueOf)
.orElse(PromoteAction.Strategy.UPSERT);
logger.info("promoteActionStrategy: {}", promoteActionStrategy);
@SuppressWarnings("unchecked")
Class<? extends Oaf> rowClazz = (Class<? extends Oaf>) Class.forName(graphTableClassName);
@SuppressWarnings("unchecked")
@ -97,7 +104,8 @@ public class PromoteActionPayloadForGraphTableJob {
inputGraphTablePath,
inputActionPayloadPath,
outputGraphTablePath,
strategy,
mergeAndGetStrategy,
promoteActionStrategy,
rowClazz,
actionPayloadClazz,
shouldGroupById);
@ -124,14 +132,16 @@ public class PromoteActionPayloadForGraphTableJob {
String inputGraphTablePath,
String inputActionPayloadPath,
String outputGraphTablePath,
MergeAndGet.Strategy strategy,
MergeAndGet.Strategy mergeAndGetStrategy,
PromoteAction.Strategy promoteActionStrategy,
Class<G> rowClazz,
Class<A> actionPayloadClazz, Boolean shouldGroupById) {
Dataset<G> rowDS = readGraphTable(spark, inputGraphTablePath, rowClazz);
Dataset<A> actionPayloadDS = readActionPayload(spark, inputActionPayloadPath, actionPayloadClazz);
Dataset<G> result = promoteActionPayloadForGraphTable(
rowDS, actionPayloadDS, strategy, rowClazz, actionPayloadClazz, shouldGroupById)
rowDS, actionPayloadDS, mergeAndGetStrategy, promoteActionStrategy, rowClazz, actionPayloadClazz,
shouldGroupById)
.map((MapFunction<G, G>) value -> value, Encoders.bean(rowClazz));
saveGraphTable(result, outputGraphTablePath);
@ -183,7 +193,8 @@ public class PromoteActionPayloadForGraphTableJob {
private static <G extends Oaf, A extends Oaf> Dataset<G> promoteActionPayloadForGraphTable(
Dataset<G> rowDS,
Dataset<A> actionPayloadDS,
MergeAndGet.Strategy strategy,
MergeAndGet.Strategy mergeAndGetStrategy,
PromoteAction.Strategy promoteActionStrategy,
Class<G> rowClazz,
Class<A> actionPayloadClazz,
Boolean shouldGroupById) {
@ -195,8 +206,9 @@ public class PromoteActionPayloadForGraphTableJob {
SerializableSupplier<Function<G, String>> rowIdFn = ModelSupport::idFn;
SerializableSupplier<Function<A, String>> actionPayloadIdFn = ModelSupport::idFn;
SerializableSupplier<BiFunction<G, A, G>> mergeRowWithActionPayloadAndGetFn = MergeAndGet.functionFor(strategy);
SerializableSupplier<BiFunction<G, G, G>> mergeRowsAndGetFn = MergeAndGet.functionFor(strategy);
SerializableSupplier<BiFunction<G, A, G>> mergeRowWithActionPayloadAndGetFn = MergeAndGet
.functionFor(mergeAndGetStrategy);
SerializableSupplier<BiFunction<G, G, G>> mergeRowsAndGetFn = MergeAndGet.functionFor(mergeAndGetStrategy);
SerializableSupplier<G> zeroFn = zeroFn(rowClazz);
SerializableSupplier<Function<G, Boolean>> isNotZeroFn = PromoteActionPayloadForGraphTableJob::isNotZeroFnUsingIdOrSourceAndTarget;
@ -207,6 +219,7 @@ public class PromoteActionPayloadForGraphTableJob {
rowIdFn,
actionPayloadIdFn,
mergeRowWithActionPayloadAndGetFn,
promoteActionStrategy,
rowClazz,
actionPayloadClazz);

View File

@ -34,6 +34,7 @@ public class PromoteActionPayloadFunctions {
* @param rowIdFn Function used to get the id of graph table row
* @param actionPayloadIdFn Function used to get id of action payload instance
* @param mergeAndGetFn Function used to merge graph table row and action payload instance
* @param promoteActionStrategy the Actionset promotion strategy
* @param rowClazz Class of graph table
* @param actionPayloadClazz Class of action payload
* @param <G> Type of graph table row
@ -46,6 +47,7 @@ public class PromoteActionPayloadFunctions {
SerializableSupplier<Function<G, String>> rowIdFn,
SerializableSupplier<Function<A, String>> actionPayloadIdFn,
SerializableSupplier<BiFunction<G, A, G>> mergeAndGetFn,
PromoteAction.Strategy promoteActionStrategy,
Class<G> rowClazz,
Class<A> actionPayloadClazz) {
if (!isSubClass(rowClazz, actionPayloadClazz)) {
@ -61,7 +63,7 @@ public class PromoteActionPayloadFunctions {
.joinWith(
actionPayloadWithIdDS,
rowWithIdDS.col("_1").equalTo(actionPayloadWithIdDS.col("_1")),
"full_outer")
PromoteAction.joinTypeForStrategy(promoteActionStrategy))
.map(
(MapFunction<Tuple2<Tuple2<String, G>, Tuple2<String, A>>, G>) value -> {
Optional<G> rowOpt = Optional.ofNullable(value._1()).map(Tuple2::_2);

View File

@ -41,6 +41,12 @@
"paramDescription": "strategy for merging graph table objects with action payload instances, MERGE_FROM_AND_GET or SELECT_NEWER_AND_GET",
"paramRequired": true
},
{
"paramName": "pas",
"paramLongName": "promoteActionStrategy",
"paramDescription": "strategy for promoting the actionset contents into the graph tables, ENRICH or UPSERT (default)",
"paramRequired": false
},
{
"paramName": "sgid",
"paramLongName": "shouldGroupById",

View File

@ -115,6 +115,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputGraphTablePath</arg><arg>${workingDir}/dataset</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="DecisionPromoteResultActionPayloadForDatasetTable"/>
@ -167,6 +168,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Result</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/dataset</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="End"/>

View File

@ -106,6 +106,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/datasource</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -106,6 +106,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/organization</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -114,6 +114,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputGraphTablePath</arg><arg>${workingDir}/otherresearchproduct</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="DecisionPromoteResultActionPayloadForOtherResearchProductTable"/>
@ -166,6 +167,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Result</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/otherresearchproduct</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="End"/>

View File

@ -106,6 +106,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/project</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -115,6 +115,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputGraphTablePath</arg><arg>${workingDir}/publication</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="DecisionPromoteResultActionPayloadForPublicationTable"/>
@ -167,6 +168,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Result</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/publication</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="End"/>

View File

@ -107,6 +107,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/relation</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -114,6 +114,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputGraphTablePath</arg><arg>${workingDir}/software</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="DecisionPromoteResultActionPayloadForSoftwareTable"/>
@ -166,6 +167,7 @@
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Result</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/software</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
<arg>--shouldGroupById</arg><arg>${shouldGroupById}</arg>
</spark>
<ok to="End"/>

View File

@ -54,7 +54,7 @@ public class PromoteActionPayloadFunctionsTest {
RuntimeException.class,
() -> PromoteActionPayloadFunctions
.joinGraphTableWithActionPayloadAndMerge(
null, null, null, null, null, OafImplSubSub.class, OafImpl.class));
null, null, null, null, null, null, OafImplSubSub.class, OafImpl.class));
}
@Test
@ -104,6 +104,7 @@ public class PromoteActionPayloadFunctionsTest {
rowIdFn,
actionPayloadIdFn,
mergeAndGetFn,
PromoteAction.Strategy.UPSERT,
OafImplSubSub.class,
OafImplSubSub.class)
.collectAsList();
@ -183,6 +184,7 @@ public class PromoteActionPayloadFunctionsTest {
rowIdFn,
actionPayloadIdFn,
mergeAndGetFn,
PromoteAction.Strategy.UPSERT,
OafImplSubSub.class,
OafImplSub.class)
.collectAsList();

View File

@ -165,7 +165,7 @@ public class OaiIterator implements Iterator<String> {
} catch (final DocumentException e1) {
final String resumptionToken = extractResumptionToken(xml);
if (resumptionToken == null) {
report.put(e1.getClass().getName(), e1.getMessage());
report.put(e1.getClass().getName(), e1.getMessage());
throw new CollectorException("Error parsing cleaned document:\n" + cleaned, e1);
}
return resumptionToken;

View File

@ -122,22 +122,41 @@ public class DedupRecordFactory {
}
return Stream
.concat(Stream.of(agg.getDedupId()), agg.aliases.stream())
.map(id -> {
try {
OafEntity res = (OafEntity) BeanUtils.cloneBean(agg.entity);
res.setId(id);
res.setDataInfo(dataInfo);
res.setLastupdatetimestamp(ts);
return res;
} catch (Exception e) {
throw new RuntimeException(e);
}
})
.concat(
Stream
.of(agg.getDedupId())
.map(id -> createDedupOafEntity(id, agg.entity, dataInfo, ts)),
agg.aliases
.stream()
.map(id -> createMergedDedupAliasOafEntity(id, agg.entity, dataInfo, ts)))
.iterator();
}, beanEncoder);
}
private static OafEntity createDedupOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
try {
OafEntity res = (OafEntity) BeanUtils.cloneBean(base);
res.setId(id);
res.setDataInfo(dataInfo);
res.setLastupdatetimestamp(ts);
return res;
} catch (Exception e) {
throw new RuntimeException(e);
}
}
private static OafEntity createMergedDedupAliasOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) {
try {
OafEntity res = createDedupOafEntity(id, base, dataInfo, ts);
DataInfo ds = (DataInfo) BeanUtils.cloneBean(dataInfo);
ds.setDeletedbyinference(true);
res.setDataInfo(ds);
return res;
} catch (Exception e) {
throw new RuntimeException(e);
}
}
private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) {
if (duplicate == null) {

View File

@ -1,6 +1,26 @@
[
{"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}
{
"paramName": "t",
"paramLongName": "targetPath",
"paramDescription": "the path of the OAF Orcid transformed",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "isLookupUrl",
"paramDescription": "the isLookup URL",
"paramRequired": true
},
{
"paramName": "s",
"paramLongName": "sourcePath",
"paramDescription": "the source path ",
"paramRequired": false
},
{
"paramName": "m",
"paramLongName": "master",
"paramDescription": "the master name",
"paramRequired": true
}
]

View File

@ -1,5 +1,6 @@
package eu.dnetlib.doiboost.crossref
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf._
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, IdentifierFactory, OafMapperUtils}
@ -47,67 +48,16 @@ case object Crossref2Oaf {
json.extract[List[funderInfo]]
}
val mappingCrossrefType = Map(
"book-section" -> "publication",
"book" -> "publication",
"book-chapter" -> "publication",
"book-part" -> "publication",
"book-series" -> "publication",
"book-set" -> "publication",
"book-track" -> "publication",
"edited-book" -> "publication",
"reference-book" -> "publication",
"monograph" -> "publication",
"journal-article" -> "publication",
"dissertation" -> "publication",
"other" -> "publication",
"peer-review" -> "publication",
"proceedings" -> "publication",
"proceedings-article" -> "publication",
"reference-entry" -> "publication",
"report" -> "publication",
"report-series" -> "publication",
"standard" -> "publication",
"standard-series" -> "publication",
"posted-content" -> "publication",
"dataset" -> "dataset"
)
val mappingCrossrefSubType = Map(
"book-section" -> "0013 Part of book or chapter of book",
"book" -> "0002 Book",
"book-chapter" -> "0013 Part of book or chapter of book",
"book-part" -> "0013 Part of book or chapter of book",
"book-series" -> "0002 Book",
"book-set" -> "0002 Book",
"book-track" -> "0002 Book",
"edited-book" -> "0002 Book",
"reference-book" -> "0002 Book",
"monograph" -> "0002 Book",
"journal-article" -> "0001 Article",
"dissertation" -> "0044 Thesis",
"other" -> "0038 Other literature type",
"peer-review" -> "0015 Review",
"proceedings" -> "0004 Conference object",
"proceedings-article" -> "0004 Conference object",
"reference-entry" -> "0013 Part of book or chapter of book",
"report" -> "0017 Report",
"report-series" -> "0017 Report",
"standard" -> "0038 Other literature type",
"standard-series" -> "0038 Other literature type",
"dataset" -> "0021 Dataset",
"preprint" -> "0016 Preprint",
"report" -> "0017 Report"
)
def getIrishId(doi: String): Option[String] = {
def getIrishId(doi: String): Option[String] = {
val id = doi.split("/").last
irishFunder
.find(f => id.equalsIgnoreCase(f.id) || (f.synonym.nonEmpty && f.synonym.exists(s => s.equalsIgnoreCase(id))))
.map(f => f.id)
}
def mappingResult(result: Result, json: JValue, cobjCategory: String, originalType: String): Result = {
def mappingResult(result: Result, json: JValue, instanceType:Qualifier, originalType: String): Result = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
//MAPPING Crossref DOI into PID
@ -275,27 +225,20 @@ case object Crossref2Oaf {
instance.setAccessright(
decideAccessRight(instance.getLicense, result.getDateofacceptance.getValue)
)
instance.setInstancetype(
OafMapperUtils.qualifier(
cobjCategory.substring(0, 4),
cobjCategory.substring(5),
ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE
)
)
instance.setInstancetype(instanceType)
//ADD ORIGINAL TYPE to the mapping
val itm = new InstanceTypeMapping
itm.setOriginalType(originalType)
itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1)
instance.setInstanceTypeMapping(List(itm).asJava)
result.setResourcetype(
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.setCollectedfrom(createCrossrefCollectedFrom())
if (StringUtils.isNotBlank(issuedDate)) {
@ -354,7 +297,40 @@ case object Crossref2Oaf {
a
}
def convert(input: String): List[Oaf] = {
/** *
* Use the vocabulary dnet:publication_resource to find a synonym to one of these terms and get the instance.type.
* Using the dnet:result_typologies vocabulary, we look up the instance.type synonym
* to generate one of the following main entities:
* - publication
* - dataset
* - software
* - otherresearchproduct
*
* @param resourceType
* @param vocabularies
* @return
*/
def getTypeQualifier(
resourceType: String,
vocabularies: VocabularyGroup
): (Qualifier, Qualifier, String) = {
if (resourceType != null && resourceType.nonEmpty) {
val typeQualifier =
vocabularies.getSynonymAsQualifier(ModelConstants.DNET_PUBLICATION_RESOURCE, resourceType)
if (typeQualifier != null)
return (
typeQualifier,
vocabularies.getSynonymAsQualifier(
ModelConstants.DNET_RESULT_TYPOLOGIES,
typeQualifier.getClassid
),
resourceType
)
}
null
}
def convert(input: String, vocabularies: VocabularyGroup): List[Oaf] = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
@ -364,17 +340,17 @@ case object Crossref2Oaf {
val objectSubType = (json \ "subtype").extractOrElse[String](null)
if (objectType == null)
return resultList
val typology =getTypeQualifier(objectType, vocabularies)
val result = generateItemFromType(objectType, objectSubType)
if (typology == null)
return List()
val result = generateItemFromType(typology._2)
if (result == null)
return List()
val cOBJCategory = mappingCrossrefSubType.getOrElse(
objectType,
mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type")
)
val originalType = if (mappingCrossrefSubType.contains(objectType)) objectType else objectSubType
mappingResult(result, json, cOBJCategory, originalType)
mappingResult(result, json, typology._1, typology._3)
if (result == null || result.getId == null)
return List()
@ -392,7 +368,7 @@ case object Crossref2Oaf {
}
result match {
case publication: Publication => convertPublication(publication, json, cOBJCategory)
case publication: Publication => convertPublication(publication, json, typology._1)
case dataset: Dataset => convertDataset(dataset)
}
@ -622,12 +598,12 @@ case object Crossref2Oaf {
// TODO check if there are other info to map into the Dataset
}
def convertPublication(publication: Publication, json: JValue, cobjCategory: String): Unit = {
def convertPublication(publication: Publication, json: JValue, cobjCategory: Qualifier): Unit = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
val containerTitles = for { JString(ct) <- json \ "container-title" } yield ct
//Mapping book
if (cobjCategory.toLowerCase.contains("book")) {
if (cobjCategory.getClassname.toLowerCase.contains("book")) {
val ISBN = for { JString(isbn) <- json \ "ISBN" } yield isbn
if (ISBN.nonEmpty && containerTitles.nonEmpty) {
val source = s"${containerTitles.head} ISBN: ${ISBN.head}"
@ -708,12 +684,24 @@ case object Crossref2Oaf {
null
}
def generateItemFromType(objectType: String, objectSubType: String): Result = {
if (mappingCrossrefType.contains(objectType)) {
if (mappingCrossrefType(objectType).equalsIgnoreCase("publication"))
return new Publication()
if (mappingCrossrefType(objectType).equalsIgnoreCase("dataset"))
return new Dataset()
def generateItemFromType(objectType: Qualifier): Result = {
if (objectType.getClassid.equalsIgnoreCase("publication")) {
val item = new Publication
item.setResourcetype(objectType)
return item
} else if (objectType.getClassid.equalsIgnoreCase("dataset")) {
val item = new Dataset
item.setResourcetype(objectType)
return item
}
else if (objectType.getClassid.equalsIgnoreCase("software")){
val item = new Software
item.setResourcetype(objectType)
return item
}else if (objectType.getClassid.equalsIgnoreCase("OtherResearchProduct")){
val item = new OtherResearchProduct
item.setResourcetype(objectType)
return item
}
null
}

View File

@ -1,8 +1,10 @@
package eu.dnetlib.doiboost.crossref
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.oaf
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql._
@ -40,11 +42,17 @@ object SparkMapDumpIntoOAF {
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
val targetPath = parser.get("targetPath")
val isLookupUrl: String = parser.get("isLookupUrl")
logger.info("isLookupUrl: {}", isLookupUrl)
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
require(vocabularies != null)
spark.read
.load(parser.get("sourcePath"))
.as[CrossrefDT]
.flatMap(k => Crossref2Oaf.convert(k.json))
.flatMap(k => Crossref2Oaf.convert(k.json, vocabularies))
.filter(o => o != null)
.write
.mode(SaveMode.Overwrite)

View File

@ -0,0 +1,51 @@
package eu.dnetlib.dhp.aggregation;
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.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
public abstract class AbstractVocabularyTest {
@Mock
protected ISLookUpService isLookUpService;
protected VocabularyGroup vocabularies;
public void setUpVocabulary() throws ISLookUpException, IOException {
lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs());
lenient()
.when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY))
.thenReturn(synonyms());
vocabularies = VocabularyGroup.loadVocsFromIS(isLookUpService);
}
private static List<String> vocs() throws IOException {
return IOUtils
.readLines(
Objects
.requireNonNull(
AbstractVocabularyTest.class.getResourceAsStream("/eu/dnetlib/dhp/doiboost/terms.txt")));
}
private static List<String> synonyms() throws IOException {
return IOUtils
.readLines(
Objects
.requireNonNull(
AbstractVocabularyTest.class.getResourceAsStream("/eu/dnetlib/dhp/doiboost/synonyms.txt")));
}
}

File diff suppressed because it is too large Load Diff

View File

@ -57,7 +57,7 @@
]
]
},
"type": "posted-content",
"type": "journal-article",
"URL": "http://dx.doi.org/10.1101/030080",
"is-referenced-by-count": 2,
"link": [

View File

@ -1,31 +1,42 @@
package eu.dnetlib.dhp.doiboost.crossref
import com.fasterxml.jackson.databind.SerializationFeature
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf._
import eu.dnetlib.dhp.utils.DHPUtils
import eu.dnetlib.doiboost.crossref.Crossref2Oaf
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
import com.fasterxml.jackson.databind.ObjectMapper
import org.json4s
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.{DefaultFormats, JValue}
import org.json4s.jackson.JsonMethods
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.extension.ExtendWith
import org.junit.jupiter.api.{BeforeEach, Test}
import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.{Logger, LoggerFactory}
import java.nio.file.Files
import scala.collection.JavaConverters._
import scala.io.Source
import scala.util.matching.Regex
class CrossrefMappingTest {
@ExtendWith(Array(classOf[MockitoExtension]))
class CrossrefMappingTest extends AbstractVocabularyTest{
val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass)
val mapper = new ObjectMapper()
@BeforeEach
def setUp(): Unit = {
super.setUpVocabulary()
}
@Test
def testMissingAuthorParser():Unit = {
val json: String = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/s41567-022-01757-y.json")).mkString
val result = Crossref2Oaf.convert(json)
val result = Crossref2Oaf.convert(json, vocabularies)
result.filter(o => o.isInstanceOf[Publication]).map(p=> p.asInstanceOf[Publication]).foreach(p =>assertTrue(p.getAuthor.size()>0))
}
@ -45,13 +56,13 @@ class CrossrefMappingTest {
for (line <- funder_doi.linesWithSeparators.map(l => l.stripLineEnd)) {
val json = template.replace("%s", line)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
checkRelation(resultList)
}
for (line <- funder_name.linesWithSeparators.map(l => l.stripLineEnd)) {
val json = template.replace("%s", line)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
checkRelation(resultList)
}
@ -91,7 +102,7 @@ class CrossrefMappingTest {
Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/issue_date.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Result])
@ -110,14 +121,14 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Result])
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
items.foreach(p => println(mapper.writeValueAsString(p)))
items.foreach(p => println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p)))
}
@ -137,7 +148,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty)
val result: List[Oaf] = Crossref2Oaf.convert(json)
val result: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(result.nonEmpty)
@ -158,8 +169,8 @@ class CrossrefMappingTest {
assertEquals(doisReference.size, relationList.size)
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
relationList.foreach(p => println(mapper.writeValueAsString(p)))
relationList.foreach(p => println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p)))
}
@Test
@ -173,14 +184,14 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Result])
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
items.foreach(p => println(mapper.writeValueAsString(p)))
items.foreach(p => println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p)))
}
@ -189,18 +200,17 @@ class CrossrefMappingTest {
val json = Source
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/prwTest.json"))
.mkString
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Result])
items.foreach(p => logger.info(mapper.writeValueAsString(p)))
items.foreach(p => logger.info(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(p)))
}
@ -230,7 +240,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val rels: List[Relation] =
@ -250,7 +260,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -261,7 +271,7 @@ class CrossrefMappingTest {
val result: Result = items.head.asInstanceOf[Result]
assertNotNull(result)
logger.info(mapper.writeValueAsString(result));
logger.info(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(result));
assertNotNull(result.getDataInfo, "Datainfo test not null Failed");
assertNotNull(
@ -326,7 +336,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -410,7 +420,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -458,7 +468,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -537,7 +547,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -563,7 +573,7 @@ class CrossrefMappingTest {
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -573,7 +583,8 @@ class CrossrefMappingTest {
assert(items.size == 1)
val result: Result = items.head.asInstanceOf[Publication]
assertNotNull(result)
logger.info(mapper.writeValueAsString(result));
logger.info(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(result));
}
@Test
@ -586,7 +597,7 @@ class CrossrefMappingTest {
val line: String =
"\"funder\": [{\"name\": \"Wellcome Trust Masters Fellowship\",\"award\": [\"090633\"]}],"
val json = template.replace("%s", line)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Publication])
val result: Result = items.head.asInstanceOf[Publication]
@ -605,7 +616,7 @@ class CrossrefMappingTest {
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/article.json"))
.mkString
val resultList: List[Oaf] = Crossref2Oaf.convert(template)
val resultList: List[Oaf] = Crossref2Oaf.convert(template, vocabularies)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Publication])
val result: Result = items.head.asInstanceOf[Publication]
@ -629,14 +640,14 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
val item: Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result]
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
println(mapper.writeValueAsString(item))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(item))
assertTrue(
item.getInstance().asScala exists (i => i.getLicense.getValue.equals("https://www.springer.com/vor"))
@ -659,7 +670,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -676,8 +687,8 @@ class CrossrefMappingTest {
assertTrue(
item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == OpenAccessRoute.hybrid)
)
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
println(mapper.writeValueAsString(item))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(item))
}
@ -694,7 +705,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -711,8 +722,7 @@ class CrossrefMappingTest {
assertTrue(
item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == OpenAccessRoute.hybrid)
)
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
println(mapper.writeValueAsString(item))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(item))
}
@ -729,7 +739,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -746,8 +756,7 @@ class CrossrefMappingTest {
item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("EMBARGO"))
)
assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == null))
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
println(mapper.writeValueAsString(item))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(item))
}
@ -764,7 +773,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -781,8 +790,8 @@ class CrossrefMappingTest {
item.getInstance().asScala exists (i => i.getAccessright.getClassid.equals("EMBARGO"))
)
assertTrue(item.getInstance().asScala exists (i => i.getAccessright.getOpenAccessRoute == null))
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
println(mapper.writeValueAsString(item))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(item))
}
@ -797,7 +806,7 @@ class CrossrefMappingTest {
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
val resultList: List[Oaf] = Crossref2Oaf.convert(json, vocabularies)
assertTrue(resultList.nonEmpty)
@ -807,9 +816,8 @@ class CrossrefMappingTest {
assertEquals(1, item.getInstance().get(0).getUrl().size())
assertEquals(
"https://doi.org/10.1016/j.jas.2019.105013",
item.getInstance().get(0).getUrl().get(0)
item.getInstance().get(0).getUrl.get(0)
)
//println(mapper.writeValueAsString(item))
}

View File

@ -114,7 +114,7 @@
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/preparedInfo/targetOrcidAssoc</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/targetOrcidAssoc</arg>
<arg>--allowedsemrels</arg><arg>${allowedsemrels}</arg>
</spark>
<ok to="wait"/>
@ -142,7 +142,7 @@
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/preparedInfo/targetOrcidAssoc</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/targetOrcidAssoc</arg>
<arg>--allowedsemrels</arg><arg>${allowedsemrels}</arg>
</spark>
<ok to="wait"/>
@ -170,7 +170,7 @@
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/preparedInfo/targetOrcidAssoc</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/targetOrcidAssoc</arg>
<arg>--allowedsemrels</arg><arg>${allowedsemrels}</arg>
</spark>
<ok to="wait"/>
@ -198,7 +198,7 @@
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/preparedInfo/targetOrcidAssoc</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/targetOrcidAssoc</arg>
<arg>--allowedsemrels</arg><arg>${allowedsemrels}</arg>
</spark>
<ok to="wait"/>
@ -225,8 +225,8 @@
--conf spark.dynamicAllocation.enabled=true
--conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/orcid/orcidprop</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/orcidprop/mergedOrcidAssoc</arg>
<arg>--sourcePath</arg><arg>${workingDir}/orcid/targetOrcidAssoc</arg>
<arg>--outputPath</arg><arg>${workingDir}/orcid/mergedOrcidAssoc</arg>
</spark>
<ok to="fork-join-exec-propagation"/>
<error to="Kill"/>
@ -247,9 +247,10 @@
<class>eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob</class>
<jar>dhp-enrichment-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--executor-cores=4
--executor-memory=4G
--driver-memory=${sparkDriverMemory}
--conf spark.executor.memoryOverhead=5G
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
@ -259,9 +260,9 @@
--conf spark.speculation=false
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
--conf spark.sql.shuffle.partitions=3840
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/orcidprop/mergedOrcidAssoc</arg>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/mergedOrcidAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/publication</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
<arg>--outputPath</arg><arg>${outputPath}/publication</arg>
@ -291,7 +292,7 @@
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
</spark-opts>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/orcidprop/mergedOrcidAssoc</arg>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/mergedOrcidAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/dataset</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
<arg>--outputPath</arg><arg>${outputPath}/dataset</arg>
@ -321,7 +322,7 @@
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
</spark-opts>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/orcidprop/mergedOrcidAssoc</arg>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/mergedOrcidAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/otherresearchproduct</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
<arg>--outputPath</arg><arg>${outputPath}/otherresearchproduct</arg>
@ -351,7 +352,7 @@
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
</spark-opts>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/orcidprop/mergedOrcidAssoc</arg>
<arg>--possibleUpdatesPath</arg><arg>${workingDir}/orcid/mergedOrcidAssoc</arg>
<arg>--sourcePath</arg><arg>${sourcePath}/software</arg>
<arg>--resultTableName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
<arg>--outputPath</arg><arg>${outputPath}/software</arg>

View File

@ -238,11 +238,23 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper {
(Element) doc
.selectSingleNode(
"//*[local-name()='metadata']/*[local-name() = 'resource']/*[local-name() = 'resourceType']"))
.map(element -> {
final String resourceTypeURI = element.attributeValue("uri");
final String resourceTypeAnyURI = element.attributeValue("anyURI");
final String resourceTypeTxt = element.getText();
final String resourceTypeGeneral = element.attributeValue("resourceTypeGeneral");
.map(e -> {
final String resourceTypeURI = Optional
.ofNullable(e.attributeValue("uri"))
.filter(StringUtils::isNotBlank)
.orElse(null);
final String resourceTypeAnyURI = Optional
.ofNullable(e.attributeValue("anyURI"))
.filter(StringUtils::isNotBlank)
.orElse(null);
final String resourceTypeTxt = Optional
.ofNullable(e.getText())
.filter(StringUtils::isNotBlank)
.orElse(null);
final String resourceTypeGeneral = Optional
.ofNullable(e.attributeValue("resourceTypeGeneral"))
.filter(StringUtils::isNotBlank)
.orElse(null);
return ObjectUtils
.firstNonNull(resourceTypeURI, resourceTypeAnyURI, resourceTypeTxt, resourceTypeGeneral);

View File

@ -49,10 +49,10 @@
<action name="reset_outputpath">
<fs>
<delete path="${graphPath}/datasource"/>
<delete path="${graphPath}/organization"/>
<delete path="${graphPath}/project"/>
<delete path="${graphPath}/relation"/>
<delete path="${targetPath}/datasource"/>
<delete path="${targetPath}/organization"/>
<delete path="${targetPath}/project"/>
<delete path="${targetPath}/relation"/>
</fs>
<ok to="copy_datasource"/>
<error to="Kill"/>

View File

@ -1171,6 +1171,34 @@ class MappersTest {
}
@Test
void test_Zenodo2() throws IOException {
final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("odf_zenodo2.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
assertEquals(3, list.size());
Publication p = cleanup((Publication) list.get(0), vocs);
assertNotNull(p.getInstance());
assertEquals(1, p.getInstance().size());
final Instance instance = p.getInstance().get(0);
assertNotNull(instance.getInstanceTypeMapping());
assertEquals(1, instance.getInstanceTypeMapping().size());
Optional<InstanceTypeMapping> coarType = instance
.getInstanceTypeMapping()
.stream()
.filter(itm -> ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName()))
.findFirst();
assertTrue(coarType.isPresent());
assertNotNull(coarType.get().getOriginalType());
assertNull(coarType.get().getTypeCode());
assertNull(coarType.get().getTypeLabel());
}
@Test
void testROHub2() throws IOException {
final String xml = IOUtils
@ -1229,7 +1257,7 @@ class MappersTest {
}
@Test
public void testD4ScienceTraining() throws IOException {
void testD4ScienceTraining() throws IOException {
final String xml = IOUtils
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-1-training.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
@ -1240,7 +1268,7 @@ class MappersTest {
}
@Test
public void testD4ScienceDataset() throws IOException {
void testD4ScienceDataset() throws IOException {
final String xml = IOUtils
.toString(Objects.requireNonNull(getClass().getResourceAsStream("d4science-2-dataset.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
@ -1250,6 +1278,21 @@ class MappersTest {
System.out.println("***************");
}
@Test
void testIRISPub() throws IOException, DocumentException {
final String xml = IOUtils.toString(Objects.requireNonNull(getClass().getResourceAsStream("iris-odf.xml")));
final List<Oaf> list = new OdfToOafMapper(vocs, false, true).processMdRecord(xml);
System.out.println("***************");
System.out.println(new ObjectMapper().writeValueAsString(list));
System.out.println("***************");
final Publication p = (Publication) list.get(0);
assertNotNull(p.getInstance().get(0).getUrl().get(0));
assertValidId(p.getId());
System.out.println(p.getInstance().get(0).getUrl());
p.getPid().forEach(x -> System.out.println(x.getValue()));
p.getInstance().get(0).getAlternateIdentifier().forEach(x -> System.out.println(x.getValue()));
}
@Test
void testNotWellFormed() throws IOException {
final String xml = IOUtils

View File

@ -0,0 +1,215 @@
<?xml version="1.0" encoding="UTF-8"?>
<record xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
xmlns:oaire="http://namespace.openaire.eu/schema/oaire/"
xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<oai:header xmlns="http://namespace.openaire.eu/" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<identifier>oai:air.unimi.it:2434/907506</identifier>
<datestamp>2024-01-04T12:42:51Z</datestamp>
<setSpec>com_2434_73555</setSpec>
<setSpec>col_2434_73557</setSpec>
<setSpec>openaire</setSpec>
<dr:dateOfTransformation>2024-01-29T16:56:50.632Z</dr:dateOfTransformation>
<dri:objIdentifier>od______1261::ff2d9e058e7bea90a27f41c31078e601</dri:objIdentifier>
<dri:recordIdentifier>oai:air.unimi.it:2434/907506</dri:recordIdentifier>
<dri:dateOfCollection/>
<dri:mdFormat/>
<dri:mdFormatInterpretation/>
<dri:repositoryId/>
<oaf:datasourceprefix> od______1261</oaf:datasourceprefix>
</oai:header>
<metadata>
<oaire:resource xmlns:oaire="http://namespace.openaire.eu/schema/oaire/"
xmlns:exslt="http://exslt.org/common"
xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:rdf="http://www.w3.org/TR/rdf-concepts/"
xmlns:doc="http://www.lyncode.com/xoai"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:vc="http://www.w3.org/2007/XMLSchema-versioning"
xmlns="http://www.openarchives.org/OAI/2.0/"
xsi:schemaLocation="http://namespace.openaire.eu/schema/oaire/ https://www.openaire.eu/schema/repo-lit/4.0/openaire.xsd">
<datacite:titles>
<datacite:title xml:lang="en">Ensuring tests of conservation interventions build on existing literature</datacite:title>
</datacite:titles>
<datacite:creator>
<datacite:creator>
<datacite:creatorName>W.J. Sutherland</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>S.T. Alvarez-Castaneda</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>T. Amano</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>R. Ambrosini</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>P. Atkinson</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>J.M. Baxter</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>A.L. Bond</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>P.J. Boon</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>K.L. Buchanan</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>J. Barlow</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>G. Bogliani</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>O.M. Bragg</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M. Burgman</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M.W. Cadotte</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M. Calver</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>S.J. Cooke</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>R.T. Corlett</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>V. Devictor</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>J.G. Ewen</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M. Fisher</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>G. Freeman</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>E. Game</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>B.J. Godley</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>C. Gortazar</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>I.R. Hartley</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>D.L. Hawksworth</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>K.A. Hobson</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M.-. Lu</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>B. Martin-Lopez</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>K. Ma</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>A. Machado</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>D. Mae</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M. Mangiacotti</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>D.J. Mccafferty</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>V. Melfi</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>S. Molur</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>A.J. Moore</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>S.D. Murphy</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>D. Norri</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>A.P.E. van Oudenhoven</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>J. Power</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>E.C. Ree</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>M.W. Schwartz</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>I. Storch</datacite:creatorName>
</datacite:creator>
<datacite:creator>
<datacite:creatorName>C. Wordley</datacite:creatorName>
</datacite:creator>
</datacite:creator>
<datacite:relatedIdentifiers>
</datacite:relatedIdentifiers>
<datacite:dates>
<datacite:date dateType="Accepted">2020</datacite:date>
<datacite:date dateType="Issued">2020</datacite:date>
<datacite:date dateType="Available">2022-06-20</datacite:date>
</datacite:dates>
<dc:language>eng</dc:language>
<dc:publisher>Wiley Blackwell Publishing</dc:publisher>
<oaire:resourceType resourceTypeGeneral="literature"
uri="http://purl.org/coar/resource_type/c_6501">journal article</oaire:resourceType>
<dc:format>application/pdf</dc:format>
<datacite:identifier xmlns:datacite="http://datacite.org/schema/kernel-3"
identifierType="Handle">2434/907506</datacite:identifier>
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_abf2">open access</datacite:rights>
<datacite:subjects>
<datacite:subject>Conservation of Natural Resources</datacite:subject>
</datacite:subjects>
<datacite:sizes/>
<datacite:sizes/>
<datacite:sizes>
<datacite:size>191802 bytes</datacite:size>
</datacite:sizes>
<oaire:file accessRightsURI="" mimeType="application/pdf" objectType="fulltext">https://air.unimi.it/bitstream/2434/907506/4/Full%20manuscript%20resubmitted.pdf</oaire:file>
</oaire:resource>
<oaf:identifier identifierType="DOI">10.1111/cobi.13555</oaf:identifier>
<oaf:identifier identifierType="PMID">32779884</oaf:identifier>
<oaf:fulltext>https://air.unimi.it/bitstream/2434/907506/4/Full%20manuscript%20resubmitted.pdf</oaf:fulltext>
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
<oaf:dateAccepted>2020-01-01</oaf:dateAccepted>
<oaf:accessrights>OPEN</oaf:accessrights>
<oaf:language>eng</oaf:language>
<oaf:hostedBy name="Archivio Istituzionale della Ricerca dell'Università degli Studi di Milano"
id="opendoar____::1261"/>
<oaf:collectedFrom name="Archivio Istituzionale della Ricerca dell'Università degli Studi di Milano"
id="opendoar____::1261"/>
</metadata>
</record>

View File

@ -0,0 +1,59 @@
<?xml version="1.0" encoding="UTF-8"?>
<record xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
xmlns:datacite="http://datacite.org/schema/kernel-3"
xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:dri="http://www.driver-repository.eu/namespace/dri">
<header xmlns="http://www.openarchives.org/OAI/2.0/">
<identifier>oai:zenodo.org:1596086</identifier>
<datestamp>2020-01-20T13:50:28Z</datestamp>
<setSpec>openaire</setSpec>
<dr:dateOfTransformation>2024-02-08T11:03:10.994Z</dr:dateOfTransformation>
<dri:objIdentifier>od______2659::036d5555a6688ed00c8d0da97bdece3b</dri:objIdentifier>
<dri:dateOfCollection>2024-02-08T11:03:10.994Z</dri:dateOfCollection>
<dri:dateOfTransformation>2024-02-08T11:03:10.994Z</dri:dateOfTransformation>
</header>
<metadata>
<resource xmlns="http://datacite.org/schema/kernel-4"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://datacite.org/schema/kernel-4 http://schema.datacite.org/meta/kernel-4.1/metadata.xsd">
<identifier identifierType="URL">https://zenodo.org/record/1596086</identifier>
<alternateIdentifiers xmlns="http://datacite.org/schema/kernel-3"/>
<creators>
<creator>
<creatorName>Bonney, T. G.</creatorName>
<givenName>T. G.</givenName>
<familyName>Bonney</familyName>
</creator>
</creators>
<titles>
<title>Ice Blocks on a Moraine</title>
</titles>
<publisher>Zenodo</publisher>
<publicationYear>1889</publicationYear>
<dates>
<date dateType="Issued">1889-08-22</date>
</dates>
<resourceType resourceTypeGeneral="JournalArticle"/>
<relatedIdentifiers>
<relatedIdentifier relatedIdentifierType="DOI" relationType="IsIdenticalTo"
>10.1038/040391a0</relatedIdentifier>
</relatedIdentifiers>
<rightsList>
<rights rightsURI="https://creativecommons.org/publicdomain/zero/1.0/legalcode"
>Creative Commons Zero v1.0 Universal</rights>
<rights rightsURI="info:eu-repo/semantics/openAccess">Open Access</rights>
</rightsList>
<descriptions>
<description descriptionType="Abstract">n/a</description>
</descriptions>
</resource>
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
<oaf:dateAccepted>1889-08-22</oaf:dateAccepted>
<oaf:accessrights>OPEN</oaf:accessrights>
<oaf:license>http://creativecommons.org/publicdomain/zero/1.0/legalcode</oaf:license>
<oaf:language/>
<oaf:hostedBy name="ZENODO" id="opendoar____::2659"/>
<oaf:collectedFrom name="ZENODO" id="opendoar____::2659"/>
</metadata>
</record>

View File

@ -244,4 +244,27 @@ public class XmlRecordFactoryTest {
}
@Test
public void testIrisGuidelines4() throws DocumentException, IOException {
final ContextMapper contextMapper = new ContextMapper();
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
XmlConverterJob.schemaLocation);
final Publication p = OBJECT_MAPPER
.readValue(
IOUtils.toString(getClass().getResourceAsStream("iris-odf-4.json")),
Publication.class);
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
assertNotNull(xml);
final Document doc = new SAXReader().read(new StringReader(xml));
assertNotNull(doc);
System.out.println(doc.asXML());
}
}

File diff suppressed because one or more lines are too long