Merge branch 'beta' of code-repo.d4science.org:D-Net/dnet-hadoop into beta

This commit is contained in:
Michele Artini 2024-04-03 09:50:41 +02:00
commit 71d6e02886
208 changed files with 109053 additions and 2426 deletions

View File

@ -1,6 +1,24 @@
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;
@ -12,182 +30,165 @@ 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>) 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));
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);
}
});
}
}

View File

@ -0,0 +1,76 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.HashSet;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class CleaningFunctions {
public static final String DOI_PREFIX_REGEX = "(^10\\.|\\/10\\.)";
public static final String DOI_PREFIX = "10.";
public static final Set<String> PID_BLACKLIST = new HashSet<>();
static {
PID_BLACKLIST.add("none");
PID_BLACKLIST.add("na");
}
public CleaningFunctions() {
}
/**
* Utility method that filter PID values on a per-type basis.
* @param s the PID whose value will be checked.
* @return false if the pid matches the filter criteria, true otherwise.
*/
public static boolean pidFilter(StructuredProperty s) {
final String pidValue = s.getValue();
if (Objects.isNull(s.getQualifier()) ||
StringUtils.isBlank(pidValue) ||
StringUtils.isBlank(pidValue.replaceAll("(?:\\n|\\r|\\t|\\s)", ""))) {
return false;
}
if (CleaningFunctions.PID_BLACKLIST.contains(pidValue)) {
return false;
}
return !PidBlacklistProvider.getBlacklist(s.getQualifier().getClassid()).contains(pidValue);
}
/**
* Utility method that normalises PID values on a per-type basis.
* @param pid the PID whose value will be normalised.
* @return the PID containing the normalised value.
*/
public static StructuredProperty normalizePidValue(StructuredProperty pid) {
pid
.setValue(
normalizePidValue(
pid.getQualifier().getClassid(),
pid.getValue()));
return pid;
}
public static String normalizePidValue(String pidType, String pidValue) {
String value = Optional
.ofNullable(pidValue)
.map(String::trim)
.orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty"));
switch (pidType) {
// TODO add cleaning for more PID types as needed
case "doi":
return value.toLowerCase().replaceFirst(DOI_PREFIX_REGEX, DOI_PREFIX);
}
return value;
}
}

View File

