forked from D-Net/dnet-hadoop
Merge branch 'beta' into oaf_country_beta
This commit is contained in:
commit
538b180fe0
|
@ -145,105 +145,6 @@ public class AuthorMerger {
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method tries to figure out when two author are the same in the contest
|
||||
* of ORCID enrichment
|
||||
*
|
||||
* @param left Author in the OAF entity
|
||||
* @param right Author ORCID
|
||||
* @return based on a heuristic on the names of the authors if they are the same.
|
||||
*/
|
||||
public static boolean checkORCIDSimilarity(final Author left, final Author right) {
|
||||
final Person pl = parse(left);
|
||||
final Person pr = parse(right);
|
||||
|
||||
// If one of them didn't have a surname we verify if they have the fullName not empty
|
||||
// and verify if the normalized version is equal
|
||||
if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) &&
|
||||
pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) {
|
||||
|
||||
if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null
|
||||
&& !pr.getFullname().isEmpty()) {
|
||||
return pl
|
||||
.getFullname()
|
||||
.stream()
|
||||
.anyMatch(
|
||||
fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr))));
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
// The Authors have one surname in common
|
||||
if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) {
|
||||
|
||||
// If one of them has only a surname and is the same we can say that they are the same author
|
||||
if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) ||
|
||||
(pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank)))
|
||||
return true;
|
||||
// The authors have the same initials of Name in common
|
||||
if (pl
|
||||
.getName()
|
||||
.stream()
|
||||
.anyMatch(
|
||||
nl -> pr
|
||||
.getName()
|
||||
.stream()
|
||||
.anyMatch(nr -> nr.equalsIgnoreCase(nl))))
|
||||
return true;
|
||||
}
|
||||
|
||||
// Sometimes we noticed that publication have author wrote in inverse order Surname, Name
|
||||
// We verify if we have an exact match between name and surname
|
||||
if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) &&
|
||||
pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl))))
|
||||
return true;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
//
|
||||
|
||||
/**
|
||||
* Method to enrich ORCID information in one list of authors based on another list
|
||||
*
|
||||
* @param baseAuthor the Author List in the OAF Entity
|
||||
* @param orcidAuthor The list of ORCID Author intersected
|
||||
* @return The Author List of the OAF Entity enriched with the orcid Author
|
||||
*/
|
||||
public static List<Author> enrichOrcid(List<Author> baseAuthor, List<Author> orcidAuthor) {
|
||||
|
||||
if (baseAuthor == null || baseAuthor.isEmpty())
|
||||
return orcidAuthor;
|
||||
|
||||
if (orcidAuthor == null || orcidAuthor.isEmpty())
|
||||
return baseAuthor;
|
||||
|
||||
if (baseAuthor.size() == 1 && orcidAuthor.size() > 10)
|
||||
return baseAuthor;
|
||||
|
||||
final List<Author> oAuthor = new ArrayList<>();
|
||||
oAuthor.addAll(orcidAuthor);
|
||||
|
||||
baseAuthor.forEach(ba -> {
|
||||
Optional<Author> aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst();
|
||||
if (aMatch.isPresent()) {
|
||||
final Author sameAuthor = aMatch.get();
|
||||
addPid(ba, sameAuthor.getPid());
|
||||
oAuthor.remove(sameAuthor);
|
||||
}
|
||||
});
|
||||
return baseAuthor;
|
||||
}
|
||||
|
||||
private static void addPid(final Author a, final List<StructuredProperty> pids) {
|
||||
|
||||
if (a.getPid() == null) {
|
||||
a.setPid(new ArrayList<>());
|
||||
}
|
||||
|
||||
a.getPid().addAll(pids);
|
||||
|
||||
}
|
||||
|
||||
public static String pidToComparableString(StructuredProperty pid) {
|
||||
final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase()
|
||||
: "";
|
||||
|
|
|
@ -1,24 +1,6 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.merge;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
import static org.apache.spark.sql.functions.when;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ForkJoinPool;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
|
@ -26,169 +8,186 @@ import eu.dnetlib.dhp.schema.common.EntityType;
|
|||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ForkJoinPool;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
import static org.apache.spark.sql.functions.when;
|
||||
|
||||
/**
|
||||
* Groups the graph content by entity identifier to ensure ID uniqueness
|
||||
*/
|
||||
public class GroupEntitiesSparkJob {
|
||||
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class);
|
||||
private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class);
|
||||
|
||||
private static final Encoder<OafEntity> OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class);
|
||||
private static final Encoder<OafEntity> OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class);
|
||||
|
||||
private ArgumentApplicationParser parser;
|
||||
private ArgumentApplicationParser parser;
|
||||
|
||||
public GroupEntitiesSparkJob(ArgumentApplicationParser parser) {
|
||||
this.parser = parser;
|
||||
}
|
||||
public GroupEntitiesSparkJob(ArgumentApplicationParser parser) {
|
||||
this.parser = parser;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
GroupEntitiesSparkJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json"));
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
GroupEntitiesSparkJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json"));
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String isLookupUrl = parser.get("isLookupUrl");
|
||||
log.info("isLookupUrl: {}", isLookupUrl);
|
||||
final String isLookupUrl = parser.get("isLookupUrl");
|
||||
log.info("isLookupUrl: {}", isLookupUrl);
|
||||
|
||||
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||
|
||||
new GroupEntitiesSparkJob(parser).run(isSparkSessionManaged, isLookupService);
|
||||
}
|
||||
new GroupEntitiesSparkJob(parser).run(isSparkSessionManaged, isLookupService);
|
||||
}
|
||||
|
||||
public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService)
|
||||
throws ISLookUpException {
|
||||
public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService)
|
||||
throws ISLookUpException {
|
||||
|
||||
String graphInputPath = parser.get("graphInputPath");
|
||||
log.info("graphInputPath: {}", graphInputPath);
|
||||
String graphInputPath = parser.get("graphInputPath");
|
||||
log.info("graphInputPath: {}", graphInputPath);
|
||||
|
||||
String checkpointPath = parser.get("checkpointPath");
|
||||
log.info("checkpointPath: {}", checkpointPath);
|
||||
String checkpointPath = parser.get("checkpointPath");
|
||||
log.info("checkpointPath: {}", checkpointPath);
|
||||
|
||||
String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible"));
|
||||
log.info("filterInvisible: {}", filterInvisible);
|
||||
boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible"));
|
||||
log.info("filterInvisible: {}", filterInvisible);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||
|
||||
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService);
|
||||
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService);
|
||||
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration());
|
||||
groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible, vocs);
|
||||
});
|
||||
}
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration());
|
||||
groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible, vocs);
|
||||
});
|
||||
}
|
||||
|
||||
private static void groupEntities(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String checkpointPath,
|
||||
String outputPath,
|
||||
boolean filterInvisible, VocabularyGroup vocs) {
|
||||
private static void groupEntities(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String checkpointPath,
|
||||
String outputPath,
|
||||
boolean filterInvisible, VocabularyGroup vocs) {
|
||||
|
||||
Dataset<OafEntity> allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC);
|
||||
Dataset<OafEntity> allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC);
|
||||
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
String entityInputPath = inputPath + "/" + entity;
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
String entityInputPath = inputPath + "/" + entity;
|
||||
|
||||
if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) {
|
||||
continue;
|
||||
}
|
||||
if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) {
|
||||
continue;
|
||||
}
|
||||
|
||||
allEntities = allEntities
|
||||
.union(
|
||||
((Dataset<OafEntity>) spark
|
||||
.read()
|
||||
.schema(Encoders.bean(entityClass).schema())
|
||||
.json(entityInputPath)
|
||||
.filter("length(id) > 0")
|
||||
.as(Encoders.bean(entityClass)))
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, OAFENTITY_KRYO_ENC));
|
||||
}
|
||||
allEntities = allEntities
|
||||
.union(
|
||||
((Dataset<OafEntity>) spark
|
||||
.read()
|
||||
.schema(Encoders.bean(entityClass).schema())
|
||||
.json(entityInputPath)
|
||||
.filter("length(id) > 0")
|
||||
.as(Encoders.bean(entityClass)))
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, OAFENTITY_KRYO_ENC));
|
||||
}
|
||||
|
||||
Dataset<?> groupedEntities = allEntities
|
||||
.map(
|
||||
(MapFunction<OafEntity, OafEntity>) entity -> GraphCleaningFunctions
|
||||
.applyCoarVocabularies(entity, vocs),
|
||||
OAFENTITY_KRYO_ENC)
|
||||
.groupByKey((MapFunction<OafEntity, String>) OafEntity::getId, Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<OafEntity>) OafMapperUtils::mergeEntities)
|
||||
.map(
|
||||
(MapFunction<Tuple2<String, OafEntity>, Tuple2<String, OafEntity>>) t -> new Tuple2<>(
|
||||
t._2().getClass().getName(), t._2()),
|
||||
Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC));
|
||||
Dataset<?> groupedEntities = allEntities
|
||||
.map(
|
||||
(MapFunction<OafEntity, OafEntity>) entity -> GraphCleaningFunctions
|
||||
.applyCoarVocabularies(entity, vocs),
|
||||
OAFENTITY_KRYO_ENC)
|
||||
.groupByKey((MapFunction<OafEntity, String>) OafEntity::getId, Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<OafEntity>) MergeUtils::checkedMerge)
|
||||
.map(
|
||||
(MapFunction<Tuple2<String, OafEntity>, Tuple2<String, OafEntity>>) t -> new Tuple2<>(
|
||||
t._2().getClass().getName(), t._2()),
|
||||
Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC));
|
||||
|
||||
// pivot on "_1" (classname of the entity)
|
||||
// created columns containing only entities of the same class
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
// pivot on "_1" (classname of the entity)
|
||||
// created columns containing only entities of the same class
|
||||
for (Map.Entry<EntityType, Class> e : ModelSupport.entityTypes.entrySet()) {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
|
||||
groupedEntities = groupedEntities
|
||||
.withColumn(
|
||||
entity,
|
||||
when(col("_1").equalTo(entityClass.getName()), col("_2")));
|
||||
}
|
||||
groupedEntities = groupedEntities
|
||||
.withColumn(
|
||||
entity,
|
||||
when(col("_1").equalTo(entityClass.getName()), col("_2")));
|
||||
}
|
||||
|
||||
groupedEntities
|
||||
.drop("_1", "_2")
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.save(checkpointPath);
|
||||
groupedEntities
|
||||
.drop("_1", "_2")
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.save(checkpointPath);
|
||||
|
||||
ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size());
|
||||
ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size());
|
||||
|
||||
ModelSupport.entityTypes
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(e -> parPool.submit(() -> {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
ModelSupport.entityTypes
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(e -> parPool.submit(() -> {
|
||||
String entity = e.getKey().name();
|
||||
Class<? extends OafEntity> entityClass = e.getValue();
|
||||
|
||||
spark
|
||||
.read()
|
||||
.load(checkpointPath)
|
||||
.select(col(entity).as("value"))
|
||||
.filter("value IS NOT NULL")
|
||||
.as(OAFENTITY_KRYO_ENC)
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, (Encoder<OafEntity>) Encoders.bean(entityClass))
|
||||
.filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE")
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + "/" + entity);
|
||||
}))
|
||||
.collect(Collectors.toList())
|
||||
.forEach(t -> {
|
||||
try {
|
||||
t.get();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
spark
|
||||
.read()
|
||||
.load(checkpointPath)
|
||||
.select(col(entity).as("value"))
|
||||
.filter("value IS NOT NULL")
|
||||
.as(OAFENTITY_KRYO_ENC)
|
||||
.map((MapFunction<OafEntity, OafEntity>) r -> r, (Encoder<OafEntity>) Encoders.bean(entityClass))
|
||||
.filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE")
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + "/" + entity);
|
||||
}))
|
||||
.collect(Collectors.toList())
|
||||
.forEach(t -> {
|
||||
try {
|
||||
t.get();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -506,6 +506,8 @@ public class GraphCleaningFunctions extends CleaningFunctions {
|
|||
.filter(Objects::nonNull)
|
||||
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
|
||||
.map(GraphCleaningFunctions::cleanValue)
|
||||
.sorted((s1, s2) -> s2.getValue().length() - s1.getValue().length())
|
||||
.limit(ModelHardLimits.MAX_ABSTRACTS)
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {
|
||||
|
|
|
@ -0,0 +1,79 @@
|
|||
package eu.dnetlib.dhp.schema.oaf.utils;
|
||||
|
||||
//
|
||||
// Source code recreated from a .class file by IntelliJ IDEA
|
||||
// (powered by FernFlower decompiler)
|
||||
//
|
||||
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class MergeComparator implements Comparator<Oaf> {
|
||||
public MergeComparator() {
|
||||
}
|
||||
|
||||
public int compare(Oaf left, Oaf right) {
|
||||
// nulls at the end
|
||||
if (left == null && right == null) {
|
||||
return 0;
|
||||
} else if (left == null) {
|
||||
return -1;
|
||||
} else if (right == null) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
// invisible
|
||||
if (left.getDataInfo() != null && left.getDataInfo().getInvisible() == true) {
|
||||
if (right.getDataInfo() != null && right.getDataInfo().getInvisible() == false) {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
// collectedfrom
|
||||
HashSet<String> lCf = getCollectedFromIds(left);
|
||||
HashSet<String> rCf = getCollectedFromIds(right);
|
||||
if (lCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2") && !rCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")) {
|
||||
return -1;
|
||||
} else if (!lCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2") && rCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
|
||||
|
||||
SubEntityType lClass = SubEntityType.fromClass(left.getClass());
|
||||
SubEntityType rClass = SubEntityType.fromClass(right.getClass());
|
||||
return lClass.ordinal() - rClass.ordinal();
|
||||
|
||||
}
|
||||
|
||||
protected HashSet<String> getCollectedFromIds(Oaf left) {
|
||||
return (HashSet) Optional.ofNullable(left.getCollectedfrom()).map((cf) -> {
|
||||
return (HashSet) cf.stream().map(KeyValue::getKey).collect(Collectors.toCollection(HashSet::new));
|
||||
}).orElse(new HashSet());
|
||||
}
|
||||
|
||||
enum SubEntityType {
|
||||
publication, dataset, software, otherresearchproduct, datasource, organization, project;
|
||||
|
||||
/**
|
||||
* Resolves the EntityType, given the relative class name
|
||||
*
|
||||
* @param clazz the given class name
|
||||
* @param <T> actual OafEntity subclass
|
||||
* @return the EntityType associated to the given class
|
||||
*/
|
||||
public static <T extends Oaf> SubEntityType fromClass(Class<T> clazz) {
|
||||
return valueOf(clazz.getSimpleName().toLowerCase());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,707 @@
|
|||
|
||||
package eu.dnetlib.dhp.schema.oaf.utils;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.AccessRightComparator;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.text.ParseException;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static com.google.common.base.Objects.firstNonNull;
|
||||
import static com.google.common.base.Preconditions.checkArgument;
|
||||
|
||||
public class MergeUtils {
|
||||
|
||||
public static <T extends Oaf> T checkedMerge(final T left, final T right) {
|
||||
return (T) merge(left, right, false);
|
||||
}
|
||||
|
||||
public static Oaf merge(final Oaf left, final Oaf right) {
|
||||
return merge(left, right, false);
|
||||
}
|
||||
|
||||
public static Oaf merge(final Oaf left, final Oaf right, boolean checkDelegatedAuthority) {
|
||||
if (sameClass(left, right, OafEntity.class)) {
|
||||
return mergeEntities(left, right, checkDelegatedAuthority);
|
||||
} else if (sameClass(left, right, Relation.class)) {
|
||||
return mergeRelation((Relation) left, (Relation) right);
|
||||
} else {
|
||||
throw new RuntimeException(
|
||||
String
|
||||
.format(
|
||||
"MERGE_FROM_AND_GET incompatible types: %s, %s",
|
||||
left.getClass().getCanonicalName(), right.getClass().getCanonicalName()));
|
||||
}
|
||||
}
|
||||
|
||||
private static <T extends Oaf> boolean sameClass(Object left, Object right, Class<T> cls) {
|
||||
return cls.isAssignableFrom(left.getClass()) && cls.isAssignableFrom(right.getClass());
|
||||
}
|
||||
|
||||
private static Oaf mergeEntities(Oaf left, Oaf right, boolean checkDelegatedAuthority) {
|
||||
|
||||
if (sameClass(left, right, Result.class)) {
|
||||
if (!left.getClass().equals(right.getClass()) || checkDelegatedAuthority) {
|
||||
return mergeResultsOfDifferentTypes((Result)left, (Result) right);
|
||||
}
|
||||
|
||||
if (sameClass(left, right, Publication.class)) {
|
||||
return mergePublication((Publication) left, (Publication) right);
|
||||
}
|
||||
if (sameClass(left, right, Dataset.class)) {
|
||||
return mergeDataset((Dataset) left, (Dataset) right);
|
||||
}
|
||||
if (sameClass(left, right, OtherResearchProduct.class)) {
|
||||
return mergeORP((OtherResearchProduct) left, (OtherResearchProduct) right);
|
||||
}
|
||||
if (sameClass(left, right, Software.class)) {
|
||||
return mergeSoftware((Software) left, (Software) right);
|
||||
}
|
||||
|
||||
return mergeResult((Result) left, (Result) right);
|
||||
} else if (sameClass(left, right, Datasource.class)) {
|
||||
// TODO
|
||||
final int trust = compareTrust(left, right);
|
||||
return mergeOafEntityFields((Datasource) left, (Datasource) right, trust);
|
||||
} else if (sameClass(left, right, Organization.class)) {
|
||||
return mergeOrganization((Organization) left, (Organization) right);
|
||||
} else if (sameClass(left, right, Project.class)) {
|
||||
return mergeProject((Project) left, (Project) right);
|
||||
} else {
|
||||
throw new RuntimeException(
|
||||
String
|
||||
.format(
|
||||
"MERGE_FROM_AND_GET incompatible types: %s, %s",
|
||||
left.getClass().getCanonicalName(), right.getClass().getCanonicalName()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is used in the global result grouping phase. It checks if one of the two is from a delegated authority
|
||||
* https://graph.openaire.eu/docs/data-model/pids-and-identifiers#delegated-authorities and in that case it prefers
|
||||
* such version.
|
||||
* <p>
|
||||
* Otherwise, it considers a resulttype priority order implemented in {@link ResultTypeComparator}
|
||||
* and proceeds with the canonical property merging.
|
||||
*
|
||||
* @param left
|
||||
* @param right
|
||||
* @return
|
||||
*/
|
||||
private static <T extends Result> T mergeResultsOfDifferentTypes(T left, T right) {
|
||||
|
||||
final boolean leftFromDelegatedAuthority = isFromDelegatedAuthority(left);
|
||||
final boolean rightFromDelegatedAuthority = isFromDelegatedAuthority(right);
|
||||
|
||||
if (leftFromDelegatedAuthority && !rightFromDelegatedAuthority) {
|
||||
return left;
|
||||
}
|
||||
if (!leftFromDelegatedAuthority && rightFromDelegatedAuthority) {
|
||||
return right;
|
||||
}
|
||||
//TODO: raise trust to have preferred fields from one or the other??
|
||||
if (new ResultTypeComparator().compare(left, right) < 0) {
|
||||
return mergeResult(left, right);
|
||||
} else {
|
||||
return mergeResult(right, left);
|
||||
}
|
||||
}
|
||||
|
||||
private static DataInfo chooseDataInfo(DataInfo left, DataInfo right, int trust) {
|
||||
if (trust > 0) {
|
||||
return left;
|
||||
} else if (trust == 0) {
|
||||
if (left == null || (left.getInvisible() != null && left.getInvisible().equals(Boolean.TRUE))) {
|
||||
return right;
|
||||
} else {
|
||||
return left;
|
||||
}
|
||||
} else {
|
||||
return right;
|
||||
}
|
||||
}
|
||||
|
||||
private static String chooseString(String left, String right, int trust) {
|
||||
if (trust > 0) {
|
||||
return left;
|
||||
} else if (trust == 0) {
|
||||
return StringUtils.isNotBlank(left) ? left : right;
|
||||
} else {
|
||||
return right;
|
||||
}
|
||||
}
|
||||
|
||||
private static <T> T chooseReference(T left, T right, int trust) {
|
||||
if (trust > 0) {
|
||||
return left;
|
||||
} else if (trust == 0) {
|
||||
return left != null ? left : right;
|
||||
} else {
|
||||
return right;
|
||||
}
|
||||
}
|
||||
|
||||
private static Long max(Long left, Long right) {
|
||||
if (left == null)
|
||||
return right;
|
||||
if (right == null)
|
||||
return left;
|
||||
|
||||
return Math.max(left, right);
|
||||
}
|
||||
|
||||
// trust ??
|
||||
private static Boolean booleanOR(Boolean a, Boolean b) {
|
||||
if (a == null) {
|
||||
return b;
|
||||
} else if (b == null) {
|
||||
return a;
|
||||
}
|
||||
|
||||
return a || b;
|
||||
}
|
||||
|
||||
|
||||
private static <T> List<T> unionDistinctLists(final List<T> left, final List<T> right, int trust) {
|
||||
if (left == null) {
|
||||
return right;
|
||||
} else if (right == null) {
|
||||
return left;
|
||||
}
|
||||
|
||||
List<T> h = trust >= 0 ? left : right;
|
||||
List<T> l = trust >= 0 ? right : left;
|
||||
|
||||
return Stream.concat(h.stream(), l.stream())
|
||||
.filter(Objects::nonNull)
|
||||
.distinct()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static List<String> unionDistinctListOfString(final List<String> l, final List<String> r) {
|
||||
if (l == null) {
|
||||
return r;
|
||||
} else if (r == null) {
|
||||
return l;
|
||||
}
|
||||
|
||||
return Stream.concat(l.stream(), r.stream())
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.distinct()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
//TODO review
|
||||
private static List<KeyValue> mergeKeyValue(List<KeyValue> left, List<KeyValue> right, int trust) {
|
||||
if (trust < 0) {
|
||||
List<KeyValue> s = left;
|
||||
left = right;
|
||||
right = s;
|
||||
}
|
||||
|
||||
HashMap<String, KeyValue> values = new HashMap<>();
|
||||
left.forEach(kv -> values.put(kv.getKey(), kv));
|
||||
right.forEach(kv -> values.putIfAbsent(kv.getKey(), kv));
|
||||
|
||||
return new ArrayList<>(values.values());
|
||||
}
|
||||
|
||||
private static List<StructuredProperty> unionTitle(List<StructuredProperty> left, List<StructuredProperty> right, int trust) {
|
||||
if (left == null) {
|
||||
return right;
|
||||
} else if (right == null) {
|
||||
return left;
|
||||
}
|
||||
|
||||
List<StructuredProperty> h = trust >= 0 ? left : right;
|
||||
List<StructuredProperty> l = trust >= 0 ? right : left;
|
||||
|
||||
return Stream.concat(h.stream(), l.stream())
|
||||
.filter(Objects::isNull)
|
||||
.distinct()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal utility that merges the common OafEntity fields
|
||||
*
|
||||
* @param merged
|
||||
* @param enrich
|
||||
* @param <T>
|
||||
* @return
|
||||
*/
|
||||
private static <T extends Oaf> T mergeOafFields(T merged, T enrich, int trust) {
|
||||
|
||||
//TODO: union of all values, but what does it mean with KeyValue pairs???
|
||||
merged.setCollectedfrom(mergeKeyValue(merged.getCollectedfrom(), enrich.getCollectedfrom(), trust));
|
||||
merged.setDataInfo(chooseDataInfo(merged.getDataInfo(), enrich.getDataInfo(), trust));
|
||||
merged.setLastupdatetimestamp(max(merged.getLastupdatetimestamp(), enrich.getLastupdatetimestamp()));
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal utility that merges the common OafEntity fields
|
||||
*
|
||||
* @param original
|
||||
* @param enrich
|
||||
* @param <T>
|
||||
* @return
|
||||
*/
|
||||
private static <T extends OafEntity> T mergeOafEntityFields(T original, T enrich, int trust) {
|
||||
final T merged = mergeOafFields(original, enrich, trust);
|
||||
|
||||
merged.setOriginalId(unionDistinctListOfString(merged.getOriginalId(), enrich.getOriginalId()));
|
||||
merged.setPid(unionDistinctLists(merged.getPid(), enrich.getPid(), trust));
|
||||
// dateofcollection mettere today quando si fa merge
|
||||
merged.setDateofcollection(chooseString(merged.getDateofcollection(), enrich.getDateofcollection(), trust));
|
||||
// setDateoftransformation mettere vuota in dedup, nota per Claudio
|
||||
merged.setDateoftransformation(chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust));
|
||||
// TODO: was missing in OafEntity.merge
|
||||
merged.setExtraInfo(unionDistinctLists(merged.getExtraInfo(), enrich.getExtraInfo(), trust));
|
||||
//oaiprovenanze da mettere a null quando si genera merge
|
||||
merged.setOaiprovenance(chooseReference(merged.getOaiprovenance(), enrich.getOaiprovenance(), trust));
|
||||
merged.setMeasures(unionDistinctLists(merged.getMeasures(), enrich.getMeasures(), trust));
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
|
||||
public static <T extends Relation> T mergeRelation(T original, T enrich) {
|
||||
int trust = compareTrust(original, enrich);
|
||||
T merge = mergeOafFields(original, enrich, trust);
|
||||
|
||||
checkArgument(Objects.equals(merge.getSource(), enrich.getSource()), "source ids must be equal");
|
||||
checkArgument(Objects.equals(merge.getTarget(), enrich.getTarget()), "target ids must be equal");
|
||||
checkArgument(Objects.equals(merge.getRelType(), enrich.getRelType()), "relType(s) must be equal");
|
||||
checkArgument(
|
||||
Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal");
|
||||
checkArgument(Objects.equals(merge.getRelClass(), enrich.getRelClass()), "relClass(es) must be equal");
|
||||
|
||||
//merge.setProvenance(mergeLists(merge.getProvenance(), enrich.getProvenance()));
|
||||
|
||||
//TODO: trust ??
|
||||
merge.setValidated(booleanOR(merge.getValidated(), enrich.getValidated()));
|
||||
try {
|
||||
merge.setValidationDate(ModelSupport.oldest(merge.getValidationDate(), enrich.getValidationDate()));
|
||||
} catch (ParseException e) {
|
||||
throw new IllegalArgumentException(String
|
||||
.format(
|
||||
"invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(),
|
||||
merge.getTarget(),
|
||||
merge.getValidationDate()));
|
||||
}
|
||||
|
||||
// TODO keyvalue merge
|
||||
merge.setProperties(mergeKeyValue(merge.getProperties(), enrich.getProperties(), trust));
|
||||
|
||||
return merge;
|
||||
}
|
||||
|
||||
public static <T extends Result> T mergeResult(T original, T enrich) {
|
||||
final int trust = compareTrust(original, enrich);
|
||||
T merge = mergeOafEntityFields(original, enrich, trust);
|
||||
|
||||
if (merge.getProcessingchargeamount() == null || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) {
|
||||
merge.setProcessingchargeamount(enrich.getProcessingchargeamount());
|
||||
merge.setProcessingchargecurrency(enrich.getProcessingchargecurrency());
|
||||
}
|
||||
|
||||
// author = usare la stessa logica che in dedup
|
||||
merge.setAuthor(chooseReference(merge.getAuthor(), enrich.getAuthor(), trust));
|
||||
// il primo che mi arriva secondo l'ordinamento per priorita'
|
||||
merge.setResulttype(chooseReference(merge.getResulttype(), enrich.getResulttype(), trust));
|
||||
// gestito come il resulttype perche' e' un subtype
|
||||
merge.setMetaResourceType(chooseReference(merge.getMetaResourceType(), enrich.getMetaResourceType(), trust));
|
||||
// spostiamo nell'instance e qui prendo il primo che arriva
|
||||
merge.setLanguage(chooseReference(merge.getLanguage(), enrich.getLanguage(), trust));
|
||||
// country lasicamo,o cosi' -> parentesi sul datainfo
|
||||
merge.setCountry(unionDistinctLists(merge.getCountry(), enrich.getCountry(), trust));
|
||||
//ok
|
||||
merge.setSubject(unionDistinctLists(merge.getSubject(), enrich.getSubject(), trust));
|
||||
// union per priority quindi vanno in append
|
||||
merge.setTitle(unionTitle(merge.getTitle(), enrich.getTitle(), trust));
|
||||
//ok
|
||||
merge.setRelevantdate(unionDistinctLists(merge.getRelevantdate(), enrich.getRelevantdate(), trust));
|
||||
// prima trust e poi longest list
|
||||
merge.setDescription(longestLists(merge.getDescription(), enrich.getDescription()));
|
||||
// trust piu' alto e poi piu' vecchia
|
||||
merge.setDateofacceptance(chooseReference(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust));
|
||||
// ok, ma publisher va messo ripetibile
|
||||
merge.setPublisher(chooseReference(merge.getPublisher(), enrich.getPublisher(), trust));
|
||||
// ok
|
||||
merge.setEmbargoenddate(chooseReference(merge.getEmbargoenddate(), enrich.getEmbargoenddate(), trust));
|
||||
// ok
|
||||
merge.setSource(unionDistinctLists(merge.getSource(), enrich.getSource(), trust));
|
||||
// ok
|
||||
merge.setFulltext(unionDistinctLists(merge.getFulltext(), enrich.getFulltext(), trust));
|
||||
// ok
|
||||
merge.setFormat(unionDistinctLists(merge.getFormat(), enrich.getFormat(), trust));
|
||||
// ok
|
||||
merge.setContributor(unionDistinctLists(merge.getContributor(), enrich.getContributor(), trust));
|
||||
|
||||
// prima prendo l'higher trust, su questo prendo il valore migliore nelle istanze TODO
|
||||
// trust maggiore ma a parita' di trust il piu' specifico (base del vocabolario)
|
||||
// vedi note
|
||||
merge.setResourcetype(firstNonNull(merge.getResourcetype(), enrich.getResourcetype()));
|
||||
|
||||
// ok
|
||||
merge.setCoverage(unionDistinctLists(merge.getCoverage(), enrich.getCoverage(), trust));
|
||||
|
||||
// most open ok
|
||||
if (enrich.getBestaccessright() != null
|
||||
&& new AccessRightComparator<>()
|
||||
.compare(enrich.getBestaccessright(), merge.getBestaccessright()) < 0) {
|
||||
merge.setBestaccessright(enrich.getBestaccessright());
|
||||
}
|
||||
|
||||
// TODO merge of datainfo given same id
|
||||
merge.setContext(unionDistinctLists(merge.getContext(), enrich.getContext(), trust));
|
||||
|
||||
//ok
|
||||
merge.setExternalReference(unionDistinctLists(merge.getExternalReference(), enrich.getExternalReference(), trust));
|
||||
|
||||
//instance enrichment or union
|
||||
// review instance equals => add pid to comparision
|
||||
if (!isAnEnrichment(merge) && !isAnEnrichment(enrich))
|
||||
merge.setInstance(unionDistinctLists(merge.getInstance(), enrich.getInstance(), trust));
|
||||
else {
|
||||
final List<Instance> enrichmentInstances = isAnEnrichment(merge) ? merge.getInstance()
|
||||
: enrich.getInstance();
|
||||
final List<Instance> enrichedInstances = isAnEnrichment(merge) ? enrich.getInstance()
|
||||
: merge.getInstance();
|
||||
if (isAnEnrichment(merge))
|
||||
merge.setDataInfo(enrich.getDataInfo());
|
||||
merge.setInstance(enrichInstances(enrichedInstances, enrichmentInstances));
|
||||
}
|
||||
|
||||
merge.setEoscifguidelines(unionDistinctLists(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust));
|
||||
merge.setIsGreen(booleanOR(merge.getIsGreen(), enrich.getIsGreen()));
|
||||
// OK but should be list of values
|
||||
merge.setOpenAccessColor(chooseReference(merge.getOpenAccessColor(), enrich.getOpenAccessColor(), trust));
|
||||
merge.setIsInDiamondJournal(booleanOR(merge.getIsInDiamondJournal(), enrich.getIsInDiamondJournal()));
|
||||
merge.setPubliclyFunded(booleanOR(merge.getPubliclyFunded(), enrich.getPubliclyFunded()));
|
||||
|
||||
return merge;
|
||||
}
|
||||
|
||||
private static <T extends OtherResearchProduct> T mergeORP(T original, T enrich) {
|
||||
int trust = compareTrust(original, enrich);
|
||||
final T merge = mergeResult(original, enrich);
|
||||
|
||||
merge.setContactperson(unionDistinctLists(merge.getContactperson(), enrich.getContactperson(), trust));
|
||||
merge.setContactgroup(unionDistinctLists(merge.getContactgroup(), enrich.getContactgroup(), trust));
|
||||
merge.setTool(unionDistinctLists(merge.getTool(), enrich.getTool(), trust));
|
||||
|
||||
return merge;
|
||||
}
|
||||
|
||||
private static <T extends Software> T mergeSoftware(T original, T enrich) {
|
||||
int trust = compareTrust(original, enrich);
|
||||
final T merge = mergeResult(original, enrich);
|
||||
|
||||
merge.setDocumentationUrl(unionDistinctLists(merge.getDocumentationUrl(), enrich.getDocumentationUrl(), trust));
|
||||
merge.setLicense(unionDistinctLists(merge.getLicense(), enrich.getLicense(), trust));
|
||||
merge.setCodeRepositoryUrl(chooseReference(merge.getCodeRepositoryUrl(), enrich.getCodeRepositoryUrl(), trust));
|
||||
merge.setProgrammingLanguage(chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust));
|
||||
|
||||
return merge;
|
||||
}
|
||||
|
||||
private static <T extends Dataset> T mergeDataset(T original, T enrich) {
|
||||
int trust = compareTrust(original, enrich);
|
||||
T merge = mergeResult(original, enrich);
|
||||
|
||||
merge.setStoragedate(chooseReference(merge.getStoragedate(), enrich.getStoragedate(), trust));
|
||||
merge.setDevice(chooseReference(merge.getDevice(), enrich.getDevice(), trust));
|
||||
merge.setSize(chooseReference(merge.getSize(), enrich.getSize(), trust));
|
||||
merge.setVersion(chooseReference(merge.getVersion(), enrich.getVersion(), trust));
|
||||
merge.setLastmetadataupdate(chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust));
|
||||
merge.setMetadataversionnumber(chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust));
|
||||
merge.setGeolocation(unionDistinctLists(merge.getGeolocation(), enrich.getGeolocation(), trust));
|
||||
|
||||
return merge;
|
||||
}
|
||||
|
||||
public static <T extends Publication> T mergePublication(T original, T enrich) {
|
||||
final int trust = compareTrust(original, enrich);
|
||||
T merged = mergeResult(original, enrich);
|
||||
|
||||
merged.setJournal(chooseReference(merged.getJournal(), enrich.getJournal(), trust));
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
private static <T extends Organization> T mergeOrganization(T left, T enrich) {
|
||||
int trust = compareTrust(left, enrich);
|
||||
T merged = mergeOafEntityFields(left, enrich, trust);
|
||||
|
||||
merged.setLegalshortname(chooseReference(merged.getLegalshortname(), enrich.getLegalshortname(), trust));
|
||||
merged.setLegalname(chooseReference(merged.getLegalname(), enrich.getLegalname(), trust));
|
||||
merged.setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust));
|
||||
merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust));
|
||||
merged.setLogourl(chooseReference(merged.getLogourl(), enrich.getLogourl(), trust));
|
||||
merged.setEclegalbody(chooseReference(merged.getEclegalbody(), enrich.getEclegalbody(), trust));
|
||||
merged.setEclegalperson(chooseReference(merged.getEclegalperson(), enrich.getEclegalperson(), trust));
|
||||
merged.setEcnonprofit(chooseReference(merged.getEcnonprofit(), enrich.getEcnonprofit(), trust));
|
||||
merged.setEcresearchorganization(chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust));
|
||||
merged.setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust));
|
||||
merged.setEcinternationalorganizationeurinterests(chooseReference(merged.getEcinternationalorganizationeurinterests(), enrich.getEcinternationalorganizationeurinterests(), trust));
|
||||
merged.setEcinternationalorganization(chooseReference(merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust));
|
||||
merged.setEcenterprise(chooseReference(merged.getEcenterprise(), enrich.getEcenterprise(), trust));
|
||||
merged.setEcsmevalidated(chooseReference(merged.getEcsmevalidated(), enrich.getEcsmevalidated(), trust));
|
||||
merged.setEcnutscode(chooseReference(merged.getEcnutscode(), enrich.getEcnutscode(), trust));
|
||||
merged.setCountry(chooseReference(merged.getCountry(), enrich.getCountry(), trust));
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
public static <T extends Project> T mergeProject(T original, T enrich) {
|
||||
int trust = compareTrust(original, enrich);
|
||||
T merged = mergeOafEntityFields(original, enrich, trust);
|
||||
|
||||
merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust));
|
||||
merged.setCode(chooseReference(merged.getCode(), enrich.getCode(), trust));
|
||||
merged.setAcronym(chooseReference(merged.getAcronym(), enrich.getAcronym(), trust));
|
||||
merged.setTitle(chooseReference(merged.getTitle(), enrich.getTitle(), trust));
|
||||
merged.setStartdate(chooseReference(merged.getStartdate(), enrich.getStartdate(), trust));
|
||||
merged.setEnddate(chooseReference(merged.getEnddate(), enrich.getEnddate(), trust));
|
||||
merged.setCallidentifier(chooseReference(merged.getCallidentifier(), enrich.getCallidentifier(), trust));
|
||||
merged.setKeywords(chooseReference(merged.getKeywords(), enrich.getKeywords(), trust));
|
||||
merged.setDuration(chooseReference(merged.getDuration(), enrich.getDuration(), trust));
|
||||
merged.setEcsc39(chooseReference(merged.getEcsc39(), enrich.getEcsc39(), trust));
|
||||
merged.setOamandatepublications(chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust));
|
||||
merged.setEcarticle29_3(chooseReference(merged.getEcarticle29_3(), enrich.getEcarticle29_3(), trust));
|
||||
merged.setSubjects(unionDistinctLists(merged.getSubjects(), enrich.getSubjects(), trust));
|
||||
merged.setFundingtree(unionDistinctLists(merged.getFundingtree(), enrich.getFundingtree(), trust));
|
||||
merged.setContracttype(chooseReference(merged.getContracttype(), enrich.getContracttype(), trust));
|
||||
merged.setOptional1(chooseReference(merged.getOptional1(), enrich.getOptional1(), trust));
|
||||
merged.setOptional2(chooseReference(merged.getOptional2(), enrich.getOptional2(), trust));
|
||||
merged.setJsonextrainfo(chooseReference(merged.getJsonextrainfo(), enrich.getJsonextrainfo(), trust));
|
||||
merged.setContactfullname(chooseReference(merged.getContactfullname(), enrich.getContactfullname(), trust));
|
||||
merged.setContactfax(chooseReference(merged.getContactfax(), enrich.getContactfax(), trust));
|
||||
merged.setContactphone(chooseReference(merged.getContactphone(), enrich.getContactphone(), trust));
|
||||
merged.setContactemail(chooseReference(merged.getContactemail(), enrich.getContactemail(), trust));
|
||||
merged.setSummary(chooseReference(merged.getSummary(), enrich.getSummary(), trust));
|
||||
merged.setCurrency(chooseReference(merged.getCurrency(), enrich.getCurrency(), trust));
|
||||
|
||||
//missin in Project.merge
|
||||
merged.setTotalcost(chooseReference(merged.getTotalcost(), enrich.getTotalcost(), trust));
|
||||
merged.setFundedamount(chooseReference(merged.getFundedamount(), enrich.getFundedamount(), trust));
|
||||
|
||||
// trust ??
|
||||
if (enrich.getH2020topiccode() != null && StringUtils.isEmpty(merged.getH2020topiccode())) {
|
||||
merged.setH2020topiccode(enrich.getH2020topiccode());
|
||||
merged.setH2020topicdescription(enrich.getH2020topicdescription());
|
||||
}
|
||||
|
||||
merged.setH2020classification(unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust));
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Longest lists list.
|
||||
*
|
||||
* @param a the a
|
||||
* @param b the b
|
||||
* @return the list
|
||||
*/
|
||||
public static List<Field<String>> longestLists(List<Field<String>> a, List<Field<String>> b) {
|
||||
if (a == null || b == null)
|
||||
return a == null ? b : a;
|
||||
|
||||
return a.size() >= b.size() ? a : b;
|
||||
}
|
||||
|
||||
/**
|
||||
* This main method apply the enrichment of the instances
|
||||
*
|
||||
* @param toEnrichInstances the instances that could be enriched
|
||||
* @param enrichmentInstances the enrichment instances
|
||||
* @return list of instances possibly enriched
|
||||
*/
|
||||
private static List<Instance> enrichInstances(final List<Instance> toEnrichInstances,
|
||||
final List<Instance> enrichmentInstances) {
|
||||
final List<Instance> enrichmentResult = new ArrayList<>();
|
||||
|
||||
if (toEnrichInstances == null) {
|
||||
return enrichmentResult;
|
||||
}
|
||||
if (enrichmentInstances == null) {
|
||||
return enrichmentResult;
|
||||
}
|
||||
Map<String, Instance> ri = toInstanceMap(enrichmentInstances);
|
||||
|
||||
toEnrichInstances.forEach(i -> {
|
||||
final List<Instance> e = findEnrichmentsByPID(i.getPid(), ri);
|
||||
if (e != null && e.size() > 0) {
|
||||
e.forEach(enr -> applyEnrichment(i, enr));
|
||||
} else {
|
||||
final List<Instance> a = findEnrichmentsByPID(i.getAlternateIdentifier(), ri);
|
||||
if (a != null && a.size() > 0) {
|
||||
a.forEach(enr -> applyEnrichment(i, enr));
|
||||
}
|
||||
}
|
||||
enrichmentResult.add(i);
|
||||
});
|
||||
return enrichmentResult;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method converts the list of instance enrichments
|
||||
* into a Map where the key is the normalized identifier
|
||||
* and the value is the instance itself
|
||||
*
|
||||
* @param ri the list of enrichment instances
|
||||
* @return the result map
|
||||
*/
|
||||
private static Map<String, Instance> toInstanceMap(final List<Instance> ri) {
|
||||
return ri
|
||||
.stream()
|
||||
.filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null)
|
||||
.flatMap(i -> {
|
||||
final List<Pair<String, Instance>> result = new ArrayList<>();
|
||||
if (i.getPid() != null)
|
||||
i
|
||||
.getPid()
|
||||
.stream()
|
||||
.filter(MergeUtils::validPid)
|
||||
.forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i)));
|
||||
if (i.getAlternateIdentifier() != null)
|
||||
i
|
||||
.getAlternateIdentifier()
|
||||
.stream()
|
||||
.filter(MergeUtils::validPid)
|
||||
.forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i)));
|
||||
return result.stream();
|
||||
})
|
||||
.collect(
|
||||
Collectors
|
||||
.toMap(
|
||||
Pair::getLeft,
|
||||
Pair::getRight,
|
||||
(a, b) -> a));
|
||||
}
|
||||
|
||||
private static boolean isFromDelegatedAuthority(Result r) {
|
||||
return Optional
|
||||
.ofNullable(r.getInstance())
|
||||
.map(
|
||||
instance -> instance
|
||||
.stream()
|
||||
.filter(i -> Objects.nonNull(i.getCollectedfrom()))
|
||||
.map(i -> i.getCollectedfrom().getKey())
|
||||
.anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId)))
|
||||
.orElse(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Valid pid boolean.
|
||||
*
|
||||
* @param p the p
|
||||
* @return the boolean
|
||||
*/
|
||||
private static boolean validPid(final StructuredProperty p) {
|
||||
return p.getValue() != null && p.getQualifier() != null && p.getQualifier().getClassid() != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalize pid string.
|
||||
*
|
||||
* @param pid the pid
|
||||
* @return the string
|
||||
*/
|
||||
private static String extractKeyFromPid(final StructuredProperty pid) {
|
||||
if (pid == null)
|
||||
return null;
|
||||
final StructuredProperty normalizedPid = CleaningFunctions.normalizePidValue(pid);
|
||||
|
||||
return String.format("%s::%s", normalizedPid.getQualifier().getClassid(), normalizedPid.getValue());
|
||||
}
|
||||
|
||||
/**
|
||||
* This utility method finds the list of enrichment instances
|
||||
* that match one or more PIDs in the input list
|
||||
*
|
||||
* @param pids the list of PIDs
|
||||
* @param enrichments the List of enrichment instances having the same pid
|
||||
* @return the list
|
||||
*/
|
||||
private static List<Instance> findEnrichmentsByPID(final List<StructuredProperty> pids,
|
||||
final Map<String, Instance> enrichments) {
|
||||
if (pids == null || enrichments == null)
|
||||
return null;
|
||||
return pids
|
||||
.stream()
|
||||
.map(MergeUtils::extractKeyFromPid)
|
||||
.map(enrichments::get)
|
||||
.filter(Objects::nonNull)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Is an enrichment boolean.
|
||||
*
|
||||
* @param e the e
|
||||
* @return the boolean
|
||||
*/
|
||||
private static boolean isAnEnrichment(OafEntity e) {
|
||||
return e.getDataInfo() != null &&
|
||||
e.getDataInfo().getProvenanceaction() != null
|
||||
&& ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid());
|
||||
}
|
||||
|
||||
/**
|
||||
* This method apply enrichment on a single instance
|
||||
* The enrichment consists of replacing values on
|
||||
* single attribute only if in the current instance is missing
|
||||
* The only repeatable field enriched is measures
|
||||
*
|
||||
* @param merge the current instance
|
||||
* @param enrichment the enrichment instance
|
||||
*/
|
||||
private static void applyEnrichment(final Instance merge, final Instance enrichment) {
|
||||
if (merge == null || enrichment == null)
|
||||
return;
|
||||
|
||||
merge.setLicense(firstNonNull(merge.getLicense(), enrichment.getLicense()));
|
||||
merge.setAccessright(firstNonNull(merge.getAccessright(), enrichment.getAccessright()));
|
||||
merge.setInstancetype(firstNonNull(merge.getInstancetype(), enrichment.getInstancetype()));
|
||||
merge.setInstanceTypeMapping(firstNonNull(merge.getInstanceTypeMapping(), enrichment.getInstanceTypeMapping()));
|
||||
merge.setHostedby(firstNonNull(merge.getHostedby(), enrichment.getHostedby()));
|
||||
merge.setUrl(unionDistinctLists(merge.getUrl(), enrichment.getUrl(), 0));
|
||||
merge.setDistributionlocation(firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation()));
|
||||
merge.setCollectedfrom(firstNonNull(merge.getCollectedfrom(), enrichment.getCollectedfrom()));
|
||||
// pid and alternateId are used for matching
|
||||
merge.setDateofacceptance(firstNonNull(merge.getDateofacceptance(), enrichment.getDateofacceptance()));
|
||||
merge.setProcessingchargeamount(firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount()));
|
||||
merge.setProcessingchargecurrency(firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency()));
|
||||
merge.setRefereed(firstNonNull(merge.getRefereed(), enrichment.getRefereed()));
|
||||
merge.setMeasures(unionDistinctLists(merge.getMeasures(), enrichment.getMeasures(), 0));
|
||||
merge.setFulltext(firstNonNull(merge.getFulltext(), enrichment.getFulltext()));
|
||||
}
|
||||
|
||||
private static int compareTrust(Oaf a, Oaf b) {
|
||||
String left = Optional
|
||||
.ofNullable(a.getDataInfo())
|
||||
.map(DataInfo::getTrust)
|
||||
.orElse("0.0");
|
||||
|
||||
String right = Optional
|
||||
.ofNullable(b.getDataInfo())
|
||||
.map(DataInfo::getTrust)
|
||||
.orElse("0.0");
|
||||
|
||||
return left.compareTo(right);
|
||||
}
|
||||
|
||||
}
|
|
@ -14,7 +14,6 @@ import java.util.stream.Collectors;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.AccessRightComparator;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
|
||||
public class OafMapperUtils {
|
||||
|
@ -22,65 +21,6 @@ public class OafMapperUtils {
|
|||
private OafMapperUtils() {
|
||||
}
|
||||
|
||||
public static Oaf merge(final Oaf left, final Oaf right) {
|
||||
if (ModelSupport.isSubClass(left, OafEntity.class)) {
|
||||
return mergeEntities((OafEntity) left, (OafEntity) right);
|
||||
} else if (ModelSupport.isSubClass(left, Relation.class)) {
|
||||
((Relation) left).mergeFrom((Relation) right);
|
||||
} else {
|
||||
throw new IllegalArgumentException("invalid Oaf type:" + left.getClass().getCanonicalName());
|
||||
}
|
||||
return left;
|
||||
}
|
||||
|
||||
public static OafEntity mergeEntities(OafEntity left, OafEntity right) {
|
||||
if (ModelSupport.isSubClass(left, Result.class)) {
|
||||
return mergeResults((Result) left, (Result) right);
|
||||
} else if (ModelSupport.isSubClass(left, Datasource.class)) {
|
||||
left.mergeFrom(right);
|
||||
} else if (ModelSupport.isSubClass(left, Organization.class)) {
|
||||
left.mergeFrom(right);
|
||||
} else if (ModelSupport.isSubClass(left, Project.class)) {
|
||||
left.mergeFrom(right);
|
||||
} else {
|
||||
throw new IllegalArgumentException("invalid OafEntity subtype:" + left.getClass().getCanonicalName());
|
||||
}
|
||||
return left;
|
||||
}
|
||||
|
||||
public static Result mergeResults(Result left, Result right) {
|
||||
|
||||
final boolean leftFromDelegatedAuthority = isFromDelegatedAuthority(left);
|
||||
final boolean rightFromDelegatedAuthority = isFromDelegatedAuthority(right);
|
||||
|
||||
if (leftFromDelegatedAuthority && !rightFromDelegatedAuthority) {
|
||||
return left;
|
||||
}
|
||||
if (!leftFromDelegatedAuthority && rightFromDelegatedAuthority) {
|
||||
return right;
|
||||
}
|
||||
|
||||
if (new ResultTypeComparator().compare(left, right) < 0) {
|
||||
left.mergeFrom(right);
|
||||
return left;
|
||||
} else {
|
||||
right.mergeFrom(left);
|
||||
return right;
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean isFromDelegatedAuthority(Result r) {
|
||||
return Optional
|
||||
.ofNullable(r.getInstance())
|
||||
.map(
|
||||
instance -> instance
|
||||
.stream()
|
||||
.filter(i -> Objects.nonNull(i.getCollectedfrom()))
|
||||
.map(i -> i.getCollectedfrom().getKey())
|
||||
.anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId)))
|
||||
.orElse(false);
|
||||
}
|
||||
|
||||
public static KeyValue keyValue(final String k, final String v) {
|
||||
final KeyValue kv = new KeyValue();
|
||||
kv.setKey(k);
|
||||
|
|
|
@ -0,0 +1,111 @@
|
|||
|
||||
package eu.dnetlib.dhp.schema.oaf.utils;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.*;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
|
||||
public class MergeUtilsTest {
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
|
||||
@Test
|
||||
void testMergePubs() throws IOException {
|
||||
Publication p1 = read("publication_1.json", Publication.class);
|
||||
Publication p2 = read("publication_2.json", Publication.class);
|
||||
Dataset d1 = read("dataset_1.json", Dataset.class);
|
||||
Dataset d2 = read("dataset_2.json", Dataset.class);
|
||||
|
||||
assertEquals(1, p1.getCollectedfrom().size());
|
||||
assertEquals(ModelConstants.CROSSREF_ID, p1.getCollectedfrom().get(0).getKey());
|
||||
assertEquals(1, d2.getCollectedfrom().size());
|
||||
assertFalse(cfId(d2.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID));
|
||||
|
||||
assertEquals(1, p2.getCollectedfrom().size());
|
||||
assertFalse(cfId(p2.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID));
|
||||
assertEquals(1, d1.getCollectedfrom().size());
|
||||
assertTrue(cfId(d1.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID));
|
||||
|
||||
final Result p1d2 = MergeUtils.checkedMerge(p1, d2);
|
||||
assertEquals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, p1d2.getResulttype().getClassid());
|
||||
assertTrue(p1d2 instanceof Publication);
|
||||
assertEquals(p1.getId(), p1d2.getId());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMergePubs_1() throws IOException {
|
||||
Publication p2 = read("publication_2.json", Publication.class);
|
||||
Dataset d1 = read("dataset_1.json", Dataset.class);
|
||||
|
||||
final Result p2d1 = MergeUtils.checkedMerge(p2, d1);
|
||||
assertEquals((ModelConstants.DATASET_RESULTTYPE_CLASSID), p2d1.getResulttype().getClassid());
|
||||
assertTrue(p2d1 instanceof Dataset);
|
||||
assertEquals(d1.getId(), p2d1.getId());
|
||||
assertEquals(2, p2d1.getCollectedfrom().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMergePubs_2() throws IOException {
|
||||
Publication p1 = read("publication_1.json", Publication.class);
|
||||
Publication p2 = read("publication_2.json", Publication.class);
|
||||
|
||||
Result p1p2 = MergeUtils.checkedMerge(p1, p2);
|
||||
assertTrue(p1p2 instanceof Publication);
|
||||
assertEquals(p1.getId(), p1p2.getId());
|
||||
assertEquals(2, p1p2.getCollectedfrom().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDelegatedAuthority_1() throws IOException {
|
||||
Dataset d1 = read("dataset_2.json", Dataset.class);
|
||||
Dataset d2 = read("dataset_delegated.json", Dataset.class);
|
||||
|
||||
assertEquals(1, d2.getCollectedfrom().size());
|
||||
assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID));
|
||||
|
||||
Result res = (Result) MergeUtils.merge(d1, d2, true);
|
||||
|
||||
assertEquals(d2, res);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testDelegatedAuthority_2() throws IOException {
|
||||
Dataset p1 = read("publication_1.json", Dataset.class);
|
||||
Dataset d2 = read("dataset_delegated.json", Dataset.class);
|
||||
|
||||
assertEquals(1, d2.getCollectedfrom().size());
|
||||
assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID));
|
||||
|
||||
Result res = (Result) MergeUtils.merge(p1, d2, true);
|
||||
|
||||
assertEquals(d2, res);
|
||||
}
|
||||
|
||||
protected HashSet<String> cfId(List<KeyValue> collectedfrom) {
|
||||
return collectedfrom.stream().map(KeyValue::getKey).collect(Collectors.toCollection(HashSet::new));
|
||||
}
|
||||
|
||||
protected <T extends Result> T read(String filename, Class<T> clazz) throws IOException {
|
||||
final String json = IOUtils.toString(getClass().getResourceAsStream(filename));
|
||||
return OBJECT_MAPPER.readValue(json, clazz);
|
||||
}
|
||||
|
||||
}
|
|
@ -149,7 +149,7 @@ class OafMapperUtilsTest {
|
|||
void testDate() {
|
||||
final String date = GraphCleaningFunctions.cleanDate("23-FEB-1998");
|
||||
assertNotNull(date);
|
||||
System.out.println(date);
|
||||
assertEquals("1998-02-23", date);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -166,8 +166,8 @@ class OafMapperUtilsTest {
|
|||
|
||||
assertEquals(
|
||||
ModelConstants.PUBLICATION_RESULTTYPE_CLASSID,
|
||||
OafMapperUtils
|
||||
.mergeResults(p1, d2)
|
||||
MergeUtils
|
||||
.mergeResult(p1, d2)
|
||||
.getResulttype()
|
||||
.getClassid());
|
||||
|
||||
|
@ -178,8 +178,8 @@ class OafMapperUtilsTest {
|
|||
|
||||
assertEquals(
|
||||
ModelConstants.DATASET_RESULTTYPE_CLASSID,
|
||||
OafMapperUtils
|
||||
.mergeResults(p2, d1)
|
||||
MergeUtils
|
||||
.mergeResult(p2, d1)
|
||||
.getResulttype()
|
||||
.getClassid());
|
||||
}
|
||||
|
@ -192,7 +192,7 @@ class OafMapperUtilsTest {
|
|||
assertEquals(1, d2.getCollectedfrom().size());
|
||||
assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID));
|
||||
|
||||
Result res = OafMapperUtils.mergeResults(d1, d2);
|
||||
Result res = MergeUtils.mergeResult(d1, d2);
|
||||
|
||||
assertEquals(d2, res);
|
||||
|
||||
|
|
|
@ -1,14 +1,13 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.promote;
|
||||
|
||||
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
|
||||
import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
|
||||
import java.util.function.BiFunction;
|
||||
|
||||
import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier;
|
||||
import eu.dnetlib.dhp.schema.oaf.Oaf;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
|
||||
|
||||
/** OAF model merging support. */
|
||||
public class MergeAndGet {
|
||||
|
@ -46,20 +45,7 @@ public class MergeAndGet {
|
|||
}
|
||||
|
||||
private static <G extends Oaf, A extends Oaf> G mergeFromAndGet(G x, A y) {
|
||||
if (isSubClass(x, Relation.class) && isSubClass(y, Relation.class)) {
|
||||
((Relation) x).mergeFrom((Relation) y);
|
||||
return x;
|
||||
} else if (isSubClass(x, OafEntity.class)
|
||||
&& isSubClass(y, OafEntity.class)
|
||||
&& isSubClass(x, y)) {
|
||||
((OafEntity) x).mergeFrom((OafEntity) y);
|
||||
return x;
|
||||
}
|
||||
throw new RuntimeException(
|
||||
String
|
||||
.format(
|
||||
"MERGE_FROM_AND_GET incompatible types: %s, %s",
|
||||
x.getClass().getCanonicalName(), y.getClass().getCanonicalName()));
|
||||
return (G) MergeUtils.merge(x, y);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
|
|
@ -64,6 +64,9 @@ public class PrepareAffiliationRelations implements Serializable {
|
|||
final String pubmedInputPath = parser.get("pubmedInputPath");
|
||||
log.info("pubmedInputPath: {}", pubmedInputPath);
|
||||
|
||||
final String openapcInputPath = parser.get("openapcInputPath");
|
||||
log.info("openapcInputPath: {}", openapcInputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
|
@ -85,8 +88,14 @@ public class PrepareAffiliationRelations implements Serializable {
|
|||
JavaPairRDD<Text, Text> pubmedRelations = prepareAffiliationRelations(
|
||||
spark, pubmedInputPath, collectedFromPubmed);
|
||||
|
||||
List<KeyValue> collectedFromOpenAPC = OafMapperUtils
|
||||
.listKeyValues(ModelConstants.OPEN_APC_ID, "OpenAPC");
|
||||
JavaPairRDD<Text, Text> openAPCRelations = prepareAffiliationRelations(
|
||||
spark, openapcInputPath, collectedFromOpenAPC);
|
||||
|
||||
crossrefRelations
|
||||
.union(pubmedRelations)
|
||||
.union(openAPCRelations)
|
||||
.saveAsHadoopFile(
|
||||
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
|
||||
|
|
|
@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable {
|
|||
|
||||
return projectScores.map((MapFunction<BipProjectModel, Project>) bipProjectScores -> {
|
||||
Project project = new Project();
|
||||
project.setId(bipProjectScores.getProjectId());
|
||||
//project.setId(bipProjectScores.getProjectId());
|
||||
project.setMeasures(bipProjectScores.toMeasures());
|
||||
return project;
|
||||
}, Encoders.bean(Project.class))
|
||||
|
|
|
@ -34,6 +34,11 @@ public class BipProjectModel {
|
|||
|
||||
String totalCitationCount;
|
||||
|
||||
public String getProjectId() {
|
||||
return projectId;
|
||||
}
|
||||
|
||||
|
||||
// each project bip measure has exactly one value, hence one key-value pair
|
||||
private Measure createMeasure(String measureId, String measureValue) {
|
||||
|
||||
|
|
|
@ -75,6 +75,7 @@ public class GetFOSSparkJob implements Serializable {
|
|||
fosData.map((MapFunction<Row, FOSDataModel>) r -> {
|
||||
FOSDataModel fosDataModel = new FOSDataModel();
|
||||
fosDataModel.setDoi(r.getString(0).toLowerCase());
|
||||
fosDataModel.setOaid(r.getString(1).toLowerCase());
|
||||
fosDataModel.setLevel1(r.getString(2));
|
||||
fosDataModel.setLevel2(r.getString(3));
|
||||
fosDataModel.setLevel3(r.getString(4));
|
||||
|
|
|
@ -16,12 +16,14 @@ import org.apache.spark.sql.Dataset;
|
|||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.jetbrains.annotations.NotNull;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.dhp.schema.oaf.Subject;
|
||||
|
@ -52,62 +54,90 @@ public class PrepareFOSSparkJob implements Serializable {
|
|||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final Boolean distributeDOI = Optional
|
||||
.ofNullable(parser.get("distributeDoi"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
distributeFOSdois(
|
||||
spark,
|
||||
sourcePath,
|
||||
if (distributeDOI)
|
||||
distributeFOSdois(
|
||||
spark,
|
||||
sourcePath,
|
||||
|
||||
outputPath);
|
||||
outputPath);
|
||||
else
|
||||
distributeFOSoaid(spark, sourcePath, outputPath);
|
||||
});
|
||||
}
|
||||
|
||||
private static void distributeFOSoaid(SparkSession spark, String sourcePath, String outputPath) {
|
||||
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
|
||||
|
||||
fosDataset
|
||||
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getOaid().toLowerCase(), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, FOSDataModel, Result>) (k, it) -> {
|
||||
return getResult(ModelSupport.getIdPrefix(Result.class) + "|" + k, it);
|
||||
}, Encoders.bean(Result.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + "/fos");
|
||||
}
|
||||
|
||||
@NotNull
|
||||
private static Result getResult(String k, Iterator<FOSDataModel> it) {
|
||||
Result r = new Result();
|
||||
FOSDataModel first = it.next();
|
||||
r.setId(k);
|
||||
|
||||
HashSet<String> level1 = new HashSet<>();
|
||||
HashSet<String> level2 = new HashSet<>();
|
||||
HashSet<String> level3 = new HashSet<>();
|
||||
HashSet<String> level4 = new HashSet<>();
|
||||
addLevels(level1, level2, level3, level4, first);
|
||||
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
|
||||
List<Subject> sbjs = new ArrayList<>();
|
||||
level1
|
||||
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
|
||||
level2
|
||||
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
|
||||
level3
|
||||
.forEach(
|
||||
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
|
||||
level4
|
||||
.forEach(
|
||||
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
|
||||
r.setSubject(sbjs);
|
||||
r
|
||||
.setDataInfo(
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, true,
|
||||
false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
ModelConstants.PROVENANCE_ENRICH,
|
||||
null,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
null));
|
||||
return r;
|
||||
}
|
||||
|
||||
private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) {
|
||||
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
|
||||
|
||||
fosDataset
|
||||
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getDoi().toLowerCase(), Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, FOSDataModel, Result>) (k, it) -> {
|
||||
Result r = new Result();
|
||||
FOSDataModel first = it.next();
|
||||
r.setId(DHPUtils.generateUnresolvedIdentifier(k, DOI));
|
||||
|
||||
HashSet<String> level1 = new HashSet<>();
|
||||
HashSet<String> level2 = new HashSet<>();
|
||||
HashSet<String> level3 = new HashSet<>();
|
||||
HashSet<String> level4 = new HashSet<>();
|
||||
addLevels(level1, level2, level3, level4, first);
|
||||
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
|
||||
List<Subject> sbjs = new ArrayList<>();
|
||||
level1
|
||||
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
|
||||
level2
|
||||
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
|
||||
level3
|
||||
.forEach(
|
||||
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
|
||||
level4
|
||||
.forEach(
|
||||
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
|
||||
r.setSubject(sbjs);
|
||||
r
|
||||
.setDataInfo(
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, true,
|
||||
false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
ModelConstants.PROVENANCE_ENRICH,
|
||||
null,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
null));
|
||||
return r;
|
||||
}, Encoders.bean(Result.class))
|
||||
.mapGroups(
|
||||
(MapGroupsFunction<String, FOSDataModel, Result>) (k,
|
||||
it) -> getResult(DHPUtils.generateUnresolvedIdentifier(k, DOI), it),
|
||||
Encoders.bean(Result.class))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
|
|
|
@ -0,0 +1,92 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.fosnodoi;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.*;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class CreateActionSetSparkJob implements Serializable {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
CreateActionSetSparkJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json"))));
|
||||
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("sourcePath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> createActionSet(spark, inputPath, outputPath));
|
||||
|
||||
}
|
||||
|
||||
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
|
||||
spark
|
||||
.read()
|
||||
.textFile(inputPath)
|
||||
.map(
|
||||
(MapFunction<String, Result>) value -> OBJECT_MAPPER.readValue(value, Result.class),
|
||||
Encoders.bean(Result.class))
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p))
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||
.saveAsHadoopFile(
|
||||
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
}
|
||||
|
||||
}
|
|
@ -22,12 +22,14 @@ import org.apache.spark.sql.SparkSession;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.*;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
|
@ -37,16 +39,12 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
|
||||
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
|
||||
|
||||
// DOI-to-DOI citations
|
||||
public static final String COCI = "COCI";
|
||||
|
||||
// PMID-to-PMID citations
|
||||
public static final String POCI = "POCI";
|
||||
|
||||
private static final String DOI_PREFIX = "50|doi_________::";
|
||||
|
||||
private static final String PMID_PREFIX = "50|pmid________::";
|
||||
private static final String ARXIV_PREFIX = "50|arXiv_______::";
|
||||
|
||||
private static final String PMCID_PREFIX = "50|pmcid_______::";
|
||||
private static final String TRUST = "0.91";
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
|
@ -79,38 +77,30 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
final boolean shouldDuplicateRels = Optional
|
||||
.ofNullable(parser.get("shouldDuplicateRels"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.FALSE);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> extractContent(spark, inputPath, outputPath, shouldDuplicateRels));
|
||||
spark -> extractContent(spark, inputPath, outputPath));
|
||||
|
||||
}
|
||||
|
||||
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
|
||||
boolean shouldDuplicateRels) {
|
||||
private static void extractContent(SparkSession spark, String inputPath, String outputPath) {
|
||||
|
||||
getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, COCI)
|
||||
.union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI))
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
getTextTextJavaPairRDD(spark, inputPath)
|
||||
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);// , GzipCodec.class);
|
||||
}
|
||||
|
||||
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath,
|
||||
boolean shouldDuplicateRels, String prefix) {
|
||||
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath) {
|
||||
return spark
|
||||
.read()
|
||||
.textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*")
|
||||
.textFile(inputPath)
|
||||
.map(
|
||||
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
|
||||
Encoders.bean(COCI.class))
|
||||
.flatMap(
|
||||
(FlatMapFunction<COCI, Relation>) value -> createRelation(
|
||||
value, shouldDuplicateRels, prefix)
|
||||
value)
|
||||
.iterator(),
|
||||
Encoders.bean(Relation.class))
|
||||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||
|
@ -121,34 +111,68 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
|
||||
}
|
||||
|
||||
private static List<Relation> createRelation(COCI value, boolean duplicate, String p) {
|
||||
private static List<Relation> createRelation(COCI value) throws JsonProcessingException {
|
||||
|
||||
List<Relation> relationList = new ArrayList<>();
|
||||
String prefix;
|
||||
|
||||
String citing;
|
||||
String cited;
|
||||
|
||||
switch (p) {
|
||||
case COCI:
|
||||
prefix = DOI_PREFIX;
|
||||
citing = prefix
|
||||
switch (value.getCiting_pid()) {
|
||||
case "doi":
|
||||
citing = DOI_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
break;
|
||||
case "pmid":
|
||||
citing = PMID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
|
||||
break;
|
||||
case "arxiv":
|
||||
citing = ARXIV_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), value.getCiting()));
|
||||
break;
|
||||
case "pmcid":
|
||||
citing = PMCID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), value.getCiting()));
|
||||
break;
|
||||
case "isbn":
|
||||
case "issn":
|
||||
return relationList;
|
||||
|
||||
default:
|
||||
throw new IllegalStateException("Invalid prefix: " + new ObjectMapper().writeValueAsString(value));
|
||||
}
|
||||
|
||||
switch (value.getCited_pid()) {
|
||||
case "doi":
|
||||
cited = DOI_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCited()));
|
||||
break;
|
||||
case POCI:
|
||||
prefix = PMID_PREFIX;
|
||||
citing = prefix
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
|
||||
cited = prefix
|
||||
case "pmid":
|
||||
cited = PMID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCited()));
|
||||
break;
|
||||
case "arxiv":
|
||||
cited = ARXIV_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), value.getCited()));
|
||||
break;
|
||||
case "pmcid":
|
||||
cited = PMCID_PREFIX
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), value.getCited()));
|
||||
break;
|
||||
case "isbn":
|
||||
case "issn":
|
||||
return relationList;
|
||||
default:
|
||||
throw new IllegalStateException("Invalid prefix: " + p);
|
||||
throw new IllegalStateException("Invalid prefix: " + new ObjectMapper().writeValueAsString(value));
|
||||
}
|
||||
|
||||
if (!citing.equals(cited)) {
|
||||
|
@ -157,15 +181,6 @@ public class CreateActionSetSparkJob implements Serializable {
|
|||
getRelation(
|
||||
citing,
|
||||
cited, ModelConstants.CITES));
|
||||
|
||||
if (duplicate && value.getCiting().endsWith(".refs")) {
|
||||
citing = prefix + IdentifierFactory
|
||||
.md5(
|
||||
CleaningFunctions
|
||||
.normalizePidValue(
|
||||
"doi", value.getCiting().substring(0, value.getCiting().indexOf(".refs"))));
|
||||
relationList.add(getRelation(citing, cited, ModelConstants.CITES));
|
||||
}
|
||||
}
|
||||
|
||||
return relationList;
|
||||
|
|
|
@ -12,10 +12,7 @@ import java.util.zip.ZipInputStream;
|
|||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -37,17 +34,17 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
|
||||
parser.parseArgument(args);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
// final String[] inputFile = parser.get("inputFile").split(";");
|
||||
// log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String hdfsNameNode = parser.get("hdfsNameNode");
|
||||
log.info("hdfsNameNode {}", hdfsNameNode);
|
||||
|
||||
final String prefix = parser.get("prefix");
|
||||
log.info("prefix {}", prefix);
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
@ -56,41 +53,42 @@ public class GetOpenCitationsRefs implements Serializable {
|
|||
|
||||
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
|
||||
|
||||
for (String file : inputFile) {
|
||||
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix);
|
||||
}
|
||||
ocr.doExtract(inputPath, outputPath, fileSystem);
|
||||
|
||||
}
|
||||
|
||||
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix)
|
||||
private void doExtract(String inputPath, String outputPath, FileSystem fileSystem)
|
||||
throws IOException {
|
||||
|
||||
final Path path = new Path(inputFile);
|
||||
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
||||
.listFiles(
|
||||
new Path(inputPath), true);
|
||||
while (fileStatusListIterator.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
||||
// do stuff with the file like ...
|
||||
FSDataInputStream oc_zip = fileSystem.open(fileStatus.getPath());
|
||||
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
FSDataInputStream oc_zip = fileSystem.open(path);
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
fileName = fileName.substring(0, fileName.lastIndexOf("."));
|
||||
// count++;
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(outputPath + "/" + fileName + ".gz"));
|
||||
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
|
||||
|
||||
// int count = 1;
|
||||
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
// fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
|
||||
fileName = fileName.substring(0, fileName.lastIndexOf("."));
|
||||
// count++;
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz"));
|
||||
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
|
||||
|
||||
IOUtils.copy(zis, gzipOs);
|
||||
IOUtils.copy(zis, gzipOs);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,171 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.opencitations;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.ZipEntry;
|
||||
import java.util.zip.ZipInputStream;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.ForeachFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 29/02/24
|
||||
*/
|
||||
public class MapOCIdsInPids implements Serializable {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
private static final String DELIMITER = ",";
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
MapOCIdsInPids.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/remap_parameters.json"))));
|
||||
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
final String nameNode = parser.get("nameNode");
|
||||
log.info("nameNode {}", nameNode);
|
||||
|
||||
unzipCorrespondenceFile(inputPath, nameNode);
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> mapIdentifiers(spark, inputPath, outputPath));
|
||||
|
||||
}
|
||||
|
||||
private static void unzipCorrespondenceFile(String inputPath, String hdfsNameNode) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
||||
final Path path = new Path(inputPath + "/correspondence/omid.zip");
|
||||
FileSystem fileSystem = FileSystem.get(conf);
|
||||
|
||||
FSDataInputStream project_zip = fileSystem.open(path);
|
||||
|
||||
try (ZipInputStream zis = new ZipInputStream(project_zip)) {
|
||||
ZipEntry entry = null;
|
||||
while ((entry = zis.getNextEntry()) != null) {
|
||||
|
||||
if (!entry.isDirectory()) {
|
||||
String fileName = entry.getName();
|
||||
byte buffer[] = new byte[1024];
|
||||
int count;
|
||||
|
||||
try (
|
||||
FSDataOutputStream out = fileSystem
|
||||
.create(new Path(inputPath + "/correspondence/omid.csv"))) {
|
||||
|
||||
while ((count = zis.read(buffer, 0, buffer.length)) != -1)
|
||||
out.write(buffer, 0, count);
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static void mapIdentifiers(SparkSession spark, String inputPath, String outputPath) {
|
||||
Dataset<COCI> coci = spark
|
||||
.read()
|
||||
.textFile(inputPath + "/JSON")
|
||||
.map(
|
||||
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
|
||||
Encoders.bean(COCI.class));
|
||||
|
||||
Dataset<Tuple2<String, String>> correspondenceData = spark
|
||||
.read()
|
||||
.format("csv")
|
||||
.option("sep", DELIMITER)
|
||||
.option("inferSchema", "true")
|
||||
.option("header", "true")
|
||||
.option("quotes", "\"")
|
||||
.load(inputPath + "/correspondence/omid.csv")
|
||||
.repartition(5000)
|
||||
.flatMap((FlatMapFunction<Row, Tuple2<String, String>>) r -> {
|
||||
String ocIdentifier = r.getAs("omid");
|
||||
String[] correspondentIdentifiers = ((String) r.getAs("id")).split(" ");
|
||||
return Arrays
|
||||
.stream(correspondentIdentifiers)
|
||||
.map(ci -> new Tuple2<String, String>(ocIdentifier, ci))
|
||||
.collect(Collectors.toList())
|
||||
.iterator();
|
||||
}, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||
|
||||
Dataset<COCI> mappedCitingDataset = coci
|
||||
.joinWith(correspondenceData, coci.col("citing").equalTo(correspondenceData.col("_1")))
|
||||
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
|
||||
String correspondent = t2._2()._2();
|
||||
t2._1().setCiting_pid(correspondent.substring(0, correspondent.indexOf(":")));
|
||||
t2._1().setCiting(correspondent.substring(correspondent.indexOf(":") + 1));
|
||||
return t2._1();
|
||||
}, Encoders.bean(COCI.class));
|
||||
|
||||
mappedCitingDataset
|
||||
.joinWith(correspondenceData, mappedCitingDataset.col("cited").equalTo(correspondenceData.col("_1")))
|
||||
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
|
||||
String correspondent = t2._2()._2();
|
||||
t2._1().setCited_pid(correspondent.substring(0, correspondent.indexOf(":")));
|
||||
t2._1().setCited(correspondent.substring(correspondent.indexOf(":") + 1));
|
||||
return t2._1();
|
||||
}, Encoders.bean(COCI.class))
|
||||
.write()
|
||||
.mode(SaveMode.Append)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -12,11 +12,9 @@ import java.util.Optional;
|
|||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.*;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -42,19 +40,21 @@ public class ReadCOCI implements Serializable {
|
|||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final String[] inputFile = parser.get("inputFile").split(";");
|
||||
log.info("inputFile {}", Arrays.asList(inputFile));
|
||||
final String hdfsNameNode = parser.get("hdfsNameNode");
|
||||
log.info("hdfsNameNode {}", hdfsNameNode);
|
||||
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String workingPath = parser.get("workingPath");
|
||||
final String workingPath = parser.get("inputPath");
|
||||
log.info("workingPath {}", workingPath);
|
||||
|
||||
final String format = parser.get("format");
|
||||
log.info("format {}", format);
|
||||
|
||||
SparkConf sconf = new SparkConf();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.set("fs.defaultFS", hdfsNameNode);
|
||||
|
||||
FileSystem fileSystem = FileSystem.get(conf);
|
||||
final String delimiter = Optional
|
||||
.ofNullable(parser.get("delimiter"))
|
||||
.orElse(DEFAULT_DELIMITER);
|
||||
|
@ -66,20 +66,21 @@ public class ReadCOCI implements Serializable {
|
|||
doRead(
|
||||
spark,
|
||||
workingPath,
|
||||
inputFile,
|
||||
fileSystem,
|
||||
outputPath,
|
||||
delimiter,
|
||||
format);
|
||||
delimiter);
|
||||
});
|
||||
}
|
||||
|
||||
private static void doRead(SparkSession spark, String workingPath, String[] inputFiles,
|
||||
private static void doRead(SparkSession spark, String workingPath, FileSystem fileSystem,
|
||||
String outputPath,
|
||||
String delimiter, String format) {
|
||||
|
||||
for (String inputFile : inputFiles) {
|
||||
String pString = workingPath + "/" + inputFile + ".gz";
|
||||
|
||||
String delimiter) throws IOException {
|
||||
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
|
||||
.listFiles(
|
||||
new Path(workingPath), true);
|
||||
while (fileStatusListIterator.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fileStatusListIterator.next();
|
||||
log.info("extracting file {}", fileStatus.getPath().toString());
|
||||
Dataset<Row> cociData = spark
|
||||
.read()
|
||||
.format("csv")
|
||||
|
@ -87,26 +88,26 @@ public class ReadCOCI implements Serializable {
|
|||
.option("inferSchema", "true")
|
||||
.option("header", "true")
|
||||
.option("quotes", "\"")
|
||||
.load(pString)
|
||||
.load(fileStatus.getPath().toString())
|
||||
.repartition(100);
|
||||
|
||||
cociData.map((MapFunction<Row, COCI>) row -> {
|
||||
|
||||
COCI coci = new COCI();
|
||||
if (format.equals("COCI")) {
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
} else {
|
||||
coci.setCiting(String.valueOf(row.getInt(1)));
|
||||
coci.setCited(String.valueOf(row.getInt(2)));
|
||||
}
|
||||
|
||||
coci.setCiting(row.getString(1));
|
||||
coci.setCited(row.getString(2));
|
||||
|
||||
coci.setOci(row.getString(0));
|
||||
|
||||
return coci;
|
||||
}, Encoders.bean(COCI.class))
|
||||
.filter((FilterFunction<COCI>) c -> c != null)
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.mode(SaveMode.Append)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath + inputFile);
|
||||
.json(outputPath);
|
||||
fileSystem.rename(fileStatus.getPath(), new Path("/tmp/miriam/OC/DONE"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -9,8 +9,10 @@ public class COCI implements Serializable {
|
|||
private String oci;
|
||||
|
||||
private String citing;
|
||||
private String citing_pid;
|
||||
|
||||
private String cited;
|
||||
private String cited_pid;
|
||||
|
||||
public String getOci() {
|
||||
return oci;
|
||||
|
@ -25,6 +27,8 @@ public class COCI implements Serializable {
|
|||
}
|
||||
|
||||
public void setCiting(String citing) {
|
||||
if (citing != null && citing.startsWith("omid:"))
|
||||
citing = citing.substring(5);
|
||||
this.citing = citing;
|
||||
}
|
||||
|
||||
|
@ -33,7 +37,24 @@ public class COCI implements Serializable {
|
|||
}
|
||||
|
||||
public void setCited(String cited) {
|
||||
if (cited != null && cited.startsWith("omid:"))
|
||||
cited = cited.substring(5);
|
||||
this.cited = cited;
|
||||
}
|
||||
|
||||
public String getCiting_pid() {
|
||||
return citing_pid;
|
||||
}
|
||||
|
||||
public void setCiting_pid(String citing_pid) {
|
||||
this.citing_pid = citing_pid;
|
||||
}
|
||||
|
||||
public String getCited_pid() {
|
||||
return cited_pid;
|
||||
}
|
||||
|
||||
public void setCited_pid(String cited_pid) {
|
||||
this.cited_pid = cited_pid;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,12 +1,20 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.project;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.H2020Classification;
|
||||
import eu.dnetlib.dhp.schema.oaf.H2020Programme;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Project;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
|
@ -18,24 +26,14 @@ import org.apache.spark.sql.Encoders;
|
|||
import org.apache.spark.sql.SparkSession;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.EXCELTopic;
|
||||
import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.H2020Classification;
|
||||
import eu.dnetlib.dhp.schema.oaf.H2020Programme;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Project;
|
||||
import eu.dnetlib.dhp.utils.DHPUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
/**
|
||||
* Class that makes the ActionSet. To prepare the AS two joins are needed
|
||||
*
|
||||
|
@ -160,9 +158,11 @@ public class SparkAtomicActionJob {
|
|||
(MapFunction<Project, String>) OafEntity::getId,
|
||||
Encoders.STRING())
|
||||
.mapGroups((MapGroupsFunction<String, Project, Project>) (s, it) -> {
|
||||
Project first = it.next();
|
||||
it.forEachRemaining(first::mergeFrom);
|
||||
return first;
|
||||
Project merge = it.next();
|
||||
while (it.hasNext()) {
|
||||
merge = MergeUtils.mergeProject(merge, it.next());
|
||||
}
|
||||
return merge;
|
||||
}, Encoders.bean(Project.class))
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(Project.class, p))
|
||||
|
|
|
@ -0,0 +1,195 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.transformativeagreement;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.GzipCodec;
|
||||
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.transformativeagreement.model.TransformativeAgreementModel;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Country;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.*;
|
||||
import scala.Tuple2;
|
||||
|
||||
public class CreateActionSetSparkJob implements Serializable {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static final String IREL_PROJECT = "40|100018998___::1e5e62235d094afd01cd56e65112fc63";
|
||||
private static final String TRANSFORMATIVE_AGREEMENT = "openapc::transformativeagreement";
|
||||
|
||||
public static void main(final String[] args) throws IOException, ParseException {
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
CreateActionSetSparkJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/actionmanager/transformativeagreement/as_parameters.json"))));
|
||||
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = Optional
|
||||
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
final String inputPath = parser.get("inputPath");
|
||||
log.info("inputPath {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath {}", outputPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
runWithSparkSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> createActionSet(spark, inputPath, outputPath));
|
||||
|
||||
}
|
||||
|
||||
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
|
||||
JavaRDD<AtomicAction> relations = spark
|
||||
.read()
|
||||
.textFile(inputPath)
|
||||
.map(
|
||||
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
||||
.readValue(value, TransformativeAgreementModel.class),
|
||||
Encoders.bean(TransformativeAgreementModel.class))
|
||||
.flatMap(
|
||||
(FlatMapFunction<TransformativeAgreementModel, Relation>) value -> createRelation(
|
||||
value)
|
||||
.iterator(),
|
||||
Encoders.bean(Relation.class))
|
||||
.filter((FilterFunction<Relation>) Objects::nonNull)
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p));
|
||||
//TODO relations in stand-by waiting to know if we need to create them or not In case we need just make a union before saving the sequence file
|
||||
spark
|
||||
.read()
|
||||
.textFile(inputPath)
|
||||
.map(
|
||||
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
|
||||
.readValue(value, TransformativeAgreementModel.class),
|
||||
Encoders.bean(TransformativeAgreementModel.class))
|
||||
.map(
|
||||
(MapFunction<TransformativeAgreementModel, Result>) value -> createResult(
|
||||
value),
|
||||
Encoders.bean(Result.class))
|
||||
.filter((FilterFunction<Result>) r -> r != null)
|
||||
.toJavaRDD()
|
||||
.map(p -> new AtomicAction(p.getClass(), p))
|
||||
.mapToPair(
|
||||
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
|
||||
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
|
||||
.saveAsHadoopFile(
|
||||
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
|
||||
|
||||
}
|
||||
|
||||
private static Result createResult(TransformativeAgreementModel value) {
|
||||
Result r = new Result();
|
||||
r
|
||||
.setId(
|
||||
"50|doi_________::"
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getDoi())));
|
||||
r.setTransformativeAgreement(value.getAgreement());
|
||||
Country country = new Country();
|
||||
country.setClassid(value.getCountry());
|
||||
country.setClassname(value.getCountry());
|
||||
country
|
||||
.setDataInfo(
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, ModelConstants.SYSIMPORT_ACTIONSET, false, false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
"openapc::transformativeagreement",
|
||||
"Harvested from Trnasformative Agreement file from OpenAPC",
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
"0.9"));
|
||||
country.setSchemeid(ModelConstants.DNET_COUNTRY_TYPE);
|
||||
country.setSchemename(ModelConstants.DNET_COUNTRY_TYPE);
|
||||
r.setCountry(Arrays.asList(country));
|
||||
return r;
|
||||
}
|
||||
|
||||
private static List<Relation> createRelation(TransformativeAgreementModel value) {
|
||||
|
||||
List<Relation> relationList = new ArrayList<>();
|
||||
|
||||
if (value.getAgreement().startsWith("IReL")) {
|
||||
String paper;
|
||||
|
||||
paper = "50|doi_________::"
|
||||
+ IdentifierFactory
|
||||
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getDoi()));
|
||||
|
||||
relationList
|
||||
.add(
|
||||
getRelation(
|
||||
paper,
|
||||
IREL_PROJECT, ModelConstants.IS_PRODUCED_BY));
|
||||
|
||||
relationList.add(getRelation(IREL_PROJECT, paper, ModelConstants.PRODUCES));
|
||||
}
|
||||
return relationList;
|
||||
}
|
||||
|
||||
public static Relation getRelation(
|
||||
String source,
|
||||
String target,
|
||||
String relClass) {
|
||||
|
||||
return OafMapperUtils
|
||||
.getRelation(
|
||||
source,
|
||||
target,
|
||||
ModelConstants.RESULT_PROJECT,
|
||||
ModelConstants.OUTCOME,
|
||||
relClass,
|
||||
Arrays
|
||||
.asList(
|
||||
OafMapperUtils.keyValue(ModelConstants.OPEN_APC_ID, ModelConstants.OPEN_APC_NAME)),
|
||||
OafMapperUtils
|
||||
.dataInfo(
|
||||
false, null, false, false,
|
||||
OafMapperUtils
|
||||
.qualifier(
|
||||
TRANSFORMATIVE_AGREEMENT, "Transformative Agreement",
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
|
||||
"0.9"),
|
||||
null);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.transformativeagreement.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 18/12/23
|
||||
*/
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
|
||||
public class TransformativeAgreementModel implements Serializable {
|
||||
private String institution;
|
||||
private String doi;
|
||||
private String agreement;
|
||||
private String country;
|
||||
|
||||
public String getCountry() {
|
||||
return country;
|
||||
}
|
||||
|
||||
public void setCountry(String country) {
|
||||
this.country = country;
|
||||
}
|
||||
|
||||
public String getInstitution() {
|
||||
return institution;
|
||||
}
|
||||
|
||||
public void setInstitution(String institution) {
|
||||
this.institution = institution;
|
||||
}
|
||||
|
||||
public String getDoi() {
|
||||
return doi;
|
||||
}
|
||||
|
||||
public void setDoi(String doi) {
|
||||
this.doi = doi;
|
||||
}
|
||||
|
||||
public String getAgreement() {
|
||||
return agreement;
|
||||
}
|
||||
|
||||
public void setAgreement(String agreement) {
|
||||
this.agreement = agreement;
|
||||
}
|
||||
}
|
|
@ -45,15 +45,22 @@ public class BaseCollectorPlugin implements CollectorPlugin {
|
|||
|
||||
@Override
|
||||
public Stream<String> collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException {
|
||||
// get path to file
|
||||
// the path of the dump file on HDFS
|
||||
// http://oai.base-search.net/initial_load/base_oaipmh_dump-current.tar
|
||||
// it could be downloaded from iis-cdh5-test-gw.ocean.icm.edu.pl and then copied on HDFS
|
||||
final Path filePath = Optional
|
||||
.ofNullable(api.getBaseUrl())
|
||||
.map(Path::new)
|
||||
.orElseThrow(() -> new CollectorException("missing baseUrl"));
|
||||
|
||||
// get the parameters for the connection to the OpenAIRE database.
|
||||
// the database is used to obtain the list of the datasources that the plugin will collect
|
||||
final String dbUrl = api.getParams().get("dbUrl");
|
||||
final String dbUser = api.getParams().get("dbUser");
|
||||
final String dbPassword = api.getParams().get("dbPassword");
|
||||
|
||||
// the types(comma separated, empty value for all) that the plugin will collect,
|
||||
// the types should be expressed in the format of the normalized types of BASE (for example 1,121,...)
|
||||
final String acceptedNormTypesString = api.getParams().get("acceptedNormTypes");
|
||||
|
||||
log.info("baseUrl: {}", filePath);
|
||||
|
|
|
@ -48,7 +48,7 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
|
|||
@Override
|
||||
public MetadataRecord call(MetadataRecord value) {
|
||||
aggregationCounter.getTotalItems().add(1);
|
||||
try {
|
||||
|
||||
Processor processor = new Processor(false);
|
||||
|
||||
processor.registerExtensionFunction(cleanFunction);
|
||||
|
@ -60,11 +60,18 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
|
|||
comp.setParameter(datasourceIDParam, new XdmAtomicValue(value.getProvenance().getDatasourceId()));
|
||||
QName datasourceNameParam = new QName(DATASOURCE_NAME_PARAM);
|
||||
comp.setParameter(datasourceNameParam, new XdmAtomicValue(value.getProvenance().getDatasourceName()));
|
||||
XsltExecutable xslt = comp
|
||||
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
|
||||
XdmNode source = processor
|
||||
.newDocumentBuilder()
|
||||
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
|
||||
XsltExecutable xslt;
|
||||
XdmNode source;
|
||||
try {
|
||||
xslt = comp
|
||||
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
|
||||
source = processor
|
||||
.newDocumentBuilder()
|
||||
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
|
||||
} catch (Throwable e) {
|
||||
throw new RuntimeException("Error on parsing xslt", e);
|
||||
}
|
||||
try {
|
||||
XsltTransformer trans = xslt.load();
|
||||
trans.setInitialContextNode(source);
|
||||
final StringWriter output = new StringWriter();
|
||||
|
|
|
@ -17,6 +17,12 @@
|
|||
"paramDescription": "the path to get the input data from Pubmed",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "oip",
|
||||
"paramLongName": "openapcInputPath",
|
||||
"paramDescription": "the path to get the input data from OpenAPC",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "o",
|
||||
"paramLongName": "outputPath",
|
||||
|
|
|
@ -31,6 +31,7 @@ spark2SqlQueryExecutionListeners=com.cloudera.spark.lineage.NavigatorQueryListen
|
|||
# The following is needed as a property of a workflow
|
||||
oozie.wf.application.path=${oozieTopWfApplicationPath}
|
||||
|
||||
crossrefInputPath=/data/bip-affiliations/data.json
|
||||
crossrefInputPath=/data/bip-affiliations/crossref-data.json
|
||||
pubmedInputPath=/data/bip-affiliations/pubmed-data.json
|
||||
openapcInputPath=/data/bip-affiliations/openapc-data.json
|
||||
outputPath=/tmp/crossref-affiliations-output-v5
|
||||
|
|
|
@ -9,6 +9,10 @@
|
|||
<name>pubmedInputPath</name>
|
||||
<description>the path where to find the inferred affiliation relations from Pubmed</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>openapcInputPath</name>
|
||||
<description>the path where to find the inferred affiliation relations from OpenAPC</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>outputPath</name>
|
||||
<description>the path where to store the actionset</description>
|
||||
|
@ -102,6 +106,7 @@
|
|||
</spark-opts>
|
||||
<arg>--crossrefInputPath</arg><arg>${crossrefInputPath}</arg>
|
||||
<arg>--pubmedInputPath</arg><arg>${pubmedInputPath}</arg>
|
||||
<arg>--openapcInputPath</arg><arg>${openapcInputPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -16,5 +16,10 @@
|
|||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the path of the new ActionSet",
|
||||
"paramRequired": true
|
||||
}
|
||||
}, {
|
||||
"paramName": "fd",
|
||||
"paramLongName": "distributeDoi",
|
||||
"paramDescription": "the path of the new ActionSet",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"paramName": "sp",
|
||||
"paramLongName": "sourcePath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,30 @@
|
|||
<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>hiveMetastoreUris</name>
|
||||
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hiveJdbcUrl</name>
|
||||
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hiveDbName</name>
|
||||
<value>openaire</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,153 @@
|
|||
|
||||
<workflow-app name="FOS no doi" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>fosPath</name>
|
||||
<description>the input path of the resources to be extended</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>outputPath</name>
|
||||
<description>the path where to store the actionset</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>oozieActionShareLibForSpark2</name>
|
||||
<description>oozie action sharelib for spark 2.*</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||
<description>spark 2.* extra listeners classname</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||
<description>spark 2.* sql query execution listeners classname</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<description>spark 2.* yarn history server address</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<description>spark 2.* event log dir location</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<global>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapreduce.job.queuename</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapred.job.queue.name</name>
|
||||
<value>${oozieLauncherQueueName}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>${oozieActionShareLibForSpark2}</value>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
</global>
|
||||
<start to="getFOS"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
|
||||
|
||||
<action name="getFOS">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Gets Data from FOS csv file</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${fosPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingDir}/input/fos</arg>
|
||||
<arg>--delimiter</arg><arg>${delimiter}</arg>
|
||||
</spark>
|
||||
<ok to="prepareFos"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="prepareFos">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Produces the results from FOS</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/input/fos</arg>
|
||||
<arg>--outputPath</arg><arg>${workingDir}/prepared</arg>
|
||||
<arg>--distributeDoi</arg><arg>false</arg>
|
||||
</spark>
|
||||
<ok to="produceActionSet"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
|
||||
<action name="produceActionSet">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Save the action set grouping results with the same id</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.fosnodoi.CreateActionSetSparkJob</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${workingDir}/prepared/fos</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -1,13 +1,13 @@
|
|||
[
|
||||
{
|
||||
"paramName": "if",
|
||||
"paramLongName": "inputFile",
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "wp",
|
||||
"paramLongName": "workingPath",
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
|
@ -16,11 +16,5 @@
|
|||
"paramLongName": "hdfsNameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "p",
|
||||
"paramLongName": "prefix",
|
||||
"paramDescription": "COCI or POCI",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
[
|
||||
{
|
||||
"paramName": "wp",
|
||||
"paramLongName": "workingPath",
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
|
@ -24,15 +24,9 @@
|
|||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "if",
|
||||
"paramLongName": "inputFile",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}, {
|
||||
"paramName": "f",
|
||||
"paramLongName": "format",
|
||||
}, {
|
||||
"paramName": "nn",
|
||||
"paramLongName": "hdfsNameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
|
|
|
@ -27,7 +27,9 @@
|
|||
<case to="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
|
||||
<case to="extract">${wf:conf('resumeFrom') eq 'ExtractContent'}</case>
|
||||
<case to="read">${wf:conf('resumeFrom') eq 'ReadContent'}</case>
|
||||
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
|
||||
<case to="remap">${wf:conf('resumeFrom') eq 'MapContent'}</case>
|
||||
<case to="create_actionset">${wf:conf('resumeFrom') eq 'CreateAS'}</case>
|
||||
<default to="deleteoutputpath"/> <!-- first action to be done when downloadDump is to be performed -->
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
|
@ -35,6 +37,15 @@
|
|||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="deleteoutputpath">
|
||||
<fs>
|
||||
<delete path='${inputPath}'/>
|
||||
<mkdir path='${inputPath}'/>
|
||||
</fs>
|
||||
<ok to="download"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="download">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
|
@ -47,7 +58,28 @@
|
|||
</configuration>
|
||||
<exec>download.sh</exec>
|
||||
<argument>${filelist}</argument>
|
||||
<argument>${workingPath}/${prefix}/Original</argument>
|
||||
<argument>${inputPath}/Original</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
</shell>
|
||||
<ok to="download_correspondence"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<!-- downloads the correspondence from the omid and the pid (doi, pmid etc)-->
|
||||
<action name="download_correspondence">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapred.job.queue.name</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<exec>download_corr.sh</exec>
|
||||
<argument>${filecorrespondence}</argument>
|
||||
<argument>${inputPath}/correspondence</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
|
@ -60,9 +92,19 @@
|
|||
<java>
|
||||
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFile}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}</arg>
|
||||
<arg>--prefix</arg><arg>${prefix}</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/Original</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/Extracted</arg>
|
||||
</java>
|
||||
<ok to="read"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="extract_correspondence">
|
||||
<java>
|
||||
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/correspondence</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/correspondence_extracted</arg>
|
||||
</java>
|
||||
<ok to="read"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -85,11 +127,35 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${workingPath}/${prefix}/${prefix}</arg>
|
||||
<arg>--outputPath</arg><arg>${workingPath}/${prefix}/${prefix}_JSON/</arg>
|
||||
<arg>--inputPath</arg><arg>${inputPath}/Extracted</arg>
|
||||
<arg>--outputPath</arg><arg>${inputPath}/JSON</arg>
|
||||
<arg>--delimiter</arg><arg>${delimiter}</arg>
|
||||
<arg>--inputFile</arg><arg>${inputFileCoci}</arg>
|
||||
<arg>--format</arg><arg>${prefix}</arg>
|
||||
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
|
||||
</spark>
|
||||
<ok to="remap"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="remap">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Produces the AS for OC</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.opencitations.MapOCIdsInPids</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${inputPath}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPathExtraction}</arg>
|
||||
<arg>--nameNode</arg><arg>${nameNode}</arg>
|
||||
</spark>
|
||||
<ok to="create_actionset"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -112,7 +178,7 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--inputPath</arg><arg>${outputPathExtraction}</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
[
|
||||
{
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
},{
|
||||
"paramName": "nn",
|
||||
"paramLongName": "nameNode",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"paramName": "ip",
|
||||
"paramLongName": "inputPath",
|
||||
"paramDescription": "the zipped opencitations file",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the working path",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
}
|
||||
]
|
|
@ -0,0 +1,30 @@
|
|||
[
|
||||
|
||||
{
|
||||
"paramName": "issm",
|
||||
"paramLongName": "isSparkSessionManaged",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "d",
|
||||
"paramLongName": "delimiter",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": false
|
||||
},
|
||||
{
|
||||
"paramName": "op",
|
||||
"paramLongName": "outputPath",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "if",
|
||||
"paramLongName": "inputFile",
|
||||
"paramDescription": "the hdfs name node",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
||||
|
||||
|
||||
|
|
@ -0,0 +1,58 @@
|
|||
<configuration>
|
||||
<property>
|
||||
<name>jobTracker</name>
|
||||
<value>yarnRM</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>nameNode</name>
|
||||
<value>hdfs://nameservice1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.use.system.libpath</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hive_metastore_uris</name>
|
||||
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorNumber</name>
|
||||
<value>4</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2EventLogDir</name>
|
||||
<value>/user/spark/spark2ApplicationHistory</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<value>15G</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<value>6G</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<value>1</value>
|
||||
</property>
|
||||
</configuration>
|
|
@ -0,0 +1,2 @@
|
|||
#!/bin/bash
|
||||
curl -L $1 | hdfs dfs -put - $2
|
|
@ -0,0 +1,82 @@
|
|||
<workflow-app name="Transfomative Agreement Integration" xmlns="uri:oozie:workflow:0.5">
|
||||
|
||||
<global>
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapreduce.job.queuename</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapred.job.queue.name</name>
|
||||
<value>${oozieLauncherQueueName}</value>
|
||||
</property>
|
||||
<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="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
|
||||
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
|
||||
</switch>
|
||||
</decision>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="download">
|
||||
<shell xmlns="uri:oozie:shell-action:0.2">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapred.job.queue.name</name>
|
||||
<value>${queueName}</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<exec>download.sh</exec>
|
||||
<argument>${inputFile}</argument>
|
||||
<argument>${workingDir}/transformativeagreement/transformativeAgreement.json</argument>
|
||||
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
|
||||
<file>download.sh</file>
|
||||
<capture-output/>
|
||||
</shell>
|
||||
<ok to="create_actionset"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
|
||||
<action name="create_actionset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Produces the AS for the Transformative Agreement</name>
|
||||
<class>eu.dnetlib.dhp.actionmanager.transformativeagreement.CreateActionSetSparkJob</class>
|
||||
<jar>dhp-aggregation-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
|
||||
</spark-opts>
|
||||
<arg>--inputPath</arg><arg>${workingDir}/transformativeagreement/</arg>
|
||||
<arg>--outputPath</arg><arg>${outputPath}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -108,7 +108,7 @@ INSERT INTO dsm_apiparams(
|
|||
'api_________::openaire____::base_search::dump@@acceptedNormTypes',
|
||||
'api_________::openaire____::base_search::dump',
|
||||
'acceptedNormTypes',
|
||||
''
|
||||
'1,11,111,121,14,15,18,181,182,183,1A,6,7'
|
||||
);
|
||||
|
||||
COMMIT;
|
|
@ -2,6 +2,8 @@ select s.id as id
|
|||
from dsm_services s
|
||||
where collectedfrom = 'openaire____::opendoar'
|
||||
and jurisdiction = 'Institutional'
|
||||
and s.id not in (
|
||||
select service from dsm_api where coalesce(compatibility_override, compatibility) like '%openaire%' or last_collection_total > 0
|
||||
);
|
||||
and s.id in (
|
||||
select service from dsm_api where coalesce(compatibility_override, compatibility) = 'driver' or coalesce(compatibility_override, compatibility) = 'UNKNOWN'
|
||||
) and s.id not in (
|
||||
select service from dsm_api where coalesce(compatibility_override, compatibility) like '%openaire%'
|
||||
);
|
||||
|
|
|
@ -66,10 +66,6 @@ dc:language (I used base_dc:lang)
|
|||
base_dc:link (I used dc:identifier)
|
||||
-->
|
||||
|
||||
|
||||
<xsl:variable name="varBaseNormType" select="vocabulary:clean(//base_dc:typenorm, 'base:normalized_types')" />
|
||||
|
||||
|
||||
<metadata>
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:title" />
|
||||
|
@ -112,9 +108,13 @@ base_dc:link (I used dc:identifier)
|
|||
<xsl:with-param name="targetElement" select="'dc:format'" />
|
||||
</xsl:call-template>
|
||||
|
||||
<dc:type>
|
||||
<xsl:value-of select="$varBaseNormType" />
|
||||
</dc:type>
|
||||
|
||||
<xsl:for-each select="//base_dc:typenorm">
|
||||
<dc:type>
|
||||
<xsl:value-of select="vocabulary:clean(., 'base:normalized_types')" />
|
||||
</dc:type>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:type" />
|
||||
<xsl:with-param name="targetElement" select="'dc:type'" />
|
||||
|
@ -162,16 +162,150 @@ base_dc:link (I used dc:identifier)
|
|||
</xsl:if>
|
||||
</xsl:for-each>
|
||||
|
||||
<dr:CobjCategory>
|
||||
<xsl:variable name="varCobjCategory" select="vocabulary:clean($varBaseNormType, 'dnet:publication_resource')" />
|
||||
<xsl:variable name="varSuperType" select="vocabulary:clean($varCobjCategory, 'dnet:result_typologies')" />
|
||||
<xsl:attribute name="type" select="$varSuperType" />
|
||||
<xsl:value-of select="$varCobjCategory" />
|
||||
</dr:CobjCategory>
|
||||
<xsl:choose>
|
||||
<!-- I used an inline mapping because the field typenorm could be repeated and I have to specify a list of priority -->
|
||||
|
||||
<!-- Book part -->
|
||||
<xsl:when test="//base_dc:typenorm = '111'">
|
||||
<dr:CobjCategory type="publication">0013</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Book -->
|
||||
<xsl:when test="//base_dc:typenorm = '11'">
|
||||
<dr:CobjCategory type="publication">0002</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Article contribution -->
|
||||
<xsl:when test="//base_dc:typenorm = '121'">
|
||||
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
|
||||
<!-- Journal/Newspaper -->
|
||||
<xsl:when test="//base_dc:typenorm = '12'">
|
||||
<dr:CobjCategory type="publication">0043</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Report -->
|
||||
<xsl:when test="//base_dc:typenorm = '14'">
|
||||
<dr:CobjCategory type="publication">0017</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Review -->
|
||||
<xsl:when test="//base_dc:typenorm = '15'">
|
||||
<dr:CobjCategory type="publication">0015</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Lecture -->
|
||||
<xsl:when test="//base_dc:typenorm = '17'">
|
||||
<dr:CobjCategory type="publication">0010</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Bachelor's thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '181'">
|
||||
<dr:CobjCategory type="publication">0008</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Master's thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '182'">
|
||||
<dr:CobjCategory type="publication">0007</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Doctoral and postdoctoral thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '183'">
|
||||
<dr:CobjCategory type="publication">0006</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '18'">
|
||||
<dr:CobjCategory type="publication">0044</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Patent -->
|
||||
<xsl:when test="//base_dc:typenorm = '1A'">
|
||||
<dr:CobjCategory type="publication">0019</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Text -->
|
||||
<xsl:when test="//base_dc:typenorm = '1'">
|
||||
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Software -->
|
||||
<xsl:when test="//base_dc:typenorm = '6'">
|
||||
<dr:CobjCategory type="software">0029</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Dataset -->
|
||||
<xsl:when test="//base_dc:typenorm = '7'">
|
||||
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Still image -->
|
||||
<xsl:when test="//base_dc:typenorm = '51'">
|
||||
<dr:CobjCategory type="other">0025</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Moving image/Video -->
|
||||
<xsl:when test="//base_dc:typenorm = '52'">
|
||||
<dr:CobjCategory type="other">0024</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Image/Video -->
|
||||
<xsl:when test="//base_dc:typenorm = '5'">
|
||||
<dr:CobjCategory type="other">0033</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Audio -->
|
||||
<xsl:when test="//base_dc:typenorm = '4'">
|
||||
<dr:CobjCategory type="other">0030</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Musical notation -->
|
||||
<xsl:when test="//base_dc:typenorm = '2'">
|
||||
<dr:CobjCategory type="other">0020</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Map -->
|
||||
<xsl:when test="//base_dc:typenorm = '3'">
|
||||
<dr:CobjCategory type="other">0020</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Other non-article -->
|
||||
<xsl:when test="//base_dc:typenorm = '122'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Course material -->
|
||||
<xsl:when test="//base_dc:typenorm = '16'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Manuscript -->
|
||||
<xsl:when test="//base_dc:typenorm = '19'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Conference object -->
|
||||
<xsl:when test="//base_dc:typenorm = '13'">
|
||||
<dr:CobjCategory type="publication">0004</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Unknown -->
|
||||
<xsl:when test="//base_dc:typenorm = 'F'">
|
||||
<dr:CobjCategory type="other">0000</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
<xsl:otherwise>
|
||||
<dr:CobjCategory type="other">0000</dr:CobjCategory>
|
||||
</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
|
||||
|
||||
<oaf:accessrights>
|
||||
<xsl:choose>
|
||||
<xsl:when test="//base_dc:oa[.='0']">CLOSED</xsl:when>
|
||||
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
|
||||
<xsl:when test="//base_dc:oa[.='2']">UNKNOWN</xsl:when>
|
||||
<xsl:when test="//base_dc:rightsnorm">
|
||||
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
|
||||
</xsl:when>
|
||||
|
@ -208,7 +342,7 @@ base_dc:link (I used dc:identifier)
|
|||
|
||||
<oaf:identifier identifierType="oai-original">
|
||||
<xsl:value-of
|
||||
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
|
||||
select="//oai:header/oai:identifier" />
|
||||
</oaf:identifier>
|
||||
|
||||
<oaf:hostedBy>
|
||||
|
|
|
@ -51,10 +51,6 @@
|
|||
base_dc:link (I used dc:identifier)
|
||||
-->
|
||||
|
||||
|
||||
<xsl:variable name="varBaseNormType" select="vocabulary:clean(//base_dc:typenorm, 'base:normalized_types')" />
|
||||
|
||||
|
||||
<metadata>
|
||||
<datacite:resource>
|
||||
|
||||
|
@ -86,14 +82,17 @@
|
|||
|
||||
<datacite:identifier alternateIdentifierType="oai-original">
|
||||
<xsl:value-of
|
||||
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
|
||||
select="//oai:header/oai:identifier" />
|
||||
</datacite:identifier>
|
||||
|
||||
</datacite:alternateIdentifiers>
|
||||
|
||||
<datacite:relatedIdentifiers />
|
||||
|
||||
|
||||
<datacite:resourceType><xsl:value-of select="$varBaseNormType" /></datacite:resourceType>
|
||||
<xsl:for-each select="//base_dc:typenorm">
|
||||
<datacite:resourceType><xsl:value-of select="vocabulary:clean(., 'base:normalized_types')" /></datacite:resourceType>
|
||||
</xsl:for-each>
|
||||
|
||||
<datacite:titles>
|
||||
<xsl:for-each select="//dc:title">
|
||||
|
@ -154,14 +153,18 @@
|
|||
</xsl:for-each>
|
||||
</datacite:subjects>
|
||||
|
||||
<datacite:publisher>
|
||||
<xsl:value-of select="normalize-space(//dc:publisher)" />
|
||||
</datacite:publisher>
|
||||
|
||||
<datacite:publicationYear>
|
||||
<xsl:value-of select="normalize-space(//base_dc:year)" />
|
||||
</datacite:publicationYear>
|
||||
<xsl:for-each select="//dc:publisher">
|
||||
<datacite:publisher>
|
||||
<xsl:value-of select="normalize-space(.)" />
|
||||
</datacite:publisher>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:for-each select="//base_dc:year">
|
||||
<datacite:publicationYear>
|
||||
<xsl:value-of select="normalize-space(.)" />
|
||||
</datacite:publicationYear>
|
||||
</xsl:for-each>
|
||||
|
||||
<datacite:formats>
|
||||
<xsl:for-each select="//dc:format">
|
||||
<datacite:format>
|
||||
|
@ -175,6 +178,9 @@
|
|||
</datacite:language>
|
||||
|
||||
<oaf:accessrights>
|
||||
<xsl:if test="//base_dc:oa[.='0']">
|
||||
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_16ec">restricted access</datacite:rights>
|
||||
</xsl:if>
|
||||
<xsl:if test="//base_dc:oa[.='1']">
|
||||
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_abf2">open access</datacite:rights>
|
||||
</xsl:if>
|
||||
|
@ -185,29 +191,162 @@
|
|||
|
||||
</datacite:resource>
|
||||
|
||||
<xsl:for-each select="//dc:relation">
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
</xsl:for-each>
|
||||
<xsl:for-each select="//dc:relation">
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
</xsl:for-each>
|
||||
|
||||
<dr:CobjCategory>
|
||||
<xsl:variable name="varCobjCategory" select="vocabulary:clean($varBaseNormType, 'dnet:publication_resource')" />
|
||||
<xsl:variable name="varSuperType" select="vocabulary:clean($varCobjCategory, 'dnet:result_typologies')" />
|
||||
<xsl:attribute name="type" select="$varSuperType" />
|
||||
<xsl:value-of select="$varCobjCategory" />
|
||||
</dr:CobjCategory>
|
||||
<xsl:choose>
|
||||
<!-- I used an inline mapping because the field typenorm could be repeated and I have to specify a list of priority -->
|
||||
|
||||
<!-- Book part -->
|
||||
<xsl:when test="//base_dc:typenorm = '111'">
|
||||
<dr:CobjCategory type="publication">0013</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Book -->
|
||||
<xsl:when test="//base_dc:typenorm = '11'">
|
||||
<dr:CobjCategory type="publication">0002</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Article contribution -->
|
||||
<xsl:when test="//base_dc:typenorm = '121'">
|
||||
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
|
||||
<!-- Journal/Newspaper -->
|
||||
<xsl:when test="//base_dc:typenorm = '12'">
|
||||
<dr:CobjCategory type="publication">0043</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Report -->
|
||||
<xsl:when test="//base_dc:typenorm = '14'">
|
||||
<dr:CobjCategory type="publication">0017</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Review -->
|
||||
<xsl:when test="//base_dc:typenorm = '15'">
|
||||
<dr:CobjCategory type="publication">0015</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Lecture -->
|
||||
<xsl:when test="//base_dc:typenorm = '17'">
|
||||
<dr:CobjCategory type="publication">0010</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Bachelor's thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '181'">
|
||||
<dr:CobjCategory type="publication">0008</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Master's thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '182'">
|
||||
<dr:CobjCategory type="publication">0007</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Doctoral and postdoctoral thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '183'">
|
||||
<dr:CobjCategory type="publication">0006</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Thesis -->
|
||||
<xsl:when test="//base_dc:typenorm = '18'">
|
||||
<dr:CobjCategory type="publication">0044</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Patent -->
|
||||
<xsl:when test="//base_dc:typenorm = '1A'">
|
||||
<dr:CobjCategory type="publication">0019</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Text -->
|
||||
<xsl:when test="//base_dc:typenorm = '1'">
|
||||
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Software -->
|
||||
<xsl:when test="//base_dc:typenorm = '6'">
|
||||
<dr:CobjCategory type="software">0029</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Dataset -->
|
||||
<xsl:when test="//base_dc:typenorm = '7'">
|
||||
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Still image -->
|
||||
<xsl:when test="//base_dc:typenorm = '51'">
|
||||
<dr:CobjCategory type="other">0025</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Moving image/Video -->
|
||||
<xsl:when test="//base_dc:typenorm = '52'">
|
||||
<dr:CobjCategory type="other">0024</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Image/Video -->
|
||||
<xsl:when test="//base_dc:typenorm = '5'">
|
||||
<dr:CobjCategory type="other">0033</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Audio -->
|
||||
<xsl:when test="//base_dc:typenorm = '4'">
|
||||
<dr:CobjCategory type="other">0030</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Musical notation -->
|
||||
<xsl:when test="//base_dc:typenorm = '2'">
|
||||
<dr:CobjCategory type="other">0020</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Map -->
|
||||
<xsl:when test="//base_dc:typenorm = '3'">
|
||||
<dr:CobjCategory type="other">0020</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Other non-article -->
|
||||
<xsl:when test="//base_dc:typenorm = '122'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Course material -->
|
||||
<xsl:when test="//base_dc:typenorm = '16'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Manuscript -->
|
||||
<xsl:when test="//base_dc:typenorm = '19'">
|
||||
<dr:CobjCategory type="publication">0038</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Conference object -->
|
||||
<xsl:when test="//base_dc:typenorm = '13'">
|
||||
<dr:CobjCategory type="publication">0004</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
|
||||
<!-- Unknown -->
|
||||
<xsl:when test="//base_dc:typenorm = 'F'">
|
||||
<dr:CobjCategory type="other">0000</dr:CobjCategory>
|
||||
</xsl:when>
|
||||
<xsl:otherwise>
|
||||
<dr:CobjCategory type="other">0000</dr:CobjCategory>
|
||||
</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
|
||||
<oaf:accessrights>
|
||||
<xsl:choose>
|
||||
<xsl:when test="//base_dc:oa[.='0']">CLOSED</xsl:when>
|
||||
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
|
||||
<xsl:when test="//base_dc:oa[.='2']">UNKNOWN</xsl:when>
|
||||
<xsl:when test="//base_dc:rightsnorm">
|
||||
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
|
||||
</xsl:when>
|
||||
|
@ -245,7 +384,7 @@
|
|||
|
||||
<oaf:identifier identifierType="oai-original">
|
||||
<xsl:value-of
|
||||
select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'identifier']" />
|
||||
select="//oai:header/oai:identifier" />
|
||||
</oaf:identifier>
|
||||
|
||||
<oaf:hostedBy>
|
||||
|
|
|
@ -78,10 +78,6 @@ public class PrepareAffiliationRelationsTest {
|
|||
.getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json")
|
||||
.getPath();
|
||||
|
||||
String pubmedAffiliationRelationsPath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json")
|
||||
.getPath();
|
||||
|
||||
String outputPath = workingDir.toString() + "/actionSet";
|
||||
|
||||
PrepareAffiliationRelations
|
||||
|
@ -89,7 +85,8 @@ public class PrepareAffiliationRelationsTest {
|
|||
new String[] {
|
||||
"-isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"-crossrefInputPath", crossrefAffiliationRelationPath,
|
||||
"-pubmedInputPath", pubmedAffiliationRelationsPath,
|
||||
"-pubmedInputPath", crossrefAffiliationRelationPath,
|
||||
"-openapcInputPath", crossrefAffiliationRelationPath,
|
||||
"-outputPath", outputPath
|
||||
});
|
||||
|
||||
|
@ -106,7 +103,7 @@ public class PrepareAffiliationRelationsTest {
|
|||
// );
|
||||
// }
|
||||
// count the number of relations
|
||||
assertEquals(40, tmp.count());
|
||||
assertEquals(60, tmp.count());
|
||||
|
||||
Dataset<Relation> dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
|
||||
dataset.createOrReplaceTempView("result");
|
||||
|
@ -117,7 +114,7 @@ public class PrepareAffiliationRelationsTest {
|
|||
// verify that we have equal number of bi-directional relations
|
||||
Assertions
|
||||
.assertEquals(
|
||||
20, execVerification
|
||||
30, execVerification
|
||||
.filter(
|
||||
"relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'")
|
||||
.collectAsList()
|
||||
|
@ -125,7 +122,7 @@ public class PrepareAffiliationRelationsTest {
|
|||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
20, execVerification
|
||||
30, execVerification
|
||||
.filter(
|
||||
"relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'")
|
||||
.collectAsList()
|
||||
|
|
|
@ -0,0 +1,104 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.fosnodoi;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob;
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareTest;
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest;
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 13/02/23
|
||||
*/
|
||||
public class GetFosTest {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(ProduceTest.class);
|
||||
|
||||
private static Path workingDir;
|
||||
private static SparkSession spark;
|
||||
private static LocalFileSystem fs;
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName());
|
||||
|
||||
fs = FileSystem.getLocal(new Configuration());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(ProduceTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(PrepareTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
@Disabled
|
||||
void test3() throws Exception {
|
||||
final String sourcePath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv")
|
||||
.getPath();
|
||||
|
||||
final String outputPath = workingDir.toString() + "/fos.json";
|
||||
GetFOSSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"--isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"--sourcePath", sourcePath,
|
||||
|
||||
"-outputPath", outputPath,
|
||||
"-delimiter", ","
|
||||
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<FOSDataModel> tmp = sc
|
||||
.textFile(outputPath)
|
||||
.map(item -> OBJECT_MAPPER.readValue(item, FOSDataModel.class));
|
||||
|
||||
tmp.foreach(t -> Assertions.assertTrue(t.getOaid() != null));
|
||||
tmp.foreach(t -> Assertions.assertTrue(t.getLevel1() != null));
|
||||
tmp.foreach(t -> Assertions.assertTrue(t.getLevel2() != null));
|
||||
tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null));
|
||||
|
||||
tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t)));
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.fosnodoi;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob;
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareSDGSparkJob;
|
||||
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
|
||||
public class PrepareTest {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(ProduceTest.class);
|
||||
|
||||
private static Path workingDir;
|
||||
private static SparkSession spark;
|
||||
private static LocalFileSystem fs;
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName());
|
||||
|
||||
fs = FileSystem.getLocal(new Configuration());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(ProduceTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(PrepareTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
void fosPrepareTest() throws Exception {
|
||||
final String sourcePath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json")
|
||||
.getPath();
|
||||
|
||||
PrepareFOSSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"--isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"--sourcePath", sourcePath,
|
||||
|
||||
"-outputPath", workingDir.toString() + "/work",
|
||||
"-distributeDoi", Boolean.FALSE.toString()
|
||||
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Result> tmp = sc
|
||||
.textFile(workingDir.toString() + "/work/fos")
|
||||
.map(item -> OBJECT_MAPPER.readValue(item, Result.class));
|
||||
|
||||
tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t)));
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -76,7 +76,7 @@ public class CreateOpenCitationsASTest {
|
|||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI/inputremap/jsonforas")
|
||||
.getPath();
|
||||
|
||||
CreateActionSetSparkJob
|
||||
|
@ -84,8 +84,6 @@ public class CreateOpenCitationsASTest {
|
|||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-shouldDuplicateRels",
|
||||
Boolean.TRUE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
|
@ -99,9 +97,10 @@ public class CreateOpenCitationsASTest {
|
|||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
assertEquals(31, tmp.count());
|
||||
Assertions.assertEquals(27, tmp.count());
|
||||
tmp.foreach(r -> Assertions.assertEquals(1, r.getCollectedfrom().size()));
|
||||
|
||||
// tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
|
||||
tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.opencitations;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.spark.SparkConf;
|
||||
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.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 07/03/24
|
||||
*/
|
||||
public class RemapTest {
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
private static Path workingDir;
|
||||
private static final Logger log = LoggerFactory
|
||||
.getLogger(RemapTest.class);
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files
|
||||
.createTempDirectory(RemapTest.class.getSimpleName());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(RemapTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(RemapTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRemap() throws Exception {
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI/inputremap")
|
||||
.getPath();
|
||||
|
||||
MapOCIdsInPids
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/out/",
|
||||
"-nameNode", "input1;input2;input3;input4;input5"
|
||||
});
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,324 @@
|
|||
|
||||
package eu.dnetlib.dhp.actionmanager.transformativeagreement;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob;
|
||||
import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest;
|
||||
import eu.dnetlib.dhp.schema.action.AtomicAction;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 13/02/24
|
||||
*/
|
||||
public class CreateTAActionSetTest {
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
private static Path workingDir;
|
||||
private static final Logger log = LoggerFactory
|
||||
.getLogger(CreateOpenCitationsASTest.class);
|
||||
|
||||
@BeforeAll
|
||||
public static void beforeAll() throws IOException {
|
||||
workingDir = Files
|
||||
.createTempDirectory(CreateTAActionSetTest.class.getSimpleName());
|
||||
log.info("using work dir {}", workingDir);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.setAppName(CreateTAActionSetTest.class.getSimpleName());
|
||||
|
||||
conf.setMaster("local[*]");
|
||||
conf.set("spark.driver.host", "localhost");
|
||||
conf.set("hive.metastore.local", "true");
|
||||
conf.set("spark.ui.enabled", "false");
|
||||
conf.set("spark.sql.warehouse.dir", workingDir.toString());
|
||||
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
|
||||
|
||||
spark = SparkSession
|
||||
.builder()
|
||||
.appName(CreateTAActionSetTest.class.getSimpleName())
|
||||
.config(conf)
|
||||
.getOrCreate();
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static void afterAll() throws IOException {
|
||||
FileUtils.deleteDirectory(workingDir.toFile());
|
||||
spark.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
void createActionSet() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/transformativeagreement/facts.json")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.transformativeagreement.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet1"
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testNumberofRelations2() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet2"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet2", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
assertEquals(23, tmp.count());
|
||||
|
||||
// tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsCollectedFrom() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet3"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet3", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> {
|
||||
assertEquals(ModelConstants.OPENOCITATIONS_NAME, r.getCollectedfrom().get(0).getValue());
|
||||
assertEquals(ModelConstants.OPENOCITATIONS_ID, r.getCollectedfrom().get(0).getKey());
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsDataInfo() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet4"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet4", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> {
|
||||
assertEquals(false, r.getDataInfo().getInferred());
|
||||
assertEquals(false, r.getDataInfo().getDeletedbyinference());
|
||||
assertEquals("0.91", r.getDataInfo().getTrust());
|
||||
assertEquals(
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob.OPENCITATIONS_CLASSID,
|
||||
r.getDataInfo().getProvenanceaction().getClassid());
|
||||
assertEquals(
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob.OPENCITATIONS_CLASSNAME,
|
||||
r.getDataInfo().getProvenanceaction().getClassname());
|
||||
assertEquals(ModelConstants.DNET_PROVENANCE_ACTIONS, r.getDataInfo().getProvenanceaction().getSchemeid());
|
||||
assertEquals(ModelConstants.DNET_PROVENANCE_ACTIONS, r.getDataInfo().getProvenanceaction().getSchemename());
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsSemantics() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet5"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet5", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> {
|
||||
assertEquals("citation", r.getSubRelType());
|
||||
assertEquals("resultResult", r.getRelType());
|
||||
});
|
||||
assertEquals(23, tmp.filter(r -> r.getRelClass().equals("Cites")).count());
|
||||
assertEquals(0, tmp.filter(r -> r.getRelClass().equals("IsCitedBy")).count());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsSourceTargetPrefix() throws Exception {
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet6"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet6", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
tmp.foreach(r -> {
|
||||
assertEquals("50|doi_________::", r.getSource().substring(0, 17));
|
||||
assertEquals("50|doi_________::", r.getTarget().substring(0, 17));
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
void testRelationsSourceTargetCouple() throws Exception {
|
||||
final String doi1 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-015-3684-x"));
|
||||
final String doi2 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x"));
|
||||
final String doi3 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-014-2114-9"));
|
||||
final String doi4 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069"));
|
||||
final String doi5 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-009-9913-4"));
|
||||
final String doi6 = "50|doi_________::"
|
||||
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5"));
|
||||
|
||||
String inputPath = getClass()
|
||||
.getResource(
|
||||
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
|
||||
.getPath();
|
||||
|
||||
CreateActionSetSparkJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged",
|
||||
Boolean.FALSE.toString(),
|
||||
"-inputPath",
|
||||
inputPath,
|
||||
"-outputPath",
|
||||
workingDir.toString() + "/actionSet7"
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Relation> tmp = sc
|
||||
.sequenceFile(workingDir.toString() + "/actionSet7", Text.class, Text.class)
|
||||
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
|
||||
.map(aa -> ((Relation) aa.getPayload()));
|
||||
|
||||
JavaRDD<Relation> check = tmp.filter(r -> r.getSource().equals(doi1) || r.getTarget().equals(doi1));
|
||||
|
||||
assertEquals(5, check.count());
|
||||
|
||||
// check.foreach(r -> {
|
||||
// if (r.getSource().equals(doi2) || r.getSource().equals(doi3) || r.getSource().equals(doi4) ||
|
||||
// r.getSource().equals(doi5) || r.getSource().equals(doi6)) {
|
||||
// assertEquals(ModelConstants.IS_CITED_BY, r.getRelClass());
|
||||
// assertEquals(doi1, r.getTarget());
|
||||
// }
|
||||
// });
|
||||
|
||||
assertEquals(5, check.filter(r -> r.getSource().equals(doi1)).count());
|
||||
check.filter(r -> r.getSource().equals(doi1)).foreach(r -> assertEquals(ModelConstants.CITES, r.getRelClass()));
|
||||
|
||||
}
|
||||
}
|
|
@ -1,6 +1,8 @@
|
|||
|
||||
package eu.dnetlib.dhp.collection.plugin.base;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -8,7 +10,6 @@ import org.apache.spark.SparkConf;
|
|||
import org.apache.spark.util.LongAccumulator;
|
||||
import org.dom4j.io.SAXReader;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Disabled;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
@ -20,7 +21,7 @@ import eu.dnetlib.dhp.schema.mdstore.Provenance;
|
|||
import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
|
||||
@Disabled
|
||||
// @Disabled
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class BaseTransfomationTest extends AbstractVocabularyTest {
|
||||
|
||||
|
@ -64,11 +65,26 @@ public class BaseTransfomationTest extends AbstractVocabularyTest {
|
|||
System.out.println(result.getBody());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBase2ODF_wrong_date() throws Exception {
|
||||
|
||||
final MetadataRecord mr = new MetadataRecord();
|
||||
mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX"));
|
||||
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record_wrong_1.xml")));
|
||||
|
||||
final XSLTTransformationFunction tr = loadTransformationRule("xml/base2oaf.transformationRule.xml");
|
||||
|
||||
assertThrows(NullPointerException.class, () -> {
|
||||
final MetadataRecord result = tr.call(mr);
|
||||
System.out.println(result.getBody());
|
||||
});
|
||||
}
|
||||
|
||||
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
|
||||
final String xslt = new SAXReader()
|
||||
.read(this.getClass().getResourceAsStream(path))
|
||||
.selectSingleNode("//CODE/*")
|
||||
.asXML();
|
||||
.read(this.getClass().getResourceAsStream(path))
|
||||
.selectSingleNode("//CODE/*")
|
||||
.asXML();
|
||||
|
||||
final LongAccumulator la = new LongAccumulator();
|
||||
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.transformation;
|
|||
|
||||
import static eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
|
@ -279,6 +280,19 @@ class TransformationJobTest extends AbstractVocabularyTest {
|
|||
// TODO Create significant Assert
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidXSLT() throws Exception{
|
||||
final MetadataRecord mr = new MetadataRecord();
|
||||
|
||||
mr.setProvenance(new Provenance("openaire____::cnr_explora", "CNR ExploRA", "cnr_________"));
|
||||
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input_cnr_explora.xml")));
|
||||
// We Load the XSLT transformation Rule from the classpath
|
||||
final XSLTTransformationFunction tr = loadTransformationRule("/eu/dnetlib/dhp/transform/invalid.xslt");
|
||||
|
||||
assertThrows(RuntimeException.class,()->tr.call(mr));
|
||||
|
||||
}
|
||||
|
||||
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
|
||||
final String trValue = IOUtils.toString(this.getClass().getResourceAsStream(path));
|
||||
final LongAccumulator la = new LongAccumulator();
|
||||
|
|
|
@ -0,0 +1,19 @@
|
|||
DOI,OAID,level1,level2,level3,level4,score_for_L3,score_for_L4
|
||||
N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.5,0.0
|
||||
N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0303 health sciences,030304 developmental biology,N/A,0.5,0.0
|
||||
N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050109 social psychology,05010904 Group processes/Collective identity,0.5589094161987305,0.5166763067245483
|
||||
N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050105 experimental psychology,05010501 Emotion/Affective science,0.44109055399894714,0.4833236634731293
|
||||
N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050203 business & management,05020302 Supply chain management/Business terms,0.5459638833999634,0.5460261106491089
|
||||
N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050211 marketing,05021102 Services marketing/Retailing,0.4540362060070038,0.4539738595485687
|
||||
N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010504 meteorology & atmospheric sciences,01050407 Geomagnetism/Ionosphere,0.5131047964096069,0.4990350902080536
|
||||
N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010502 geochemistry & geophysics,01050203 Seismology/Seismology measurement,0.4868951737880707,0.500964879989624
|
||||
N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022002 Medical imaging/Medical physics,0.5068133473396301,0.10231181626910052
|
||||
N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030204 cardiovascular system & hematology,N/A,0.49318668246269226,0.0
|
||||
N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5
|
||||
N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5
|
||||
N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021702 Aging-associated diseases/Cognitive disorders,0.5134317874908447,0.09614889098529535
|
||||
N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.48656824231147766,0.0
|
||||
N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,0209 industrial biotechnology,020901 industrial engineering & automation,02090105 Control theory/Advanced driver assistance systems,0.5178514122962952,0.5198937654495239
|
||||
N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,"0202 electrical engineering, electronic engineering, information engineering",020201 artificial intelligence & image processing,02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience,0.48214852809906006,0.4801062345504761
|
||||
N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021708 Neurotrauma/Stroke,0.5014800429344177,0.5109656453132629
|
||||
N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,02 engineering and technology,0206 medical engineering,020601 biomedical engineering,02060102 Medical terminology/Patient,0.4985199570655823,0.4890343248844147
|
|
|
@ -0,0 +1,18 @@
|
|||
{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"}
|
||||
{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"}
|
||||
{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"}
|
||||
{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0303 health sciences","level3":"030304 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"}
|
||||
{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021702 Aging-associated diseases/Cognitive disorders","scoreL3":"0.5134317874908447","scoreL4":"0.09614889098529535"}
|
||||
{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050109 social psychology","level4":"05010904 Group processes/Collective identity","scoreL3":"0.5589094161987305","scoreL4":"0.5166763067245483"}
|
||||
{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.48656824231147766","scoreL4":"0.0"}
|
||||
{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050105 experimental psychology","level4":"05010501 Emotion/Affective science","scoreL3":"0.44109055399894714","scoreL4":"0.4833236634731293"}
|
||||
{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0209 industrial biotechnology","level3":"020901 industrial engineering & automation","level4":"02090105 Control theory/Advanced driver assistance systems","scoreL3":"0.5178514122962952","scoreL4":"0.5198937654495239"}
|
||||
{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050203 business & management","level4":"05020302 Supply chain management/Business terms","scoreL3":"0.5459638833999634","scoreL4":"0.5460261106491089"}
|
||||
{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050211 marketing","level4":"05021102 Services marketing/Retailing","scoreL3":"0.4540362060070038","scoreL4":"0.4539738595485687"}
|
||||
{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0202 electrical engineering, electronic engineering, information engineering","level3":"020201 artificial intelligence & image processing","level4":"02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience","scoreL3":"0.48214852809906006","scoreL4":"0.4801062345504761"}
|
||||
{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010504 meteorology & atmospheric sciences","level4":"01050407 Geomagnetism/Ionosphere","scoreL3":"0.5131047964096069","scoreL4":"0.4990350902080536"}
|
||||
{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021708 Neurotrauma/Stroke","scoreL3":"0.5014800429344177","scoreL4":"0.5109656453132629"}
|
||||
{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010502 geochemistry & geophysics","level4":"01050203 Seismology/Seismology measurement","scoreL3":"0.4868951737880707","scoreL4":"0.500964879989624"}
|
||||
{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"02 engineering and technology","level2":"0206 medical engineering","level3":"020601 biomedical engineering","level4":"02060102 Medical terminology/Patient","scoreL3":"0.4985199570655823","scoreL4":"0.4890343248844147"}
|
||||
{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022002 Medical imaging/Medical physics","scoreL3":"0.5068133473396301","scoreL4":"0.10231181626910052"}
|
||||
{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030204 cardiovascular system & hematology","level4":"N/A","scoreL3":"0.49318668246269226","scoreL4":"0.0"}
|
File diff suppressed because one or more lines are too long
|
@ -0,0 +1,31 @@
|
|||
{"cited":"br/061201599020", "citing":"br/06203041400","oci":"oci:06701327944-06504326071"}
|
||||
{"cited":"br/061201599020","citing":"br/06502272390","oci":"oci:06502272390-061301355525"}
|
||||
{"cited":"br/061201599020", "citing":"br/06120941789","oci":"oci:0670804699-067055659"}
|
||||
{"cited":"br/06210273177","citing":"br/06203041400","oci":"oci:061502003994-062201281456"}
|
||||
{"cited":"br/06210273177", "citing":"br/06502272390","oci":"oci:06502272390-0660806688"}
|
||||
{"cited":"br/06210273177", "citing":"br/06120941789","oci":"oci:06502307119-0620223645"}
|
||||
{"cited":"br/0660613430","citing":"br/06203041400","oci":"oci:061502004011-061902692285"}
|
||||
{"cited":"br/0660613430", "citing":"br/06502272390","oci":"oci:0660549063-0610398792"}
|
||||
{"cited":"br/0660613430", "citing":"br/06120941789","oci":"oci:06420189324-06301543046"}
|
||||
{"cited":"br/062602732073","citing":"br/06203041400","oci":"oci:06380130275-061502004367"}
|
||||
{"cited":"br/062602732073","citing":"br/06502272390","oci":"oci:062403449086-062501448395"}
|
||||
{"cited":"br/062602732073","citing":"br/06120941789","oci":"oci:06420189328-061202007182"}
|
||||
{"cited":"br/061103703697","citing":"br/06203041400","oci":"oci:062603906965-061701362658"}
|
||||
{"cited":"br/061103703697", "citing":"br/06502272390","oci":"oci:0670294309-06104327031"}
|
||||
{"cited":"br/061103703697","citing":"br/06120941789","oci":"oci:061702060228-061301712529"}
|
||||
{"cited":"br/06230199640", "citing":"br/0670517081","oci":"oci:06901104174-06503692526"}
|
||||
{"cited":"br/061703513967","citing":"br/061702310822","oci":"oci:061702310822-061703513967"}
|
||||
{"cited":"br/062104002953","citing":"br/061702311472","oci":"oci:061702311472-062104002953"}
|
||||
{"cited":"br/061101204417","citing":"br/062102701590","oci":"oci:062102701590-061101204417"}
|
||||
{"cited":"br/062403787088","citing":"br/061401499173","oci":"oci:061401499173-062403787088"}
|
||||
{"cited":"br/061203576338","citing":"br/06110279619","oci":"oci:06110279619-061203576338"}
|
||||
{"cited":"br/061601962207","citing":"br/061502004018","oci":"oci:061502004018-061601962207"}
|
||||
{"cited":"br/06101014588", "citing":"br/061502004027","oci":"oci:061502004027-06101014588"}
|
||||
{"cited":"br/06704040804", "citing":"br/06220799044","oci":"oci:06220799044-06704040804"}
|
||||
{"cited":"br/061401105151","citing":"br/061502004037","oci":"oci:061502004037-061401105151"}
|
||||
{"cited":"br/0640821079", "citing":"br/061702311537","oci":"oci:061702311537-0640821079"}
|
||||
{"cited":"br/06604165310", "citing":"br/062501970289","oci":"oci:062501970289-06604165310"}
|
||||
{"cited":"br/061501351689","citing":"br/061203895786","oci":"oci:061203895786-061501351689"}
|
||||
{"cited":"br/06202223692", "citing":"br/06110298832","oci":"oci:06110298832-06202223692"}
|
||||
{"cited":"br/06104310727", "citing":"br/0660439086","oci":"oci:0660439086-06104310727"}
|
||||
{"cited":"br/06150216214", "citing":"br/06340150329","oci":"oci:06340150329-06150216214"}
|
|
@ -0,0 +1,48 @@
|
|||
omid,id
|
||||
br/061201599020,doi:10.1142/s0219887817501687
|
||||
br/06203041400,doi:10.1111/j.1523-5378.2005.00327.x pmid:16104945
|
||||
br/06210273177,doi:10.1090/qam/20394
|
||||
br/06502272390,pmid:32235596 doi:10.3390/nano10040644
|
||||
br/0660613430,doi:10.1007/bf00470411
|
||||
br/06120941789,doi:10.1098/rspa.2006.1747
|
||||
br/062602732073,doi:10.1007/978-3-642-38844-6_25
|
||||
br/06230199640,pmid:25088780 doi:10.1016/j.ymeth.2014.07.008
|
||||
br/061103703697,pmid:2682767
|
||||
br/0670517081,doi:10.1016/j.foodpol.2021.102189
|
||||
br/06502310477,doi:10.1142/s0218127416500450
|
||||
br/06520113284,doi:10.1109/cfasta57821.2023.10243367
|
||||
br/062303652439,pmid:5962654 doi:10.1016/0020-708x(66)90001-9
|
||||
br/06250691436,doi:10.1042/bst20150052 pmid:26009172
|
||||
br/061201665577,doi:10.1097/00115550-200205000-00018
|
||||
br/06503490336,pmid:34689254 doi:10.1007/s10072-021-05687-0
|
||||
br/06220615942,pmid:25626134 doi:10.1016/j.jcis.2015.01.008
|
||||
br/061103389243,doi:10.4324/9780203702819-10
|
||||
br/062303011271,doi:10.1109/icassp.2011.5946250
|
||||
br/061302926083,doi:10.4018/978-1-6684-3937-1.ch002
|
||||
br/061402485360,doi:10.1109/iciict.2015.7396079
|
||||
br/06410101083,doi:10.1016/j.autcon.2023.104828
|
||||
br/062202243386,doi:10.1016/0001-8791(81)90022-1
|
||||
br/06170421486,doi:10.1130/0016-7606(2003)115<0166:dsagmf>2.0.co;2
|
||||
br/061201983865,doi:10.4324/9781315109008 isbn:9781315109008
|
||||
br/061701697230,doi:10.1016/j.trd.2012.07.006
|
||||
br/061201137111,doi:10.1109/access.2020.2971656
|
||||
br/06120436283,pmid:2254430 doi:10.1128/jcm.28.11.2551-2554.1990
|
||||
br/061903968916,doi:10.1111/j.1742-1241.1988.tb08627.x
|
||||
br/06201583482,doi:10.1016/0016-5085(78)93139-6
|
||||
br/06130338317,doi:10.2134/agronj1952.00021962004400080013x
|
||||
br/062601538320,doi:10.1371/journal.pone.0270593 pmid:35789338
|
||||
br/062401098626,pmid:22385804 doi:10.1016/j.talanta.2011.12.034
|
||||
br/06190436492,doi:10.1039/c7dt01499f pmid:28644489
|
||||
br/06202819247,doi:10.1007/978-3-319-45823-6_57
|
||||
br/0648013560,doi:10.1080/14772000.2012.705356
|
||||
br/0690214059,doi:10.2752/175630608x329217
|
||||
br/06601640415,doi:10.1080/18128600508685647
|
||||
br/061503394761,doi:10.1002/0471443395.img018
|
||||
br/061702861849,pmid:31203682 doi:10.1080/10428194.2019.1627538
|
||||
br/06450133713,doi:10.1093/acprof:oso/9780199670888.003.0008
|
||||
br/0628074892,doi:10.1097/hnp.0000000000000597
|
||||
br/061601032219,doi:10.1002/bdm.2102
|
||||
br/06602079930,doi:10.1101/2020.08.25.267500
|
||||
br/0604192147,doi:10.11501/3307395
|
||||
br/061101933800,doi:10.1142/s0217732398002242
|
||||
br/06504184118,pmid:10091417
|
|
|
@ -0,0 +1,27 @@
|
|||
{"oci":"oci:06701327944-06504326071","citing":"16104945","citing_pid":"pmid","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
|
||||
{"oci":"oci:06701327944-06504326071","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
|
||||
{"oci":"oci:06502272390-061301355525","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
|
||||
{"oci":"oci:06502272390-061301355525","citing":"32235596","citing_pid":"pmid","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
|
||||
{"oci":"oci:0670804699-067055659","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
|
||||
{"oci":"oci:061502003994-062201281456","citing":"16104945","citing_pid":"pmid","cited":"10.1090/qam/20394","cited_pid":"doi"}
|
||||
{"oci":"oci:061502003994-062201281456","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
|
||||
{"oci":"oci:06502272390-0660806688","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
|
||||
{"oci":"oci:06502272390-0660806688","citing":"32235596","citing_pid":"pmid","cited":"10.1090/qam/20394","cited_pid":"doi"}
|
||||
{"oci":"oci:06502307119-0620223645","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
|
||||
{"oci":"oci:061502004011-061902692285","citing":"16104945","citing_pid":"pmid","cited":"10.1007/bf00470411","cited_pid":"doi"}
|
||||
{"oci":"oci:061502004011-061902692285","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
|
||||
{"oci":"oci:0660549063-0610398792","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
|
||||
{"oci":"oci:0660549063-0610398792","citing":"32235596","citing_pid":"pmid","cited":"10.1007/bf00470411","cited_pid":"doi"}
|
||||
{"oci":"oci:06420189324-06301543046","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
|
||||
{"oci":"oci:06380130275-061502004367","citing":"16104945","citing_pid":"pmid","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
|
||||
{"oci":"oci:06380130275-061502004367","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
|
||||
{"oci":"oci:062403449086-062501448395","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
|
||||
{"oci":"oci:062403449086-062501448395","citing":"32235596","citing_pid":"pmid","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
|
||||
{"oci":"oci:06420189328-061202007182","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
|
||||
{"oci":"oci:062603906965-061701362658","citing":"16104945","citing_pid":"pmid","cited":"2682767","cited_pid":"pmid"}
|
||||
{"oci":"oci:062603906965-061701362658","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
|
||||
{"oci":"oci:0670294309-06104327031","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
|
||||
{"oci":"oci:0670294309-06104327031","citing":"32235596","citing_pid":"pmid","cited":"2682767","cited_pid":"pmid"}
|
||||
{"oci":"oci:061702060228-061301712529","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
|
||||
{"oci":"oci:06901104174-06503692526","citing":"10.1016/j.foodpol.2021.102189","citing_pid":"doi","cited":"10.1016/j.ymeth.2014.07.008","cited_pid":"doi"}
|
||||
{"oci":"oci:06901104174-06503692526","citing":"10.1016/j.foodpol.2021.102189","citing_pid":"doi","cited":"25088780","cited_pid":"pmid"}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,36 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<record>
|
||||
<header xmlns="http://www.openarchives.org/OAI/2.0/">
|
||||
<identifier>ftunivminnesdc:oai:conservancy.umn.edu:11299/109914</identifier>
|
||||
<datestamp>2023-07-18T20:05:40Z</datestamp>
|
||||
</header>
|
||||
<metadata xmlns="http://www.openarchives.org/OAI/2.0/" xmlns:base_dc="http://oai.base-search.net/base_dc/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:dc="http://purl.org/dc/elements/1.1/">
|
||||
<base_dc:dc xsi:schemaLocation="http://oai.base-search.net/base_dc/ http://oai.base-search.net/base_dc/base_dc.xsd">
|
||||
<base_dc:global_id>ftunivminnesdc:oai:conservancy.umn.edu:11299/109914</base_dc:global_id>
|
||||
<base_dc:continent>cna</base_dc:continent>
|
||||
<base_dc:country>us</base_dc:country>
|
||||
<base_dc:collection opendoar_id="1008">ftunivminnesdc</base_dc:collection>
|
||||
<base_dc:collname>University of Minnesota Digital Conservancy</base_dc:collname>
|
||||
<dc:title>An Experimental Investigation of the influence of an air bubble layer on radiated noise and surface pressure fluctuations in a turbulent boundary layer</dc:title>
|
||||
<dc:creator>Killen, John M.</dc:creator>
|
||||
<dc:subject>Boundary layer noise</dc:subject>
|
||||
<dc:subject>Kinetic energy dissipation</dc:subject>
|
||||
<dc:subject>Air bubble layer</dc:subject>
|
||||
<dc:subject>Maximum stable bubble size</dc:subject>
|
||||
<dc:subject>bubble noise spectra</dc:subject>
|
||||
<dc:description>between the noise spectra of a layer of air bubbles in a turbulent flow and' a maximum stable bubble size which can exist in the same flow. An air bubbie layer with individual bubble sizes greater than a maximum stable size was introduced into the boundary layer of water flowing along a smooth flat plate. It was found that the maximum stable bubble size was related to the turbulent kinetic energy dissipation as had been previously shown by other investigators ; Office of Naval Research Arlington, Virginia 22217</dc:description>
|
||||
<dc:date>1981-09-31T21:42:52Z</dc:date>
|
||||
<base_dc:year>1981</base_dc:year>
|
||||
<dc:type>Report</dc:type>
|
||||
<base_dc:typenorm>14</base_dc:typenorm>
|
||||
<dc:identifier>http://purl.umn.edu/109914</dc:identifier>
|
||||
<base_dc:link>http://purl.umn.edu/109914</base_dc:link>
|
||||
<dc:language>en_US</dc:language>
|
||||
<dc:relation>Project Reports</dc:relation>
|
||||
<dc:relation>202</dc:relation>
|
||||
<dc:relation>http://purl.umn.edu/109914</dc:relation>
|
||||
<base_dc:oa>2</base_dc:oa>
|
||||
<base_dc:lang>eng</base_dc:lang>
|
||||
</base_dc:dc>
|
||||
</metadata>
|
||||
</record>
|
|
@ -0,0 +1,185 @@
|
|||
<xsl:stylesheet
|
||||
xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
|
||||
xmlns:oaire="http://namespace.openaire.eu/schema/oaire/"
|
||||
xmlns:vocabulary="http://eu/dnetlib/transform/clean"
|
||||
xmlns:dateCleaner="http://eu/dnetlib/transform/dateISO"
|
||||
xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
|
||||
xmlns:datacite="http://datacite.org/schema/kernel-4"
|
||||
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||
xmlns:xs="http://www.w3.org/2001/XMLSchema"
|
||||
xmlns:dr="http://www.driver-repository.eu/namespace/dr"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
exclude-result-prefixes="xsl vocabulary dateCleaner" version="2.0">
|
||||
<xsl:param name="varOfficialName" />
|
||||
<xsl:param name="varDataSourceId" />
|
||||
<xsl:param name="varFP7" select="'corda_______::'" />
|
||||
<xsl:param name="varH2020" select="'corda__h2020::'" />
|
||||
<xsl:param name="repoCode"
|
||||
select="substring-before(//*[local-name() = 'header']/*[local-name()='recordIdentifier'], ':')" />
|
||||
<xsl:param name="index" select="0" />
|
||||
<xsl:param name="transDate" select="current-dateTime()" />
|
||||
asdf;klamsdof'sdn &
|
||||
<xsl:template match="/asdfasdf asdf&">
|
||||
c:format'" />
|
||||
</xsl:call-template>
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:type" />
|
||||
<xsl:with-param name="targetElement" select="'dc:type'" />
|
||||
</xsl:call-template>
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:source" />
|
||||
<xsl:with-param name="targetElement" select="'dc:source'" />
|
||||
</xsl:call-template>
|
||||
<dc:language>
|
||||
<xsl:value-of select="vocabulary:clean( //dc:language, 'dnet:languages')" />
|
||||
</dc:language>
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:rights" />
|
||||
<xsl:with-param name="targetElement" select="'dc:rights'" />
|
||||
</xsl:call-template>
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:relation[not(starts-with(.,'info:cnr-pdr'))]" />
|
||||
<xsl:with-param name="targetElement" select="'dc:relation'" />
|
||||
</xsl:call-template>
|
||||
|
||||
<xsl:call-template name="allElements">
|
||||
<xsl:with-param name="sourceElement" select="//dc:identifier[starts-with(., 'http')]" />
|
||||
<xsl:with-param name="targetElement" select="'dc:identifier'" />
|
||||
</xsl:call-template>
|
||||
<xsl:for-each select="//dc:relation">
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
|
||||
<oaf:projectid>
|
||||
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
|
||||
</oaf:projectid>
|
||||
</xsl:if>
|
||||
</xsl:for-each>
|
||||
|
||||
<oaf:accessrights>
|
||||
<xsl:value-of select="vocabulary:clean( //dc:rights, 'dnet:access_modes')" />
|
||||
</oaf:accessrights>
|
||||
|
||||
<xsl:variable name="varCobjCategory" select="vocabulary:clean( //dc:type, 'dnet:publication_resource')" />
|
||||
<xsl:variable name="varSuperType" select="vocabulary:clean( $varCobjCategory, 'dnet:result_typologies')" />
|
||||
<dr:CobjCategory type="{$varSuperType}"><xsl:value-of select="$varCobjCategory" /></dr:CobjCategory>
|
||||
|
||||
|
||||
<xsl:variable name="varRefereedConvt" select="for $i in (//dc:type, //dc:description, //oai:setSpec) return vocabulary:clean( normalize-space($i), 'dnet:review_levels')" />
|
||||
<xsl:variable name="varRefereedIdntf" select="(//*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '(^|.*[\.\-_/\s\(\)%\d#])pre[\.\-_/\s\(\)%\d#]?prints?([\.\-_/\s\(\)%\d#].*)?$')][count(//dc:identifier) = 1]/'0002', //*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '(^|.*[\.\-_/\s\(\)%\d#])refereed([\.\-_/\s\(\)\d%\d#].*)?$')]/'0001', //*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '.*-peer-reviewed-(fulltext-)?article-.*')]/'0001')" />
|
||||
<xsl:variable name="varRefereedSourc" select="//*[string(node-name(.)) = ('dc:source', 'dc:publisher') and matches(lower-case(.), '^(.*\s)?pre[\s\-_]*prints?([\s\.,].*)?$')]/'0002'" />
|
||||
<xsl:variable name="varRefereedDescr" select="(//dc:description[matches(lower-case(.), '.*(this\s*book|this\s*volume|it)\s*constitutes\s*the\s*(thoroughly\s*)?refereed') or matches(lower-case(.), '.*peer[\.\-_/\s\(\)]?review\s*under\s*responsibility\s*of.*') or matches(lower-case(.), '(this|a)\s*(article|preprint)\s*(has\s*been\s*)?(peer[\-\s]*)?reviewed\s*and\s*recommended\s*by\s*peer[\-\s]*community')]/'0001', //dc:description[matches(., '^version\s*(préliminaire.*|preliminary.*|0$)')]/'0002')" />
|
||||
<xsl:variable name="varRefereedTitle" select="(//dc:title[matches(lower-case(.), '.*\[.*peer[\s\-\._]*review\s*:.*\]\s*$')]/'0001', //dc:title[matches(lower-case(.), '.*\(\s*pre[\s\-\._]*prints?\s*\)\s*$')]/'0002')" />
|
||||
<xsl:variable name="varRefereedSubjt" select="(//dc:subject[matches(lower-case(.), '^\s*refereed\s*$')][//oaf:datasourceprefix = 'narcis______']/'0001', //dc:subject[matches(lower-case(.), '^\s*no[nt].{0,3}refereed\s*$')][//oaf:datasourceprefix = 'narcis______']/'0002')" />
|
||||
<xsl:variable name="varRefereed" select="($varRefereedConvt, $varRefereedIdntf, $varRefereedSourc, $varRefereedDescr, $varRefereedTitle, $varRefereedSubjt)" />
|
||||
<xsl:choose>
|
||||
<xsl:when test="count($varRefereed[. = '0001']) > 0">
|
||||
<oaf:refereed>
|
||||
<xsl:value-of select="'0001'" />
|
||||
</oaf:refereed>
|
||||
</xsl:when>
|
||||
<xsl:when test="count($varRefereed[. = '0002']) > 0">
|
||||
<oaf:refereed>
|
||||
<xsl:value-of select="'0002'" />
|
||||
</oaf:refereed>
|
||||
</xsl:when>
|
||||
</xsl:choose>
|
||||
|
||||
<oaf:dateAccepted>
|
||||
<xsl:value-of select="dateCleaner:dateISO( //dc:date[1] )" />
|
||||
</oaf:dateAccepted>
|
||||
|
||||
<xsl:if test="//dc:relation[starts-with(., 'http')] and //dc:rights[.='info:eu-repo/semantics/openAccess']">
|
||||
<oaf:fulltext>
|
||||
<xsl:value-of select="//dc:relation[starts-with(., 'http')]" />
|
||||
</oaf:fulltext>
|
||||
</xsl:if>
|
||||
|
||||
<oaf:hostedBy name="{$varOfficialName}" id="{$varDataSourceId}" />
|
||||
<oaf:collectedFrom name="{$varOfficialName}" id="{$varDataSourceId}" />
|
||||
|
||||
<xsl:variable name="varKnownFileEndings" select="('.bmp', '.doc', '.docx', '.epub', '.flv', '.jpeg', '.jpg', '.m4v', '.mp4', '.mpg', '.odp', '.pdf', '.png', '.ppt', '.tiv', '.txt', '.xls', '.xlsx', '.zip')" />
|
||||
<xsl:variable name="varIdDoi" select="distinct-values((//dc:identifier[starts-with(., '10.')][matches(., '(10[.][0-9]{4,}[^\s/>]*/[^\s>]+)')], //dc:identifier[starts-with(., 'http') and (contains(., '://dx.doi.org/10.') or contains(., '://doi.org/10.'))]/substring-after(., 'doi.org/'), //dc:identifier[starts-with(lower-case(.), 'doi:10.')]/substring-after(lower-case(.), 'doi:')))" />
|
||||
<xsl:for-each select="$varIdDoi">
|
||||
<oaf:identifier identifierType="doi">
|
||||
<xsl:value-of select="." />
|
||||
</oaf:identifier>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:variable name="varIdHdl" select="distinct-values(//dc:identifier[starts-with(., 'http') and contains(., '://hdl.handle.net/')]/substring-after(., 'hdl.handle.net/'))" />
|
||||
<xsl:for-each select="$varIdHdl" >
|
||||
<oaf:identifier identifierType="handle">
|
||||
<xsl:value-of select="." />
|
||||
</oaf:identifier>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:variable name="varIdUrn" select="distinct-values(//dc:identifier[starts-with(., 'urn:nbn:nl:') or starts-with(., 'URN:NBN:NL:')])" />
|
||||
<xsl:for-each select="$varIdUrn">
|
||||
<oaf:identifier identifierType="urn">
|
||||
<xsl:value-of select="." />
|
||||
</oaf:identifier>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:variable name="varOrigBaseUrl" select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'baseURL']" />
|
||||
<xsl:variable name="varIdLdpg" select="distinct-values(//dc:identifier[(contains(substring-after(., '://'), '/') and contains($varOrigBaseUrl, substring-before(substring-after(., '://'), '/'))) or (contains(substring-after(., '://'), ':') and contains($varOrigBaseUrl, substring-before(substring-after(., '://'), ':')))][not(replace(lower-case(.), '.*(\.[a-z]*)$', '$1') = $varKnownFileEndings)])" />
|
||||
<xsl:for-each select="$varIdLdpg">
|
||||
<oaf:identifier identifierType="landingPage">
|
||||
<xsl:value-of select="." />
|
||||
</oaf:identifier>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:variable name="varIdUrl" select="distinct-values(//dc:identifier[starts-with(., 'http')][not(contains(., '://dx.doi.org/') or contains(., '://doi.org/') or contains(., '://hdl.handle.net/'))][count(index-of($varIdLdpg, .)) = 0])" />
|
||||
<xsl:for-each select="$varIdUrl">
|
||||
<oaf:identifier identifierType="url">
|
||||
<xsl:value-of select="." />
|
||||
</oaf:identifier>
|
||||
</xsl:for-each>
|
||||
|
||||
<xsl:for-each select="//oai:setSpec">
|
||||
<xsl:variable name="rorDsId" select="vocabulary:clean(., 'cnr:institutes')" />
|
||||
<xsl:if test="contains($rorDsId, '/ror.org/')">
|
||||
<oaf:relation relType="resultOrganization" subRelType="affiliation" relClass="hasAuthorInstitution">
|
||||
<xsl:value-of select="concat('ror_________::', $rorDsId)" />
|
||||
</oaf:relation>
|
||||
</xsl:if>
|
||||
</xsl:for-each>
|
||||
|
||||
</metadata>
|
||||
|
||||
<xsl:copy-of select="//*[local-name() = 'about']" />
|
||||
</record>
|
||||
</xsl:template>
|
||||
|
||||
|
||||
<xsl:template name="allElements">
|
||||
<xsl:param name="sourceElement" />
|
||||
<xsl:param name="targetElement" />
|
||||
|
||||
<xsl:for-each select="$sourceElement">
|
||||
<xsl:element name="{$targetElement}">
|
||||
<xsl:value-of select="normalize-space(.)" />
|
||||
</xsl:element>
|
||||
</xsl:for-each>
|
||||
</xsl:template>
|
||||
|
||||
<xsl:template match="//*[local-name() = 'header']">
|
||||
<xsl:copy>
|
||||
<xsl:apply-templates select="node()|@*" />
|
||||
<xsl:element name="dr:dateOfTransformation">
|
||||
<xsl:value-of select="$transDate" />
|
||||
</xsl:element>
|
||||
</xsl:copy>
|
||||
</xsl:template>
|
||||
|
||||
<xsl:template match="node()|@*">
|
||||
<xsl:copy>
|
||||
<xsl:apply-templates select="node()|@*" />
|
||||
</xsl:copy>
|
||||
</xsl:template>
|
||||
|
||||
</xsl:stylesheet>
|
|
@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.dedup;
|
|||
import java.util.*;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import org.apache.commons.beanutils.BeanUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
@ -14,6 +15,7 @@ import org.apache.spark.sql.*;
|
|||
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
|
@ -23,190 +25,190 @@ import scala.Tuple3;
|
|||
import scala.collection.JavaConversions;
|
||||
|
||||
public class DedupRecordFactory {
|
||||
public static final class DedupRecordReduceState {
|
||||
public final String dedupId;
|
||||
public static final class DedupRecordReduceState {
|
||||
public final String dedupId;
|
||||
|
||||
public final ArrayList<String> aliases = new ArrayList<>();
|
||||
public final ArrayList<String> aliases = new ArrayList<>();
|
||||
|
||||
public final HashSet<String> acceptanceDate = new HashSet<>();
|
||||
public final HashSet<String> acceptanceDate = new HashSet<>();
|
||||
|
||||
public OafEntity entity;
|
||||
public OafEntity entity;
|
||||
|
||||
public DedupRecordReduceState(String dedupId, String id, OafEntity entity) {
|
||||
this.dedupId = dedupId;
|
||||
this.entity = entity;
|
||||
if (entity == null) {
|
||||
aliases.add(id);
|
||||
} else {
|
||||
if (Result.class.isAssignableFrom(entity.getClass())) {
|
||||
Result result = (Result) entity;
|
||||
if (result.getDateofacceptance() != null
|
||||
&& StringUtils.isNotBlank(result.getDateofacceptance().getValue())) {
|
||||
acceptanceDate.add(result.getDateofacceptance().getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
public DedupRecordReduceState(String dedupId, String id, OafEntity entity) {
|
||||
this.dedupId = dedupId;
|
||||
this.entity = entity;
|
||||
if (entity == null) {
|
||||
aliases.add(id);
|
||||
} else {
|
||||
if (Result.class.isAssignableFrom(entity.getClass())) {
|
||||
Result result = (Result) entity;
|
||||
if (result.getDateofacceptance() != null
|
||||
&& StringUtils.isNotBlank(result.getDateofacceptance().getValue())) {
|
||||
acceptanceDate.add(result.getDateofacceptance().getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public String getDedupId() {
|
||||
return dedupId;
|
||||
}
|
||||
}
|
||||
public String getDedupId() {
|
||||
return dedupId;
|
||||
}
|
||||
}
|
||||
|
||||
private static final int MAX_ACCEPTANCE_DATE = 20;
|
||||
private static final int MAX_ACCEPTANCE_DATE = 20;
|
||||
|
||||
private DedupRecordFactory() {
|
||||
}
|
||||
private DedupRecordFactory() {
|
||||
}
|
||||
|
||||
public static Dataset<OafEntity> createDedupRecord(
|
||||
final SparkSession spark,
|
||||
final DataInfo dataInfo,
|
||||
final String mergeRelsInputPath,
|
||||
final String entitiesInputPath,
|
||||
final Class<OafEntity> clazz) {
|
||||
public static Dataset<OafEntity> createDedupRecord(
|
||||
final SparkSession spark,
|
||||
final DataInfo dataInfo,
|
||||
final String mergeRelsInputPath,
|
||||
final String entitiesInputPath,
|
||||
final Class<OafEntity> clazz) {
|
||||
|
||||
final long ts = System.currentTimeMillis();
|
||||
final Encoder<OafEntity> beanEncoder = Encoders.bean(clazz);
|
||||
final Encoder<OafEntity> kryoEncoder = Encoders.kryo(clazz);
|
||||
final long ts = System.currentTimeMillis();
|
||||
final Encoder<OafEntity> beanEncoder = Encoders.bean(clazz);
|
||||
final Encoder<OafEntity> kryoEncoder = Encoders.kryo(clazz);
|
||||
|
||||
// <id, json_entity>
|
||||
Dataset<Row> entities = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(clazz).schema())
|
||||
.json(entitiesInputPath)
|
||||
.as(beanEncoder)
|
||||
.map(
|
||||
(MapFunction<OafEntity, Tuple2<String, OafEntity>>) entity -> {
|
||||
return new Tuple2<>(entity.getId(), entity);
|
||||
},
|
||||
Encoders.tuple(Encoders.STRING(), kryoEncoder))
|
||||
.selectExpr("_1 AS id", "_2 AS kryoObject");
|
||||
// <id, json_entity>
|
||||
Dataset<Row> entities = spark
|
||||
.read()
|
||||
.schema(Encoders.bean(clazz).schema())
|
||||
.json(entitiesInputPath)
|
||||
.as(beanEncoder)
|
||||
.map(
|
||||
(MapFunction<OafEntity, Tuple2<String, OafEntity>>) entity -> {
|
||||
return new Tuple2<>(entity.getId(), entity);
|
||||
},
|
||||
Encoders.tuple(Encoders.STRING(), kryoEncoder))
|
||||
.selectExpr("_1 AS id", "_2 AS kryoObject");
|
||||
|
||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||
Dataset<Row> mergeRels = spark
|
||||
.read()
|
||||
.load(mergeRelsInputPath)
|
||||
.where("relClass == 'merges'")
|
||||
.selectExpr("source as dedupId", "target as id");
|
||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||
Dataset<Row> mergeRels = spark
|
||||
.read()
|
||||
.load(mergeRelsInputPath)
|
||||
.where("relClass == 'merges'")
|
||||
.selectExpr("source as dedupId", "target as id");
|
||||
|
||||
return mergeRels
|
||||
.join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
||||
.select("dedupId", "id", "kryoObject")
|
||||
.as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder))
|
||||
.map(
|
||||
(MapFunction<Tuple3<String, String, OafEntity>, DedupRecordReduceState>) t -> new DedupRecordReduceState(
|
||||
t._1(), t._2(), t._3()),
|
||||
Encoders.kryo(DedupRecordReduceState.class))
|
||||
.groupByKey(
|
||||
(MapFunction<DedupRecordReduceState, String>) DedupRecordReduceState::getDedupId, Encoders.STRING())
|
||||
.reduceGroups(
|
||||
(ReduceFunction<DedupRecordReduceState>) (t1, t2) -> {
|
||||
if (t1.entity == null) {
|
||||
t2.aliases.addAll(t1.aliases);
|
||||
return t2;
|
||||
}
|
||||
if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) {
|
||||
t1.acceptanceDate.addAll(t2.acceptanceDate);
|
||||
}
|
||||
t1.aliases.addAll(t2.aliases);
|
||||
t1.entity = reduceEntity(t1.entity, t2.entity);
|
||||
return mergeRels
|
||||
.join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left")
|
||||
.select("dedupId", "id", "kryoObject")
|
||||
.as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder))
|
||||
.map(
|
||||
(MapFunction<Tuple3<String, String, OafEntity>, DedupRecordReduceState>) t -> new DedupRecordReduceState(
|
||||
t._1(), t._2(), t._3()),
|
||||
Encoders.kryo(DedupRecordReduceState.class))
|
||||
.groupByKey(
|
||||
(MapFunction<DedupRecordReduceState, String>) DedupRecordReduceState::getDedupId, Encoders.STRING())
|
||||
.reduceGroups(
|
||||
(ReduceFunction<DedupRecordReduceState>) (t1, t2) -> {
|
||||
if (t1.entity == null) {
|
||||
t2.aliases.addAll(t1.aliases);
|
||||
return t2;
|
||||
}
|
||||
if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) {
|
||||
t1.acceptanceDate.addAll(t2.acceptanceDate);
|
||||
}
|
||||
t1.aliases.addAll(t2.aliases);
|
||||
t1.entity = reduceEntity(t1.entity, t2.entity);
|
||||
|
||||
return t1;
|
||||
})
|
||||
.flatMap((FlatMapFunction<Tuple2<String, DedupRecordReduceState>, OafEntity>) t -> {
|
||||
String dedupId = t._1();
|
||||
DedupRecordReduceState agg = t._2();
|
||||
return t1;
|
||||
})
|
||||
.flatMap((FlatMapFunction<Tuple2<String, DedupRecordReduceState>, OafEntity>) t -> {
|
||||
String dedupId = t._1();
|
||||
DedupRecordReduceState agg = t._2();
|
||||
|
||||
if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
|
||||
return Stream
|
||||
.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);
|
||||
}
|
||||
return Stream
|
||||
.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 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 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) {
|
||||
private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) {
|
||||
|
||||
if (duplicate == null) {
|
||||
return entity;
|
||||
}
|
||||
if (duplicate == null) {
|
||||
return entity;
|
||||
}
|
||||
|
||||
int compare = new IdentifierComparator<>()
|
||||
.compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate));
|
||||
int compare = new IdentifierComparator<>()
|
||||
.compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate));
|
||||
|
||||
if (compare > 0) {
|
||||
OafEntity swap = duplicate;
|
||||
duplicate = entity;
|
||||
entity = swap;
|
||||
}
|
||||
if (compare > 0) {
|
||||
OafEntity swap = duplicate;
|
||||
duplicate = entity;
|
||||
entity = swap;
|
||||
}
|
||||
|
||||
entity.mergeFrom(duplicate);
|
||||
entity = MergeUtils.checkedMerge(entity, duplicate);
|
||||
|
||||
if (ModelSupport.isSubClass(duplicate, Result.class)) {
|
||||
Result re = (Result) entity;
|
||||
Result rd = (Result) duplicate;
|
||||
if (ModelSupport.isSubClass(duplicate, Result.class)) {
|
||||
Result re = (Result) entity;
|
||||
Result rd = (Result) duplicate;
|
||||
|
||||
List<List<Author>> authors = new ArrayList<>();
|
||||
if (re.getAuthor() != null) {
|
||||
authors.add(re.getAuthor());
|
||||
}
|
||||
if (rd.getAuthor() != null) {
|
||||
authors.add(rd.getAuthor());
|
||||
}
|
||||
List<List<Author>> authors = new ArrayList<>();
|
||||
if (re.getAuthor() != null) {
|
||||
authors.add(re.getAuthor());
|
||||
}
|
||||
if (rd.getAuthor() != null) {
|
||||
authors.add(rd.getAuthor());
|
||||
}
|
||||
|
||||
re.setAuthor(AuthorMerger.merge(authors));
|
||||
}
|
||||
re.setAuthor(AuthorMerger.merge(authors));
|
||||
}
|
||||
|
||||
return entity;
|
||||
}
|
||||
return entity;
|
||||
}
|
||||
|
||||
public static <T extends OafEntity> T entityMerger(
|
||||
String id, Iterator<Tuple2<String, T>> entities, long ts, DataInfo dataInfo, Class<T> clazz) {
|
||||
T base = entities.next()._2();
|
||||
public static <T extends OafEntity> T entityMerger(
|
||||
String id, Iterator<Tuple2<String, T>> entities, long ts, DataInfo dataInfo, Class<T> clazz) {
|
||||
T base = entities.next()._2();
|
||||
|
||||
while (entities.hasNext()) {
|
||||
T duplicate = entities.next()._2();
|
||||
if (duplicate != null)
|
||||
base = (T) reduceEntity(base, duplicate);
|
||||
}
|
||||
while (entities.hasNext()) {
|
||||
T duplicate = entities.next()._2();
|
||||
if (duplicate != null)
|
||||
base = (T) reduceEntity(base, duplicate);
|
||||
}
|
||||
|
||||
base.setId(id);
|
||||
base.setDataInfo(dataInfo);
|
||||
base.setLastupdatetimestamp(ts);
|
||||
base.setId(id);
|
||||
base.setDataInfo(dataInfo);
|
||||
base.setLastupdatetimestamp(ts);
|
||||
|
||||
return base;
|
||||
}
|
||||
return base;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup;
|
|||
|
||||
import static org.apache.spark.sql.functions.col;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
|
@ -127,10 +128,8 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
|||
(MapFunction<Relation, String>) r -> String
|
||||
.join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||
Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<Relation>) (b, a) -> {
|
||||
b.mergeFrom(a);
|
||||
return b;
|
||||
})
|
||||
.reduceGroups((ReduceFunction<Relation>) MergeUtils::mergeRelation
|
||||
)
|
||||
.map((MapFunction<Tuple2<String, Relation>, Relation>) Tuple2::_2, REL_BEAN_ENC);
|
||||
|
||||
final String outputRelationPath = graphOutputPath + "/relation";
|
||||
|
|
|
@ -3,17 +3,18 @@ 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.utils.MergeUtils
|
||||
import eu.dnetlib.dhp.schema.oaf.{Organization, Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.doiboost.mag.ConversionUtil
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql._
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||
import org.json4s.jackson.JsonMethods.parse
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object SparkGenerateDoiBoost {
|
||||
|
@ -78,8 +79,10 @@ object SparkGenerateDoiBoost {
|
|||
if (item._2 != null) {
|
||||
val otherPub = item._2._2
|
||||
if (otherPub != null) {
|
||||
crossrefPub.mergeFrom(otherPub)
|
||||
crossrefPub.setAuthor(AuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor))
|
||||
val mergedAuthor = AuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor)
|
||||
val res = MergeUtils.mergePublication(crossrefPub, otherPub)
|
||||
res.setAuthor(mergedAuthor);
|
||||
return res
|
||||
}
|
||||
}
|
||||
crossrefPub
|
||||
|
@ -130,14 +133,13 @@ object SparkGenerateDoiBoost {
|
|||
// So we have to merge
|
||||
val b1 = left._2
|
||||
val b2 = right._2
|
||||
b1.mergeFrom(b2)
|
||||
b1.mergeOAFDataInfo(b2)
|
||||
val authors = AuthorMerger.mergeAuthor(b1.getAuthor, b2.getAuthor)
|
||||
b1.setAuthor(authors)
|
||||
val merged = MergeUtils.mergePublication(b1, b2);
|
||||
merged.setAuthor(authors)
|
||||
if (b2.getId != null && b2.getId.nonEmpty)
|
||||
b1.setId(b2.getId)
|
||||
merged.setId(b2.getId)
|
||||
//Return publication Merged
|
||||
(b1.getId, b1)
|
||||
(merged.getId, merged)
|
||||
}
|
||||
} else {
|
||||
// Left is Null so we return right
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package eu.dnetlib.doiboost.mag
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory
|
||||
import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, StructuredProperty, Subject}
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, MergeUtils}
|
||||
import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, Subject}
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil
|
||||
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
|
||||
import org.json4s
|
||||
|
@ -142,8 +142,7 @@ case object ConversionUtil {
|
|||
|
||||
def mergePublication(a: Publication, b: Publication): Publication = {
|
||||
if ((a != null) && (b != null)) {
|
||||
a.mergeFrom(b)
|
||||
a
|
||||
MergeUtils.mergePublication(a, b)
|
||||
} else {
|
||||
if (a == null) b else a
|
||||
}
|
||||
|
|
|
@ -0,0 +1,39 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.actions;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 22/01/24
|
||||
*/
|
||||
public class Action implements Serializable {
|
||||
private String clazz;
|
||||
private String method;
|
||||
private List<Parameters> params;
|
||||
|
||||
public String getClazz() {
|
||||
return clazz;
|
||||
}
|
||||
|
||||
public void setClazz(String clazz) {
|
||||
this.clazz = clazz;
|
||||
}
|
||||
|
||||
public String getMethod() {
|
||||
return method;
|
||||
}
|
||||
|
||||
public void setMethod(String method) {
|
||||
this.method = method;
|
||||
}
|
||||
|
||||
public List<Parameters> getParams() {
|
||||
return params;
|
||||
}
|
||||
|
||||
public void setParams(List<Parameters> params) {
|
||||
this.params = params;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.actions;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 19/01/24
|
||||
*/
|
||||
public class ExecSubstringAction implements Serializable {
|
||||
|
||||
private String value;
|
||||
private String from;
|
||||
private String to;
|
||||
|
||||
public String getValue() {
|
||||
return value;
|
||||
}
|
||||
|
||||
public void setValue(String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public String getFrom() {
|
||||
return from;
|
||||
}
|
||||
|
||||
public void setFrom(String from) {
|
||||
this.from = from;
|
||||
}
|
||||
|
||||
public String getTo() {
|
||||
return to;
|
||||
}
|
||||
|
||||
public void setTo(String to) {
|
||||
this.to = to;
|
||||
}
|
||||
|
||||
public String execSubstring() {
|
||||
|
||||
return this.value.substring(Integer.valueOf(this.from), Integer.valueOf(this.to));
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.actions;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 22/01/24
|
||||
*/
|
||||
public class MapModel implements Serializable {
|
||||
|
||||
private String path;
|
||||
private Action action;
|
||||
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public void setPath(String path) {
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
public Action getAction() {
|
||||
return action;
|
||||
}
|
||||
|
||||
public void setAction(Action action) {
|
||||
this.action = action;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.actions;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 22/01/24
|
||||
*/
|
||||
public class Parameters implements Serializable {
|
||||
private String paramName;
|
||||
private String paramValue;
|
||||
|
||||
public String getParamName() {
|
||||
return paramName;
|
||||
}
|
||||
|
||||
public void setParamName(String paramName) {
|
||||
this.paramName = paramName;
|
||||
}
|
||||
|
||||
public String getParamValue() {
|
||||
return paramValue;
|
||||
}
|
||||
|
||||
public void setParamValue(String paramValue) {
|
||||
this.paramValue = paramValue;
|
||||
}
|
||||
}
|
|
@ -4,7 +4,9 @@ package eu.dnetlib.dhp.bulktag.community;
|
|||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
|
||||
public class ProtoMap extends HashMap<String, String> implements Serializable {
|
||||
import eu.dnetlib.dhp.bulktag.actions.MapModel;
|
||||
|
||||
public class ProtoMap extends HashMap<String, MapModel> implements Serializable {
|
||||
|
||||
public ProtoMap() {
|
||||
super();
|
||||
|
|
|
@ -5,6 +5,8 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.*;
|
|||
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
@ -15,7 +17,10 @@ import org.slf4j.LoggerFactory;
|
|||
import com.google.gson.Gson;
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.PathNotFoundException;
|
||||
|
||||
import eu.dnetlib.dhp.bulktag.actions.MapModel;
|
||||
import eu.dnetlib.dhp.bulktag.actions.Parameters;
|
||||
import eu.dnetlib.dhp.bulktag.eosc.EoscIFTag;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
|
@ -35,27 +40,59 @@ public class ResultTagger implements Serializable {
|
|||
return (tmp != clist.size());
|
||||
}
|
||||
|
||||
private Map<String, List<String>> getParamMap(final Result result, Map<String, String> params) {
|
||||
private Map<String, List<String>> getParamMap(final Result result, Map<String, MapModel> params)
|
||||
throws NoSuchMethodException, InvocationTargetException {
|
||||
Map<String, List<String>> param = new HashMap<>();
|
||||
String json = new Gson().toJson(result, Result.class);
|
||||
DocumentContext jsonContext = JsonPath.parse(json);
|
||||
|
||||
if (params == null) {
|
||||
params = new HashMap<>();
|
||||
}
|
||||
for (String key : params.keySet()) {
|
||||
MapModel mapModel = params.get(key);
|
||||
|
||||
try {
|
||||
param.put(key, jsonContext.read(params.get(key)));
|
||||
} catch (com.jayway.jsonpath.PathNotFoundException e) {
|
||||
String path = mapModel.getPath();
|
||||
Object obj = jsonContext.read(path);
|
||||
List<String> pathValue;
|
||||
if (obj instanceof java.lang.String)
|
||||
pathValue = Arrays.asList((String) obj);
|
||||
else
|
||||
pathValue = (List<String>) obj;
|
||||
if (Optional.ofNullable(mapModel.getAction()).isPresent()) {
|
||||
Class<?> c = Class.forName(mapModel.getAction().getClazz());
|
||||
Object class_instance = c.newInstance();
|
||||
Method setField = c.getMethod("setValue", String.class);
|
||||
setField.invoke(class_instance, pathValue.get(0));
|
||||
for (Parameters p : mapModel.getAction().getParams()) {
|
||||
setField = c.getMethod("set" + p.getParamName(), String.class);
|
||||
setField.invoke(class_instance, p.getParamValue());
|
||||
}
|
||||
|
||||
param
|
||||
.put(
|
||||
key, Arrays
|
||||
.asList((String) c.getMethod(mapModel.getAction().getMethod()).invoke(class_instance)));
|
||||
|
||||
}
|
||||
|
||||
else {
|
||||
param.put(key, pathValue);
|
||||
}
|
||||
|
||||
} catch (PathNotFoundException | ClassNotFoundException | InstantiationException
|
||||
| IllegalAccessException e) {
|
||||
param.put(key, new ArrayList<>());
|
||||
}
|
||||
}
|
||||
return param;
|
||||
|
||||
}
|
||||
|
||||
public <R extends Result> R enrichContextCriteria(
|
||||
final R result, final CommunityConfiguration conf, final Map<String, String> criteria) {
|
||||
|
||||
final Map<String, List<String>> param = getParamMap(result, criteria);
|
||||
final R result, final CommunityConfiguration conf, final Map<String, MapModel> criteria)
|
||||
throws InvocationTargetException, NoSuchMethodException {
|
||||
|
||||
// Verify if the entity is deletedbyinference. In case verify if to clean the context list
|
||||
// from all the zenodo communities
|
||||
|
@ -64,6 +101,8 @@ public class ResultTagger implements Serializable {
|
|||
return result;
|
||||
}
|
||||
|
||||
final Map<String, List<String>> param = getParamMap(result, criteria);
|
||||
|
||||
// Execute the EOSCTag for the services
|
||||
switch (result.getResulttype().getClassid()) {
|
||||
case PUBLICATION_RESULTTYPE_CLASSID:
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.criteria;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 11/11/22
|
||||
*/
|
||||
@VerbClass("greater_than")
|
||||
public class GreatThanVerb implements Selection, Serializable {
|
||||
|
||||
private String param;
|
||||
|
||||
public GreatThanVerb() {
|
||||
}
|
||||
|
||||
public GreatThanVerb(final String param) {
|
||||
this.param = param;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean apply(String value) {
|
||||
return value.compareTo(param) > 0;
|
||||
}
|
||||
|
||||
public String getParam() {
|
||||
return param;
|
||||
}
|
||||
|
||||
public void setParam(String param) {
|
||||
this.param = param;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
|
||||
package eu.dnetlib.dhp.bulktag.criteria;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* @author miriam.baglioni
|
||||
* @Date 11/11/22
|
||||
*/
|
||||
@VerbClass("lesser_than")
|
||||
public class LessThanVerb implements Selection, Serializable {
|
||||
|
||||
private String param;
|
||||
|
||||
public LessThanVerb() {
|
||||
}
|
||||
|
||||
public LessThanVerb(final String param) {
|
||||
this.param = param;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean apply(String value) {
|
||||
return value.compareTo(param) < 0;
|
||||
}
|
||||
|
||||
public String getParam() {
|
||||
return param;
|
||||
}
|
||||
|
||||
public void setParam(String param) {
|
||||
this.param = param;
|
||||
}
|
||||
}
|
|
@ -149,7 +149,7 @@ public class SparkResultToCommunityFromOrganizationJob {
|
|||
}
|
||||
}
|
||||
// res.setContext(propagatedContexts);
|
||||
// ret.mergeFrom(res);
|
||||
// return MergeUtils.mergeResult(ret, res);
|
||||
}
|
||||
return ret;
|
||||
};
|
||||
|
|
|
@ -13,6 +13,7 @@ import java.util.List;
|
|||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
|
@ -24,8 +25,6 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList;
|
||||
import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.Context;
|
||||
|
@ -162,7 +161,7 @@ public class SparkResultToCommunityFromProject implements Serializable {
|
|||
}
|
||||
}
|
||||
res.setContext(propagatedContexts);
|
||||
ret.mergeFrom(res);
|
||||
return MergeUtils.checkedMerge(ret, res);
|
||||
}
|
||||
return ret;
|
||||
};
|
||||
|
|
|
@ -25,118 +25,118 @@ import scala.Tuple2;
|
|||
|
||||
public class SparkResultToCommunityThroughSemRelJob {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class);
|
||||
private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
SparkResultToCommunityThroughSemRelJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"));
|
||||
String jsonConfiguration = IOUtils
|
||||
.toString(
|
||||
SparkResultToCommunityThroughSemRelJob.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"));
|
||||
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||
|
||||
parser.parseArgument(args);
|
||||
parser.parseArgument(args);
|
||||
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
|
||||
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||
|
||||
String inputPath = parser.get("sourcePath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
String inputPath = parser.get("sourcePath");
|
||||
log.info("inputPath: {}", inputPath);
|
||||
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
final String outputPath = parser.get("outputPath");
|
||||
log.info("outputPath: {}", outputPath);
|
||||
|
||||
final String preparedInfoPath = parser.get("preparedInfoPath");
|
||||
log.info("preparedInfoPath: {}", preparedInfoPath);
|
||||
final String preparedInfoPath = parser.get("preparedInfoPath");
|
||||
log.info("preparedInfoPath: {}", preparedInfoPath);
|
||||
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
|
||||
SparkConf conf = new SparkConf();
|
||||
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
|
||||
|
||||
final String resultClassName = parser.get("resultTableName");
|
||||
log.info("resultTableName: {}", resultClassName);
|
||||
final String resultClassName = parser.get("resultTableName");
|
||||
log.info("resultTableName: {}", resultClassName);
|
||||
|
||||
final Boolean saveGraph = Optional
|
||||
.ofNullable(parser.get("saveGraph"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("saveGraph: {}", saveGraph);
|
||||
final Boolean saveGraph = Optional
|
||||
.ofNullable(parser.get("saveGraph"))
|
||||
.map(Boolean::valueOf)
|
||||
.orElse(Boolean.TRUE);
|
||||
log.info("saveGraph: {}", saveGraph);
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
||||
@SuppressWarnings("unchecked")
|
||||
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
|
||||
|
||||
runWithSparkHiveSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
if (isTest(parser)) {
|
||||
removeOutputDir(spark, outputPath);
|
||||
}
|
||||
if (saveGraph) {
|
||||
execPropagation(
|
||||
spark, inputPath, outputPath, preparedInfoPath, resultClazz);
|
||||
}
|
||||
});
|
||||
}
|
||||
runWithSparkHiveSession(
|
||||
conf,
|
||||
isSparkSessionManaged,
|
||||
spark -> {
|
||||
if (isTest(parser)) {
|
||||
removeOutputDir(spark, outputPath);
|
||||
}
|
||||
if (saveGraph) {
|
||||
execPropagation(
|
||||
spark, inputPath, outputPath, preparedInfoPath, resultClazz);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static <R extends Result> void execPropagation(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String outputPath,
|
||||
String preparedInfoPath,
|
||||
Class<R> resultClazz) {
|
||||
private static <R extends Result> void execPropagation(
|
||||
SparkSession spark,
|
||||
String inputPath,
|
||||
String outputPath,
|
||||
String preparedInfoPath,
|
||||
Class<R> resultClazz) {
|
||||
|
||||
Dataset<ResultCommunityList> possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class);
|
||||
Dataset<R> result = readPath(spark, inputPath, resultClazz);
|
||||
Dataset<ResultCommunityList> possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class);
|
||||
Dataset<R> result = readPath(spark, inputPath, resultClazz);
|
||||
|
||||
result
|
||||
.joinWith(
|
||||
possibleUpdates,
|
||||
result.col("id").equalTo(possibleUpdates.col("resultId")),
|
||||
"left_outer")
|
||||
.map(contextUpdaterFn(), Encoders.bean(resultClazz))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath);
|
||||
result
|
||||
.joinWith(
|
||||
possibleUpdates,
|
||||
result.col("id").equalTo(possibleUpdates.col("resultId")),
|
||||
"left_outer")
|
||||
.map(contextUpdaterFn(), Encoders.bean(resultClazz))
|
||||
.write()
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
private static <R extends Result> MapFunction<Tuple2<R, ResultCommunityList>, R> contextUpdaterFn() {
|
||||
return value -> {
|
||||
R ret = value._1();
|
||||
Optional<ResultCommunityList> rcl = Optional.ofNullable(value._2());
|
||||
if (rcl.isPresent()) {
|
||||
Set<String> contexts = new HashSet<>();
|
||||
ret.getContext().forEach(c -> contexts.add(c.getId()));
|
||||
rcl
|
||||
.get()
|
||||
.getCommunityList()
|
||||
.stream()
|
||||
.forEach(
|
||||
c -> {
|
||||
if (!contexts.contains(c)) {
|
||||
Context newContext = new Context();
|
||||
newContext.setId(c);
|
||||
newContext
|
||||
.setDataInfo(
|
||||
Arrays
|
||||
.asList(
|
||||
getDataInfo(
|
||||
PROPAGATION_DATA_INFO_TYPE,
|
||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID,
|
||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS)));
|
||||
ret.getContext().add(newContext);
|
||||
}
|
||||
private static <R extends Result> MapFunction<Tuple2<R, ResultCommunityList>, R> contextUpdaterFn() {
|
||||
return value -> {
|
||||
R ret = value._1();
|
||||
Optional<ResultCommunityList> rcl = Optional.ofNullable(value._2());
|
||||
if (rcl.isPresent()) {
|
||||
Set<String> contexts = new HashSet<>();
|
||||
ret.getContext().forEach(c -> contexts.add(c.getId()));
|
||||
rcl
|
||||
.get()
|
||||
.getCommunityList()
|
||||
.stream()
|
||||
.forEach(
|
||||
c -> {
|
||||
if (!contexts.contains(c)) {
|
||||
Context newContext = new Context();
|
||||
newContext.setId(c);
|
||||
newContext
|
||||
.setDataInfo(
|
||||
Arrays
|
||||
.asList(
|
||||
getDataInfo(
|
||||
PROPAGATION_DATA_INFO_TYPE,
|
||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID,
|
||||
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME,
|
||||
ModelConstants.DNET_PROVENANCE_ACTIONS)));
|
||||
ret.getContext().add(newContext);
|
||||
}
|
||||
|
||||
});
|
||||
});
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
};
|
||||
}
|
||||
return ret;
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -13,6 +13,7 @@ import org.apache.commons.io.IOUtils;
|
|||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
@ -31,18 +32,26 @@ public class BulkTagJobTest {
|
|||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static final String pathMap = "{ \"author\" : \"$['author'][*]['fullname']\","
|
||||
+ " \"title\" : \"$['title'][*]['value']\","
|
||||
+ " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\","
|
||||
+ " \"contributor\" : \"$['contributor'][*]['value']\","
|
||||
+ " \"description\" : \"$['description'][*]['value']\", "
|
||||
+ " \"subject\" :\"$['subject'][*]['value']\" , " +
|
||||
"\"fos\" : \"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"," +
|
||||
"\"sdg\" : \"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"," +
|
||||
"\"hostedby\" : \"$['instance'][*]['hostedby']['key']\" , " +
|
||||
"\"collectedfrom\" : \"$['instance'][*]['collectedfrom']['key']\"," +
|
||||
"\"publisher\":\"$['publisher'].value\"," +
|
||||
"\"publicationyear\":\"$['dateofacceptance'].value\"} ";
|
||||
public static final String pathMap = "{\"author\":{\"path\":\"$['author'][*]['fullname']\"}," +
|
||||
" \"title\":{\"path\":\"$['title'][*]['value']\"}, " +
|
||||
" \"orcid\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']\"} , " +
|
||||
" \"orcid_pending\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']\"} ,"
|
||||
+
|
||||
"\"contributor\" : {\"path\":\"$['contributor'][*]['value']\"}," +
|
||||
" \"description\" : {\"path\":\"$['description'][*]['value']\"}," +
|
||||
" \"subject\" :{\"path\":\"$['subject'][*]['value']\"}, " +
|
||||
" \"fos\" : {\"path\":\"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"} , " +
|
||||
"\"sdg\" : {\"path\":\"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"}," +
|
||||
"\"journal\":{\"path\":\"$['journal'].name\"}," +
|
||||
"\"hostedby\":{\"path\":\"$['instance'][*]['hostedby']['key']\"}," +
|
||||
"\"collectedfrom\":{\"path\":\"$['instance'][*]['collectedfrom']['key']\"}," +
|
||||
"\"publisher\":{\"path\":\"$['publisher'].value\"}," +
|
||||
"\"publicationyear\":{\"path\":\"$['dateofacceptance'].value\", " +
|
||||
" \"action\":{\"clazz\":\"eu.dnetlib.dhp.bulktag.actions.ExecSubstringAction\"," +
|
||||
"\"method\":\"execSubstring\"," +
|
||||
"\"params\":[" +
|
||||
"{\"paramName\":\"From\", \"paramValue\":0}, " +
|
||||
"{\"paramName\":\"To\",\"paramValue\":4}]}}}";
|
||||
|
||||
private static SparkSession spark;
|
||||
|
||||
|
@ -1600,4 +1609,94 @@ public class BulkTagJobTest {
|
|||
Assertions.assertEquals(0, spark.sql(query).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
void pubdateTest() throws Exception {
|
||||
|
||||
final String pathMap = BulkTagJobTest.pathMap;
|
||||
SparkBulkTagJob
|
||||
.main(
|
||||
new String[] {
|
||||
"-isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"-sourcePath",
|
||||
getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/publicationyear/").getPath(),
|
||||
"-taggingConf",
|
||||
IOUtils
|
||||
.toString(
|
||||
BulkTagJobTest.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_publicationdate.xml")),
|
||||
"-outputPath", workingDir.toString() + "/",
|
||||
"-pathMap", pathMap
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
|
||||
JavaRDD<Dataset> tmp = sc
|
||||
.textFile(workingDir.toString() + "/dataset")
|
||||
.map(item -> OBJECT_MAPPER.readValue(item, Dataset.class));
|
||||
|
||||
Assertions.assertEquals(10, tmp.count());
|
||||
org.apache.spark.sql.Dataset<Dataset> verificationDataset = spark
|
||||
.createDataset(tmp.rdd(), Encoders.bean(Dataset.class));
|
||||
|
||||
verificationDataset.createOrReplaceTempView("dataset");
|
||||
|
||||
String query = "select id, MyT.id community, MyD.provenanceaction.classid "
|
||||
+ "from dataset "
|
||||
+ "lateral view explode(context) c as MyT "
|
||||
+ "lateral view explode(MyT.datainfo) d as MyD "
|
||||
+ "where MyD.inferenceprovenance = 'bulktagging'";
|
||||
|
||||
org.apache.spark.sql.Dataset<Row> queryResult = spark.sql(query);
|
||||
queryResult.show(false);
|
||||
Assertions.assertEquals(5, queryResult.count());
|
||||
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1,
|
||||
queryResult
|
||||
.filter(
|
||||
(FilterFunction<Row>) r -> r
|
||||
.getAs("id")
|
||||
.equals("50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529"))
|
||||
.count());
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1,
|
||||
queryResult
|
||||
.filter(
|
||||
(FilterFunction<Row>) r -> r
|
||||
.getAs("id")
|
||||
.equals("50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf"))
|
||||
.count());
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1,
|
||||
queryResult
|
||||
.filter(
|
||||
(FilterFunction<Row>) r -> r
|
||||
.getAs("id")
|
||||
.equals("50|od______3989::7fcbe3a03280663cddebfd3cb9203177"))
|
||||
.count());
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1,
|
||||
queryResult
|
||||
.filter(
|
||||
(FilterFunction<Row>) r -> r
|
||||
.getAs("id")
|
||||
.equals("50|od______3989::d791339867bec6d3eb2104deeb4e4961"))
|
||||
.count());
|
||||
Assertions
|
||||
.assertEquals(
|
||||
1,
|
||||
queryResult
|
||||
.filter(
|
||||
(FilterFunction<Row>) r -> r
|
||||
.getAs("id")
|
||||
.equals("50|od______3989::d90d3a1f64ad264b5ebed8a35b280343"))
|
||||
.count());
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because one or more lines are too long
|
@ -7,6 +7,7 @@ import java.util.*;
|
|||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.function.FilterFunction;
|
||||
|
@ -24,7 +25,6 @@ import eu.dnetlib.dhp.common.HdfsSupport;
|
|||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
|
@ -248,11 +248,11 @@ public class MergeGraphTableSparkJob {
|
|||
private T mergeAndGet(T b, T a) {
|
||||
if (Objects.nonNull(a) && Objects.nonNull(b)) {
|
||||
if (ModelSupport.isSubClass(a, OafEntity.class) && ModelSupport.isSubClass(b, OafEntity.class)) {
|
||||
return (T) OafMapperUtils.mergeEntities((OafEntity) b, (OafEntity) a);
|
||||
return (T) MergeUtils.merge(b, a);
|
||||
}
|
||||
if (a instanceof Relation && b instanceof Relation) {
|
||||
((Relation) a).mergeFrom(b);
|
||||
return a;
|
||||
return (T) MergeUtils.mergeRelation((Relation)a, (Relation) b);
|
||||
}
|
||||
}
|
||||
return Objects.isNull(a) ? b : a;
|
||||
|
|
|
@ -1,14 +1,16 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.graph.raw;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
@ -16,27 +18,18 @@ 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.rdd.RDD;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.dom4j.DocumentException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
|
||||
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||
|
||||
public class GenerateEntitiesApplication extends AbstractMigrationApplication {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class);
|
||||
|
@ -137,7 +130,7 @@ public class GenerateEntitiesApplication extends AbstractMigrationApplication {
|
|||
save(
|
||||
inputRdd
|
||||
.mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf))
|
||||
.reduceByKey(OafMapperUtils::merge)
|
||||
.reduceByKey(MergeUtils::merge)
|
||||
.map(Tuple2::_2),
|
||||
targetPath);
|
||||
break;
|
||||
|
|
|
@ -12,6 +12,16 @@
|
|||
<name>targetPath</name>
|
||||
<description>the output path of the graph enriched</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2ExtraListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
|
||||
<description>spark 2.* extra listeners classname</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2SqlQueryExecutionListeners</name>
|
||||
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
|
||||
<description>spark 2.* sql query execution listeners classname</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="EnrichGraph"/>
|
||||
|
@ -31,8 +41,8 @@
|
|||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.executor.memoryOverhead=2g
|
||||
--conf spark.sql.shuffle.partitions=3000
|
||||
--conf spark.executor.memoryOverhead=${sparkExecutorMemory}
|
||||
--conf spark.sql.shuffle.partitions=5000
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
|
|
|
@ -1,40 +0,0 @@
|
|||
package eu.dnetlib.dhp.enrich.orcid
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, Publication}
|
||||
import eu.dnetlib.dhp.schema.sx.OafUtils
|
||||
import org.apache.spark.sql.Row
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object AuthorEnricher extends Serializable {
|
||||
|
||||
def createAuthor(givenName: String, familyName: String, orcid: String): Author = {
|
||||
val a = new Author
|
||||
a.setName(givenName)
|
||||
a.setSurname(familyName)
|
||||
a.setFullname(s"$givenName $familyName")
|
||||
val pid = OafUtils.createSP(orcid, ModelConstants.ORCID, ModelConstants.ORCID)
|
||||
pid.setDataInfo(OafUtils.generateDataInfo())
|
||||
pid.getDataInfo.setProvenanceaction(OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT"))
|
||||
a.setPid(List(pid).asJava)
|
||||
a
|
||||
}
|
||||
|
||||
def toOAFAuthor(r: Row): java.util.List[Author] = {
|
||||
r.getList[Row](1)
|
||||
.asScala
|
||||
.map(s => createAuthor(s.getAs[String]("givenName"), s.getAs[String]("familyName"), s.getAs[String]("orcid")))
|
||||
.toList
|
||||
.asJava
|
||||
}
|
||||
|
||||
// def enrichAuthor(p:Publication,r:Row): Unit = {
|
||||
// val k:Map[String, OAuthor] =r.getList[Row](1).asScala.map(s => (s.getAs[String]("orcid"), OAuthor(s.getAs[String]("givenName") ,s.getAs[String]("familyName") ))).groupBy(_._1).mapValues(_.map(_._2).head)
|
||||
// println(k)
|
||||
//
|
||||
//
|
||||
//
|
||||
// }
|
||||
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
package eu.dnetlib.dhp.enrich.orcid
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, StructuredProperty}
|
||||
import eu.dnetlib.dhp.schema.sx.OafUtils
|
||||
|
||||
import java.util
|
||||
import scala.beans.BeanProperty
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.Breaks.{break, breakable}
|
||||
|
||||
case class ORCIDAuthorEnricherResult(
|
||||
@BeanProperty var id: String,
|
||||
@BeanProperty var enriched_author: java.util.List[Author],
|
||||
@BeanProperty var author_matched: java.util.List[MatchedAuthors],
|
||||
@BeanProperty var author_unmatched: java.util.List[Author],
|
||||
@BeanProperty var orcid_unmatched: java.util.List[OrcidAutor]
|
||||
)
|
||||
|
||||
object ORCIDAuthorEnricher extends Serializable {
|
||||
|
||||
def enrichOrcid(
|
||||
id: String,
|
||||
graph_authors: java.util.List[Author],
|
||||
orcid_authors: java.util.List[OrcidAutor]
|
||||
): ORCIDAuthorEnricherResult = {
|
||||
// Author enriching strategy:
|
||||
// 1) create a copy of graph author list in unmatched_authors
|
||||
// 2) find best match in unmatched_authors, remove it from unmatched_authors and enrich it so
|
||||
// that the enrichment is reflected in graph_authors (they share author instances)
|
||||
// 3) repeat (2) till the end of the list and then with different matching algorithms that have decreasing
|
||||
// trust in their output
|
||||
// At the end unmatched_authors will contain authors not matched with any of the matching algos
|
||||
val unmatched_authors = new util.ArrayList[Author](graph_authors)
|
||||
|
||||
val matches = {
|
||||
// Look after exact fullname match, reconstruct ORCID fullname as givenName + familyName
|
||||
extractAndEnrichMatches(
|
||||
unmatched_authors,
|
||||
orcid_authors,
|
||||
(author, orcid) =>
|
||||
ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.givenName + " " + orcid.familyName),
|
||||
"fullName"
|
||||
) ++
|
||||
// Look after exact reversed fullname match, reconstruct ORCID fullname as familyName + givenName
|
||||
extractAndEnrichMatches(
|
||||
unmatched_authors,
|
||||
orcid_authors,
|
||||
(author, orcid) =>
|
||||
ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.familyName + " " + orcid.givenName),
|
||||
"reversedFullName"
|
||||
) ++
|
||||
// split author names in tokens, order the tokens, then check for matches of full tokens or abbreviations
|
||||
extractAndEnrichMatches(
|
||||
unmatched_authors,
|
||||
orcid_authors,
|
||||
(author, orcid) =>
|
||||
ORCIDAuthorMatchers
|
||||
.matchOrderedTokenAndAbbreviations(author.getFullname, orcid.givenName + " " + orcid.familyName),
|
||||
"orderedTokens"
|
||||
) ++
|
||||
// look after exact matches of ORCID creditName
|
||||
extractAndEnrichMatches(
|
||||
unmatched_authors,
|
||||
orcid_authors,
|
||||
(author, orcid) => ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.creditName),
|
||||
"creditName"
|
||||
) ++
|
||||
// look after exact matches in ORCID otherNames
|
||||
extractAndEnrichMatches(
|
||||
unmatched_authors,
|
||||
orcid_authors,
|
||||
(author, orcid) =>
|
||||
orcid.otherNames != null && ORCIDAuthorMatchers.matchOtherNames(author.getFullname, orcid.otherNames.asScala),
|
||||
"otherNames"
|
||||
)
|
||||
}
|
||||
|
||||
ORCIDAuthorEnricherResult(id, graph_authors, matches.asJava, unmatched_authors, orcid_authors)
|
||||
}
|
||||
|
||||
private def extractAndEnrichMatches(
|
||||
graph_authors: java.util.List[Author],
|
||||
orcid_authors: java.util.List[OrcidAutor],
|
||||
matchingFunc: (Author, OrcidAutor) => Boolean,
|
||||
matchName: String
|
||||
) = {
|
||||
val matched = scala.collection.mutable.ArrayBuffer.empty[MatchedAuthors]
|
||||
|
||||
if (graph_authors != null && !graph_authors.isEmpty) {
|
||||
val ait = graph_authors.iterator
|
||||
|
||||
while (ait.hasNext) {
|
||||
val author = ait.next()
|
||||
val oit = orcid_authors.iterator
|
||||
|
||||
breakable {
|
||||
while (oit.hasNext) {
|
||||
val orcid = oit.next()
|
||||
|
||||
if (matchingFunc(author, orcid)) {
|
||||
ait.remove()
|
||||
oit.remove()
|
||||
matched += MatchedAuthors(author, orcid, matchName)
|
||||
|
||||
if (author.getPid == null) {
|
||||
author.setPid(new util.ArrayList[StructuredProperty]())
|
||||
}
|
||||
|
||||
val orcidPID = OafUtils.createSP(orcid.orcid, ModelConstants.ORCID, ModelConstants.ORCID)
|
||||
orcidPID.setDataInfo(OafUtils.generateDataInfo())
|
||||
orcidPID.getDataInfo.setProvenanceaction(
|
||||
OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT")
|
||||
)
|
||||
|
||||
author.getPid.add(orcidPID)
|
||||
|
||||
break()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
matched
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
package eu.dnetlib.dhp.enrich.orcid
|
||||
|
||||
import java.util.Locale
|
||||
import java.util.regex.Pattern
|
||||
|
||||
object ORCIDAuthorMatchers {
|
||||
val SPLIT_REGEX = Pattern.compile("[\\s,\\.]+")
|
||||
|
||||
val WORD_DIFF = 2
|
||||
|
||||
def matchEqualsIgnoreCase(a1: String, a2: String): Boolean = {
|
||||
if (a1 == null || a2 == null)
|
||||
false
|
||||
else
|
||||
a1 == a2 || a1.toLowerCase(Locale.ROOT).equals(a2.toLowerCase(Locale.ROOT))
|
||||
}
|
||||
|
||||
def matchOtherNames(fullName: String, otherNames: Seq[String]): Boolean = {
|
||||
if (otherNames != null) {
|
||||
otherNames.exists(matchEqualsIgnoreCase(fullName, _))
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
def matchOrderedTokenAndAbbreviations(a1: String, a2: String): Boolean = {
|
||||
val p1: Array[String] = SPLIT_REGEX.split(a1.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted
|
||||
val p2: Array[String] = SPLIT_REGEX.split(a2.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted
|
||||
|
||||
if (p1.length < 2 || p2.length < 2) return false
|
||||
if (Math.abs(p1.length - p2.length) > WORD_DIFF) return false // use alternative comparison algo
|
||||
|
||||
var p1Idx: Int = 0
|
||||
var p2Idx: Int = 0
|
||||
var shortMatches: Int = 0
|
||||
var longMatches: Int = 0
|
||||
while (p1Idx < p1.length && p2Idx < p2.length) {
|
||||
val e1: String = p1(p1Idx)
|
||||
val c1: Char = e1.charAt(0)
|
||||
val e2: String = p2(p2Idx)
|
||||
val c2: Char = e2.charAt(0)
|
||||
if (c1 < c2) p1Idx += 1
|
||||
else if (c1 > c2) p2Idx += 1
|
||||
else {
|
||||
var res: Boolean = false
|
||||
if (e1.length != 1 && e2.length != 1) {
|
||||
res = e1 == e2
|
||||
longMatches += 1
|
||||
} else {
|
||||
res = true
|
||||
shortMatches += 1
|
||||
}
|
||||
if (res) {
|
||||
p1Idx += 1
|
||||
p2Idx += 1
|
||||
} else {
|
||||
val diff: Int = e1.compareTo(e2)
|
||||
if (diff < 0) p1Idx += 1
|
||||
else if (diff > 0) p2Idx += 1
|
||||
}
|
||||
}
|
||||
}
|
||||
longMatches > 0 && (shortMatches + longMatches) == Math.min(p1.length, p2.length)
|
||||
}
|
||||
}
|
|
@ -1,14 +1,39 @@
|
|||
package eu.dnetlib.dhp.enrich.orcid
|
||||
|
||||
import eu.dnetlib.dhp.application.AbstractScalaApplication
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger
|
||||
import eu.dnetlib.dhp.schema.common.ModelSupport
|
||||
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software}
|
||||
import org.apache.spark.sql.functions._
|
||||
import eu.dnetlib.dhp.schema.oaf._
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.beans.BeanProperty
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
case class OrcidAutor(
|
||||
@BeanProperty var orcid: String,
|
||||
@BeanProperty var familyName: String,
|
||||
@BeanProperty var givenName: String,
|
||||
@BeanProperty var creditName: String,
|
||||
@BeanProperty var otherNames: java.util.List[String]
|
||||
) {
|
||||
def this() = this("null", "null", "null", "null", null)
|
||||
}
|
||||
|
||||
case class MatchData(
|
||||
@BeanProperty var id: String,
|
||||
@BeanProperty var graph_authors: java.util.List[Author],
|
||||
@BeanProperty var orcid_authors: java.util.List[OrcidAutor]
|
||||
) {
|
||||
def this() = this("null", null, null)
|
||||
}
|
||||
|
||||
case class MatchedAuthors(
|
||||
@BeanProperty var author: Author,
|
||||
@BeanProperty var orcid: OrcidAutor,
|
||||
@BeanProperty var `type`: String
|
||||
)
|
||||
|
||||
class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger)
|
||||
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
|
||||
|
||||
|
@ -22,107 +47,132 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String]
|
|||
log.info(s"orcidPath is '$orcidPath'")
|
||||
val targetPath = parser.get("targetPath")
|
||||
log.info(s"targetPath is '$targetPath'")
|
||||
val orcidPublication: Dataset[Row] = generateOrcidTable(spark, orcidPath)
|
||||
// ModelSupport.entityTypes.entrySet().asScala.filter(k => k.getKey.getClass isInstance(Result))
|
||||
|
||||
enrichResult(
|
||||
spark,
|
||||
s"$graphPath/publication",
|
||||
orcidPublication,
|
||||
s"$targetPath/publication",
|
||||
Encoders.bean(classOf[Publication])
|
||||
)
|
||||
enrichResult(
|
||||
spark,
|
||||
s"$graphPath/dataset",
|
||||
orcidPublication,
|
||||
s"$targetPath/dataset",
|
||||
Encoders.bean(classOf[eu.dnetlib.dhp.schema.oaf.Dataset])
|
||||
)
|
||||
enrichResult(
|
||||
spark,
|
||||
s"$graphPath/software",
|
||||
orcidPublication,
|
||||
s"$targetPath/software",
|
||||
Encoders.bean(classOf[Software])
|
||||
)
|
||||
enrichResult(
|
||||
spark,
|
||||
s"$graphPath/otherresearchproduct",
|
||||
orcidPublication,
|
||||
s"$targetPath/otherresearchproduct",
|
||||
Encoders.bean(classOf[OtherResearchProduct])
|
||||
)
|
||||
createTemporaryData(graphPath, orcidPath, targetPath)
|
||||
analisys(targetPath)
|
||||
generateGraph(graphPath, targetPath)
|
||||
}
|
||||
|
||||
private def enrichResult[T <: Result](
|
||||
spark: SparkSession,
|
||||
graphPath: String,
|
||||
orcidPublication: Dataset[Row],
|
||||
outputPath: String,
|
||||
enc: Encoder[T]
|
||||
): Unit = {
|
||||
private def generateGraph(graphPath: String, targetPath: String): Unit = {
|
||||
|
||||
val entities = spark.read
|
||||
.schema(enc.schema)
|
||||
.json(graphPath)
|
||||
.select(col("id"), col("datainfo"), col("instance"))
|
||||
.where("datainfo.deletedbyinference != true")
|
||||
.drop("datainfo")
|
||||
.withColumn("instances", explode(col("instance")))
|
||||
.withColumn("pids", explode(col("instances.pid")))
|
||||
.select(
|
||||
col("pids.qualifier.classid").alias("pid_schema"),
|
||||
col("pids.value").alias("pid_value"),
|
||||
col("id").alias("dnet_id")
|
||||
)
|
||||
ModelSupport.entityTypes.asScala
|
||||
.filter(e => ModelSupport.isResult(e._1))
|
||||
.foreach(e => {
|
||||
val resultType = e._1.name()
|
||||
val enc = Encoders.bean(e._2)
|
||||
|
||||
val orcidDnet = orcidPublication
|
||||
.join(
|
||||
entities,
|
||||
lower(col("schema")).equalTo(lower(col("pid_schema"))) &&
|
||||
lower(col("value")).equalTo(lower(col("pid_value"))),
|
||||
"inner"
|
||||
)
|
||||
.groupBy(col("dnet_id"))
|
||||
.agg(collect_set(orcidPublication("author")).alias("orcid_authors"))
|
||||
.select("dnet_id", "orcid_authors")
|
||||
.cache()
|
||||
orcidDnet.count()
|
||||
val result = spark.read.schema(enc.schema).json(graphPath).as[T](enc)
|
||||
val matched = spark.read
|
||||
.schema(Encoders.bean(classOf[ORCIDAuthorEnricherResult]).schema)
|
||||
.parquet(s"${targetPath}/${resultType}_matched")
|
||||
.selectExpr("id", "enriched_author")
|
||||
|
||||
spark.read
|
||||
.schema(enc.schema)
|
||||
.json(s"$graphPath/$resultType")
|
||||
.join(matched, Seq("id"), "left")
|
||||
.withColumn(
|
||||
"author",
|
||||
when(size(col("enriched_author")).gt(0), col("enriched_author"))
|
||||
.otherwise(col("author"))
|
||||
)
|
||||
.drop("enriched_author")
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(s"${targetPath}/${resultType}")
|
||||
|
||||
})
|
||||
|
||||
result
|
||||
.joinWith(orcidDnet, result("id").equalTo(orcidDnet("dnet_id")), "left")
|
||||
.map {
|
||||
case (r: T, null) =>
|
||||
r
|
||||
case (p: T, r: Row) =>
|
||||
p.setAuthor(AuthorMerger.enrichOrcid(p.getAuthor, AuthorEnricher.toOAFAuthor(r)))
|
||||
p
|
||||
}(enc)
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.json(outputPath)
|
||||
}
|
||||
|
||||
private def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = {
|
||||
private def createTemporaryData(graphPath: String, orcidPath: String, targetPath: String): Unit = {
|
||||
val orcidAuthors =
|
||||
spark.read.load(s"$inputPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames")
|
||||
spark.read.load(s"$orcidPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames")
|
||||
|
||||
val orcidWorks = spark.read
|
||||
.load(s"$inputPath/Works")
|
||||
.load(s"$orcidPath/Works")
|
||||
.select(col("orcid"), explode(col("pids")).alias("identifier"))
|
||||
.where(
|
||||
"identifier.schema IN('doi','pmid','pmc','arxiv','handle')"
|
||||
"identifier.schema IN('doi','pmid','pmc','arxiv','handle')" // scopus eid ?
|
||||
)
|
||||
val orcidPublication = orcidAuthors
|
||||
.join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid")))
|
||||
|
||||
val orcidWorksWithAuthors = orcidAuthors
|
||||
.join(orcidWorks, Seq("orcid"))
|
||||
.select(
|
||||
col("identifier.schema").alias("schema"),
|
||||
col("identifier.value").alias("value"),
|
||||
struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author")
|
||||
lower(col("identifier.schema")).alias("pid_schema"),
|
||||
lower(col("identifier.value")).alias("pid_value"),
|
||||
struct(
|
||||
col("orcid"),
|
||||
col("givenName"),
|
||||
col("familyName"),
|
||||
col("creditName"),
|
||||
col("otherNames")
|
||||
).alias("author")
|
||||
)
|
||||
orcidPublication.cache()
|
||||
.cache()
|
||||
|
||||
ModelSupport.entityTypes.asScala
|
||||
.filter(e => ModelSupport.isResult(e._1))
|
||||
.foreach(e => {
|
||||
val resultType = e._1.name()
|
||||
val enc = Encoders.bean(e._2)
|
||||
|
||||
val oaEntities = spark.read
|
||||
.schema(enc.schema)
|
||||
.json(s"$graphPath/$resultType")
|
||||
.select(col("id"), col("datainfo"), col("instance"))
|
||||
.where("datainfo.deletedbyinference != true")
|
||||
.drop("datainfo")
|
||||
.withColumn("instances", explode(col("instance")))
|
||||
.withColumn("pids", explode(col("instances.pid")))
|
||||
.select(
|
||||
lower(col("pids.qualifier.classid")).alias("pid_schema"),
|
||||
lower(col("pids.value")).alias("pid_value"),
|
||||
col("id")
|
||||
)
|
||||
|
||||
val orcidDnet = orcidWorksWithAuthors
|
||||
.join(
|
||||
oaEntities,
|
||||
Seq("pid_schema", "pid_value"),
|
||||
"inner"
|
||||
)
|
||||
.groupBy(col("id"))
|
||||
.agg(collect_set(col("author")).alias("orcid_authors"))
|
||||
.select("id", "orcid_authors")
|
||||
|
||||
val result =
|
||||
spark.read.schema(enc.schema).json(s"$graphPath/$resultType").selectExpr("id", "author as graph_authors")
|
||||
|
||||
result
|
||||
.join(orcidDnet, Seq("id"))
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
.option("compression", "gzip")
|
||||
.parquet(s"$targetPath/${resultType}_unmatched")
|
||||
})
|
||||
|
||||
orcidWorksWithAuthors.unpersist()
|
||||
}
|
||||
|
||||
private def analisys(targetPath: String): Unit = {
|
||||
ModelSupport.entityTypes.asScala
|
||||
.filter(e => ModelSupport.isResult(e._1))
|
||||
.foreach(e => {
|
||||
val resultType = e._1.name()
|
||||
|
||||
spark.read
|
||||
.parquet(s"$targetPath/${resultType}_unmatched")
|
||||
.where("size(graph_authors) > 0")
|
||||
.as[MatchData](Encoders.bean(classOf[MatchData]))
|
||||
.map(md => {
|
||||
ORCIDAuthorEnricher.enrichOrcid(md.id, md.graph_authors, md.orcid_authors)
|
||||
})(Encoders.bean(classOf[ORCIDAuthorEnricherResult]))
|
||||
.write
|
||||
.option("compression", "gzip")
|
||||
.mode("overwrite")
|
||||
.parquet(s"$targetPath/${resultType}_matched")
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.graph.resolution
|
|||
import com.fasterxml.jackson.databind.ObjectMapper
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.common.EntityType
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils
|
||||
import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset, _}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
|
@ -118,15 +119,12 @@ object SparkResolveEntities {
|
|||
currentEntityDataset
|
||||
.joinWith(re, currentEntityDataset("_1").equalTo(re("_1")), "left")
|
||||
.map(k => {
|
||||
|
||||
val a = k._1
|
||||
val b = k._2
|
||||
if (b == null)
|
||||
a._2
|
||||
else {
|
||||
a._2.mergeFrom(b._2)
|
||||
a._2
|
||||
}
|
||||
else
|
||||
MergeUtils.mergeResult(a._2, b._2)
|
||||
})
|
||||
.map(r => mapper.writeValueAsString(r))(Encoders.STRING)
|
||||
.write
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package eu.dnetlib.dhp.sx.graph
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils
|
||||
import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset, _}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.SparkConf
|
||||
|
@ -130,10 +131,7 @@ object SparkCreateInputGraph {
|
|||
val ds: Dataset[T] = spark.read.load(sourcePath).as[T]
|
||||
|
||||
ds.groupByKey(_.getId)
|
||||
.reduceGroups { (x: T, y: T) =>
|
||||
x.mergeFrom(y)
|
||||
x
|
||||
}
|
||||
.reduceGroups { (x: T, y: T) => MergeUtils.merge(x, y).asInstanceOf[T] }
|
||||
.map(_._2)
|
||||
.write
|
||||
.mode(SaveMode.Overwrite)
|
||||
|
|
|
@ -1,10 +1,9 @@
|
|||
|
||||
package eu.dnetlib.oa.merge;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.*;
|
||||
package eu.dnetlib.dhp.enrich.orcid;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
|
@ -14,10 +13,9 @@ import org.junit.platform.commons.util.StringUtils;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
|
||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||
|
||||
public class AuthorMergerTest {
|
||||
public class ORCIDAuthorEnricherTest {
|
||||
|
||||
@Test
|
||||
public void testEnrcichAuthor() throws Exception {
|
||||
|
@ -26,12 +24,13 @@ public class AuthorMergerTest {
|
|||
BufferedReader pr = new BufferedReader(new InputStreamReader(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
AuthorMergerTest.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json"))));
|
||||
ORCIDAuthorEnricherTest.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_publication_sample.json"))));
|
||||
BufferedReader or = new BufferedReader(new InputStreamReader(
|
||||
Objects
|
||||
.requireNonNull(
|
||||
AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json"))));
|
||||
ORCIDAuthorEnricherTest.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_orcid_sample.json"))));
|
||||
|
||||
TypeReference<List<Author>> aclass = new TypeReference<List<Author>>() {
|
||||
};
|
||||
|
@ -67,7 +66,8 @@ public class AuthorMergerTest {
|
|||
long start = System.currentTimeMillis();
|
||||
|
||||
// final List<Author> enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors);
|
||||
final List<Author> enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors);
|
||||
final List<Author> enrichedList = Collections.emptyList(); // SparkEnrichGraphWithOrcidAuthors.enrichOrcid(publicationAuthors,
|
||||
// orcidAuthors);
|
||||
|
||||
long enrichedAuthorWithPid = enrichedList
|
||||
.stream()
|
||||
|
@ -91,24 +91,4 @@ public class AuthorMergerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkSimilarityTest() {
|
||||
final Author left = new Author();
|
||||
left.setName("Anand");
|
||||
left.setSurname("Rachna");
|
||||
left.setFullname("Anand, Rachna");
|
||||
|
||||
System.out.println(AuthorMerger.normalizeFullName(left.getFullname()));
|
||||
|
||||
final Author right = new Author();
|
||||
right.setName("Rachna");
|
||||
right.setSurname("Anand");
|
||||
right.setFullname("Rachna, Anand");
|
||||
// System.out.println(AuthorMerger.normalize(right.getFullname()));
|
||||
boolean same = AuthorMerger.checkORCIDSimilarity(left, right);
|
||||
|
||||
assertTrue(same);
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,13 +1,12 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.graph.raw;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
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.MergeUtils;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.dom4j.DocumentException;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
@ -16,13 +15,12 @@ import org.junit.jupiter.api.extension.ExtendWith;
|
|||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
|
||||
import eu.dnetlib.dhp.oa.graph.clean.GraphCleaningFunctionsTest;
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
class GenerateEntitiesApplicationTest {
|
||||
|
@ -72,7 +70,7 @@ class GenerateEntitiesApplicationTest {
|
|||
|
||||
protected <T extends Result> void verifyMerge(Result publication, Result dataset, Class<T> clazz,
|
||||
String resultType) {
|
||||
final Result merge = OafMapperUtils.mergeResults(publication, dataset);
|
||||
final Result merge = MergeUtils.mergeResult(publication, dataset);
|
||||
assertTrue(clazz.isAssignableFrom(merge.getClass()));
|
||||
assertEquals(resultType, merge.getResulttype().getClassid());
|
||||
}
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
package eu.dnetlib.dhp.enrich.orcid
|
||||
|
||||
import eu.dnetlib.dhp.enrich.orcid.ORCIDAuthorMatchers.matchOrderedTokenAndAbbreviations
|
||||
import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
class ORCIDAuthorMatchersTest {
|
||||
|
||||
@Test def testShortNames(): Unit = {
|
||||
assertTrue(matchOrderedTokenAndAbbreviations("Lasagni Mariozzi Federico", "Lasagni F. Mariozzi"))
|
||||
}
|
||||
|
||||
@Test def testInvertedNames(): Unit = {
|
||||
assertTrue(matchOrderedTokenAndAbbreviations("Andrea, Paolo Marcello", "Marcello Paolo, Andrea"))
|
||||
}
|
||||
|
||||
@Test def testHomonymy(): Unit = {
|
||||
assertTrue(matchOrderedTokenAndAbbreviations("Jang Myung Lee", "J Lee"))
|
||||
}
|
||||
|
||||
@Test def testAmbiguousShortNames(): Unit = {
|
||||
assertFalse(matchOrderedTokenAndAbbreviations("P. Mariozzi", "M. Paolozzi"))
|
||||
}
|
||||
|
||||
@Test def testNonMatches(): Unit = {
|
||||
assertFalse(matchOrderedTokenAndAbbreviations("Giovanni Paolozzi", "Francesco Paolozzi"))
|
||||
assertFalse(matchOrderedTokenAndAbbreviations("G. Paolozzi", "F. Paolozzi"))
|
||||
}
|
||||
|
||||
@Test def testChineseNames(): Unit = {
|
||||
assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin"))
|
||||
// assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* Copyright (c) 2024.
|
||||
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
|
||||
* SPDX-License-Identifier: AGPL-3.0-or-later
|
||||
*/
|
||||
|
||||
package eu.dnetlib.dhp.oa.provision;
|
||||
|
||||
import java.io.StringReader;
|
||||
import java.io.StringWriter;
|
||||
|
||||
import javax.xml.transform.Transformer;
|
||||
import javax.xml.transform.TransformerException;
|
||||
import javax.xml.transform.stream.StreamResult;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
|
||||
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
|
||||
|
||||
public abstract class AbstractSolrRecordTransformJob {
|
||||
|
||||
protected static String toIndexRecord(Transformer tr, final String xmlRecord) {
|
||||
final StreamResult res = new StreamResult(new StringWriter());
|
||||
try {
|
||||
tr.transform(new StreamSource(new StringReader(xmlRecord)), res);
|
||||
return res.getWriter().toString();
|
||||
} catch (TransformerException e) {
|
||||
throw new IllegalArgumentException("XPathException on record: \n" + xmlRecord, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the XSLT responsible for building the index xml records.
|
||||
*
|
||||
* @param format Metadata format name (DMF|TMF)
|
||||
* @param xslt xslt for building the index record transformer
|
||||
* @param fields the list of fields
|
||||
* @return the javax.xml.transform.Transformer
|
||||
* @throws TransformerException could happen
|
||||
*/
|
||||
protected static String getLayoutTransformer(String format, String fields, String xslt)
|
||||
throws TransformerException {
|
||||
|
||||
final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt);
|
||||
final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter());
|
||||
|
||||
layoutTransformer.setParameter("format", format);
|
||||
layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt);
|
||||
|
||||
return layoutToXsltXslt.getWriter().toString();
|
||||
}
|
||||
|
||||
}
|
|
@ -27,14 +27,7 @@ import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
|
|||
import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
|
||||
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
|
||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
||||
import eu.dnetlib.dhp.schema.oaf.Field;
|
||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||
import eu.dnetlib.dhp.schema.oaf.Organization;
|
||||
import eu.dnetlib.dhp.schema.oaf.Project;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits;
|
||||
import scala.Tuple2;
|
||||
|
||||
|
@ -156,10 +149,33 @@ public class CreateRelatedEntitiesJob_phase1 {
|
|||
case software:
|
||||
final Result result = (Result) entity;
|
||||
|
||||
if (result.getTitle() != null && !result.getTitle().isEmpty()) {
|
||||
final StructuredProperty title = result.getTitle().stream().findFirst().get();
|
||||
title.setValue(StringUtils.left(title.getValue(), ModelHardLimits.MAX_TITLE_LENGTH));
|
||||
re.setTitle(title);
|
||||
if (Objects.nonNull(result.getTitle()) && !result.getTitle().isEmpty()) {
|
||||
result
|
||||
.getTitle()
|
||||
.stream()
|
||||
.findFirst()
|
||||
.map(StructuredProperty::getValue)
|
||||
.ifPresent(
|
||||
title -> re.getTitle().setValue(StringUtils.left(title, ModelHardLimits.MAX_TITLE_LENGTH)));
|
||||
}
|
||||
if (Objects.nonNull(result.getDescription()) && !result.getDescription().isEmpty()) {
|
||||
result
|
||||
.getDescription()
|
||||
.stream()
|
||||
.findFirst()
|
||||
.map(Field::getValue)
|
||||
.ifPresent(
|
||||
d -> re.setDescription(StringUtils.left(d, ModelHardLimits.MAX_RELATED_ABSTRACT_LENGTH)));
|
||||
}
|
||||
if (Objects.nonNull(result.getAuthor()) && !result.getAuthor().isEmpty()) {
|
||||
re
|
||||
.setAuthor(
|
||||
result
|
||||
.getAuthor()
|
||||
.stream()
|
||||
.map(Author::getFullname)
|
||||
.filter(StringUtils::isNotBlank)
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
re.setDateofacceptance(getValue(result.getDateofacceptance()));
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue