Merge branch 'master' into graph_cleaning

This commit is contained in:
Claudio Atzori 2020-06-10 15:00:35 +02:00
commit 67c7b31ba6
36 changed files with 702 additions and 235 deletions

View File

@ -1,66 +1,72 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>dhp-workflows</artifactId>
<groupId>eu.dnetlib.dhp</groupId>
<version>1.2.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>dhp-workflows</artifactId>
<groupId>eu.dnetlib.dhp</groupId>
<version>1.2.2-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>dhp-broker-events</artifactId>
<artifactId>dhp-broker-events</artifactId>
<dependencies>
<dependencies>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_2.11</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-hive_2.11</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-schemas</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-schemas</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-pace-core</artifactId>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
<dependency>
<groupId>dom4j</groupId>
<artifactId>dom4j</artifactId>
</dependency>
<dependency>
<groupId>jaxen</groupId>
<artifactId>jaxen</artifactId>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
<dependency>
<groupId>dom4j</groupId>
<artifactId>dom4j</artifactId>
</dependency>
<dependency>
<groupId>jaxen</groupId>
<artifactId>jaxen</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-openaire-broker-common</artifactId>
<version>[2.0.1,3.0.0)</version>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-openaire-broker-common</artifactId>
<version>[2.0.1,3.0.0)</version>
</dependency>
</dependencies>
</dependencies>
</project>

View File