@ -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())) {

View File

@ -0,0 +1,294 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import static com.google.common.base.Preconditions.checkArgument;
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.util.*;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.lang3.StringUtils;
import com.google.common.collect.HashBiMap;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
/**
* Factory class for OpenAIRE identifiers in the Graph
*/
public class IdentifierFactory implements Serializable {
public static final String ID_SEPARATOR = "::";
public static final String ID_PREFIX_SEPARATOR = "|";
public static final int ID_PREFIX_LEN = 12;
/**
* Declares the associations PID_TYPE -> [DATASOURCE ID, NAME] considered authoritative for that PID_TYPE.
* The id of the record (source_::id) will be rewritten as pidType_::id)
*/
public static final Map<PidType, HashBiMap<String, String>> PID_AUTHORITY = Maps.newHashMap();
static {
PID_AUTHORITY.put(PidType.doi, HashBiMap.create());
PID_AUTHORITY.get(PidType.doi).put(CROSSREF_ID, "Crossref");
PID_AUTHORITY.get(PidType.doi).put(DATACITE_ID, "Datacite");
PID_AUTHORITY.get(PidType.doi).put(ZENODO_OD_ID, "ZENODO");
PID_AUTHORITY.get(PidType.doi).put(ZENODO_R3_ID, "Zenodo");
PID_AUTHORITY.put(PidType.pmc, HashBiMap.create());
PID_AUTHORITY.get(PidType.pmc).put(EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central");
PID_AUTHORITY.get(PidType.pmc).put(PUBMED_CENTRAL_ID, "PubMed Central");
PID_AUTHORITY.put(PidType.pmid, HashBiMap.create());
PID_AUTHORITY.get(PidType.pmid).put(EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central");
PID_AUTHORITY.get(PidType.pmid).put(PUBMED_CENTRAL_ID, "PubMed Central");
PID_AUTHORITY.put(PidType.arXiv, HashBiMap.create());
PID_AUTHORITY.get(PidType.arXiv).put(ARXIV_ID, "arXiv.org e-Print Archive");
PID_AUTHORITY.put(PidType.w3id, HashBiMap.create());
PID_AUTHORITY.get(PidType.w3id).put(ROHUB_ID, "ROHub");
}
/**
* Declares the associations PID_TYPE -> [DATASOURCE ID, PID SUBSTRING] considered as delegated authority for that
* PID_TYPE. Example, Zenodo is delegated to forge DOIs that contain the 'zenodo' word.
*
* If a record with the same id (same pid) comes from 2 data sources, the one coming from a delegated source wins. E.g. Zenodo records win over those from Datacite.
* See also https://code-repo.d4science.org/D-Net/dnet-hadoop/pulls/187 and the class dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java
*/
public static final Map<PidType, Map<String, String>> DELEGATED_PID_AUTHORITY = Maps.newHashMap();
static {
DELEGATED_PID_AUTHORITY.put(PidType.doi, new HashMap<>());
DELEGATED_PID_AUTHORITY.get(PidType.doi).put(ZENODO_OD_ID, "zenodo");
DELEGATED_PID_AUTHORITY.get(PidType.doi).put(ZENODO_R3_ID, "zenodo");
DELEGATED_PID_AUTHORITY.put(PidType.w3id, new HashMap<>());
DELEGATED_PID_AUTHORITY.get(PidType.w3id).put(ROHUB_ID, "ro-id");
}
/**
* Declares the associations PID_TYPE -> [DATASOURCE ID, NAME] whose records are considered enrichment for the graph.
* Their OpenAIRE ID is built from the declared PID type. Are merged with their corresponding record, identified by
* the same OpenAIRE id.
*/
public static final Map<PidType, HashBiMap<String, String>> ENRICHMENT_PROVIDER = Maps.newHashMap();
static {
ENRICHMENT_PROVIDER.put(PidType.doi, HashBiMap.create());
ENRICHMENT_PROVIDER.get(PidType.doi).put(OPEN_APC_ID, OPEN_APC_NAME);
}
public static Set<String> delegatedAuthorityDatasourceIds() {
return DELEGATED_PID_AUTHORITY
.values()
.stream()
.flatMap(m -> m.keySet().stream())
.collect(Collectors.toCollection(HashSet::new));
}
public static List<StructuredProperty> getPids(List<StructuredProperty> pid, KeyValue collectedFrom) {
return pidFromInstance(pid, collectedFrom, true).distinct().collect(Collectors.toList());
}
public static <T extends Result> String createDOIBoostIdentifier(T entity) {
if (entity == null)
return null;
StructuredProperty pid = null;
if (entity.getPid() != null) {
pid = entity
.getPid()
.stream()
.filter(Objects::nonNull)
.filter(s -> s.getQualifier() != null && "doi".equalsIgnoreCase(s.getQualifier().getClassid()))
.filter(CleaningFunctions::pidFilter)
.findAny()
.orElse(null);
} else {
if (entity.getInstance() != null) {
pid = entity
.getInstance()
.stream()
.filter(i -> i.getPid() != null)
.flatMap(i -> i.getPid().stream())
.filter(CleaningFunctions::pidFilter)
.findAny()
.orElse(null);
}
}
if (pid != null)
return idFromPid(entity, pid, true);
return null;
}
/**
* Creates an identifier from the most relevant PID (if available) provided by a known PID authority in the given
* entity T. Returns entity.id when none of the PIDs meet the selection criteria is available.
*
* @param entity the entity providing PIDs and a default ID.
* @param <T> the specific entity type. Currently Organization and Result subclasses are supported.
* @param md5 indicates whether should hash the PID value or not.
* @return an identifier from the most relevant PID, entity.id otherwise
*/
public static <T extends OafEntity> String createIdentifier(T entity, boolean md5) {
checkArgument(StringUtils.isNoneBlank(entity.getId()), "missing entity identifier");
final Map<String, Set<StructuredProperty>> pids = extractPids(entity);
return pids
.values()
.stream()
.flatMap(Set::stream)
.min(new PidComparator<>(entity))
.map(
min -> Optional
.ofNullable(pids.get(min.getQualifier().getClassid()))
.map(
p -> p
.stream()
.sorted(new PidValueComparator())
.findFirst()
.map(s -> idFromPid(entity, s, md5))
.orElseGet(entity::getId))
.orElseGet(entity::getId))
.orElseGet(entity::getId);
}
private static <T extends OafEntity> Map<String, Set<StructuredProperty>> extractPids(T entity) {
if (entity instanceof Result) {
return Optional
.ofNullable(((Result) entity).getInstance())
.map(IdentifierFactory::mapPids)
.orElse(new HashMap<>());
} else {
return entity
.getPid()
.stream()
.map(CleaningFunctions::normalizePidValue)
.filter(CleaningFunctions::pidFilter)
.collect(
Collectors
.groupingBy(
p -> p.getQualifier().getClassid(),
Collectors.mapping(p -> p, Collectors.toCollection(HashSet::new))));
}
}
private static Map<String, Set<StructuredProperty>> mapPids(List<Instance> instance) {
return instance
.stream()
.map(i -> pidFromInstance(i.getPid(), i.getCollectedfrom(), false))
.flatMap(Function.identity())
.collect(
Collectors
.groupingBy(
p -> p.getQualifier().getClassid(),
Collectors.mapping(p -> p, Collectors.toCollection(HashSet::new))));
}
private static Stream<StructuredProperty> pidFromInstance(List<StructuredProperty> pid, KeyValue collectedFrom,
boolean mapHandles) {
return Optional
.ofNullable(pid)
.map(
pp -> pp
.stream()
// filter away PIDs provided by a DS that is not considered an authority for the
// given PID Type
.filter(p -> shouldFilterPidByCriteria(collectedFrom, p, mapHandles))
.map(CleaningFunctions::normalizePidValue)
.filter(p -> isNotFromDelegatedAuthority(collectedFrom, p))
.filter(CleaningFunctions::pidFilter))
.orElse(Stream.empty());
}
private static boolean shouldFilterPidByCriteria(KeyValue collectedFrom, StructuredProperty p, boolean mapHandles) {
final PidType pType = PidType.tryValueOf(p.getQualifier().getClassid());
if (Objects.isNull(collectedFrom)) {
return false;
}
boolean isEnrich = Optional
.ofNullable(ENRICHMENT_PROVIDER.get(pType))
.map(
enrich -> enrich.containsKey(collectedFrom.getKey())
|| enrich.containsValue(collectedFrom.getValue()))
.orElse(false);
boolean isAuthority = Optional
.ofNullable(PID_AUTHORITY.get(pType))
.map(
authorities -> authorities.containsKey(collectedFrom.getKey())
|| authorities.containsValue(collectedFrom.getValue()))
.orElse(false);
return (mapHandles && pType.equals(PidType.handle)) || isEnrich || isAuthority;
}
private static boolean isNotFromDelegatedAuthority(KeyValue collectedFrom, StructuredProperty p) {
final PidType pType = PidType.tryValueOf(p.getQualifier().getClassid());
final Map<String, String> da = DELEGATED_PID_AUTHORITY.get(pType);
if (Objects.isNull(da)) {
return true;
}
if (!da.containsKey(collectedFrom.getKey())) {
return true;
}
return StringUtils.contains(p.getValue(), da.get(collectedFrom.getKey()));
}
/**
* @see {@link IdentifierFactory#createIdentifier(OafEntity, boolean)}
*/
public static <T extends OafEntity> String createIdentifier(T entity) {
return createIdentifier(entity, true);
}
private static <T extends OafEntity> String idFromPid(T entity, StructuredProperty s, boolean md5) {
return idFromPid(ModelSupport.getIdPrefix(entity.getClass()), s.getQualifier().getClassid(), s.getValue(), md5);
}
public static String idFromPid(String numericPrefix, String pidType, String pidValue, boolean md5) {
return new StringBuilder()
.append(numericPrefix)
.append(ID_PREFIX_SEPARATOR)
.append(createPrefix(pidType))
.append(ID_SEPARATOR)
.append(md5 ? md5(pidValue) : pidValue)
.toString();
}
// create the prefix (length = 12)
private static String createPrefix(String pidType) {
StringBuilder prefix = new StringBuilder(StringUtils.left(pidType, ID_PREFIX_LEN));
while (prefix.length() < ID_PREFIX_LEN) {
prefix.append("_");
}
return prefix.substring(0, ID_PREFIX_LEN);
}
public static String md5(final String s) {
try {
final MessageDigest md = MessageDigest.getInstance("MD5");
md.update(s.getBytes(StandardCharsets.UTF_8));
return new String(Hex.encodeHex(md.digest()));
} catch (final Exception e) {
return null;
}
}
}

View File

@ -1,79 +1,78 @@
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;
//
// 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;
public class MergeComparator implements Comparator<Oaf> {
public MergeComparator() {
}
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;
}
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;
}
}
// 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;
}
// 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();
}
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;
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());
}
}
/**
* 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());
}
}
}

View File

@ -0,0 +1,27 @@
package eu.dnetlib.dhp.schema.oaf.utils;
public class ModelHardLimits {
private ModelHardLimits() {
}
public static final String LAYOUT = "index";
public static final String INTERPRETATION = "openaire";
public static final String SEPARATOR = "-";
public static final int MAX_EXTERNAL_ENTITIES = 50;
public static final int MAX_AUTHORS = 200;
public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000;
public static final int MAX_TITLE_LENGTH = 5000;
public static final int MAX_TITLES = 10;
public static final int MAX_ABSTRACTS = 10;
public static final int MAX_ABSTRACT_LENGTH = 150000;
public static final int MAX_RELATED_ABSTRACT_LENGTH = 500;
public static final int MAX_INSTANCES = 10;
public static String getCollectionName(String format) {
return format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION;
}
}

View File

@ -0,0 +1,38 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Comparator;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class OrganizationPidComparator implements Comparator<StructuredProperty> {
@Override
public int compare(StructuredProperty left, StructuredProperty right) {
PidType lClass = PidType.tryValueOf(left.getQualifier().getClassid());
PidType rClass = PidType.tryValueOf(right.getQualifier().getClassid());
if (lClass.equals(PidType.openorgs))
return -1;
if (rClass.equals(PidType.openorgs))
return 1;
if (lClass.equals(PidType.GRID))
return -1;
if (rClass.equals(PidType.GRID))
return 1;
if (lClass.equals(PidType.mag_id))
return -1;
if (rClass.equals(PidType.mag_id))
return 1;
if (lClass.equals(PidType.urn))
return -1;
if (rClass.equals(PidType.urn))
return 1;
return 0;
}
}

View File

@ -0,0 +1,8 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.HashMap;
import java.util.HashSet;
public class PidBlacklist extends HashMap<String, HashSet<String>> {
}

View File

@ -0,0 +1,40 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.io.IOException;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import com.fasterxml.jackson.databind.ObjectMapper;
public class PidBlacklistProvider {
private static final PidBlacklist blacklist;
static {
try {
String json = IOUtils.toString(IdentifierFactory.class.getResourceAsStream("pid_blacklist.json"));
blacklist = new ObjectMapper().readValue(json, PidBlacklist.class);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public static PidBlacklist getBlacklist() {
return blacklist;
}
public static Set<String> getBlacklist(String pidType) {
return Optional
.ofNullable(getBlacklist().get(pidType))
.orElse(new HashSet<>());
}
private PidBlacklistProvider() {
}
}

View File

@ -0,0 +1,48 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Comparator;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class PidComparator<T extends OafEntity> implements Comparator<StructuredProperty> {
private final T entity;
public PidComparator(T entity) {
this.entity = entity;
}
@Override
public int compare(StructuredProperty left, StructuredProperty right) {
if (left == null && right == null)
return 0;
if (left == null)
return 1;
if (right == null)
return -1;
if (ModelSupport.isSubClass(entity, Result.class)) {
return compareResultPids(left, right);
}
if (ModelSupport.isSubClass(entity, Organization.class)) {
return compareOrganizationtPids(left, right);
}
// Else (but unlikely), lexicographical ordering will do.
return left.getQualifier().getClassid().compareTo(right.getQualifier().getClassid());
}
private int compareResultPids(StructuredProperty left, StructuredProperty right) {
return new ResultPidComparator().compare(left, right);
}
private int compareOrganizationtPids(StructuredProperty left, StructuredProperty right) {
return new OrganizationPidComparator().compare(left, right);
}
}

View File

@ -0,0 +1,79 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import org.apache.commons.lang3.EnumUtils;
public enum PidType {
/**
* The DOI syntax shall be made up of a DOI prefix and a DOI suffix separated by a forward slash.
*
* There is no defined limit on the length of the DOI name, or of the DOI prefix or DOI suffix.
*
* The DOI name is case-insensitive and can incorporate any printable characters from the legal graphic characters
* of Unicode. Further constraints on character use (e.g. use of language-specific alphanumeric characters) can be
* defined for an application by the ISO 26324 Registration Authority.
*
*
* DOI prefix: The DOI prefix shall be composed of a directory indicator followed by a registrant code.
* These two components shall be separated by a full stop (period). The directory indicator shall be "10" and
* distinguishes the entire set of character strings (prefix and suffix) as digital object identifiers within the
* resolution system.
*
* Registrant code: The second element of the DOI prefix shall be the registrant code. The registrant code is a
* unique string assigned to a registrant.
*
* DOI suffix: The DOI suffix shall consist of a character string of any length chosen by the registrant.
* Each suffix shall be unique to the prefix element that precedes it. The unique suffix can be a sequential number,
* or it might incorporate an identifier generated from or based on another system used by the registrant
* (e.g. ISAN, ISBN, ISRC, ISSN, ISTC, ISNI; in such cases, a preferred construction for such a suffix can be
* specified, as in Example 1).
*
* Source: https://www.doi.org/doi_handbook/2_Numbering.html#2.2
*/
doi,
/**
* PubMed Unique Identifier (PMID)
*
* This field is a 1-to-8 digit accession number with no leading zeros. It is present on all records and is the
* accession number for managing and disseminating records. PMIDs are not reused after records are deleted.
*
* Beginning in February 2012 PMIDs include extensions following a decimal point to account for article versions
* (e.g., 21804956.2). All citations are considered version 1 until replaced. The extended PMID is not displayed
* on the MEDLINE format.
*
* View the citation in abstract format in PubMed to access additional versions when available (see the article in
* the Jan-Feb 2012 NLM Technical Bulletin).
*
* Source: https://www.nlm.nih.gov/bsd/mms/medlineelements.html#pmid
*/
pmid,
/**
* This field contains the unique identifier for the cited article in PubMed Central. The identifier begins with the
* prefix PMC.
*
* Source: https://www.nlm.nih.gov/bsd/mms/medlineelements.html#pmc
*/
pmc, handle, arXiv, nct, pdb, w3id,
// Organization
openorgs, ROR, GRID, PIC, ISNI, Wikidata, FundRef, corda, corda_h2020, mag_id, urn,
// Used by dedup
undefined, original;
public static boolean isValid(String type) {
return EnumUtils.isValidEnum(PidType.class, type);
}
public static PidType tryValueOf(String s) {
try {
return PidType.valueOf(s);
} catch (Exception e) {
return PidType.original;
}
}
}

View File

@ -0,0 +1,33 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Comparator;
import java.util.Optional;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class PidValueComparator implements Comparator<StructuredProperty> {
@Override
public int compare(StructuredProperty left, StructuredProperty right) {
if (left == null && right == null)
return 0;
if (left == null)
return 1;
if (right == null)
return -1;
StructuredProperty l = CleaningFunctions.normalizePidValue(left);
StructuredProperty r = CleaningFunctions.normalizePidValue(right);
return Optional
.ofNullable(l.getValue())
.map(
lv -> Optional
.ofNullable(r.getValue())
.map(rv -> lv.compareTo(rv))
.orElse(-1))
.orElse(1);
}
}

View File

@ -0,0 +1,39 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Comparator;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
/**
* Comparator for sorting the values from the dnet:review_levels vocabulary, implements the following ordering
*
* peerReviewed (0001) > nonPeerReviewed (0002) > UNKNOWN (0000)
*/
public class RefereedComparator implements Comparator<Qualifier> {
@Override
public int compare(Qualifier left, Qualifier right) {
String lClass = left.getClassid();
String rClass = right.getClassid();
if ("0001".equals(lClass))
return -1;
if ("0001".equals(rClass))
return 1;
if ("0002".equals(lClass))
return -1;
if ("0002".equals(rClass))
return 1;
if ("0000".equals(lClass))
return -1;
if ("0000".equals(rClass))
return 1;
return 0;
}
}

View File

@ -0,0 +1,53 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Comparator;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class ResultPidComparator implements Comparator<StructuredProperty> {
@Override
public int compare(StructuredProperty left, StructuredProperty right) {
PidType lClass = PidType.tryValueOf(left.getQualifier().getClassid());
PidType rClass = PidType.tryValueOf(right.getQualifier().getClassid());
if (lClass.equals(PidType.doi))
return -1;
if (rClass.equals(PidType.doi))
return 1;
if (lClass.equals(PidType.pmid))
return -1;
if (rClass.equals(PidType.pmid))
return 1;
if (lClass.equals(PidType.pmc))
return -1;
if (rClass.equals(PidType.pmc))
return 1;
if (lClass.equals(PidType.handle))
return -1;
if (rClass.equals(PidType.handle))
return 1;
if (lClass.equals(PidType.arXiv))
return -1;
if (rClass.equals(PidType.arXiv))
return 1;
if (lClass.equals(PidType.nct))
return -1;
if (rClass.equals(PidType.nct))
return 1;
if (lClass.equals(PidType.pdb))
return -1;
if (rClass.equals(PidType.pdb))
return 1;
return 0;
}
}

View File

@ -0,0 +1,77 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import static eu.dnetlib.dhp.schema.common.ModelConstants.CROSSREF_ID;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Optional;
import java.util.stream.Collectors;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Result;
public class ResultTypeComparator implements Comparator<Result> {
@Override
public int compare(Result left, Result right) {
if (left == null && right == null)
return 0;
if (left == null)
return 1;
if (right == null)
return -1;
HashSet<String> lCf = getCollectedFromIds(left);
HashSet<String> rCf = getCollectedFromIds(right);
if (lCf.contains(CROSSREF_ID) && !rCf.contains(CROSSREF_ID)) {
return -1;
}
if (!lCf.contains(CROSSREF_ID) && rCf.contains(CROSSREF_ID)) {
return 1;
}
String lClass = left.getResulttype().getClassid();
String rClass = right.getResulttype().getClassid();
if (lClass.equals(rClass))
return 0;
if (lClass.equals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID))
return -1;
if (rClass.equals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID))
return 1;
if (lClass.equals(ModelConstants.DATASET_RESULTTYPE_CLASSID))
return -1;
if (rClass.equals(ModelConstants.DATASET_RESULTTYPE_CLASSID))
return 1;
if (lClass.equals(ModelConstants.SOFTWARE_RESULTTYPE_CLASSID))
return -1;
if (rClass.equals(ModelConstants.SOFTWARE_RESULTTYPE_CLASSID))
return 1;
if (lClass.equals(ModelConstants.ORP_RESULTTYPE_CLASSID))
return -1;
if (rClass.equals(ModelConstants.ORP_RESULTTYPE_CLASSID))
return 1;
// Else (but unlikely), lexicographical ordering will do.
return lClass.compareTo(rClass);
}
protected HashSet<String> getCollectedFromIds(Result left) {
return Optional
.ofNullable(left.getCollectedfrom())
.map(
cf -> cf
.stream()
.map(KeyValue::getKey)
.collect(Collectors.toCollection(HashSet::new)))
.orElse(new HashSet<>());
}
}

View File

@ -0,0 +1,21 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Set;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
class BlackListProviderTest {
@Test
void blackListTest() {
Assertions.assertNotNull(PidBlacklistProvider.getBlacklist());
Assertions.assertNotNull(PidBlacklistProvider.getBlacklist().get("doi"));
Assertions.assertTrue(PidBlacklistProvider.getBlacklist().get("doi").size() > 0);
final Set<String> xxx = PidBlacklistProvider.getBlacklist("xxx");
Assertions.assertNotNull(xxx);
Assertions.assertEquals(0, xxx.size());
}
}

View File

@ -0,0 +1,87 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import java.io.IOException;
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.oaf.Publication;
class IdentifierFactoryTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
@Test
void testCreateIdentifierForPublication() throws IOException {
verifyIdentifier(
"publication_doi1.json", "50|doi_________::79dbc7a2a56dc1532659f9038843256e", true);
verifyIdentifier(
"publication_doi2.json", "50|doi_________::79dbc7a2a56dc1532659f9038843256e", true);
verifyIdentifier(
"publication_doi3.json", "50|pmc_________::94e4cb08c93f8733b48e2445d04002ac", true);
verifyIdentifier(
"publication_doi4.json", "50|od______2852::38861c44e6052a8d49f59a4c39ba5e66", true);
verifyIdentifier(
"publication_doi5.json", "50|doi_________::3bef95c0ca26dd55451fc8839ea69d27", true);
verifyIdentifier(
"publication_pmc1.json", "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", true);
verifyIdentifier(
"publication_pmc2.json", "50|pmc_________::94e4cb08c93f8733b48e2445d04002ac", true);
verifyIdentifier(
"publication_openapc.json", "50|doi_________::79dbc7a2a56dc1532659f9038843256e", true);
final String defaultID = "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f";
verifyIdentifier("publication_3.json", defaultID, true);
verifyIdentifier("publication_4.json", defaultID, true);
verifyIdentifier("publication_5.json", defaultID, true);
}
@Test
void testCreateIdentifierForPublicationNoHash() throws IOException {
verifyIdentifier("publication_doi1.json", "50|doi_________::10.1016/j.cmet.2010.03.013", false);
verifyIdentifier("publication_doi2.json", "50|doi_________::10.1016/j.cmet.2010.03.013", false);
verifyIdentifier("publication_pmc1.json", "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", false);
verifyIdentifier(
"publication_urn1.json", "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", false);
final String defaultID = "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f";
verifyIdentifier("publication_3.json", defaultID, false);
verifyIdentifier("publication_4.json", defaultID, false);
verifyIdentifier("publication_5.json", defaultID, false);
}
@Test
void testCreateIdentifierForROHub() throws IOException {
verifyIdentifier(
"orp-rohub.json", "50|w3id________::afc7592914ae190a50570db90f55f9c2", true);
}
protected void verifyIdentifier(String filename, String expectedID, boolean md5) throws IOException {
final String json = IOUtils.toString(getClass().getResourceAsStream(filename));
final Publication pub = OBJECT_MAPPER.readValue(json, Publication.class);
String id = IdentifierFactory.createIdentifier(pub, md5);
System.out.println(id);
assertNotNull(id);
assertEquals(expectedID, id);
}
}

View File