@ -12,16 +12,13 @@ import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Column;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.TypedColumn;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -55,6 +52,9 @@ import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreOpenAccess;
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMorePid;
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreSubject;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import eu.dnetlib.dhp.broker.oa.util.EventGroup;
import eu.dnetlib.dhp.broker.oa.util.ResultAggregator;
import eu.dnetlib.dhp.broker.oa.util.ResultGroup;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
@ -63,6 +63,10 @@ import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import scala.Tuple2;
public class GenerateEventsApplication {
@ -98,7 +102,11 @@ public class GenerateEventsApplication {
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedTo = new EnrichMissingDatasetIsSupplementedTo();
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedBy = new EnrichMissingDatasetIsSupplementedBy();
public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
// Aggregators
private static final TypedColumn<Tuple2<Result, Relation>, ResultGroup> resultAggrTypedColumn = new ResultAggregator()
.toColumn();
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -120,23 +128,31 @@ public class GenerateEventsApplication {
final String eventsPath = parser.get("eventsPath");
log.info("eventsPath: {}", eventsPath);
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final String dedupConfigProfileId = parser.get("dedupConfProfile");
log.info("dedupConfigProfileId: {}", dedupConfigProfileId);
final SparkConf conf = new SparkConf();
final DedupConfig dedupConfig = loadDedupConfig(isLookupUrl, dedupConfigProfileId);
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
removeOutputDir(spark, eventsPath);
final JavaRDD<Event> eventsRdd = sc.emptyRDD();
final Dataset<Event> all = spark.emptyDataset(Encoders.kryo(Event.class));
for (final Class<? extends Result> r1 : BrokerConstants.RESULT_CLASSES) {
eventsRdd.union(generateSimpleEvents(spark, graphPath, r1));
all.union(generateSimpleEvents(spark, graphPath, r1, dedupConfig));
for (final Class<? extends Result> r2 : BrokerConstants.RESULT_CLASSES) {
eventsRdd.union(generateRelationEvents(spark, graphPath, r1, r2));
all.union(generateRelationEvents(spark, graphPath, r1, r2, dedupConfig));
}
}
eventsRdd.saveAsTextFile(eventsPath, GzipCodec.class);
all.write().mode(SaveMode.Overwrite).json(eventsPath);
});
}
@ -145,59 +161,60 @@ public class GenerateEventsApplication {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
private static <R extends Result> JavaRDD<Event> generateSimpleEvents(final SparkSession spark,
private static <R extends Result> Dataset<Event> generateSimpleEvents(final SparkSession spark,
final String graphPath,
final Class<R> resultClazz) {
final Class<R> resultClazz,
final DedupConfig dedupConfig) {
final Dataset<R> results = readPath(
spark, graphPath + "/" + resultClazz.getSimpleName().toLowerCase(), resultClazz)
final Dataset<Result> results = readPath(
spark, graphPath + "/" + resultClazz.getSimpleName().toLowerCase(), Result.class)
.filter(r -> r.getDataInfo().getDeletedbyinference());
final Dataset<Relation> rels = readPath(spark, graphPath + "/relation", Relation.class)
final Dataset<Relation> mergedRels = readPath(spark, graphPath + "/relation", Relation.class)
.filter(r -> r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS));
final Column c = null; // TODO
final Dataset<Row> aa = results
.joinWith(rels, results.col("id").equalTo(rels.col("source")), "inner")
.groupBy(rels.col("target"))
.agg(c)
.filter(x -> x.size() > 1)
// generateSimpleEvents(...)
// flatMap()
// toRdd()
;
return null;
return results
.joinWith(mergedRels, results.col("id").equalTo(mergedRels.col("source")), "inner")
.groupByKey((MapFunction<Tuple2<Result, Relation>, String>) t -> t._2.getTarget(), Encoders.STRING())
.agg(resultAggrTypedColumn)
.map((MapFunction<Tuple2<String, ResultGroup>, ResultGroup>) t -> t._2, Encoders.kryo(ResultGroup.class))
.filter(ResultGroup::isValid)
.map(
(MapFunction<ResultGroup, EventGroup>) g -> GenerateEventsApplication
.generateSimpleEvents(g, dedupConfig),
Encoders.kryo(EventGroup.class))
.flatMap(group -> group.getData().iterator(), Encoders.kryo(Event.class));
}
private List<Event> generateSimpleEvents(final Collection<Result> children) {
private static EventGroup generateSimpleEvents(final ResultGroup results, final DedupConfig dedupConfig) {
final List<UpdateInfo<?>> list = new ArrayList<>();
for (final Result target : children) {
list.addAll(enrichMissingAbstract.searchUpdatesForRecord(target, children));
list.addAll(enrichMissingAuthorOrcid.searchUpdatesForRecord(target, children));
list.addAll(enrichMissingOpenAccess.searchUpdatesForRecord(target, children));
list.addAll(enrichMissingPid.searchUpdatesForRecord(target, children));
list.addAll(enrichMissingPublicationDate.searchUpdatesForRecord(target, children));
list.addAll(enrichMissingSubject.searchUpdatesForRecord(target, children));
list.addAll(enrichMoreOpenAccess.searchUpdatesForRecord(target, children));
list.addAll(enrichMorePid.searchUpdatesForRecord(target, children));
list.addAll(enrichMoreSubject.searchUpdatesForRecord(target, children));
for (final Result target : results.getData()) {
list.addAll(enrichMissingAbstract.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMissingAuthorOrcid.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMissingOpenAccess.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMissingPid.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMissingPublicationDate.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMissingSubject.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMoreOpenAccess.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMorePid.searchUpdatesForRecord(target, results.getData(), dedupConfig));
list.addAll(enrichMoreSubject.searchUpdatesForRecord(target, results.getData(), dedupConfig));
}
return list.stream().map(EventFactory::newBrokerEvent).collect(Collectors.toList());
final EventGroup events = new EventGroup();
list.stream().map(EventFactory::newBrokerEvent).forEach(events::addElement);
return events;
}
private static <SRC extends Result, TRG extends OafEntity> JavaRDD<Event> generateRelationEvents(
private static <SRC extends Result, TRG extends OafEntity> Dataset<Event> generateRelationEvents(
final SparkSession spark,
final String graphPath,
final Class<SRC> sourceClass,
final Class<TRG> targetClass) {
final Class<TRG> targetClass,
final DedupConfig dedupConfig) {
final Dataset<SRC> sources = readPath(
spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), sourceClass)
final Dataset<Result> sources = readPath(
spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), Result.class)
.filter(r -> r.getDataInfo().getDeletedbyinference());
final Dataset<TRG> targets = readPath(
@ -209,6 +226,13 @@ public class GenerateEventsApplication {
final Dataset<Relation> rels = readPath(spark, graphPath + "/relation", Relation.class)
.filter(r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS));
final Dataset<ResultGroup> duplicates = sources
.joinWith(mergedRels, sources.col("id").equalTo(rels.col("source")), "inner")
.groupByKey((MapFunction<Tuple2<Result, Relation>, String>) t -> t._2.getTarget(), Encoders.STRING())
.agg(resultAggrTypedColumn)
.map((MapFunction<Tuple2<String, ResultGroup>, ResultGroup>) t -> t._2, Encoders.kryo(ResultGroup.class))
.filter(ResultGroup::isValid);
if (targetClass == Project.class) {
// TODO join using: generateProjectsEvents
} else if (targetClass == Software.class) {
@ -222,29 +246,32 @@ public class GenerateEventsApplication {
return null;
}
private List<Event> generateProjectsEvents(final Collection<Pair<Result, List<Project>>> childrenWithProjects) {
private List<Event> generateProjectsEvents(final Collection<Pair<Result, List<Project>>> childrenWithProjects,
final DedupConfig dedupConfig) {
final List<UpdateInfo<?>> list = new ArrayList<>();
for (final Pair<Result, List<Project>> target : childrenWithProjects) {
list.addAll(enrichMissingProject.searchUpdatesForRecord(target, childrenWithProjects));
list.addAll(enrichMoreProject.searchUpdatesForRecord(target, childrenWithProjects));
list.addAll(enrichMissingProject.searchUpdatesForRecord(target, childrenWithProjects, dedupConfig));
list.addAll(enrichMoreProject.searchUpdatesForRecord(target, childrenWithProjects, dedupConfig));
}
return list.stream().map(EventFactory::newBrokerEvent).collect(Collectors.toList());
}
private List<Event> generateSoftwareEvents(final Collection<Pair<Result, List<Software>>> childrenWithSoftwares) {
private List<Event> generateSoftwareEvents(final Collection<Pair<Result, List<Software>>> childrenWithSoftwares,
final DedupConfig dedupConfig) {
final List<UpdateInfo<?>> list = new ArrayList<>();
for (final Pair<Result, List<Software>> target : childrenWithSoftwares) {
list.addAll(enrichMissingSoftware.searchUpdatesForRecord(target, childrenWithSoftwares));
list.addAll(enrichMoreSoftware.searchUpdatesForRecord(target, childrenWithSoftwares));
list.addAll(enrichMissingSoftware.searchUpdatesForRecord(target, childrenWithSoftwares, dedupConfig));
list.addAll(enrichMoreSoftware.searchUpdatesForRecord(target, childrenWithSoftwares, dedupConfig));
}
return list.stream().map(EventFactory::newBrokerEvent).collect(Collectors.toList());
}
private List<Event> generatePublicationRelatedEvents(final String relType,
final Collection<Pair<Result, Map<String, List<Publication>>>> childrenWithRels) {
final Collection<Pair<Result, Map<String, List<Publication>>>> childrenWithRels,
final DedupConfig dedupConfig) {
final List<UpdateInfo<?>> list = new ArrayList<>();
@ -257,15 +284,30 @@ public class GenerateEventsApplication {
for (final Pair<Result, List<Publication>> target : cleanedChildrens) {
if (relType.equals("isRelatedTo")) {
list.addAll(enrichMisissingPublicationIsRelatedTo.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMisissingPublicationIsRelatedTo
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("references")) {
list.addAll(enrichMissingPublicationReferences.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingPublicationReferences
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isReferencedBy")) {
list.addAll(enrichMissingPublicationIsReferencedBy.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingPublicationIsReferencedBy
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isSupplementedTo")) {
list.addAll(enrichMissingPublicationIsSupplementedTo.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingPublicationIsSupplementedTo
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isSupplementedBy")) {
list.addAll(enrichMissingPublicationIsSupplementedBy.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingPublicationIsSupplementedBy
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
}
}
@ -274,7 +316,8 @@ public class GenerateEventsApplication {
}
private List<Event> generateDatasetRelatedEvents(final String relType,
final Collection<Pair<Result, Map<String, List<eu.dnetlib.dhp.schema.oaf.Dataset>>>> childrenWithRels) {
final Collection<Pair<Result, Map<String, List<eu.dnetlib.dhp.schema.oaf.Dataset>>>> childrenWithRels,
final DedupConfig dedupConfig) {
final List<UpdateInfo<?>> list = new ArrayList<>();
@ -287,15 +330,29 @@ public class GenerateEventsApplication {
for (final Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>> target : cleanedChildrens) {
if (relType.equals("isRelatedTo")) {
list.addAll(enrichMisissingDatasetIsRelatedTo.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMisissingDatasetIsRelatedTo
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("references")) {
list.addAll(enrichMissingDatasetReferences.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingDatasetReferences.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isReferencedBy")) {
list.addAll(enrichMissingDatasetIsReferencedBy.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingDatasetIsReferencedBy
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isSupplementedTo")) {
list.addAll(enrichMissingDatasetIsSupplementedTo.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingDatasetIsSupplementedTo
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
} else if (relType.equals("isSupplementedBy")) {
list.addAll(enrichMissingDatasetIsSupplementedBy.searchUpdatesForRecord(target, cleanedChildrens));
list
.addAll(
enrichMissingDatasetIsSupplementedBy
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
}
}
@ -312,4 +369,24 @@ public class GenerateEventsApplication {
.textFile(inputPath)
.map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz));
}
private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception {
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final String conf = isLookUpService
.getResourceProfileByQuery(
String
.format(
"for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()",
profId));
final DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class);
dedupConfig.getPace().initModel();
dedupConfig.getPace().initTranslationMap();
// dedupConfig.getWf().setConfigurationId("???");
return dedupConfig;
}
}

View File

@ -12,6 +12,7 @@ import org.apache.commons.lang3.StringUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.pace.config.DedupConfig;
public abstract class UpdateMatcher<K, T> {
@ -21,13 +22,14 @@ public abstract class UpdateMatcher<K, T> {
this.multipleUpdate = multipleUpdate;
}
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final K res, final Collection<K> others) {
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final K res, final Collection<K> others,
final DedupConfig dedupConfig) {
final Map<String, UpdateInfo<T>> infoMap = new HashMap<>();
for (final K source : others) {
if (source != res) {
for (final UpdateInfo<T> info : findUpdates(source, res)) {
for (final UpdateInfo<T> info : findUpdates(source, res, dedupConfig)) {
final String s = DigestUtils.md5Hex(info.getHighlightValueAsString());
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {
} else {
@ -51,11 +53,7 @@ public abstract class UpdateMatcher<K, T> {
}
}
protected abstract List<UpdateInfo<T>> findUpdates(K source, K target);
protected abstract UpdateInfo<T> generateUpdateInfo(final T highlightValue,
final K source,
final K target);
protected abstract List<UpdateInfo<T>> findUpdates(K source, K target, DedupConfig dedupConfig);
protected static boolean isMissing(final List<Field<String>> list) {
return list == null || list.isEmpty() || StringUtils.isBlank(list.get(0).getValue());

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Dataset;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public abstract class AbstractEnrichMissingDataset
extends UpdateMatcher<Pair<Result, List<Dataset>>, eu.dnetlib.broker.objects.Dataset> {
@ -27,7 +28,8 @@ public abstract class AbstractEnrichMissingDataset
@Override
protected final List<UpdateInfo<eu.dnetlib.broker.objects.Dataset>> findUpdates(
final Pair<Result, List<Dataset>> source,
final Pair<Result, List<Dataset>> target) {
final Pair<Result, List<Dataset>> target,
final DedupConfig dedupConfig) {
final Set<String> existingDatasets = target
.getRight()
@ -40,21 +42,22 @@ public abstract class AbstractEnrichMissingDataset
.stream()
.filter(d -> !existingDatasets.contains(d.getId()))
.map(ConversionUtils::oafDatasetToBrokerDataset)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
protected final UpdateInfo<eu.dnetlib.broker.objects.Dataset> generateUpdateInfo(
final eu.dnetlib.broker.objects.Dataset highlightValue,
final Pair<Result, List<Dataset>> source,
final Pair<Result, List<Dataset>> target) {
final Pair<Result, List<Dataset>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
getTopic(),
highlightValue, source.getLeft(), target.getLeft(),
(p, rel) -> p.getDatasets().add(rel),
rel -> rel.getInstances().get(0).getUrl());
rel -> rel.getInstances().get(0).getUrl(),
dedupConfig);
}
public Topic getTopic() {

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingProject
extends UpdateMatcher<Pair<Result, List<Project>>, eu.dnetlib.broker.objects.Project> {
@ -23,7 +24,8 @@ public class EnrichMissingProject
@Override
protected List<UpdateInfo<eu.dnetlib.broker.objects.Project>> findUpdates(final Pair<Result, List<Project>> source,
final Pair<Result, List<Project>> target) {
final Pair<Result, List<Project>> target,
final DedupConfig dedupConfig) {
if (source.getRight().isEmpty()) {
return Arrays.asList();
@ -32,21 +34,21 @@ public class EnrichMissingProject
.getRight()
.stream()
.map(ConversionUtils::oafProjectToBrokerProject)
.map(p -> generateUpdateInfo(p, source, target))
.map(p -> generateUpdateInfo(p, source, target, dedupConfig))
.collect(Collectors.toList());
}
}
@Override
public UpdateInfo<eu.dnetlib.broker.objects.Project> generateUpdateInfo(
final eu.dnetlib.broker.objects.Project highlightValue,
final Pair<Result, List<Project>> source,
final Pair<Result, List<Project>> target) {
final Pair<Result, List<Project>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_PROJECT,
highlightValue, source.getLeft(), target.getLeft(),
(p, prj) -> p.getProjects().add(prj),
prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode());
prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode(), dedupConfig);
}
}

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMoreProject extends UpdateMatcher<Pair<Result, List<Project>>, eu.dnetlib.broker.objects.Project> {
@ -22,7 +23,8 @@ public class EnrichMoreProject extends UpdateMatcher<Pair<Result, List<Project>>
@Override
protected List<UpdateInfo<eu.dnetlib.broker.objects.Project>> findUpdates(final Pair<Result, List<Project>> source,
final Pair<Result, List<Project>> target) {
final Pair<Result, List<Project>> target,
final DedupConfig dedupConfig) {
final Set<String> existingProjects = source
.getRight()
@ -35,20 +37,20 @@ public class EnrichMoreProject extends UpdateMatcher<Pair<Result, List<Project>>
.stream()
.filter(p -> !existingProjects.contains(p.getId()))
.map(ConversionUtils::oafProjectToBrokerProject)
.map(p -> generateUpdateInfo(p, source, target))
.map(p -> generateUpdateInfo(p, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<eu.dnetlib.broker.objects.Project> generateUpdateInfo(
final eu.dnetlib.broker.objects.Project highlightValue,
final Pair<Result, List<Project>> source,
final Pair<Result, List<Project>> target) {
final Pair<Result, List<Project>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MORE_PROJECT,
highlightValue, source.getLeft(), target.getLeft(),
(p, prj) -> p.getProjects().add(prj),
prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode());
prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode(), dedupConfig);
}
}

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public abstract class AbstractEnrichMissingPublication
extends UpdateMatcher<Pair<Result, List<Publication>>, eu.dnetlib.broker.objects.Publication> {
@ -27,7 +28,8 @@ public abstract class AbstractEnrichMissingPublication
@Override
protected final List<UpdateInfo<eu.dnetlib.broker.objects.Publication>> findUpdates(
final Pair<Result, List<Publication>> source,
final Pair<Result, List<Publication>> target) {
final Pair<Result, List<Publication>> target,
final DedupConfig dedupConfig) {
final Set<String> existingPublications = target
.getRight()
@ -40,21 +42,21 @@ public abstract class AbstractEnrichMissingPublication
.stream()
.filter(d -> !existingPublications.contains(d.getId()))
.map(ConversionUtils::oafResultToBrokerPublication)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
protected final UpdateInfo<eu.dnetlib.broker.objects.Publication> generateUpdateInfo(
final eu.dnetlib.broker.objects.Publication highlightValue,
final Pair<Result, List<Publication>> source,
final Pair<Result, List<Publication>> target) {
final Pair<Result, List<Publication>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
getTopic(),
highlightValue, source.getLeft(), target.getLeft(),
(p, rel) -> p.getPublications().add(rel),
rel -> rel.getInstances().get(0).getUrl());
rel -> rel.getInstances().get(0).getUrl(), dedupConfig);
}
public Topic getTopic() {

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingSoftware
extends UpdateMatcher<Pair<Result, List<Software>>, eu.dnetlib.broker.objects.Software> {
@ -24,7 +25,8 @@ public class EnrichMissingSoftware
@Override
protected List<UpdateInfo<eu.dnetlib.broker.objects.Software>> findUpdates(
final Pair<Result, List<Software>> source,
final Pair<Result, List<Software>> target) {
final Pair<Result, List<Software>> target,
final DedupConfig dedupConfig) {
if (source.getRight().isEmpty()) {
return Arrays.asList();
@ -33,21 +35,21 @@ public class EnrichMissingSoftware
.getRight()
.stream()
.map(ConversionUtils::oafSoftwareToBrokerSoftware)
.map(p -> generateUpdateInfo(p, source, target))
.map(p -> generateUpdateInfo(p, source, target, dedupConfig))
.collect(Collectors.toList());
}
}
@Override
public UpdateInfo<eu.dnetlib.broker.objects.Software> generateUpdateInfo(
final eu.dnetlib.broker.objects.Software highlightValue,
final Pair<Result, List<Software>> source,
final Pair<Result, List<Software>> target) {
final Pair<Result, List<Software>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_SOFTWARE,
highlightValue, source.getLeft(), target.getLeft(),
(p, s) -> p.getSoftwares().add(s),
s -> s.getName());
s -> s.getName(), dedupConfig);
}
}

View File

@ -13,6 +13,7 @@ import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMoreSoftware
extends UpdateMatcher<Pair<Result, List<Software>>, eu.dnetlib.broker.objects.Software> {
@ -24,7 +25,8 @@ public class EnrichMoreSoftware
@Override
protected List<UpdateInfo<eu.dnetlib.broker.objects.Software>> findUpdates(
final Pair<Result, List<Software>> source,
final Pair<Result, List<Software>> target) {
final Pair<Result, List<Software>> target,
final DedupConfig dedupConfig) {
final Set<String> existingSoftwares = source
.getRight()
@ -37,20 +39,20 @@ public class EnrichMoreSoftware
.stream()
.filter(p -> !existingSoftwares.contains(p.getId()))
.map(ConversionUtils::oafSoftwareToBrokerSoftware)
.map(p -> generateUpdateInfo(p, source, target))
.map(p -> generateUpdateInfo(p, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<eu.dnetlib.broker.objects.Software> generateUpdateInfo(
final eu.dnetlib.broker.objects.Software highlightValue,
final Pair<Result, List<Software>> source,
final Pair<Result, List<Software>> target) {
final Pair<Result, List<Software>> target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MORE_SOFTWARE,
highlightValue, source.getLeft(), target.getLeft(),
(p, s) -> p.getSoftwares().add(s),
s -> s.getName());
s -> s.getName(), dedupConfig);
}
}

View File

@ -9,6 +9,7 @@ import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingAbstract extends UpdateMatcher<Result, String> {
@ -17,22 +18,24 @@ public class EnrichMissingAbstract extends UpdateMatcher<Result, String> {
}
@Override
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
if (isMissing(target.getDescription()) && !isMissing(source.getDescription())) {
return Arrays.asList(generateUpdateInfo(source.getDescription().get(0).getValue(), source, target));
return Arrays
.asList(generateUpdateInfo(source.getDescription().get(0).getValue(), source, target, dedupConfig));
}
return new ArrayList<>();
}
@Override
public UpdateInfo<String> generateUpdateInfo(final String highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_ABSTRACT,
highlightValue, source, target,
(p, s) -> p.getAbstracts().add(s),
s -> s);
s -> s, dedupConfig);
}
}

View File

@ -10,6 +10,7 @@ import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingAuthorOrcid extends UpdateMatcher<Result, Pair<String, String>> {
@ -18,19 +19,22 @@ public class EnrichMissingAuthorOrcid extends UpdateMatcher<Result, Pair<String,
}
@Override
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
// TODO
// return Arrays.asList(new EnrichMissingAbstract("xxxxxxx", 0.9f));
return Arrays.asList();
}
@Override
public UpdateInfo<Pair<String, String>> generateUpdateInfo(final Pair<String, String> highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_AUTHOR_ORCID,
highlightValue, source, target,
(p, pair) -> p.getCreators().add(pair.getLeft() + " - ORCID: " + pair.getRight()),
pair -> pair.getLeft() + "::" + pair.getRight());
pair -> pair.getLeft() + "::" + pair.getRight(),
dedupConfig);
}
}

View File

@ -12,6 +12,7 @@ import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingOpenAccess extends UpdateMatcher<Result, Instance> {
@ -20,7 +21,8 @@ public class EnrichMissingOpenAccess extends UpdateMatcher<Result, Instance> {
}
@Override
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final long count = target
.getInstance()
.stream()
@ -38,19 +40,19 @@ public class EnrichMissingOpenAccess extends UpdateMatcher<Result, Instance> {
.filter(i -> i.getAccessright().getClassid().equals(BrokerConstants.OPEN_ACCESS))
.map(ConversionUtils::oafInstanceToBrokerInstances)
.flatMap(List::stream)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Instance> generateUpdateInfo(final Instance highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_OA_VERSION,
highlightValue, source, target,
(p, i) -> p.getInstances().add(i),
Instance::getUrl);
Instance::getUrl, dedupConfig);
}
}

View File

@ -11,6 +11,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingPid extends UpdateMatcher<Result, Pid> {
@ -19,7 +20,8 @@ public class EnrichMissingPid extends UpdateMatcher<Result, Pid> {
}
@Override
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final long count = target.getPid().size();
if (count > 0) {
@ -30,17 +32,17 @@ public class EnrichMissingPid extends UpdateMatcher<Result, Pid> {
.getPid()
.stream()
.map(ConversionUtils::oafPidToBrokerPid)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target) {
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_PID,
highlightValue, source, target,
(p, pid) -> p.getPids().add(pid),
pid -> pid.getType() + "::" + pid.getValue());
pid -> pid.getType() + "::" + pid.getValue(), dedupConfig);
}
}

View File

@ -9,6 +9,7 @@ import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingPublicationDate extends UpdateMatcher<Result, String> {
@ -17,22 +18,24 @@ public class EnrichMissingPublicationDate extends UpdateMatcher<Result, String>
}
@Override
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
if (isMissing(target.getDateofacceptance()) && !isMissing(source.getDateofacceptance())) {
return Arrays.asList(generateUpdateInfo(source.getDateofacceptance().getValue(), source, target));
return Arrays
.asList(generateUpdateInfo(source.getDateofacceptance().getValue(), source, target, dedupConfig));
}
return new ArrayList<>();
}
@Override
public UpdateInfo<String> generateUpdateInfo(final String highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MISSING_PUBLICATION_DATE,
highlightValue, source, target,
(p, date) -> p.setPublicationdate(date),
s -> s);
s -> s, dedupConfig);
}
}

View File

@ -14,6 +14,7 @@ import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMissingSubject extends UpdateMatcher<Result, Pair<String, String>> {
@ -22,7 +23,8 @@ public class EnrichMissingSubject extends UpdateMatcher<Result, Pair<String, Str
}
@Override
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final Set<String> existingTypes = target
.getSubject()
.stream()
@ -35,20 +37,20 @@ public class EnrichMissingSubject extends UpdateMatcher<Result, Pair<String, Str
.stream()
.filter(pid -> !existingTypes.contains(pid.getQualifier().getClassid()))
.map(ConversionUtils::oafSubjectToPair)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Pair<String, String>> generateUpdateInfo(final Pair<String, String> highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.fromPath("ENRICH/MISSING/SUBJECT/" + highlightValue.getLeft()),
highlightValue, source, target,
(p, pair) -> p.getSubjects().add(pair.getRight()),
pair -> pair.getLeft() + "::" + pair.getRight());
pair -> pair.getLeft() + "::" + pair.getRight(), dedupConfig);
}
}

View File

@ -12,6 +12,7 @@ import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMoreOpenAccess extends UpdateMatcher<Result, Instance> {
@ -20,7 +21,8 @@ public class EnrichMoreOpenAccess extends UpdateMatcher<Result, Instance> {
}
@Override
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final Set<String> urls = target
.getInstance()
.stream()
@ -36,19 +38,19 @@ public class EnrichMoreOpenAccess extends UpdateMatcher<Result, Instance> {
.map(ConversionUtils::oafInstanceToBrokerInstances)
.flatMap(List::stream)
.filter(i -> !urls.contains(i.getUrl()))
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Instance> generateUpdateInfo(final Instance highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MORE_OA_VERSION,
highlightValue, source, target,
(p, i) -> p.getInstances().add(i),
Instance::getUrl);
Instance::getUrl, dedupConfig);
}
}

View File

@ -11,6 +11,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMorePid extends UpdateMatcher<Result, Pid> {
@ -19,7 +20,8 @@ public class EnrichMorePid extends UpdateMatcher<Result, Pid> {
}
@Override
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final Set<String> existingPids = target
.getPid()
.stream()
@ -31,17 +33,17 @@ public class EnrichMorePid extends UpdateMatcher<Result, Pid> {
.stream()
.filter(pid -> !existingPids.contains(pid.getQualifier().getClassid() + "::" + pid.getValue()))
.map(ConversionUtils::oafPidToBrokerPid)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target) {
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.ENRICH_MORE_PID,
highlightValue, source, target,
(p, pid) -> p.getPids().add(pid),
pid -> pid.getType() + "::" + pid.getValue());
pid -> pid.getType() + "::" + pid.getValue(), dedupConfig);
}
}

View File

@ -12,6 +12,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
public class EnrichMoreSubject extends UpdateMatcher<Result, Pair<String, String>> {
@ -20,7 +21,8 @@ public class EnrichMoreSubject extends UpdateMatcher<Result, Pair<String, String
}
@Override
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target) {
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
final DedupConfig dedupConfig) {
final Set<String> existingSubjects = target
.getSubject()
.stream()
@ -32,20 +34,20 @@ public class EnrichMoreSubject extends UpdateMatcher<Result, Pair<String, String
.stream()
.filter(pid -> !existingSubjects.contains(pid.getQualifier().getClassid() + "::" + pid.getValue()))
.map(ConversionUtils::oafSubjectToPair)
.map(i -> generateUpdateInfo(i, source, target))
.map(i -> generateUpdateInfo(i, source, target, dedupConfig))
.collect(Collectors.toList());
}
@Override
public UpdateInfo<Pair<String, String>> generateUpdateInfo(final Pair<String, String> highlightValue,
final Result source,
final Result target) {
final Result target,
final DedupConfig dedupConfig) {
return new UpdateInfo<>(
Topic.fromPath("ENRICH/MORE/SUBJECT/" + highlightValue.getLeft()),
highlightValue, source, target,
(p, pair) -> p.getSubjects().add(pair.getRight()),
pair -> pair.getLeft() + "::" + pair.getRight());
pair -> pair.getLeft() + "::" + pair.getRight(), dedupConfig);
}
}

View File

@ -15,6 +15,9 @@ public class BrokerConstants {
public static final String OPEN_ACCESS = "OPEN";
public static final String IS_MERGED_IN_CLASS = "isMergedIn";
public static final float MIN_TRUST = 0.25f;
public static final float MAX_TRUST = 1.00f;
public static final List<Class<? extends Result>> RESULT_CLASSES = Arrays
.asList(Publication.class, Dataset.class, Software.class, OtherResearchProduct.class);

View File

@ -0,0 +1,33 @@
package eu.dnetlib.dhp.broker.oa.util;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import eu.dnetlib.dhp.broker.model.Event;
public class EventGroup implements Serializable {
/**
*
*/
private static final long serialVersionUID = 765977943803533130L;
private final List<Event> data = new ArrayList<>();
public List<Event> getData() {
return data;
}
public EventGroup addElement(final Event elem) {
data.add(elem);
return this;
}
public EventGroup addGroup(final EventGroup group) {
data.addAll(group.getData());
return this;
}
}

View File

@ -0,0 +1,51 @@
package eu.dnetlib.dhp.broker.oa.util;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import scala.Tuple2;
public class ResultAggregator extends Aggregator<Tuple2<Result, Relation>, ResultGroup, ResultGroup> {
/**
*
*/
private static final long serialVersionUID = -1492327874705585538L;
@Override
public ResultGroup zero() {
return new ResultGroup();
}
@Override
public ResultGroup reduce(final ResultGroup group, final Tuple2<Result, Relation> t) {
return group.addElement(t._1);
}
@Override
public ResultGroup merge(final ResultGroup g1, final ResultGroup g2) {
return g1.addGroup(g2);
}
@Override
public ResultGroup finish(final ResultGroup group) {
return group;
}
@Override
public Encoder<ResultGroup> bufferEncoder() {
return Encoders.kryo(ResultGroup.class);
}
@Override
public Encoder<ResultGroup> outputEncoder() {
return Encoders.kryo(ResultGroup.class);
}
}

View File

@ -0,0 +1,36 @@
package eu.dnetlib.dhp.broker.oa.util;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import eu.dnetlib.dhp.schema.oaf.Result;
public class ResultGroup implements Serializable {
/**
*
*/
private static final long serialVersionUID = -3360828477088669296L;
private final List<Result> data = new ArrayList<>();
public List<Result> getData() {
return data;
}
public ResultGroup addElement(final Result elem) {
data.add(elem);
return this;
}
public ResultGroup addGroup(final ResultGroup group) {
data.addAll(group.getData());
return this;
}
public boolean isValid() {
return data.size() > 1;
}
}

View File

@ -0,0 +1,23 @@
package eu.dnetlib.dhp.broker.oa.util;
public class TrustUtils {
public static float rescale(final double score, final double threshold) {
if (score >= BrokerConstants.MAX_TRUST) {
return BrokerConstants.MAX_TRUST;
}
final double val = (score - threshold) * (BrokerConstants.MAX_TRUST - BrokerConstants.MIN_TRUST)
/ (BrokerConstants.MAX_TRUST - threshold);
if (val < BrokerConstants.MIN_TRUST) {
return BrokerConstants.MIN_TRUST;
}
if (val > BrokerConstants.MAX_TRUST) {
return BrokerConstants.MAX_TRUST;
}
return (float) val;
}
}

View File

@ -5,6 +5,11 @@ import java.util.List;
import java.util.function.BiConsumer;
import java.util.function.Function;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.broker.objects.OpenAireEventPayload;
import eu.dnetlib.broker.objects.Provenance;
import eu.dnetlib.broker.objects.Publication;
@ -12,6 +17,10 @@ import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.schema.oaf.Instance;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.tree.support.TreeProcessor;
import eu.dnetlib.pace.util.MapDocumentUtil;
public final class UpdateInfo<T> {
@ -29,16 +38,19 @@ public final class UpdateInfo<T> {
private final float trust;
private static final Logger log = LoggerFactory.getLogger(UpdateInfo.class);
public UpdateInfo(final Topic topic, final T highlightValue, final Result source, final Result target,
final BiConsumer<Publication, T> compileHighlight,
final Function<T, String> highlightToString) {
final Function<T, String> highlightToString,
final DedupConfig dedupConfig) {
this.topic = topic;
this.highlightValue = highlightValue;
this.source = source;
this.target = target;
this.compileHighlight = compileHighlight;
this.highlightToString = highlightToString;
this.trust = calculateTrust(source, target);
this.trust = calculateTrust(dedupConfig, source, target);
}
public T getHighlightValue() {
@ -53,9 +65,22 @@ public final class UpdateInfo<T> {
return target;
}
private float calculateTrust(final Result source, final Result target) {
// TODO
return 0.9f;
private float calculateTrust(final DedupConfig dedupConfig, final Result r1, final Result r2) {
try {
final ObjectMapper objectMapper = new ObjectMapper();
final MapDocument doc1 = MapDocumentUtil
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1));
final MapDocument doc2 = MapDocumentUtil
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2));
final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2);
final double threshold = dedupConfig.getWf().getThreshold();
return TrustUtils.rescale(score, threshold);
} catch (final Exception e) {
log.error("Error computing score between results", e);
return BrokerConstants.MIN_TRUST;
}
}
protected Topic getTopic() {

View File

@ -0,0 +1,73 @@
package eu.dnetlib.dhp.broker.oa.util;
import static org.junit.jupiter.api.Assertions.assertTrue;
import org.junit.jupiter.api.Test;
public class TrustUtilsTest {
private static final double THRESHOLD = 0.95;
@Test
public void rescaleTest_1() {
verifyValue(-0.3, BrokerConstants.MIN_TRUST);
}
@Test
public void rescaleTest_2() {
verifyValue(0.0, BrokerConstants.MIN_TRUST);
}
@Test
public void rescaleTest_3() {
verifyValue(0.5, BrokerConstants.MIN_TRUST);
}
@Test
public void rescaleTest_4() {
verifyValue(0.95, BrokerConstants.MIN_TRUST);
}
@Test
public void rescaleTest_5() {
verifyValue(0.96, BrokerConstants.MIN_TRUST);
}
@Test
public void rescaleTest_6() {
verifyValue(0.97, 0.3f);
}
@Test
public void rescaleTest_7() {
verifyValue(0.98, 0.45f);
}
@Test
public void rescaleTest_8() {
verifyValue(0.99, 0.6f);
}
@Test
public void rescaleTest_9() {
verifyValue(1.00, BrokerConstants.MAX_TRUST);
}
@Test
public void rescaleTest_10() {
verifyValue(1.01, BrokerConstants.MAX_TRUST);
}
@Test
public void rescaleTest_11() {
verifyValue(2.00, BrokerConstants.MAX_TRUST);
}
private void verifyValue(final double originalScore, final float expectedTrust) {
final float trust = TrustUtils.rescale(originalScore, THRESHOLD);
System.out.println(trust);
assertTrue(Math.abs(trust - expectedTrust) < 0.01);
}
}

View File

@ -319,15 +319,7 @@ object DoiBoostMappingUtil {
def generateIdentifier (oaf: Result, doi: String): String = {
val id = DHPUtils.md5 (doi.toLowerCase)
if (oaf.isInstanceOf[Dataset] )
return s"60|${
doiBoostNSPREFIX
}${
SEPARATOR
}${
id
}"
s"50|${
return s"50|${
doiBoostNSPREFIX
}${
SEPARATOR

View File

@ -181,6 +181,9 @@ case object Crossref2Oaf {
if (StringUtils.isNotBlank(issuedDate)) {
instance.setDateofacceptance(asField(issuedDate))
}
else {
instance.setDateofacceptance(asField(createdDate.getValue))
}
val s: String = (json \ "URL").extract[String]
val links: List[String] = ((for {JString(url) <- json \ "link" \ "URL"} yield url) ::: List(s)).filter(p => p != null).distinct
if (links.nonEmpty)
@ -242,7 +245,7 @@ case object Crossref2Oaf {
val queue = new mutable.Queue[Relation]
def snfRule(award:String): String = {
def snsfRule(award:String): String = {
var tmp1 = StringUtils.substringAfter(award,"_")
val tmp2 = StringUtils.substringBefore(tmp1,"/")
logger.debug(s"From $award to $tmp2")
@ -317,7 +320,7 @@ case object Crossref2Oaf {
case "10.13039/501100006588" |
"10.13039/501100004488" => generateSimpleRelationFromAward(funder, "irb_hr______", a=>a.replaceAll("Project No.", "").replaceAll("HRZZ-","") )
case "10.13039/501100006769"=> generateSimpleRelationFromAward(funder, "rsf_________", a=>a)
case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snfRule)
case "10.13039/501100001711"=> generateSimpleRelationFromAward(funder, "snsf________", snsfRule)
case "10.13039/501100004410"=> generateSimpleRelationFromAward(funder, "tubitakf____", a =>a)
case "10.10.13039/100004440"=> generateSimpleRelationFromAward(funder, "wt__________", a =>a)
case "10.13039/100004440"=> queue += generateRelation(sourceId,"1e5e62235d094afd01cd56e65112fc63", "wt__________" )

View File

@ -129,16 +129,16 @@ case object ConversionUtil {
val fieldOfStudy = item._2
if (fieldOfStudy != null && fieldOfStudy.subjects != null && fieldOfStudy.subjects.nonEmpty) {
val p: List[StructuredProperty] = fieldOfStudy.subjects.flatMap(s => {
val s1 = createSP(s.DisplayName, "keywords", "dnet:subject_classification_typologies")
val s1 = createSP(s.DisplayName, "keyword", "dnet:subject_classification_typologies")
val di = DoiBoostMappingUtil.generateDataInfo(s.Score.toString)
var resList: List[StructuredProperty] = List(s1)
if (s.MainType.isDefined) {
val maintp = s.MainType.get
val s2 = createSP(s.MainType.get, "keywords", "dnet:subject_classification_typologies")
val s2 = createSP(s.MainType.get, "keyword", "dnet:subject_classification_typologies")
s2.setDataInfo(di)
resList = resList ::: List(s2)
if (maintp.contains(".")) {
val s3 = createSP(maintp.split("\\.").head, "keywords", "dnet:subject_classification_typologies")
val s3 = createSP(maintp.split("\\.").head, "keyword", "dnet:subject_classification_typologies")
s3.setDataInfo(di)
resList = resList ::: List(s3)
}
@ -190,7 +190,7 @@ case object ConversionUtil {
pub.setPid(List(createSP(paper.Doi.toLowerCase, "doi", PID_TYPES)).asJava)
pub.setOriginalId(List(paper.PaperId.toString, paper.Doi.toLowerCase).asJava)
//Set identifier as {50|60} | doiboost____::md5(DOI)
//Set identifier as 50|doiboost____::md5(DOI)
pub.setId(generateIdentifier(pub, paper.Doi.toLowerCase))
val mainTitles = createSP(paper.PaperTitle, "main title", "dnet:dataCite_title")
@ -229,6 +229,8 @@ case object ConversionUtil {
pub.setPublisher(asField(journal.Publisher.get))
if (journal.Issn.isDefined)
j.setIssnPrinted(journal.Issn.get)
j.setVol(paper.Volume)
j.setIss(paper.Issue)
pub.setJournal(j)
}
pub.setCollectedfrom(List(createMAGCollectedFrom()).asJava)
@ -247,7 +249,7 @@ case object ConversionUtil {
pub.setPid(List(createSP(paper.Doi.toLowerCase, "doi", PID_TYPES)).asJava)
pub.setOriginalId(List(paper.PaperId.toString, paper.Doi.toLowerCase).asJava)
//Set identifier as {50|60} | doiboost____::md5(DOI)
//Set identifier as 50 | doiboost____::md5(DOI)
pub.setId(generateIdentifier(pub, paper.Doi.toLowerCase))
val mainTitles = createSP(paper.PaperTitle, "main title", "dnet:dataCite_title")

View File

@ -1,16 +1,10 @@
package eu.dnetlib.doiboost.orcid
import java.io.IOException
import eu.dnetlib.dhp.schema.oaf.{Author, Publication}
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{ORCID, PID_TYPES, createSP, generateDataInfo, generateIdentifier}
import eu.dnetlib.doiboost.crossref.Crossref2Oaf
import org.apache.commons.lang.StringUtils
import org.codehaus.jackson.map.ObjectMapper
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
@ -22,7 +16,7 @@ case class ORCIDItem(oid:String,name:String,surname:String,creditName:String,err
case class ORCIDElement(doi:String, authors:List[ORCIDItem]) {}
object ORCIDToOAF {
val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass)
val logger: Logger = LoggerFactory.getLogger(ORCIDToOAF.getClass)
val mapper = new ObjectMapper
def isJsonValid(inputStr: String): Boolean = {
@ -57,7 +51,7 @@ object ORCIDToOAF {
pub.setId(generateIdentifier(pub, doi.toLowerCase))
try{
pub.setAuthor(input.authors.map(a=> {
generateAuhtor(a.name, a.surname, a.creditName, a.oid)
generateAuthor(a.name, a.surname, a.creditName, a.oid)
}).asJava)
pub.setCollectedfrom(List(DoiBoostMappingUtil.createORIDCollectedFrom()).asJava)
pub.setDataInfo(DoiBoostMappingUtil.generateDataInfo())
@ -69,7 +63,7 @@ object ORCIDToOAF {
}
}
def generateAuhtor(given: String, family: String, fullName:String, orcid: String): Author = {
def generateAuthor(given: String, family: String, fullName:String, orcid: String): Author = {
val a = new Author
a.setName(given)
a.setSurname(family)

View File

@ -0,0 +1,62 @@
Crossref Field,Type,Required,Description (from Crossref),OAF field,Comments
publisher,String,Yes,Name of work's publisher,Result/Publisher,
title,Array of String,Yes,"Work titles, including translated titles","Result/Title with Qualifier(""main title"", ""dnet:dataCite_title"")",
original-title,Array of String,No,Work titles in the work's original publication language,"Result/Title with Qualifier(""alternative title"", ""dnet:dataCite_title"")",
short-title,Array of String,No,Short or abbreviated work titles,"Result/Title with Qualifier(""alternative title"", ""dnet:dataCite_title"")",
abstract,XML String,No,Abstract as a JSON string or a JATS XML snippet encoded into a JSON string,Result/description,
reference-count,Number,Yes,Deprecated Same as references-count,"- ",
references-count,Number,Yes,Count of outbound references deposited with Crossref,N/A,
is-referenced-by-count,Number,Yes,Count of inbound references deposited with Crossref,N/A,
source,String,Yes,Currently always Crossref,Result/source,
prefix,String,Yes,DOI prefix identifier of the form http://id.crossref.org/prefix/DOI_PREFIX,N/A,
DOI,String,Yes,DOI of the work,OafEntity/originalId,
,,,,OafEntity/PID,
,,,,"Oaf/id ",Use to generate the OpenAIRE id in the form 50|doiboost____::md5(DOI)
URL,URL,Yes,URL form of the work's DOI,Instance/url,
member,String,Yes,Member identifier of the form http://id.crossref.org/member/MEMBER_ID,N/A,
type,String,Yes,"Enumeration, one of the type ids from https://api.crossref.org/v1/types",Instance/instancetype,Also use to map the record as OAF Publication or Dataset according to the mapping defined in eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala
created,Date,Yes,Date on which the DOI was first registered,"Result/relevantDate with Qualifier(""created"", ""dnet:dataCite_date"")",
,,,,"Result/dateofacceptance
Instance/dateofacceptance",If crossref.issued is blank
deposited,Date,Yes,Date on which the work metadata was most recently updated,N/A,
indexed,Date,Yes,"Date on which the work metadata was most recently indexed. Re-indexing does not imply a metadata change, see deposited for the most recent metadata change date",Result/lastupdatetimestamp,
issued,Partial Date,Yes,Earliest of published-print and published-online,Result/dateofacceptance,OAF dateofacceptance is used also for the publishing date. It's the date visualised in the OpenAIRE EXPLORE portal.
,,,,Instance/dateofacceptance,
posted,Partial Date,No,Date on which posted content was made available online,"Result/relevantDate with Qualifier(""available"", ""dnet:dataCite_date"")",
accepted,Partial Date,No,"Date on which a work was accepted, after being submitted, during a submission process","Result/relevantDate with Qualifier(""accepted"", ""dnet:dataCite_date"")",
subtitle,Array of String,No,"Work subtitles, including original language and translated","Result/Title with Qualifier(""subtitle"", ""dnet:dataCite_title"")",
container-title,Array of String,No,Full titles of the containing work (usually a book or journal),Publication/Journal/name only in case of Journal title for book title see ISBN Mapping,
short-container-title,Array of String,No,Abbreviated titles of the containing work,N/A,
group-title,String,No,Group title for posted content,N/A,
issue,String,No,Issue number of an article's journal,Publication/Journal/iss,
volume,String,No,Volume number of an article's journal,Publication/Journal/vol,
page,String,No,Pages numbers of an article within its journal,"Publication/Journal/sp
Publication/Journal/ep",Obtain start and end page by splitting by '-'
article-number,String,No,,N/A,
published-print,Partial Date,No,Date on which the work was published in print,"Result/relevantDate with Qualifier(""published-print"", ""dnet:dataCite_date"")",
published-online,Partial Date,No,Date on which the work was published online,"Result/relevantDate with Qualifier(""published-online"", ""dnet:dataCite_date"")",
subject,Array of String,No,"Subject category names, a controlled vocabulary from Sci-Val. Available for most journal articles","Result/subject with Qualifier(""keywords"", ""dnet:subject_classification_typologies""). ","Future improvements: map the controlled vocabulary instead of using the generic ""keywords"" qualifier"
ISSN,Array of String,No,,"Publication/Journal/issn
Publication/Journal/lissn
Publication/Journal/eissn",The mapping depends on the value of issn-type
issn-type,Array of ISSN with Type,No,List of ISSNs with ISSN type information,N/A,Its value guides the setting of the properties in Journal (see row above)
ISBN,Array of String,No,,Publication/source,"In case of Book We can map ISBN and container title on Publication/source using this syntax container-title + ""ISBN: "" + ISBN"
archive,Array of String,No,,N/A,
license,Array of License,No,,Result/Instance/License,
funder,Array of Funder,No,,Relation,Whenever we are able to link to a funder or project integrated into OpenAIRE. Mapping to OpenAIRE funders and projects is in eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala.generateSimpleRelationFromAward
assertion,Array of Assertion,No,,N/A,
author,Array of Contributor,No,,Result/author (with orcid if available),
editor,Array of Contributor,No,,N/A,
chair,Array of Contributor,No,,N/A,
translator,Array of Contributor,No,,N/A,
update-to,Array of Update,No,,N/A,
update-policy,URL,No,Link to an update policy covering Crossmark updates for this work,N/A,
link,Array of Resource Link,No,URLs to full-text locations,Result/Instance/url,
clinical-trial-number,Array of Clinical Trial Number,No,,OafEntity/originalId,
alternative-id,String,No,Other identifiers for the work provided by the depositing member,OafEntity/originalId,
reference,Array of Reference,No,List of references made by the work,,Future improvement: map to references
content-domain,Content Domain,No,Information on domains that support Crossmark for this work,N/A,
relation,Relations,No,Relations to other works,Result/Instance/refereed,"if(relation.has-review) instance.refereed = ""peerReviewed"". "
review,Review,No,Peer review metadata,N/A,
funder,Array of Funder,No,,Relation between Result and Project,"The mapping between Crossref funder elements and OpenAIRE projects is implemented in eu.dnetlib.doiboost.crossref.Crossref2Oaf.mappingFunderToRelations.
The matching is based on Funder DOIs, Funder names, and grant codes. Different mapping approaches are applied to cover the different cases in Crossref records."
1 Crossref Field Type Required Description (from Crossref) OAF field Comments
2 publisher String Yes Name of work's publisher Result/Publisher
3 title Array of String Yes Work titles, including translated titles Result/Title with Qualifier("main title", "dnet:dataCite_title")
4 original-title Array of String No Work titles in the work's original publication language Result/Title with Qualifier("alternative title", "dnet:dataCite_title")
5 short-title Array of String No Short or abbreviated work titles Result/Title with Qualifier("alternative title", "dnet:dataCite_title")
6 abstract XML String No Abstract as a JSON string or a JATS XML snippet encoded into a JSON string Result/description
7 reference-count Number Yes Deprecated Same as references-count -
8 references-count Number Yes Count of outbound references deposited with Crossref N/A
9 is-referenced-by-count Number Yes Count of inbound references deposited with Crossref N/A
10 source String Yes Currently always Crossref Result/source
11 prefix String Yes DOI prefix identifier of the form http://id.crossref.org/prefix/DOI_PREFIX N/A
12 DOI String Yes DOI of the work OafEntity/originalId
13 OafEntity/PID
14 Oaf/id Use to generate the OpenAIRE id in the form 50|doiboost____::md5(DOI)
15 URL URL Yes URL form of the work's DOI Instance/url
16 member String Yes Member identifier of the form http://id.crossref.org/member/MEMBER_ID N/A
17 type String Yes Enumeration, one of the type ids from https://api.crossref.org/v1/types Instance/instancetype Also use to map the record as OAF Publication or Dataset according to the mapping defined in eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala
18 created Date Yes Date on which the DOI was first registered Result/relevantDate with Qualifier("created", "dnet:dataCite_date")
19 Result/dateofacceptance Instance/dateofacceptance If crossref.issued is blank
20 deposited Date Yes Date on which the work metadata was most recently updated N/A
21 indexed Date Yes Date on which the work metadata was most recently indexed. Re-indexing does not imply a metadata change, see deposited for the most recent metadata change date Result/lastupdatetimestamp
22 issued Partial Date Yes Earliest of published-print and published-online Result/dateofacceptance OAF dateofacceptance is used also for the publishing date. It's the date visualised in the OpenAIRE EXPLORE portal.
23 Instance/dateofacceptance
24 posted Partial Date No Date on which posted content was made available online Result/relevantDate with Qualifier("available", "dnet:dataCite_date")
25 accepted Partial Date No Date on which a work was accepted, after being submitted, during a submission process Result/relevantDate with Qualifier("accepted", "dnet:dataCite_date")
26 subtitle Array of String No Work subtitles, including original language and translated Result/Title with Qualifier("subtitle", "dnet:dataCite_title")
27 container-title Array of String No Full titles of the containing work (usually a book or journal) Publication/Journal/name only in case of Journal title for book title see ISBN Mapping
28 short-container-title Array of String No Abbreviated titles of the containing work N/A
29 group-title String No Group title for posted content N/A
30 issue String No Issue number of an article's journal Publication/Journal/iss
31 volume String No Volume number of an article's journal Publication/Journal/vol
32 page String No Pages numbers of an article within its journal Publication/Journal/sp Publication/Journal/ep Obtain start and end page by splitting by '-'
33 article-number String No N/A
34 published-print Partial Date No Date on which the work was published in print Result/relevantDate with Qualifier("published-print", "dnet:dataCite_date")
35 published-online Partial Date No Date on which the work was published online Result/relevantDate with Qualifier("published-online", "dnet:dataCite_date")
36 subject Array of String No Subject category names, a controlled vocabulary from Sci-Val. Available for most journal articles Result/subject with Qualifier("keywords", "dnet:subject_classification_typologies"). Future improvements: map the controlled vocabulary instead of using the generic "keywords" qualifier
37 ISSN Array of String No Publication/Journal/issn Publication/Journal/lissn Publication/Journal/eissn The mapping depends on the value of issn-type
38 issn-type Array of ISSN with Type No List of ISSNs with ISSN type information N/A Its value guides the setting of the properties in Journal (see row above)
39 ISBN Array of String No Publication/source In case of Book We can map ISBN and container title on Publication/source using this syntax container-title + "ISBN: " + ISBN
40 archive Array of String No N/A
41 license Array of License No Result/Instance/License
42 funder Array of Funder No Relation Whenever we are able to link to a funder or project integrated into OpenAIRE. Mapping to OpenAIRE funders and projects is in eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala.generateSimpleRelationFromAward
43 assertion Array of Assertion No N/A
44 author Array of Contributor No Result/author (with orcid if available)
45 editor Array of Contributor No N/A
46 chair Array of Contributor No N/A
47 translator Array of Contributor No N/A
48 update-to Array of Update No N/A
49 update-policy URL No Link to an update policy covering Crossmark updates for this work N/A
50 link Array of Resource Link No URLs to full-text locations Result/Instance/url
51 clinical-trial-number Array of Clinical Trial Number No OafEntity/originalId
52 alternative-id String No Other identifiers for the work provided by the depositing member OafEntity/originalId
53 reference Array of Reference No List of references made by the work Future improvement: map to references
54 content-domain Content Domain No Information on domains that support Crossmark for this work N/A
55 relation Relations No Relations to other works Result/Instance/refereed if(relation.has-review) instance.refereed = "peerReviewed".
56 review Review No Peer review metadata N/A
57 funder Array of Funder No Relation between Result and Project The mapping between Crossref funder elements and OpenAIRE projects is implemented in eu.dnetlib.doiboost.crossref.Crossref2Oaf.mappingFunderToRelations. The matching is based on Funder DOIs, Funder names, and grant codes. Different mapping approaches are applied to cover the different cases in Crossref records.

View File

@ -0,0 +1,37 @@
Micorsoft Academic Graph Field,OAF field,Comments
Papers/DOI,OafEntity/originalId,
,OafEntity/PID,
,Oaf/id in the form 50|doiboost____::md5(DOI),
Papers/PaperId,OafEntity/originalId,
Papers/PaperTitle,"Result/Title with Qualifier(""main title"", ""dnet:dataCite_title"")",
Papers/OriginalTitle,"Result/Title with Qualifier(""alternative title"", ""dnet:dataCite_title"")",
Papers/BookTitle,Publication/Source,
Papers/Year,N/A,
Papers/Date,Result/dateofacceptance,
Papers/Publisher,Result/Publisher,Possibly overridden by Journal/Publisher
Papers/JournalId,N/A,
Journal/Rank,N/A,
Journal/NormalizedName,N/A,
Journal/DisplayName,Publication/Journal/Name,
Journal/Issn,Publication/Journal/issnPrinted,
Journal/Publisher,Result/publisher,"If avalable, it overrides value in Papers/Publisher"
Journal/Webpage,N/A,
Journal/PaperCount,N/A,
Journal/CitationCount,N/A,
Journal/CreatedDate,N/A,
ConferenceInstances/DisplayName,Publication/Journal/Name,
ConferenceInstances/Location,Publication/Journal/Conferenceplace,
ConferenceInstances/StartDate,Publication/Journal/Conferencedate,subjectTo be constructed as StartDate - EndDate (only the first 10 chars ofthe dates are kept)
ConferenceInstances/EndDate,Publication/Journal/Conferencedate,To be constructed as StartDate - EndDate (only the first 10 chars ofthe dates are kept)
Papers/Volume,Publication/Journal/vol,
Papers/Issue,Publication/Journal/iss,
Papers/FirstPage,Publication/Journal/sp,
Papers/LastPage,Publication/Journal/ep,
Papers/ReferenceCount,N/A,
Papers/CitationCount,N/A,
Papers/EstimatedCitation,N/A,
Papers/OriginalVenue,N/A,
Papers/FamilyId,N/A,
Papers/CreatedDate,Result/LastUpdateTimeStamp,
Papers/FieldOfStudy/DisplayName,"Result/subject with Qualifier(""keyword"", ""dnet:subject_classification_typologies"")",Some FieldOfStudy come from the UMLS controlled vocabulary. Future releases of DOIBoost will include such terms with a specific Qualifier (i.e. not as generic keywords)
Papers/FieldOfStudy/MainType,"Result/subject with Qualifier(""keyword"", ""dnet:subject_classification_typologies"")","If MainType is splittable on . (like 'food.type_of_dish', 'aviation.airport', 'soccer.team'), then the first token (i.e. food, aviation, soccer) is also added as additional subject."
1 Micorsoft Academic Graph Field OAF field Comments
2 Papers/DOI OafEntity/originalId
3 OafEntity/PID
4 Oaf/id in the form 50|doiboost____::md5(DOI)
5 Papers/PaperId OafEntity/originalId
6 Papers/PaperTitle Result/Title with Qualifier("main title", "dnet:dataCite_title")
7 Papers/OriginalTitle Result/Title with Qualifier("alternative title", "dnet:dataCite_title")
8 Papers/BookTitle Publication/Source
9 Papers/Year N/A
10 Papers/Date Result/dateofacceptance
11 Papers/Publisher Result/Publisher Possibly overridden by Journal/Publisher
12 Papers/JournalId N/A
13 Journal/Rank N/A
14 Journal/NormalizedName N/A
15 Journal/DisplayName Publication/Journal/Name
16 Journal/Issn Publication/Journal/issnPrinted
17 Journal/Publisher Result/publisher If avalable, it overrides value in Papers/Publisher
18 Journal/Webpage N/A
19 Journal/PaperCount N/A
20 Journal/CitationCount N/A
21 Journal/CreatedDate N/A
22 ConferenceInstances/DisplayName Publication/Journal/Name
23 ConferenceInstances/Location Publication/Journal/Conferenceplace
24 ConferenceInstances/StartDate Publication/Journal/Conferencedate subjectTo be constructed as StartDate - EndDate (only the first 10 chars ofthe dates are kept)
25 ConferenceInstances/EndDate Publication/Journal/Conferencedate To be constructed as StartDate - EndDate (only the first 10 chars ofthe dates are kept)
26 Papers/Volume Publication/Journal/vol
27 Papers/Issue Publication/Journal/iss
28 Papers/FirstPage Publication/Journal/sp
29 Papers/LastPage Publication/Journal/ep
30 Papers/ReferenceCount N/A
31 Papers/CitationCount N/A
32 Papers/EstimatedCitation N/A
33 Papers/OriginalVenue N/A
34 Papers/FamilyId N/A
35 Papers/CreatedDate Result/LastUpdateTimeStamp
36 Papers/FieldOfStudy/DisplayName Result/subject with Qualifier("keyword", "dnet:subject_classification_typologies") Some FieldOfStudy come from the UMLS controlled vocabulary. Future releases of DOIBoost will include such terms with a specific Qualifier (i.e. not as generic keywords)
37 Papers/FieldOfStudy/MainType Result/subject with Qualifier("keyword", "dnet:subject_classification_typologies") If MainType is splittable on . (like 'food.type_of_dish', 'aviation.airport', 'soccer.team'), then the first token (i.e. food, aviation, soccer) is also added as additional subject.

View File

@ -0,0 +1,6 @@
ORCID field,OAF mapping,Comment
doi,Result/pid,
oid,Result/Author/pid,
name,Result/Author/name,
surname,Result/Author/surname,
creditName,N/A,Result/Author/fullname is generated by concatenating name and surname
1 ORCID field OAF mapping Comment
2 doi Result/pid
3 oid Result/Author/pid
4 name Result/Author/name
5 surname Result/Author/surname
6 creditName N/A Result/Author/fullname is generated by concatenating name and surname

View File

@ -0,0 +1,11 @@
Unpaywall field,Description,OAF Mapping,Comment
doi,The DOI of this resource.,Result/pid,
is_oa,Is there an OA copy of this resource.,N/A,Add a Result/Instance only if is_oa is true
best_oa_location,The best OA Location Object we could find for this DOI.,,Create one Result/Instance with Open Access right
best_oa_location.url,The url_for_pdf if there is one; otherwise landing page URL.,Result/instance/url,
best_oa_location.license,The license under which this copy is published.,Result/instance/license,
oa_status,"The OA status, or color, of this resource. Classifies OA resources by location and license terms as one of: gold, hybrid, bronze, green or closed.",N/A,New field to be introduced in the OpenAIRE data model
published_date,"The date this resource was published. As reported by the publishers, who unfortunately have inconsistent definitions of what counts as officially ""published.""",N/A,"Not used in the mapping. Could be used to improve the coverage of dates, if needed."
title,The title of this resource.,N/A,
oa_locations,List of all the OA Location objects associated with this resource.,N/A,"Coud be useful if we plan to ""patch"" existing instances. If the instance URL is a URL of a oa_location, then it's Open.
"
1 Unpaywall field Description OAF Mapping Comment
2 doi The DOI of this resource. Result/pid
3 is_oa Is there an OA copy of this resource. N/A Add a Result/Instance only if is_oa is true
4 best_oa_location The best OA Location Object we could find for this DOI. Create one Result/Instance with Open Access right
5 best_oa_location.url The url_for_pdf if there is one; otherwise landing page URL. Result/instance/url
6 best_oa_location.license The license under which this copy is published. Result/instance/license
7 oa_status The OA status, or color, of this resource. Classifies OA resources by location and license terms as one of: gold, hybrid, bronze, green or closed. N/A New field to be introduced in the OpenAIRE data model
8 published_date The date this resource was published. As reported by the publishers, who unfortunately have inconsistent definitions of what counts as officially "published." N/A Not used in the mapping. Could be used to improve the coverage of dates, if needed.
9 title The title of this resource. N/A
10 oa_locations List of all the OA Location objects associated with this resource. N/A Coud be useful if we plan to "patch" existing instances. If the instance URL is a URL of a oa_location, then it's Open.

View File

@ -1,16 +1,14 @@
package eu.dnetlib.doiboost.orcid
import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.doiboost.crossref.Crossref2Oaf
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
import org.codehaus.jackson.map.ObjectMapper
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.slf4j.{Logger, LoggerFactory}
import org.junit.jupiter.api.Assertions._
import scala.io.Source
class MappingORCIDToOAFTest {
val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass)
val logger: Logger = LoggerFactory.getLogger(ORCIDToOAF.getClass)
val mapper = new ObjectMapper()
@Test

View File

@ -11,6 +11,7 @@ import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.io.SAXReader;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
@ -19,6 +20,8 @@ import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
//TODO to enable it we need to update the joined_entity.json test file
@Disabled
public class XmlRecordFactoryTest {
private static final String otherDsTypeId = "scholarcomminfra,infospace,pubsrepository::mock,entityregistry,entityregistry::projects,entityregistry::repositories,websource";

View File

@ -193,7 +193,6 @@
<groupId>net.sf.saxon</groupId>
<artifactId>Saxon-HE</artifactId>
<version>9.9.1-6</version>
<scope>provided</scope>
</dependency>
<dependency>