@ -5,28 +5,47 @@ import static org.junit.jupiter.api.Assertions.*;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.util.HashSet;
import java.util.List;
import java.util.stream.Collectors;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import org.apache.commons.beanutils.BeanUtils;
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 com.google.common.collect.Lists;
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;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
public class MergeUtilsTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
@Test
void testMergePubs_new() throws IOException {
Publication pt = read("publication_test.json", Publication.class);
Publication p1 = read("publication_test.json", Publication.class);
assertEquals(1, pt.getCollectedfrom().size());
assertEquals(ModelConstants.CROSSREF_ID, pt.getCollectedfrom().get(0).getKey());
Instance i = new Instance();
i.setUrl(Lists.newArrayList("https://..."));
p1.getInstance().add(i);
Publication ptp1 = MergeUtils.mergePublication(pt, p1);
assertNotNull(ptp1.getInstance());
assertEquals(2, ptp1.getInstance().size());
}
@Test
void testMergePubs() throws IOException {
Publication p1 = read("publication_1.json", Publication.class);
@ -45,7 +64,7 @@ public class MergeUtilsTest {
assertTrue(cfId(d1.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID));
final Result p1d2 = MergeUtils.checkedMerge(p1, d2);
assertEquals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, p1d2.getResulttype().getClassid());
assertEquals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, p1d2.getResulttype().getClassid());
assertTrue(p1d2 instanceof Publication);
assertEquals(p1.getId(), p1d2.getId());
}

View File

@ -178,10 +178,10 @@ class OafMapperUtilsTest {
assertEquals(
ModelConstants.DATASET_RESULTTYPE_CLASSID,
MergeUtils
.mergeResult(p2, d1)
.getResulttype()
.getClassid());
((Result) MergeUtils
.merge(p2, d1))
.getResulttype()
.getClassid());
}
@Test

View File

@ -0,0 +1,12 @@
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0000/ra.v2i3.114::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"4.65008652949e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0000/ra.v2i3.114"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0001/(aj).v3i6.458::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"4.01810569717e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0001/(aj).v3i6.458"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0001/1587::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.39172290649e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0001/1587"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0001/462::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"6.33235333753e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.36"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.00285265116e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0001/462"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0001/731::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"4.01810569717e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0001/731"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0001/ijllis.v9i4.2066.g2482::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"8.48190886761e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0001/ijllis.v9i4.2066.g2482"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0118/alfahim.v3i1.140::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"9.88840807598e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0118/alfahim.v3i1.140"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0166/fk2.stagefigshare.6442896.v3::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"7.28336930301e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0166/fk2.stagefigshare.6442896.v3"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0301/jttb.v2i1.64::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"7.28336930301e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0301/jttb.v2i1.64"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0809/seruni.v1i1.567::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"2.62959564033e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0809/seruni.v1i1.567"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0809/seruni.v2i1.765::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"9.40178571921e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0559872"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"3.67659957614e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0809/seruni.v2i1.765"}]}]}
{"dataInfo":{"deletedbyinference":false,"inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:enrich","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"id":"unresolved::10.0901/jkip.v7i3.485::doi","instance":[{"measures":[{"id":"influence","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"5.91019644836e-09"}]},{"id":"popularity_alt","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"0.0"}]},{"id":"popularity","unit":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"update","inferred":true,"invisible":false,"provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":""},"key":"score","value":"6.26204125721e-09"}]}],"pid":[{"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.0901/jkip.v7i3.485"}]}]}

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1 @@
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f","pid":[{"qualifier":{"classid":"scp-number"},"value":"79953761260"}]}

View File

@ -0,0 +1 @@
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f","pid":[]}

View File

@ -0,0 +1 @@
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f"}

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,33 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"instance": [
{
"collectedfrom": {
"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2",
"value": "Crossref"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
}
]
},
{
"pid": [
{
"qualifier": {"classid": "urn"},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {"classid": "scp-number"},
"value": "79953761260"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
}
]
}
]
}

View File

@ -0,0 +1,37 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"instance": [
{
"collectedfrom": {
"key": "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2",
"value": "Crossref"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
}
]
},
{
"collectedfrom": {
"key": "10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c",
"value": "Europe PubMed Central"
},
"pid": [
{
"qualifier": {"classid": "urn"},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {"classid": "scp-number"},
"value": "79953761260"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
}
]
}
]
}

View File

@ -0,0 +1,37 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"instance": [
{
"collectedfrom": {
"key": "10|opendoar____::358aee4cc897452c00244351e4d91f69",
"value": "Zenodo"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
}
]
},
{
"collectedfrom": {
"key": "10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c",
"value": "Europe PubMed Central"
},
"pid": [
{
"qualifier": {"classid": "urn"},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {"classid": "scp-number"},
"value": "79953761260"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
}
]
}
]
}

View File

@ -0,0 +1,37 @@
{
"id": "50|od______2852::38861c44e6052a8d49f59a4c39ba5e66",
"instance": [
{
"collectedfrom": {
"key": "10|opendoar____::358aee4cc897452c00244351e4d91f69",
"value": "Zenodo"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
},
{
"qualifier": {"classid": "handle"},
"value": "11012/83840"
}
]
},
{
"collectedfrom": {
"key": "10|opendoar____::2852",
"value": "Digital library of Brno University of Technology"
},
"pid": [
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
},
{
"qualifier": {"classid": "handle"},
"value": "11012/83840"
}
]
}
]
}

View File

@ -0,0 +1,37 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"instance": [
{
"collectedfrom": {
"key": "10|opendoar____::358aee4cc897452c00244351e4d91f69",
"value": "Zenodo"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.5281/zenodo.5121485"
}
]
},
{
"collectedfrom": {
"key": "10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c",
"value": "Europe PubMed Central"
},
"pid": [
{
"qualifier": {"classid": "urn"},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {"classid": "scp-number"},
"value": "79953761260"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
}
]
}
]
}

View File

@ -0,0 +1,3 @@
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", "resulttype" : { "classid" : "publication" }, "pid":[{"qualifier":{"classid":"doi"},"value":"10.1016/j.cmet.2011.03.013"},{"qualifier":{"classid":"urn"},"value":"urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"},{"qualifier":{"classid":"scp-number"},"value":"79953761260"},{"qualifier":{"classid":"pmc"},"value":"21459329"}], "collectedfrom" : [ { "key" : "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value" : "Crossref"} ], "isGreen": null, "openAccessColor": "gold", "isInDiamondJournal": null, "publiclyFunded": null}
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1g", "resulttype" : { "classid" : "publication" }, "isGreen": true, "openAccessColor": "gold", "isInDiamondJournal": true, "publiclyFunded": false }
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1h", "resulttype" : { "classid" : "publication" }, "isGreen": false, "openAccessColor": null, "isInDiamondJournal": true, "publiclyFunded": false }

View File

@ -0,0 +1,3 @@
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f", "resulttype" : { "classid" : "publication" }, "pid":[{"qualifier":{"classid":"doi"},"value":"10.1016/j.cmet.2011.03.013"},{"qualifier":{"classid":"urn"},"value":"urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"},{"qualifier":{"classid":"scp-number"},"value":"79953761260"},{"qualifier":{"classid":"pmc"},"value":"21459329"}], "collectedfrom" : [ { "key" : "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "value" : "Crossref"} ], "isGreen": null, "openAccessColor": "gold", "isInDiamondJournal": null, "publiclyFunded": null}
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1g", "resulttype" : { "classid" : "publication" }, "isGreen": true, "openAccessColor": "bronze", "isInDiamondJournal": true, "publiclyFunded": false }
{"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1h", "resulttype" : { "classid" : "publication" }, "isGreen": false, "openAccessColor": null, "isInDiamondJournal": true, "publiclyFunded": false }

View File

@ -0,0 +1,31 @@
{
"id": "50|openapc_____::000023f9cb6e3a247c764daec4273cbc",
"resuttype": {
"classid": "publication"
},
"instance": [
{
"collectedfrom": {
"key": "10|apc_________::e2b1600b229fc30663c8a1f662debddf",
"value": "OpenAPC Global Initiative"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
},
{
"qualifier": {"classid": "pmid"},
"value": "25811027"
}
],
"url":["https://doi.org/10.1155/2015/439379"]
}
]
}

View File

@ -0,0 +1,17 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"pid": [
{
"qualifier": {"classid": "urn"},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {"classid": "scp-number"},
"value": "79953761260"
},
{
"qualifier": {"classid": "pmc"},
"value": "21459329"
}
]
}

View File

@ -0,0 +1,21 @@
{
"id":"50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"instance": [
{
"collectedfrom": {
"key": "10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c",
"value": "Europe PubMed Central"
},
"pid": [
{
"qualifier": {"classid": "doi"},
"value": "10.1016/j.cmet.2010.03.013"
},
{
"qualifier":{"classid":"pmc"},
"value":"21459329"
}
]
}
]
}

View File

@ -0,0 +1,428 @@
{
"author": [
{
"affiliation": null,
"fullname": "Deymier, Ghislaine",
"name": "Ghislaine",
"pid": [],
"rank": 1,
"surname": "Deymier"
},
{
"affiliation": null,
"fullname": "Gaschet, Frédéric",
"name": "Frédéric",
"pid": [],
"rank": 2,
"surname": "Gaschet"
},
{
"affiliation": null,
"fullname": "Pouyanne, Guillaume",
"name": "Guillaume",
"pid": [],
"rank": 3,
"surname": "Pouyanne"
}
],
"bestaccessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": [
{
"key" : "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2",
"value" : "Crossref"
}
],
"context": [],
"contributor": [],
"country": [],
"coverage": [],
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2013-11-30"
},
"dateofcollection": "2024-02-28T00:22:13+0000",
"dateoftransformation": "2024-03-06T08:43:13.253Z",
"description": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "For analyzing the reciprocal interaction between urban sprawl and car use, research has first focused on the link between urban density and mobility. By looking for a reduction in energy consumption, cities have favoured a compact planning development. Then reflection has broadened from the simple density to the wider, multi-dimensional concept of urban form. This controversy has led to a renewal of analysis in term of the costs of urban growth, notably by comparing the costs of \"compact\" and \"sprawled\" development. The idea is to compare the mobility costs of different urban forms. However, most often because of a lack of data, such studies are scarce. This paper suggests an innovative method to compute mobility costs at an infra-urban scale : The Spatialized Travel Account (STA). It is based on the CERTU's travel account methodology at a metropolitan scale. It puts forward an accurate estimate of the mobility costs for each transport mode (individual and public) and for each type of payer (households, firms, local authorities...). In order to test the relationships between mobility costs and urban form, we link the computed costs to morphological characteristics of infra-urban zones, taking in account sociodemographic characteristics of households."
},
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "L'interaction réciproque entre étalement urbain et usage de l'automobile a conduit la recherche à se focaliser sur le lien entre les densités urbaines et la mobilité. En cherchant à réduire leur consommation d'énergie pour les transports, et donc leurs émissions de Gaz à Effet de Serre, les villes ont alors cherché à planifier la \" ville compacte \", privilégiant notamment la reconstruction de la ville sur elle-même et la densification. Par la suite, la réflexion s'est élargie de la simple densité à la notion de forme urbaine et à toutes ses dimensions. Cette controverse devait conduire à un renouveau des analyses en termes de coûts de la croissance urbaine : le débat reste vif, encore aujourd'hui, sur les coûts comparés de la ville étalée et de la ville compacte. Plus largement, il s'agit d'explorer les coûts des différentes formes urbaines en termes de mobilité. Malgré cela, généralement pour des raisons de disponibilité de données, les études sur le sujet restent extrêmement rares. Cet article propose un outil novateur pour mesurer les coûts de la mobilité à l'échelle intraurbaine : le Compte Déplacements Territorialisé (CDT). Il s'inspire de la méthode développée par le CERTU pour l'établissement des Comptes Déplacements Voyageurs à l'échelle métropolitaine. Le CDT propose, pour chacune des zones de l'agglomération, une estimation précise de l'ensemble des coûts liés aux déplacements de personnes, ventilés par mode de transport (individuels et collectifs) et par type de financeurs (ménages, entreprises, collectivités territoriales, etc.). Nous proposons une application de cette méthode à la controverse sur le lien entre forme urbaine et coûts de la mobilité. Les coûts sont reliés aux caractéristiques morphologiques des zones (en termes de densité et de diversité, notamment), en prenant soin de contrôler les facteurs socio-économiques qui influent traditionnellement sur les comportements de mobilité (taille du ménage, revenu, etc.)."
}
],
"eoscifguidelines": [],
"externalReference": [],
"extraInfo": [],
"format": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "application/pdf"
}
],
"fulltext": [],
"id": "50|06cdd3ff4700::4826ac62a11a957fe332e2c291dcfcca",
"instance": [
{
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "Digital Object Identifier",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.46298/cst.12132"
}
],
"collectedfrom": {
"key": "10|openaire____::6824b298c96ba906a3e6a70593affbf5",
"value": "Episciences"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2013-11-30"
},
"hostedby": {
"key": "10|openaire____::6824b298c96ba906a3e6a70593affbf5",
"value": "Episciences"
},
"instanceTypeMapping": [
{
"originalType": "http://purl.org/coar/resource_type/c_6501",
"typeCode": "http://purl.org/coar/resource_type/c_6501",
"typeLabel": "journal article",
"vocabularyName": "openaire::coar_resource_types_3_1"
},
{
"originalType": "http://purl.org/coar/resource_type/c_6501",
"typeCode": "Article",
"typeLabel": "Article",
"vocabularyName": "openaire::user_resource_types"
}
],
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:publication_resource",
"schemename": "dnet:publication_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "CC BY NC SA"
},
"pid": [],
"refereed": {
"classid": "0002",
"classname": "nonPeerReviewed",
"schemeid": "dnet:review_levels",
"schemename": "dnet:review_levels"
},
"url": [
"https://doi.org/10.46298/cst.12132",
"https://cst.episciences.org/12132"
]
}
],
"language": {
"classid": "fra/fre",
"classname": "French",
"schemeid": "dnet:languages",
"schemename": "dnet:languages"
},
"lastupdatetimestamp": 1710636106633,
"metaResourceType": {
"classid": "Research Literature",
"classname": "Research Literature",
"schemeid": "openaire::meta_resource_types",
"schemename": "openaire::meta_resource_types"
},
"originalId": [
"oai:episciences.org:cst:12132",
"50|06cdd3ff4700::4826ac62a11a957fe332e2c291dcfcca"
],
"pid": [],
"publisher": {
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "episciences.org"
},
"relevantdate": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "Accepted",
"classname": "Accepted",
"schemeid": "dnet:dataCite_date",
"schemename": "dnet:dataCite_date"
},
"value": "2024-02-11"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "issued",
"classname": "issued",
"schemeid": "dnet:dataCite_date",
"schemename": "dnet:dataCite_date"
},
"value": "2013-11-30"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "available",
"classname": "available",
"schemeid": "dnet:dataCite_date",
"schemename": "dnet:dataCite_date"
},
"value": "2013-11-30"
}
],
"resourcetype": {
"classid": "journal article",
"classname": "journal article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"resulttype": {
"classid": "publication",
"classname": "publication",
"schemeid": "dnet:result_typologies",
"schemename": "dnet:result_typologies"
},
"source": [],
"subject": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "keyword",
"classname": "keyword",
"schemeid": "dnet:subject_classification_typologies",
"schemename": "dnet:subject_classification_typologies"
},
"value": "JEL: H - Public Economics/H.H7 - State and Local Government • Intergovernmental Relations/H.H7.H72 - State and Local Budget and Expenditures"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "keyword",
"classname": "keyword",
"schemeid": "dnet:subject_classification_typologies",
"schemename": "dnet:subject_classification_typologies"
},
"value": "Local public finance"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "keyword",
"classname": "keyword",
"schemeid": "dnet:subject_classification_typologies",
"schemename": "dnet:subject_classification_typologies"
},
"value": "JEL: R - Urban, Rural, Regional, Real Estate, and Transportation Economics/R.R5 - Regional Government Analysis/R.R5.R51 - Finance in Urban and Rural Economies"
}
],
"title": [
{
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:repository",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "Urban form and the costs of daily mobility. The spatialized travel account tool and its application to the Bordeaux metropolitan area"
}
]
}

View File

@ -0,0 +1,23 @@
{
"id": "50|DansKnawCris::0829b5191605bdbea36d6502b8c1ce1f",
"pid": [
{
"qualifier": {
"classid": "urn"
},
"value": "urn:nbn:nl:ui:29-f3ed5f9e-edf6-457e-8848-61b58a4075e2"
},
{
"qualifier": {
"classid": "scp-number"
},
"value": "79953761260"
},
{
"qualifier": {
"classid": "pmcid"
},
"value": "21459329"
}
]
}

File diff suppressed because one or more lines are too long

View File

@ -1,14 +1,14 @@
package eu.dnetlib.dhp.actionmanager.promote;
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
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.utils.MergeUtils;
import java.util.function.BiFunction;
import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass;
/** OAF model merging support. */
public class MergeAndGet {

View File

@ -8,6 +8,7 @@ import static org.mockito.Mockito.*;
import java.util.function.BiFunction;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Nested;
import org.junit.jupiter.api.Test;
@ -85,6 +86,7 @@ public class MergeAndGetTest {
}
@Test
@Disabled
void shouldBehaveProperlyForRelationAndRelation() {
// given
Relation a = mock(Relation.class);
@ -96,7 +98,9 @@ public class MergeAndGetTest {
// then
Oaf x = fn.get().apply(a, b);
assertTrue(Relation.class.isAssignableFrom(x.getClass()));
verify(a).mergeFrom(b);
// TODO should be reimplemented
// verify(a).mergeFrom(b);
assertEquals(a, x);
}
@ -145,6 +149,7 @@ public class MergeAndGetTest {
}
@Test
@Disabled
void shouldBehaveProperlyForOafEntityAndOafEntity() {
// given
OafEntity a = mock(OafEntity.class);
@ -156,7 +161,9 @@ public class MergeAndGetTest {
// then
Oaf x = fn.get().apply(a, b);
assertTrue(OafEntity.class.isAssignableFrom(x.getClass()));
verify(a).mergeFrom(b);
// TODO should be reimplemented
// verify(a).mergeFrom(b);
assertEquals(a, x);
}
}

View File

@ -64,6 +64,12 @@ 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 dataciteInputPath = parser.get("dataciteInputPath");
log.info("dataciteInputPath: {}", dataciteInputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
@ -85,8 +91,20 @@ 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);
List<KeyValue> collectedFromDatacite = OafMapperUtils
.listKeyValues(ModelConstants.DATACITE_ID, "Datacite");
JavaPairRDD<Text, Text> dataciteRelations = prepareAffiliationRelations(
spark, dataciteInputPath, collectedFromDatacite);
crossrefRelations
.union(pubmedRelations)
.union(openAPCRelations)
.union(dataciteRelations)
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);

View File

@ -38,7 +38,6 @@ public class BipProjectModel {
return projectId;
}
// each project bip measure has exactly one value, hence one key-value pair
private Measure createMeasure(String measureId, String measureValue) {

View File

@ -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));

View File

@ -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")

View File

@ -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);
}
}

View File

@ -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))
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;

View File

@ -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);
}
}
}
}
}
}

View File

@ -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);
}
}

View File

@ -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"));
}
}

View File

@ -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;
}
}

View File

@ -1,7 +1,26 @@
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 org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.Dataset;
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.JsonTopic;
@ -15,25 +34,8 @@ 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;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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
*

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -5,6 +5,7 @@ import static eu.dnetlib.dhp.actionmanager.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
@ -13,7 +14,9 @@ import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
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.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
@ -68,18 +71,59 @@ public class SparkAtomicActionUsageJob implements Serializable {
final String workingPath = parser.get("workingPath");
final String datasourcePath = parser.get("datasourcePath");
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
removeOutputDir(spark, outputPath);
prepareData(dbname, spark, workingPath + "/usageDb", "usage_stats", "result_id");
prepareResultData(
dbname, spark, workingPath + "/usageDb",
"usage_stats",
"result_id",
"repository_id",
datasourcePath);
prepareData(dbname, spark, workingPath + "/projectDb", "project_stats", "id");
prepareData(dbname, spark, workingPath + "/datasourceDb", "datasource_stats", "repository_id");
writeActionSet(spark, workingPath, outputPath);
});
}
private static void prepareResultData(String dbname, SparkSession spark, String workingPath, String tableName,
String resultAttributeName, String datasourceAttributeName,
String datasourcePath) {
Dataset<UsageStatsResultModel> resultModel = spark
.sql(
String
.format(
"select %s as id, %s as datasourceId, sum(downloads) as downloads, sum(views) as views " +
"from %s.%s group by %s, %s",
resultAttributeName, datasourceAttributeName, dbname, tableName, resultAttributeName,
datasourceAttributeName))
.as(Encoders.bean(UsageStatsResultModel.class));
Dataset<Datasource> datasource = readPath(spark, datasourcePath, Datasource.class)
.filter((FilterFunction<Datasource>) d -> !d.getDataInfo().getDeletedbyinference())
.map((MapFunction<Datasource, Datasource>) d -> {
d.setId(d.getId().substring(3));
return d;
}, Encoders.bean(Datasource.class));
resultModel
.joinWith(datasource, resultModel.col("datasourceId").equalTo(datasource.col("id")), "left")
.map((MapFunction<Tuple2<UsageStatsResultModel, Datasource>, UsageStatsResultModel>) t2 -> {
UsageStatsResultModel usrm = t2._1();
if(Optional.ofNullable(t2._2()).isPresent())
usrm.setDatasourceId(usrm.getDatasourceId() + "||" + t2._2().getOfficialname().getValue());
else
usrm.setDatasourceId(usrm.getDatasourceId() + "||NO_MATCH_FOUND");
return usrm;
}, Encoders.bean(UsageStatsResultModel.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath);
}
private static void prepareData(String dbname, SparkSession spark, String workingPath, String tableName,
String attribute_name) {
spark
@ -115,15 +159,62 @@ public class SparkAtomicActionUsageJob implements Serializable {
}
public static Measure newMeasureInstance(String id) {
Measure m = new Measure();
m.setId(id);
m.setUnit(new ArrayList<>());
return m;
}
private static Dataset<Result> getFinalIndicatorsResult(SparkSession spark, String inputPath) {
return readPath(spark, inputPath, UsageStatsModel.class)
.map((MapFunction<UsageStatsModel, Result>) usm -> {
return readPath(spark, inputPath, UsageStatsResultModel.class)
.groupByKey((MapFunction<UsageStatsResultModel, String>) usm -> usm.getId(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, UsageStatsResultModel, Result>) (k, it) -> {
Result r = new Result();
r.setId("50|" + usm.getId());
r.setMeasures(getMeasure(usm.getDownloads(), usm.getViews()));
r.setId("50|" + k);
// id = download or view and unit = list of key value pairs
Measure download = newMeasureInstance("downloads");
Measure view = newMeasureInstance("views");
UsageStatsResultModel first = it.next();
addCountForDatasource(download, first, view);
it.forEachRemaining(usm -> {
addCountForDatasource(download, usm, view);
});
r.setMeasures(Arrays.asList(download, view));
return r;
}, Encoders.bean(Result.class));
}, Encoders.bean(Result.class))
// .map((MapFunction<UsageStatsResultModel, Result>) usm -> {
// Result r = new Result();
// r.setId("50|" + usm.getId());
// r.setMeasures(getMeasure(usm.getDownloads(), usm.getViews()));
// return r;
// }, Encoders.bean(Result.class));
;
}
private static void addCountForDatasource(Measure download, UsageStatsResultModel usm, Measure view) {
DataInfo dataInfo = OafMapperUtils
.dataInfo(
false,
UPDATE_DATA_INFO_TYPE,
true,
false,
OafMapperUtils
.qualifier(
UPDATE_MEASURE_USAGE_COUNTS_CLASS_ID,
UPDATE_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"");
download
.getUnit()
.add(
OafMapperUtils
.newKeyValueInstance(usm.getDatasourceId(), String.valueOf(usm.getDownloads()), dataInfo));
view
.getUnit()
.add(OafMapperUtils.newKeyValueInstance(usm.getDatasourceId(), String.valueOf(usm.getViews()), dataInfo));
}
private static Dataset<Project> getFinalIndicatorsProject(SparkSession spark, String inputPath) {

View File

@ -0,0 +1,18 @@
package eu.dnetlib.dhp.actionmanager.usagestats;
/**
* @author miriam.baglioni
* @Date 30/06/23
*/
public class UsageStatsResultModel extends UsageStatsModel {
private String datasourceId;
public String getDatasourceId() {
return datasourceId;
}
public void setDatasourceId(String datasourceId) {
this.datasourceId = datasourceId;
}
}

View File

@ -48,23 +48,30 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
@Override
public MetadataRecord call(MetadataRecord value) {
aggregationCounter.getTotalItems().add(1);
Processor processor = new Processor(false);
processor.registerExtensionFunction(cleanFunction);
processor.registerExtensionFunction(new DateCleaner());
processor.registerExtensionFunction(new PersonCleaner());
final XsltCompiler comp = processor.newXsltCompiler();
QName datasourceIDParam = new QName(DATASOURCE_ID_PARAM);
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;
XdmNode source;
try {
Processor processor = new Processor(false);
processor.registerExtensionFunction(cleanFunction);
processor.registerExtensionFunction(new DateCleaner());
processor.registerExtensionFunction(new PersonCleaner());
final XsltCompiler comp = processor.newXsltCompiler();
QName datasourceIDParam = new QName(DATASOURCE_ID_PARAM);
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
xslt = comp
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
XdmNode source = processor
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();

View File

@ -17,6 +17,18 @@
"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": "dip",
"paramLongName": "dataciteInputPath",
"paramDescription": "the path to get the input data from Datacite",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",

View File

@ -31,6 +31,9 @@ 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
dataciteInputPath=/data/bip-affiliations/datacite-data.json
outputPath=/tmp/crossref-affiliations-output-v5

View File

@ -9,6 +9,14 @@
<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>dataciteInputPath</name>
<description>the path where to find the inferred affiliation relations from Datacite</description>
</property>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
@ -102,6 +110,9 @@
</spark-opts>
<arg>--crossrefInputPath</arg><arg>${crossrefInputPath}</arg>
<arg>--pubmedInputPath</arg><arg>${pubmedInputPath}</arg>
<arg>--openapcInputPath</arg><arg>${openapcInputPath}</arg>
<arg>--dataciteInputPath</arg><arg>${dataciteInputPath}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>

View File

@ -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
}
]

View File

@ -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
}
]

View File

@ -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>

View File

@ -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>

View File

@ -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
}
]

View File

@ -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
}

View File

@ -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"/>

View File

@ -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
}
]

View File

@ -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
}
]

View File

@ -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
}
]

View File

@ -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>

View File

@ -0,0 +1,2 @@
#!/bin/bash
curl -L $1 | hdfs dfs -put - $2

View File

@ -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>

View File

@ -28,5 +28,11 @@
"paramLongName": "workingPath",
"paramDescription": "the workingPath where to save the content of the usage_stats table",
"paramRequired": true
},
{
"paramName": "dp",
"paramLongName": "datasourcePath",
"paramDescription": "the workingPath where to save the content of the usage_stats table",
"paramRequired": true
}
]

View File

@ -90,6 +90,7 @@
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--usagestatsdb</arg><arg>${usagestatsdb}</arg>
<arg>--workingPath</arg><arg>${workingDir}</arg>
<arg>--datasourcePath</arg><arg>${datasourcePath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -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,9 @@ public class PrepareAffiliationRelationsTest {
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-crossrefInputPath", crossrefAffiliationRelationPath,
"-pubmedInputPath", pubmedAffiliationRelationsPath,
"-pubmedInputPath", crossrefAffiliationRelationPath,
"-openapcInputPath", crossrefAffiliationRelationPath,
"-dataciteInputPath", crossrefAffiliationRelationPath,
"-outputPath", outputPath
});
@ -106,7 +104,7 @@ public class PrepareAffiliationRelationsTest {
// );
// }
// count the number of relations
assertEquals(40, tmp.count());
assertEquals(80, tmp.count());
Dataset<Relation> dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
dataset.createOrReplaceTempView("result");
@ -117,7 +115,7 @@ public class PrepareAffiliationRelationsTest {
// verify that we have equal number of bi-directional relations
Assertions
.assertEquals(
20, execVerification
40, execVerification
.filter(
"relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'")
.collectAsList()
@ -125,7 +123,7 @@ public class PrepareAffiliationRelationsTest {
Assertions
.assertEquals(
20, execVerification
40, execVerification
.filter(
"relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'")
.collectAsList()

View File

@ -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)));
}
}

View File

@ -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)));
}
}

View File

@ -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)));
}

View File

@ -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"
});
}
}

View File

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

View File

@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.oaf.Measure;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
@ -66,10 +67,380 @@ public class SparkAtomicActionCountJobTest {
spark.stop();
}
@Test
void testUsageStatsDb2() {
String usageScoresPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/usagestats/test2")
.getPath();
SparkAtomicActionUsageJob.writeActionSet(spark, usageScoresPath, workingDir.toString() + "/actionSet");
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<AtomicAction> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(usm -> OBJECT_MAPPER.readValue(usm._2.getBytes(), AtomicAction.class));
// .map(aa -> (Result) aa.getPayload());
Assertions.assertEquals(7, tmp.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("50|")).count());
Assertions.assertEquals(9, tmp.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("10|")).count());
Assertions.assertEquals(9, tmp.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("40|")).count());
tmp.foreach(r -> Assertions.assertEquals(2, ((OafEntity) r.getPayload()).getMeasures().size()));
tmp
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(u -> Assertions.assertFalse(u.getDataInfo().getDeletedbyinference()))));
tmp
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m.getUnit().stream().forEach(u -> Assertions.assertTrue(u.getDataInfo().getInferred()))));
tmp
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(u -> Assertions.assertFalse(u.getDataInfo().getInvisible()))));
tmp
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"measure:usage_counts",
u.getDataInfo().getProvenanceaction().getClassid()))));
tmp
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"Inferred by OpenAIRE",
u.getDataInfo().getProvenanceaction().getClassname()))));
tmp
.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("40|"))
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"count",
u.getKey()))));
Assertions
.assertEquals(
1,
tmp
.filter(
r -> ((OafEntity) r.getPayload())
.getId()
.equals("50|dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6"))
.count());
OafEntity entity = (OafEntity) tmp
.filter(
aa -> ((OafEntity) aa.getPayload()).getId().equals("50|dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6"))
.first()
.getPayload();
entity
.getMeasures()
.stream()
.forEach(
m -> Assertions.assertEquals(3, m.getUnit().size()));
Measure downloads = entity
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.findFirst()
.get();
Assertions
.assertEquals(
String.valueOf(0),
downloads.getUnit().stream().filter(u -> u.getKey().equals("10|fake1")).findFirst().get().getValue());
Assertions
.assertEquals(
String.valueOf(0),
downloads.getUnit().stream().filter(u -> u.getKey().equals("10|fake2")).findFirst().get().getValue());
Assertions
.assertEquals(
String.valueOf(1),
downloads.getUnit().stream().filter(u -> u.getKey().equals("10|fake3")).findFirst().get().getValue());
Measure views = entity
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.findFirst()
.get();
Assertions
.assertEquals(
String.valueOf(5),
views.getUnit().stream().filter(u -> u.getKey().equals("10|fake1")).findFirst().get().getValue());
Assertions
.assertEquals(
String.valueOf(1),
views.getUnit().stream().filter(u -> u.getKey().equals("10|fake2")).findFirst().get().getValue());
Assertions
.assertEquals(
String.valueOf(3),
views.getUnit().stream().filter(u -> u.getKey().equals("10|fake3")).findFirst().get().getValue());
tmp
.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("10|"))
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"count",
u.getKey()))));
Assertions
.assertEquals(
"0",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f1__________::53575dc69e9ace947e02d47ecd54a7a6"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"5",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f1__________::53575dc69e9ace947e02d47ecd54a7a6"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"0",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f11_________::17eda2ff77407538fbe5d3d719b9d1c0"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"1",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f11_________::17eda2ff77407538fbe5d3d719b9d1c0"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"2",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f12_________::3085e4c6e051378ca6157fe7f0430c1f"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"6",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("40|f12_________::3085e4c6e051378ca6157fe7f0430c1f"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"0",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d1__________::53575dc69e9ace947e02d47ecd54a7a6"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"5",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d1__________::53575dc69e9ace947e02d47ecd54a7a6"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"0",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d11_________::17eda2ff77407538fbe5d3d719b9d1c0"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"1",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d11_________::17eda2ff77407538fbe5d3d719b9d1c0"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"2",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d12_________::3085e4c6e051378ca6157fe7f0430c1f"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("downloads"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"6",
tmp
.map(r -> ((OafEntity) r.getPayload()))
.filter(r -> r.getId().equals("10|d12_________::3085e4c6e051378ca6157fe7f0430c1f"))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(m -> m.getId().equals("views"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
}
@Test
void testMatch() {
String usageScoresPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/usagestats")
.getResource("/eu/dnetlib/dhp/actionmanager/usagestats/test1")
.getPath();
SparkAtomicActionUsageJob.writeActionSet(spark, usageScoresPath, workingDir.toString() + "/actionSet");
@ -144,6 +515,39 @@ public class SparkAtomicActionCountJobTest {
u.getDataInfo().getProvenanceaction().getClassname()))));
tmp
.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("40|"))
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"count",
u.getKey()))));
tmp
.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("50|"))
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
.stream()
.forEach(
m -> m
.getUnit()
.stream()
.forEach(
u -> Assertions
.assertEquals(
"10|fake1",
u.getKey()))));
tmp
.filter(aa -> ((OafEntity) aa.getPayload()).getId().startsWith("10|"))
.foreach(
r -> ((OafEntity) r.getPayload())
.getMeasures()
@ -465,5 +869,4 @@ public class SparkAtomicActionCountJobTest {
.get(0)
.getValue());
}
}

View File

@ -82,9 +82,9 @@ public class BaseTransfomationTest extends AbstractVocabularyTest {
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();

View File

@ -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();

View File

@ -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
1 DOI OAID level1 level2 level3 level4 score_for_L3 score_for_L4
2 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.5 0.0
3 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0303 health sciences 030304 developmental biology N/A 0.5 0.0
4 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
5 N/A od______2806::a1da9d2678b12969a9ab5f50b5e71d0a 05 social sciences 0501 psychology and cognitive sciences 050105 experimental psychology 05010501 Emotion/Affective science 0.44109055399894714 0.4833236634731293
6 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050203 business & management 05020302 Supply chain management/Business terms 0.5459638833999634 0.5460261106491089
7 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050211 marketing 05021102 Services marketing/Retailing 0.4540362060070038 0.4539738595485687
8 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
9 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
10 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
11 N/A od______2806::4b9a664dd6b8b04204cb613e7bc9c873 03 medical and health sciences 0302 clinical medicine 030204 cardiovascular system & hematology N/A 0.49318668246269226 0.0
12 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
13 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
14 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
15 N/A od______3978::6704dcced0fe3dd6fbf985dc2507f61c 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.48656824231147766 0.0
16 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
17 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
18 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 03 medical and health sciences 0302 clinical medicine 030217 neurology & neurosurgery 03021708 Neurotrauma/Stroke 0.5014800429344177 0.5109656453132629
19 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 02 engineering and technology 0206 medical engineering 020601 biomedical engineering 02060102 Medical terminology/Patient 0.4985199570655823 0.4890343248844147

View File

@ -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

View File

@ -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"}

View File

@ -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
1 omid id
2 br/061201599020 doi:10.1142/s0219887817501687
3 br/06203041400 doi:10.1111/j.1523-5378.2005.00327.x pmid:16104945
4 br/06210273177 doi:10.1090/qam/20394
5 br/06502272390 pmid:32235596 doi:10.3390/nano10040644
6 br/0660613430 doi:10.1007/bf00470411
7 br/06120941789 doi:10.1098/rspa.2006.1747
8 br/062602732073 doi:10.1007/978-3-642-38844-6_25
9 br/06230199640 pmid:25088780 doi:10.1016/j.ymeth.2014.07.008
10 br/061103703697 pmid:2682767
11 br/0670517081 doi:10.1016/j.foodpol.2021.102189
12 br/06502310477 doi:10.1142/s0218127416500450
13 br/06520113284 doi:10.1109/cfasta57821.2023.10243367
14 br/062303652439 pmid:5962654 doi:10.1016/0020-708x(66)90001-9
15 br/06250691436 doi:10.1042/bst20150052 pmid:26009172
16 br/061201665577 doi:10.1097/00115550-200205000-00018
17 br/06503490336 pmid:34689254 doi:10.1007/s10072-021-05687-0
18 br/06220615942 pmid:25626134 doi:10.1016/j.jcis.2015.01.008
19 br/061103389243 doi:10.4324/9780203702819-10
20 br/062303011271 doi:10.1109/icassp.2011.5946250
21 br/061302926083 doi:10.4018/978-1-6684-3937-1.ch002
22 br/061402485360 doi:10.1109/iciict.2015.7396079
23 br/06410101083 doi:10.1016/j.autcon.2023.104828
24 br/062202243386 doi:10.1016/0001-8791(81)90022-1
25 br/06170421486 doi:10.1130/0016-7606(2003)115<0166:dsagmf>2.0.co;2
26 br/061201983865 doi:10.4324/9781315109008 isbn:9781315109008
27 br/061701697230 doi:10.1016/j.trd.2012.07.006
28 br/061201137111 doi:10.1109/access.2020.2971656
29 br/06120436283 pmid:2254430 doi:10.1128/jcm.28.11.2551-2554.1990
30 br/061903968916 doi:10.1111/j.1742-1241.1988.tb08627.x
31 br/06201583482 doi:10.1016/0016-5085(78)93139-6
32 br/06130338317 doi:10.2134/agronj1952.00021962004400080013x
33 br/062601538320 doi:10.1371/journal.pone.0270593 pmid:35789338
34 br/062401098626 pmid:22385804 doi:10.1016/j.talanta.2011.12.034
35 br/06190436492 doi:10.1039/c7dt01499f pmid:28644489
36 br/06202819247 doi:10.1007/978-3-319-45823-6_57
37 br/0648013560 doi:10.1080/14772000.2012.705356
38 br/0690214059 doi:10.2752/175630608x329217
39 br/06601640415 doi:10.1080/18128600508685647
40 br/061503394761 doi:10.1002/0471443395.img018
41 br/061702861849 pmid:31203682 doi:10.1080/10428194.2019.1627538
42 br/06450133713 doi:10.1093/acprof:oso/9780199670888.003.0008
43 br/0628074892 doi:10.1097/hnp.0000000000000597
44 br/061601032219 doi:10.1002/bdm.2102
45 br/06602079930 doi:10.1101/2020.08.25.267500
46 br/0604192147 doi:10.11501/3307395
47 br/061101933800 doi:10.1142/s0217732398002242
48 br/06504184118 pmid:10091417

View File

@ -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"}

View File

@ -0,0 +1,9 @@
{"id":"dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6","datasourceId":"10|fake1","downloads":0,"views":5}
{"id":"doi_________::17eda2ff77407538fbe5d3d719b9d1c0","datasourceId":"10|fake1","downloads":0,"views":1}
{"id":"doi_________::1d4dc08605fd0a2be1105d30c63bfea1","datasourceId":"10|fake1","downloads":1,"views":3}
{"id":"doi_________::2e3527822854ca9816f6dfea5bff61a8","datasourceId":"10|fake1","downloads":1,"views":1}
{"id":"doi_________::3085e4c6e051378ca6157fe7f0430c1f","datasourceId":"10|fake1","downloads":2,"views":6}
{"id":"doi_________::33f710e6dd30cc5e67e35b371ddc33cf","datasourceId":"10|fake1","downloads":0,"views":1}
{"id":"doi_________::39738ebf10654732dd3a7af9f24655f8","datasourceId":"10|fake1","downloads":1,"views":3}
{"id":"doi_________::3c3b65f07c1a06c7894397eda1d11bbf","datasourceId":"10|fake1","downloads":1,"views":10}
{"id":"doi_________::4938a71a884dd481d329657aa543b850","datasourceId":"10|fake1","downloads":0,"views":3}

View File

@ -0,0 +1,9 @@
{"id":"d1__________::53575dc69e9ace947e02d47ecd54a7a6","downloads":0,"views":5}
{"id":"d11_________::17eda2ff77407538fbe5d3d719b9d1c0","downloads":0,"views":1}
{"id":"d11_________::1d4dc08605fd0a2be1105d30c63bfea1","downloads":1,"views":3}
{"id":"d11_________::2e3527822854ca9816f6dfea5bff61a8","downloads":1,"views":1}
{"id":"d12_________::3085e4c6e051378ca6157fe7f0430c1f","downloads":2,"views":6}
{"id":"d12_________::33f710e6dd30cc5e67e35b371ddc33cf","downloads":0,"views":1}
{"id":"d12_________::39738ebf10654732dd3a7af9f24655f8","downloads":1,"views":3}
{"id":"d13_________::3c3b65f07c1a06c7894397eda1d11bbf","downloads":1,"views":10}
{"id":"d13_________::4938a71a884dd481d329657aa543b850","downloads":0,"views":3}

View File

@ -0,0 +1,9 @@
{"id":"f1__________::53575dc69e9ace947e02d47ecd54a7a6","downloads":0,"views":5}
{"id":"f11_________::17eda2ff77407538fbe5d3d719b9d1c0","downloads":0,"views":1}
{"id":"f11_________::1d4dc08605fd0a2be1105d30c63bfea1","downloads":1,"views":3}
{"id":"f11_________::2e3527822854ca9816f6dfea5bff61a8","downloads":1,"views":1}
{"id":"f12_________::3085e4c6e051378ca6157fe7f0430c1f","downloads":2,"views":6}
{"id":"f12_________::33f710e6dd30cc5e67e35b371ddc33cf","downloads":0,"views":1}
{"id":"f12_________::39738ebf10654732dd3a7af9f24655f8","downloads":1,"views":3}
{"id":"f13_________::3c3b65f07c1a06c7894397eda1d11bbf","downloads":1,"views":10}
{"id":"f13_________::4938a71a884dd481d329657aa543b850","downloads":0,"views":3}

View File

@ -0,0 +1,9 @@
{"id":"dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6","datasourceId":"10|fake1","downloads":0,"views":5}
{"id":"dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6","datasourceId":"10|fake2","downloads":0,"views":1}
{"id":"dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6","datasourceId":"10|fake3","downloads":1,"views":3}
{"id":"doi_________::2e3527822854ca9816f6dfea5bff61a8","datasourceId":"10|fake1","downloads":1,"views":1}
{"id":"doi_________::3085e4c6e051378ca6157fe7f0430c1f","datasourceId":"10|fake1","downloads":2,"views":6}
{"id":"doi_________::33f710e6dd30cc5e67e35b371ddc33cf","datasourceId":"10|fake1","downloads":0,"views":1}
{"id":"doi_________::39738ebf10654732dd3a7af9f24655f8","datasourceId":"10|fake1","downloads":1,"views":3}
{"id":"doi_________::3c3b65f07c1a06c7894397eda1d11bbf","datasourceId":"10|fake1","downloads":1,"views":10}
{"id":"doi_________::4938a71a884dd481d329657aa543b850","datasourceId":"10|fake1","downloads":0,"views":3}

View File

@ -1,9 +0,0 @@
{"id":"dedup_wf_001::53575dc69e9ace947e02d47ecd54a7a6","downloads":0,"views":5}
{"id":"doi_________::17eda2ff77407538fbe5d3d719b9d1c0","downloads":0,"views":1}
{"id":"doi_________::1d4dc08605fd0a2be1105d30c63bfea1","downloads":1,"views":3}
{"id":"doi_________::2e3527822854ca9816f6dfea5bff61a8","downloads":1,"views":1}
{"id":"doi_________::3085e4c6e051378ca6157fe7f0430c1f","downloads":2,"views":6}
{"id":"doi_________::33f710e6dd30cc5e67e35b371ddc33cf","downloads":0,"views":1}
{"id":"doi_________::39738ebf10654732dd3a7af9f24655f8","downloads":1,"views":3}
{"id":"doi_________::3c3b65f07c1a06c7894397eda1d11bbf","downloads":1,"views":10}
{"id":"doi_________::4938a71a884dd481d329657aa543b850","downloads":0,"views":3}

View File

@ -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']) &gt; 0">
<oaf:refereed>
<xsl:value-of select="'0001'" />
</oaf:refereed>
</xsl:when>
<xsl:when test="count($varRefereed[. = '0002']) &gt; 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/&gt;]*/[^\s&gt;]+)')], //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>

Some files were not shown because too many files have changed in this diff Show More