merge branch with fork master

This commit is contained in:
Miriam Baglioni 2020-07-02 14:25:39 +02:00
commit 94500a581b
96 changed files with 3041 additions and 611 deletions

View File

@ -1,5 +1,7 @@
<?xml version="1.0" encoding="UTF-8"?> <?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"> <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> <parent>
<artifactId>dhp-workflows</artifactId> <artifactId>dhp-workflows</artifactId>
<groupId>eu.dnetlib.dhp</groupId> <groupId>eu.dnetlib.dhp</groupId>
@ -24,7 +26,11 @@
<groupId>org.apache.spark</groupId> <groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId> <artifactId>spark-sql_2.11</artifactId>
</dependency> </dependency>
<dependency>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch-hadoop</artifactId>
</dependency>
<dependency> <dependency>
<groupId>eu.dnetlib.dhp</groupId> <groupId>eu.dnetlib.dhp</groupId>
@ -53,7 +59,7 @@
<dependency> <dependency>
<groupId>eu.dnetlib</groupId> <groupId>eu.dnetlib</groupId>
<artifactId>dnet-openaire-broker-common</artifactId> <artifactId>dnet-openaire-broker-common</artifactId>
<version>[3.0.3,4.0.0)</version> <version>[3.0.4,4.0.0)</version>
</dependency> </dependency>
</dependencies> </dependencies>

View File

@ -2,7 +2,6 @@
package eu.dnetlib.dhp.broker.model; package eu.dnetlib.dhp.broker.model;
import java.io.Serializable; import java.io.Serializable;
import java.util.Map;
public class Event implements Serializable { public class Event implements Serializable {
@ -25,7 +24,7 @@ public class Event implements Serializable {
private boolean instantMessage; private boolean instantMessage;
private Map<String, Object> map; private MappedFields map;
public Event() { public Event() {
} }
@ -33,7 +32,7 @@ public class Event implements Serializable {
public Event(final String producerId, final String eventId, final String topic, final String payload, public Event(final String producerId, final String eventId, final String topic, final String payload,
final Long creationDate, final Long expiryDate, final Long creationDate, final Long expiryDate,
final boolean instantMessage, final boolean instantMessage,
final Map<String, Object> map) { final MappedFields map) {
this.producerId = producerId; this.producerId = producerId;
this.eventId = eventId; this.eventId = eventId;
this.topic = topic; this.topic = topic;
@ -100,11 +99,11 @@ public class Event implements Serializable {
this.instantMessage = instantMessage; this.instantMessage = instantMessage;
} }
public Map<String, Object> getMap() { public MappedFields getMap() {
return this.map; return this.map;
} }
public void setMap(final Map<String, Object> map) { public void setMap(final MappedFields map) {
this.map = map; this.map = map;
} }
} }

View File

@ -3,9 +3,8 @@ package eu.dnetlib.dhp.broker.model;
import java.text.ParseException; import java.text.ParseException;
import java.util.Date; import java.util.Date;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.stream.Collectors;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
@ -30,10 +29,10 @@ public class EventFactory {
final Event res = new Event(); final Event res = new Event();
final Map<String, Object> map = createMapFromResult(updateInfo); final MappedFields map = createMapFromResult(updateInfo);
final String eventId = calculateEventId( final String eventId = calculateEventId(
updateInfo.getTopicPath(), updateInfo.getTarget().getOriginalId(), updateInfo.getHighlightValueAsString()); updateInfo.getTopicPath(), updateInfo.getTarget().getOpenaireId(), updateInfo.getHighlightValueAsString());
res.setEventId(eventId); res.setEventId(eventId);
res.setProducerId(PRODUCER_ID); res.setProducerId(PRODUCER_ID);
@ -43,38 +42,41 @@ public class EventFactory {
res.setCreationDate(now); res.setCreationDate(now);
res.setExpiryDate(calculateExpiryDate(now)); res.setExpiryDate(calculateExpiryDate(now));
res.setInstantMessage(false); res.setInstantMessage(false);
return res; return res;
} }
private static Map<String, Object> createMapFromResult(final UpdateInfo<?> updateInfo) { private static MappedFields createMapFromResult(final UpdateInfo<?> updateInfo) {
final Map<String, Object> map = new HashMap<>(); final MappedFields map = new MappedFields();
final OaBrokerMainEntity source = updateInfo.getSource(); final OaBrokerMainEntity source = updateInfo.getSource();
final OaBrokerMainEntity target = updateInfo.getTarget(); final OaBrokerMainEntity target = updateInfo.getTarget();
map.put("target_datasource_id", target.getCollectedFromId()); map.setTargetDatasourceId(target.getCollectedFromId());
map.put("target_datasource_name", target.getCollectedFromName()); map.setTargetDatasourceName(target.getCollectedFromName());
map.setTargetDatasourceType(target.getCollectedFromType());
map.put("target_publication_id", target.getOriginalId()); map.setTargetResultId(target.getOpenaireId());
final List<String> titles = target.getTitles(); final List<String> titles = target.getTitles();
if (titles.size() > 0) { if (titles.size() > 0) {
map.put("target_publication_title", titles.get(0)); map.setTargetResultTitle(titles.get(0));
} }
final long date = parseDateTolong(target.getPublicationdate()); final long date = parseDateTolong(target.getPublicationdate());
if (date > 0) { if (date > 0) {
map.put("target_dateofacceptance", date); map.setTargetDateofacceptance(date);
} }
map.put("target_publication_subject_list", target.getSubjects()); map.setTargetSubjects(target.getSubjects().stream().map(s -> s.getValue()).collect(Collectors.toList()));
map.put("target_publication_author_list", target.getCreators()); map.setTargetAuthors(target.getCreators().stream().map(a -> a.getFullname()).collect(Collectors.toList()));
// PROVENANCE INFO // PROVENANCE INFO
map.put("trust", updateInfo.getTrust()); map.setTrust(updateInfo.getTrust());
map.put("provenance_datasource_id", source.getCollectedFromId()); map.setProvenanceDatasourceId(source.getCollectedFromId());
map.put("provenance_datasource_name", source.getCollectedFromName()); map.setProvenanceDatasourceName(source.getCollectedFromName());
map.put("provenance_publication_id_list", source.getOriginalId()); map.setProvenanceDatasourceType(source.getCollectedFromType());
map.setProvenanceResultId(source.getOpenaireId());
return map; return map;
} }

View File

@ -0,0 +1,136 @@
package eu.dnetlib.dhp.broker.model;
import java.io.Serializable;
import java.util.List;
public class MappedFields implements Serializable {
/**
*
*/
private static final long serialVersionUID = -7999704113195802008L;
private String targetDatasourceId;
private String targetDatasourceName;
private String targetDatasourceType;
private String targetResultId;
private String targetResultTitle;
private long targetDateofacceptance;
private List<String> targetSubjects;
private List<String> targetAuthors;
private float trust;
private String provenanceDatasourceId;
private String provenanceDatasourceName;
private String provenanceDatasourceType;
private String provenanceResultId;
public String getTargetDatasourceId() {
return targetDatasourceId;
}
public void setTargetDatasourceId(final String targetDatasourceId) {
this.targetDatasourceId = targetDatasourceId;
}
public String getTargetDatasourceName() {
return targetDatasourceName;
}
public void setTargetDatasourceName(final String targetDatasourceName) {
this.targetDatasourceName = targetDatasourceName;
}
public String getTargetDatasourceType() {
return targetDatasourceType;
}
public void setTargetDatasourceType(final String targetDatasourceType) {
this.targetDatasourceType = targetDatasourceType;
}
public String getTargetResultId() {
return targetResultId;
}
public void setTargetResultId(final String targetResultId) {
this.targetResultId = targetResultId;
}
public String getTargetResultTitle() {
return targetResultTitle;
}
public void setTargetResultTitle(final String targetResultTitle) {
this.targetResultTitle = targetResultTitle;
}
public long getTargetDateofacceptance() {
return targetDateofacceptance;
}
public void setTargetDateofacceptance(final long targetDateofacceptance) {
this.targetDateofacceptance = targetDateofacceptance;
}
public List<String> getTargetSubjects() {
return targetSubjects;
}
public void setTargetSubjects(final List<String> targetSubjects) {
this.targetSubjects = targetSubjects;
}
public List<String> getTargetAuthors() {
return targetAuthors;
}
public void setTargetAuthors(final List<String> targetAuthors) {
this.targetAuthors = targetAuthors;
}
public float getTrust() {
return trust;
}
public void setTrust(final float trust) {
this.trust = trust;
}
public String getProvenanceDatasourceId() {
return provenanceDatasourceId;
}
public void setProvenanceDatasourceId(final String provenanceDatasourceId) {
this.provenanceDatasourceId = provenanceDatasourceId;
}
public String getProvenanceDatasourceName() {
return provenanceDatasourceName;
}
public void setProvenanceDatasourceName(final String provenanceDatasourceName) {
this.provenanceDatasourceName = provenanceDatasourceName;
}
public String getProvenanceDatasourceType() {
return provenanceDatasourceType;
}
public void setProvenanceDatasourceType(final String provenanceDatasourceType) {
this.provenanceDatasourceType = provenanceDatasourceType;
}
public String getProvenanceResultId() {
return provenanceResultId;
}
public void setProvenanceResultId(final String provenanceResultId) {
this.provenanceResultId = provenanceResultId;
}
public static long getSerialversionuid() {
return serialVersionUID;
}
}

View File

@ -3,14 +3,20 @@ package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.SparkContext;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode; import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -18,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.model.Event; import eu.dnetlib.dhp.broker.model.Event;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.EventFinder; import eu.dnetlib.dhp.broker.oa.util.EventFinder;
import eu.dnetlib.dhp.broker.oa.util.EventGroup; import eu.dnetlib.dhp.broker.oa.util.EventGroup;
@ -44,9 +51,6 @@ public class GenerateEventsJob {
.orElse(Boolean.TRUE); .orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged); log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String graphPath = parser.get("graphPath");
log.info("graphPath: {}", graphPath);
final String workingPath = parser.get("workingPath"); final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath); log.info("workingPath: {}", workingPath);
@ -56,9 +60,18 @@ public class GenerateEventsJob {
final String dedupConfigProfileId = parser.get("dedupConfProfile"); final String dedupConfigProfileId = parser.get("dedupConfProfile");
log.info("dedupConfigProfileId: {}", dedupConfigProfileId); log.info("dedupConfigProfileId: {}", dedupConfigProfileId);
final String eventsPath = workingPath + "/eventsPath"; final String eventsPath = workingPath + "/events";
log.info("eventsPath: {}", eventsPath); log.info("eventsPath: {}", eventsPath);
final Set<String> dsIdWhitelist = parseParamAsList(parser, "datasourceIdWhitelist");
log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ","));
final Set<String> dsTypeWhitelist = parseParamAsList(parser, "datasourceTypeWhitelist");
log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ","));
final Set<String> dsIdBlacklist = parseParamAsList(parser, "datasourceIdBlacklist");
log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ","));
final SparkConf conf = new SparkConf(); final SparkConf conf = new SparkConf();
// TODO UNCOMMENT // TODO UNCOMMENT
@ -69,21 +82,54 @@ public class GenerateEventsJob {
ClusterUtils.removeDir(spark, eventsPath); ClusterUtils.removeDir(spark, eventsPath);
final Map<String, LongAccumulator> accumulators = prepareAccumulators(spark.sparkContext());
final LongAccumulator total = spark.sparkContext().longAccumulator("total_events");
final Dataset<ResultGroup> groups = ClusterUtils final Dataset<ResultGroup> groups = ClusterUtils
.readPath(spark, graphPath + "/relation", ResultGroup.class); .readPath(spark, workingPath + "/duplicates", ResultGroup.class);
final Dataset<Event> events = groups final Dataset<Event> dataset = groups
.map( .map(
(MapFunction<ResultGroup, EventGroup>) g -> EventFinder.generateEvents(g, dedupConfig), g -> EventFinder
Encoders.bean(EventGroup.class)) .generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, dedupConfig, accumulators),
.flatMap(group -> group.getData().iterator(), Encoders.bean(Event.class)); Encoders
.bean(EventGroup.class))
.flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class));
events.write().mode(SaveMode.Overwrite).json(eventsPath); ClusterUtils.save(dataset, eventsPath, Event.class, total);
}); });
} }
private static Set<String> parseParamAsList(final ArgumentApplicationParser parser, final String key) {
final String s = parser.get(key).trim();
final Set<String> res = new HashSet<>();
if (s.length() > 1) { // A value of a single char (for example: '-') indicates an empty list
Arrays
.stream(s.split(","))
.map(String::trim)
.filter(StringUtils::isNotBlank)
.forEach(res::add);
}
return res;
}
public static Map<String, LongAccumulator> prepareAccumulators(final SparkContext sc) {
return EventFinder
.getMatchers()
.stream()
.map(UpdateMatcher::accumulatorName)
.distinct()
.collect(Collectors.toMap(s -> s, s -> sc.longAccumulator(s)));
}
private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception { private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception {
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl); final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl);

View File

@ -0,0 +1,71 @@
package eu.dnetlib.dhp.broker.oa;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.elasticsearch.spark.rdd.api.java.JavaEsSpark;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.model.Event;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
public class IndexOnESJob {
private static final Logger log = LoggerFactory.getLogger(IndexOnESJob.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
IndexOnESJob.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_es.json")));
parser.parseArgument(args);
final SparkConf conf = new SparkConf();
final String eventsPath = parser.get("workingPath") + "/events";
log.info("eventsPath: {}", eventsPath);
final String index = parser.get("index");
log.info("index: {}", index);
final String indexHost = parser.get("esHost");
log.info("indexHost: {}", indexHost);
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
final JavaRDD<String> inputRdd = ClusterUtils
.readPath(spark, eventsPath, Event.class)
.limit(10000) // TODO REMOVE
.map(IndexOnESJob::eventAsJsonString, Encoders.STRING())
.javaRDD();
final Map<String, String> esCfg = new HashMap<>();
// esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54");
esCfg.put("es.nodes", indexHost);
esCfg.put("es.mapping.id", "eventId"); // THE PRIMARY KEY
esCfg.put("es.batch.write.retry.count", "8");
esCfg.put("es.batch.write.retry.wait", "60s");
esCfg.put("es.batch.size.entries", "200");
esCfg.put("es.nodes.wan.only", "true");
JavaEsSpark.saveJsonToEs(inputRdd, index, esCfg);
}
private static String eventAsJsonString(final Event f) throws JsonProcessingException {
return new ObjectMapper().writeValueAsString(f);
}
}

View File

@ -1,94 +0,0 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.TypedColumn;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.OaBrokerMainEntityAggregator;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDataset;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedProject;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedPublication;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedSoftware;
import scala.Tuple2;
public class JoinEntitiesJob {
private static final Logger log = LoggerFactory.getLogger(JoinEntitiesJob.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinEntitiesJob.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String graphPath = parser.get("graphPath");
log.info("graphPath: {}", graphPath);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String joinedEntitiesPath = workingPath + "/joinedEntities";
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, joinedEntitiesPath);
final Dataset<OaBrokerMainEntity> r0 = ClusterUtils
.readPath(spark, graphPath + "/simpleEntities", OaBrokerMainEntity.class);
final Dataset<OaBrokerMainEntity> r1 = join(
r0, ClusterUtils.readPath(spark, graphPath + "/relatedProjects", RelatedProject.class));
final Dataset<OaBrokerMainEntity> r2 = join(
r1, ClusterUtils.readPath(spark, graphPath + "/relatedDatasets", RelatedDataset.class));
final Dataset<OaBrokerMainEntity> r3 = join(
r2, ClusterUtils.readPath(spark, graphPath + "/relatedPublications", RelatedPublication.class));
final Dataset<OaBrokerMainEntity> r4 = join(
r3, ClusterUtils.readPath(spark, graphPath + "/relatedSoftwares", RelatedSoftware.class));
r4.write().mode(SaveMode.Overwrite).json(joinedEntitiesPath);
});
}
private static <T> Dataset<OaBrokerMainEntity> join(final Dataset<OaBrokerMainEntity> sources,
final Dataset<T> typedRels) {
final TypedColumn<Tuple2<OaBrokerMainEntity, T>, OaBrokerMainEntity> aggr = new OaBrokerMainEntityAggregator<T>()
.toColumn();
return sources
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
.groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, T>, String>) t -> t._1.getOpenaireId(), Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.AddDatasourceTypeAggregator;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo;
import scala.Tuple2;
public class JoinStep0Job {
private static final Logger log = LoggerFactory.getLogger(JoinStep0Job.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinStep0Job.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String graphPath = parser.get("graphPath");
log.info("graphPath: {}", graphPath);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String outputPath = workingPath + "/joinedEntities_step0";
log.info("outputPath: {}", outputPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, outputPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
.readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class);
final Dataset<SimpleDatasourceInfo> datasources = ClusterUtils
.readPath(spark, workingPath + "/datasources", SimpleDatasourceInfo.class);
final TypedColumn<Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo>, OaBrokerMainEntity> aggr = new AddDatasourceTypeAggregator()
.toColumn();
final Dataset<OaBrokerMainEntity> dataset = sources
.joinWith(datasources, sources.col("collectedFromId").equalTo(datasources.col("id")), "inner")
.groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
ClusterUtils.save(dataset, outputPath, OaBrokerMainEntity.class, total);
});
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedProject;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedProjectAggregator;
import scala.Tuple2;
public class JoinStep1Job {
private static final Logger log = LoggerFactory.getLogger(JoinStep1Job.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinStep1Job.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String joinedEntitiesPath = workingPath + "/joinedEntities_step1";
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, joinedEntitiesPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
.readPath(spark, workingPath + "/joinedEntities_step0", OaBrokerMainEntity.class);
final Dataset<RelatedProject> typedRels = ClusterUtils
.readPath(spark, workingPath + "/relatedProjects", RelatedProject.class);
final TypedColumn<Tuple2<OaBrokerMainEntity, RelatedProject>, OaBrokerMainEntity> aggr = new RelatedProjectAggregator()
.toColumn();
final Dataset<OaBrokerMainEntity> dataset = sources
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
.groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, RelatedProject>, String>) t -> t._1.getOpenaireId(),
Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total);
});
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedSoftware;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedSoftwareAggregator;
import scala.Tuple2;
public class JoinStep2Job {
private static final Logger log = LoggerFactory.getLogger(JoinStep2Job.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinStep2Job.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String joinedEntitiesPath = workingPath + "/joinedEntities_step2";
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, joinedEntitiesPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
.readPath(spark, workingPath + "/joinedEntities_step1", OaBrokerMainEntity.class);
final Dataset<RelatedSoftware> typedRels = ClusterUtils
.readPath(spark, workingPath + "/relatedSoftwares", RelatedSoftware.class);
final TypedColumn<Tuple2<OaBrokerMainEntity, RelatedSoftware>, OaBrokerMainEntity> aggr = new RelatedSoftwareAggregator()
.toColumn();
final Dataset<OaBrokerMainEntity> dataset = sources
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
.groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, RelatedSoftware>, String>) t -> t._1.getOpenaireId(),
Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total);
});
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDataset;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasetAggregator;
import scala.Tuple2;
public class JoinStep3Job {
private static final Logger log = LoggerFactory.getLogger(JoinStep3Job.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinStep3Job.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String joinedEntitiesPath = workingPath + "/joinedEntities_step3";
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, joinedEntitiesPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
.readPath(spark, workingPath + "/joinedEntities_step2", OaBrokerMainEntity.class);
final Dataset<RelatedDataset> typedRels = ClusterUtils
.readPath(spark, workingPath + "/relatedDatasets", RelatedDataset.class);
final TypedColumn<Tuple2<OaBrokerMainEntity, RelatedDataset>, OaBrokerMainEntity> aggr = new RelatedDatasetAggregator()
.toColumn();
final Dataset<OaBrokerMainEntity> dataset = sources
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
.groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, RelatedDataset>, String>) t -> t._1.getOpenaireId(),
Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total);
});
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedPublication;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedPublicationAggregator;
import scala.Tuple2;
public class JoinStep4Job {
private static final Logger log = LoggerFactory.getLogger(JoinStep4Job.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
JoinStep4Job.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String joinedEntitiesPath = workingPath + "/joinedEntities_step4";
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, joinedEntitiesPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
.readPath(spark, workingPath + "/joinedEntities_step3", OaBrokerMainEntity.class);
final Dataset<RelatedPublication> typedRels = ClusterUtils
.readPath(spark, workingPath + "/relatedPublications", RelatedPublication.class);
final TypedColumn<Tuple2<OaBrokerMainEntity, RelatedPublication>, OaBrokerMainEntity> aggr = new RelatedPublicationAggregator()
.toColumn();
final Dataset<OaBrokerMainEntity> dataset = sources
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
.groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, RelatedPublication>, String>) t -> t._1.getOpenaireId(),
Encoders.STRING())
.agg(aggr)
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total);
});
}
}

View File

@ -10,8 +10,8 @@ import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.TypedColumn; import org.apache.spark.sql.TypedColumn;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -32,7 +32,7 @@ public class PrepareGroupsJob {
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils IOUtils
.toString( .toString(
JoinEntitiesJob.class PrepareGroupsJob.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args); parser.parseArgument(args);
@ -48,7 +48,7 @@ public class PrepareGroupsJob {
final String workingPath = parser.get("workingPath"); final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath); log.info("workingPath: {}", workingPath);
final String groupsPath = workingPath + "/groups"; final String groupsPath = workingPath + "/duplicates";
log.info("groupsPath: {}", groupsPath); log.info("groupsPath: {}", groupsPath);
final SparkConf conf = new SparkConf(); final SparkConf conf = new SparkConf();
@ -57,8 +57,10 @@ public class PrepareGroupsJob {
ClusterUtils.removeDir(spark, groupsPath); ClusterUtils.removeDir(spark, groupsPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_groups");
final Dataset<OaBrokerMainEntity> results = ClusterUtils final Dataset<OaBrokerMainEntity> results = ClusterUtils
.readPath(spark, graphPath + "/joinedEntities", OaBrokerMainEntity.class); .readPath(spark, workingPath + "/joinedEntities_step4", OaBrokerMainEntity.class);
final Dataset<Relation> mergedRels = ClusterUtils final Dataset<Relation> mergedRels = ClusterUtils
.readPath(spark, graphPath + "/relation", Relation.class) .readPath(spark, graphPath + "/relation", Relation.class)
@ -67,20 +69,16 @@ public class PrepareGroupsJob {
final TypedColumn<Tuple2<OaBrokerMainEntity, Relation>, ResultGroup> aggr = new ResultAggregator() final TypedColumn<Tuple2<OaBrokerMainEntity, Relation>, ResultGroup> aggr = new ResultAggregator()
.toColumn(); .toColumn();
final Dataset<ResultGroup> groups = results final Dataset<ResultGroup> dataset = results
.joinWith(mergedRels, results.col("openaireId").equalTo(mergedRels.col("source")), "inner") .joinWith(mergedRels, results.col("openaireId").equalTo(mergedRels.col("source")), "inner")
.groupByKey( .groupByKey(
(MapFunction<Tuple2<OaBrokerMainEntity, Relation>, String>) t -> t._2.getTarget(), (MapFunction<Tuple2<OaBrokerMainEntity, Relation>, String>) t -> t._2.getTarget(),
Encoders.STRING()) Encoders.STRING())
.agg(aggr) .agg(aggr)
.map( .map(t -> t._2, Encoders.bean(ResultGroup.class))
(MapFunction<Tuple2<String, ResultGroup>, ResultGroup>) t -> t._2, Encoders.bean(ResultGroup.class))
.filter(rg -> rg.getData().size() > 1); .filter(rg -> rg.getData().size() > 1);
groups ClusterUtils.save(dataset, groupsPath, ResultGroup.class, total);
.write()
.mode(SaveMode.Overwrite)
.json(groupsPath);
}); });
} }

View File

@ -9,14 +9,16 @@ import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode; import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerRelatedDataset;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils; import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDataset; import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDataset;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Relation;
public class PrepareRelatedDatasetsJob { public class PrepareRelatedDatasetsJob {
@ -52,22 +54,30 @@ public class PrepareRelatedDatasetsJob {
ClusterUtils.removeDir(spark, relsPath); ClusterUtils.removeDir(spark, relsPath);
final Dataset<eu.dnetlib.dhp.schema.oaf.Dataset> datasets = ClusterUtils final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels");
.readPath(spark, graphPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class);
final Dataset<Relation> rels = ClusterUtils.readPath(spark, graphPath + "/relation", Relation.class); final Dataset<OaBrokerRelatedDataset> datasets = ClusterUtils
.readPath(spark, graphPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class)
.filter(d -> !ClusterUtils.isDedupRoot(d.getId()))
.map(ConversionUtils::oafDatasetToBrokerDataset, Encoders.bean(OaBrokerRelatedDataset.class));
rels final Dataset<Relation> rels = ClusterUtils
.joinWith(datasets, datasets.col("id").equalTo(rels.col("target")), "inner") .readPath(spark, graphPath + "/relation", Relation.class)
.map( .filter(r -> r.getDataInfo().getDeletedbyinference())
t -> new RelatedDataset( .filter(r -> r.getRelType().equals(ModelConstants.RESULT_RESULT))
t._1.getSource(), .filter(r -> ClusterUtils.isValidResultResultClass(r.getRelClass()))
t._1.getRelType(), .filter(r -> !ClusterUtils.isDedupRoot(r.getSource()))
ConversionUtils.oafDatasetToBrokerDataset(t._2)), .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget()));
Encoders.bean(RelatedDataset.class))
.write() final Dataset<RelatedDataset> dataset = rels
.mode(SaveMode.Overwrite) .joinWith(datasets, datasets.col("openaireId").equalTo(rels.col("target")), "inner")
.json(relsPath); .map(t -> {
final RelatedDataset rel = new RelatedDataset(t._1.getSource(), t._2);
rel.getRelDataset().setRelType(t._1.getRelClass());
return rel;
}, Encoders.bean(RelatedDataset.class));
ClusterUtils.save(dataset, relsPath, RelatedDataset.class, total);
}); });

View File

@ -0,0 +1,68 @@
package eu.dnetlib.dhp.broker.oa;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo;
import eu.dnetlib.dhp.schema.oaf.Datasource;
public class PrepareRelatedDatasourcesJob {
private static final Logger log = LoggerFactory.getLogger(PrepareRelatedDatasourcesJob.class);
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
PrepareRelatedDatasourcesJob.class
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String graphPath = parser.get("graphPath");
log.info("graphPath: {}", graphPath);
final String workingPath = parser.get("workingPath");
log.info("workingPath: {}", workingPath);
final String relsPath = workingPath + "/datasources";
log.info("relsPath: {}", relsPath);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
ClusterUtils.removeDir(spark, relsPath);
final LongAccumulator total = spark.sparkContext().longAccumulator("total_datasources");
final Dataset<SimpleDatasourceInfo> dataset = ClusterUtils
.readPath(spark, graphPath + "/datasource", Datasource.class)
.map(
ds -> new SimpleDatasourceInfo(ds.getId(), ds.getDatasourcetype().getClassid()),
Encoders.bean(SimpleDatasourceInfo.class));
ClusterUtils.save(dataset, relsPath, SimpleDatasourceInfo.class, total);
});
}
}

View File

@ -9,13 +9,13 @@ import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode; import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.broker.objects.OaBrokerProject;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils; import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedProject; import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedProject;
@ -27,8 +27,6 @@ public class PrepareRelatedProjectsJob {
private static final Logger log = LoggerFactory.getLogger(PrepareRelatedProjectsJob.class); private static final Logger log = LoggerFactory.getLogger(PrepareRelatedProjectsJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception { public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils IOUtils
@ -58,23 +56,27 @@ public class PrepareRelatedProjectsJob {
ClusterUtils.removeDir(spark, relsPath); ClusterUtils.removeDir(spark, relsPath);
final Dataset<Project> projects = ClusterUtils.readPath(spark, graphPath + "/project", Project.class); final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels");
final Dataset<OaBrokerProject> projects = ClusterUtils
.readPath(spark, graphPath + "/project", Project.class)
.filter(p -> !ClusterUtils.isDedupRoot(p.getId()))
.map(ConversionUtils::oafProjectToBrokerProject, Encoders.bean(OaBrokerProject.class));
final Dataset<Relation> rels = ClusterUtils final Dataset<Relation> rels = ClusterUtils
.readPath(spark, graphPath + "/relation", Relation.class) .readPath(spark, graphPath + "/relation", Relation.class)
.filter(r -> r.getRelType().equals(ModelConstants.RESULT_PROJECT)); .filter(r -> r.getDataInfo().getDeletedbyinference())
.filter(r -> r.getRelType().equals(ModelConstants.RESULT_PROJECT))
.filter(r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS))
.filter(r -> !ClusterUtils.isDedupRoot(r.getSource()))
.filter(r -> !ClusterUtils.isDedupRoot(r.getTarget()));
final Dataset<RelatedProject> dataset = rels
.joinWith(projects, projects.col("openaireId").equalTo(rels.col("target")), "inner")
.map(t -> new RelatedProject(t._1.getSource(), t._2), Encoders.bean(RelatedProject.class));
ClusterUtils.save(dataset, relsPath, RelatedProject.class, total);
rels
.joinWith(projects, projects.col("id").equalTo(rels.col("target")), "inner")
.map(
t -> new RelatedProject(
t._1.getSource(),
t._1.getRelType(),
ConversionUtils.oafProjectToBrokerProject(t._2)),
Encoders.bean(RelatedProject.class))
.write()
.mode(SaveMode.Overwrite)
.json(relsPath);
}); });
} }

View File

@ -9,16 +9,16 @@ import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode; import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.broker.objects.OaBrokerRelatedPublication;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils; import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedPublication; import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedPublication;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Publication; import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Relation;
@ -26,8 +26,6 @@ public class PrepareRelatedPublicationsJob {
private static final Logger log = LoggerFactory.getLogger(PrepareRelatedPublicationsJob.class); private static final Logger log = LoggerFactory.getLogger(PrepareRelatedPublicationsJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception { public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils IOUtils
@ -57,22 +55,32 @@ public class PrepareRelatedPublicationsJob {
ClusterUtils.removeDir(spark, relsPath); ClusterUtils.removeDir(spark, relsPath);
final Dataset<Publication> pubs = ClusterUtils final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels");
.readPath(spark, graphPath + "/publication", Publication.class);
final Dataset<Relation> rels = ClusterUtils.readPath(spark, graphPath + "/relation", Relation.class); final Dataset<OaBrokerRelatedPublication> pubs = ClusterUtils
.readPath(spark, graphPath + "/publication", Publication.class)
rels .filter(p -> !ClusterUtils.isDedupRoot(p.getId()))
.joinWith(pubs, pubs.col("id").equalTo(rels.col("target")), "inner")
.map( .map(
t -> new RelatedPublication( ConversionUtils::oafPublicationToBrokerPublication,
t._1.getSource(), Encoders.bean(OaBrokerRelatedPublication.class));
t._1.getRelType(),
ConversionUtils.oafPublicationToBrokerPublication(t._2)), final Dataset<Relation> rels = ClusterUtils
Encoders.bean(RelatedPublication.class)) .readPath(spark, graphPath + "/relation", Relation.class)
.write() .filter(r -> r.getDataInfo().getDeletedbyinference())
.mode(SaveMode.Overwrite) .filter(r -> r.getRelType().equals(ModelConstants.RESULT_RESULT))
.json(relsPath); .filter(r -> ClusterUtils.isValidResultResultClass(r.getRelClass()))
.filter(r -> !ClusterUtils.isDedupRoot(r.getSource()))
.filter(r -> !ClusterUtils.isDedupRoot(r.getTarget()));
final Dataset<RelatedPublication> dataset = rels
.joinWith(pubs, pubs.col("openaireId").equalTo(rels.col("target")), "inner")
.map(t -> {
final RelatedPublication rel = new RelatedPublication(t._1.getSource(), t._2);
rel.getRelPublication().setRelType(t._1.getRelClass());
return rel;
}, Encoders.bean(RelatedPublication.class));
ClusterUtils.save(dataset, relsPath, RelatedPublication.class, total);
}); });

View File

@ -9,16 +9,17 @@ import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode; import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils; import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedSoftware; import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedSoftware;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Software; import eu.dnetlib.dhp.schema.oaf.Software;
@ -26,8 +27,6 @@ public class PrepareRelatedSoftwaresJob {
private static final Logger log = LoggerFactory.getLogger(PrepareRelatedSoftwaresJob.class); private static final Logger log = LoggerFactory.getLogger(PrepareRelatedSoftwaresJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception { public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser( final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils IOUtils
@ -57,21 +56,26 @@ public class PrepareRelatedSoftwaresJob {
ClusterUtils.removeDir(spark, relsPath); ClusterUtils.removeDir(spark, relsPath);
final Dataset<Software> softwares = ClusterUtils.readPath(spark, graphPath + "/software", Software.class); final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels");
final Dataset<Relation> rels = ClusterUtils.readPath(spark, graphPath + "/relation", Relation.class); final Dataset<OaBrokerRelatedSoftware> softwares = ClusterUtils
.readPath(spark, graphPath + "/software", Software.class)
.filter(sw -> !ClusterUtils.isDedupRoot(sw.getId()))
.map(ConversionUtils::oafSoftwareToBrokerSoftware, Encoders.bean(OaBrokerRelatedSoftware.class));
rels final Dataset<Relation> rels = ClusterUtils
.joinWith(softwares, softwares.col("id").equalTo(rels.col("target")), "inner") .readPath(spark, graphPath + "/relation", Relation.class)
.map( .filter(r -> r.getDataInfo().getDeletedbyinference())
t -> new RelatedSoftware( .filter(r -> r.getRelType().equals(ModelConstants.RESULT_RESULT))
t._1.getSource(), .filter(r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS))
t._1.getRelType(), .filter(r -> !ClusterUtils.isDedupRoot(r.getSource()))
ConversionUtils.oafSoftwareToBrokerSoftware(t._2)), .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget()));
Encoders.bean(RelatedSoftware.class))
.write() final Dataset<RelatedSoftware> dataset = rels
.mode(SaveMode.Overwrite) .joinWith(softwares, softwares.col("openaireId").equalTo(rels.col("target")), "inner")
.json(relsPath); .map(t -> new RelatedSoftware(t._1.getSource(), t._2), Encoders.bean(RelatedSoftware.class));
ClusterUtils.save(dataset, relsPath, RelatedSoftware.class, total);
}); });

View File

@ -9,8 +9,8 @@ import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -56,13 +56,14 @@ public class PrepareSimpleEntititiesJob {
ClusterUtils.removeDir(spark, simpleEntitiesPath); ClusterUtils.removeDir(spark, simpleEntitiesPath);
prepareSimpleEntities(spark, graphPath, Publication.class) final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
final Dataset<OaBrokerMainEntity> dataset = prepareSimpleEntities(spark, graphPath, Publication.class)
.union(prepareSimpleEntities(spark, graphPath, eu.dnetlib.dhp.schema.oaf.Dataset.class)) .union(prepareSimpleEntities(spark, graphPath, eu.dnetlib.dhp.schema.oaf.Dataset.class))
.union(prepareSimpleEntities(spark, graphPath, Software.class)) .union(prepareSimpleEntities(spark, graphPath, Software.class))
.union(prepareSimpleEntities(spark, graphPath, OtherResearchProduct.class)) .union(prepareSimpleEntities(spark, graphPath, OtherResearchProduct.class));
.write()
.mode(SaveMode.Overwrite) ClusterUtils.save(dataset, simpleEntitiesPath, OaBrokerMainEntity.class, total);
.json(simpleEntitiesPath);
}); });
} }
@ -74,6 +75,7 @@ public class PrepareSimpleEntititiesJob {
return ClusterUtils return ClusterUtils
.readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), sourceClass) .readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), sourceClass)
.filter(r -> !ClusterUtils.isDedupRoot(r.getId()))
.filter(r -> r.getDataInfo().getDeletedbyinference()) .filter(r -> r.getDataInfo().getDeletedbyinference())
.map(ConversionUtils::oafResultToBrokerResult, Encoders.bean(OaBrokerMainEntity.class)); .map(ConversionUtils::oafResultToBrokerResult, Encoders.bean(OaBrokerMainEntity.class));
} }

View File

@ -1,16 +1,18 @@
package eu.dnetlib.dhp.broker.oa.matchers; package eu.dnetlib.dhp.broker.oa.matchers;
import java.util.Arrays; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.BiConsumer; import java.util.function.BiConsumer;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.spark.util.LongAccumulator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
@ -19,15 +21,15 @@ import eu.dnetlib.pace.config.DedupConfig;
public abstract class UpdateMatcher<T> { public abstract class UpdateMatcher<T> {
private final boolean multipleUpdate; private final int maxNumber;
private final Function<T, Topic> topicFunction; private final Function<T, Topic> topicFunction;
private final BiConsumer<OaBrokerMainEntity, T> compileHighlightFunction; private final BiConsumer<OaBrokerMainEntity, T> compileHighlightFunction;
private final Function<T, String> highlightToStringFunction; private final Function<T, String> highlightToStringFunction;
public UpdateMatcher(final boolean multipleUpdate, final Function<T, Topic> topicFunction, public UpdateMatcher(final int maxNumber, final Function<T, Topic> topicFunction,
final BiConsumer<OaBrokerMainEntity, T> compileHighlightFunction, final BiConsumer<OaBrokerMainEntity, T> compileHighlightFunction,
final Function<T, String> highlightToStringFunction) { final Function<T, String> highlightToStringFunction) {
this.multipleUpdate = multipleUpdate; this.maxNumber = maxNumber;
this.topicFunction = topicFunction; this.topicFunction = topicFunction;
this.compileHighlightFunction = compileHighlightFunction; this.compileHighlightFunction = compileHighlightFunction;
this.highlightToStringFunction = highlightToStringFunction; this.highlightToStringFunction = highlightToStringFunction;
@ -35,7 +37,8 @@ public abstract class UpdateMatcher<T> {
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final OaBrokerMainEntity res, public Collection<UpdateInfo<T>> searchUpdatesForRecord(final OaBrokerMainEntity res,
final Collection<OaBrokerMainEntity> others, final Collection<OaBrokerMainEntity> others,
final DedupConfig dedupConfig) { final DedupConfig dedupConfig,
final Map<String, LongAccumulator> accumulators) {
final Map<String, UpdateInfo<T>> infoMap = new HashMap<>(); final Map<String, UpdateInfo<T>> infoMap = new HashMap<>();
@ -43,28 +46,34 @@ public abstract class UpdateMatcher<T> {
if (source != res) { if (source != res) {
for (final T hl : findDifferences(source, res)) { for (final T hl : findDifferences(source, res)) {
final Topic topic = getTopicFunction().apply(hl); final Topic topic = getTopicFunction().apply(hl);
final UpdateInfo<T> info = new UpdateInfo<>(topic, hl, source, res, getCompileHighlightFunction(), if (topic != null) {
getHighlightToStringFunction(), dedupConfig); final UpdateInfo<T> info = new UpdateInfo<>(topic, hl, source, res,
final String s = DigestUtils.md5Hex(info.getHighlightValueAsString()); getCompileHighlightFunction(),
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) { getHighlightToStringFunction(), dedupConfig);
} else {
infoMap.put(s, info); final String s = DigestUtils.md5Hex(info.getHighlightValueAsString());
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {
infoMap.put(s, info);
}
} }
} }
} }
} }
final Collection<UpdateInfo<T>> values = infoMap.values(); final List<UpdateInfo<T>> values = infoMap
.values()
.stream()
.sorted((o1, o2) -> Float.compare(o2.getTrust(), o1.getTrust())) // DESCENDING
.collect(Collectors.toList());
if (values.isEmpty() || multipleUpdate) { if (values.isEmpty()) {
return values; return new ArrayList<>();
} else if (values.size() > maxNumber) {
incrementAccumulator(accumulators, maxNumber);
return values.subList(0, maxNumber);
} else { } else {
final UpdateInfo<T> v = values incrementAccumulator(accumulators, values.size());
.stream() return values;
.sorted((o1, o2) -> Float.compare(o1.getTrust(), o2.getTrust()))
.findFirst()
.get();
return Arrays.asList(v);
} }
} }
@ -74,12 +83,12 @@ public abstract class UpdateMatcher<T> {
return list == null || list.isEmpty() || StringUtils.isBlank(list.get(0)); return list == null || list.isEmpty() || StringUtils.isBlank(list.get(0));
} }
protected boolean isMissing(final String field) { protected boolean isMissing(final String s) {
return StringUtils.isBlank(field); return StringUtils.isBlank(s);
} }
public boolean isMultipleUpdate() { public int getMaxNumber() {
return multipleUpdate; return maxNumber;
} }
public Function<T, Topic> getTopicFunction() { public Function<T, Topic> getTopicFunction() {
@ -94,4 +103,14 @@ public abstract class UpdateMatcher<T> {
return highlightToStringFunction; return highlightToStringFunction;
} }
public String accumulatorName() {
return "event_matcher_" + getClass().getSimpleName().toLowerCase();
}
public void incrementAccumulator(final Map<String, LongAccumulator> accumulators, final long n) {
if (accumulators != null && accumulators.containsKey(accumulatorName())) {
accumulators.get(accumulatorName()).add(n);
}
}
} }

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,14 +10,15 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerRelatedDataset; import eu.dnetlib.broker.objects.OaBrokerRelatedDataset;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public abstract class AbstractEnrichMissingDataset extends UpdateMatcher<OaBrokerRelatedDataset> { public abstract class AbstractEnrichMissingDataset extends UpdateMatcher<OaBrokerRelatedDataset> {
public AbstractEnrichMissingDataset(final Topic topic) { public AbstractEnrichMissingDataset(final Topic topic) {
super(true, super(10,
rel -> topic, rel -> topic,
(p, rel) -> p.getDatasets().add(rel), (p, rel) -> p.getDatasets().add(rel),
rel -> rel.getOriginalId()); rel -> rel.getOpenaireId());
} }
protected abstract boolean filterByType(String relType); protected abstract boolean filterByType(String relType);
@ -25,18 +27,22 @@ public abstract class AbstractEnrichMissingDataset extends UpdateMatcher<OaBroke
protected final List<OaBrokerRelatedDataset> findDifferences(final OaBrokerMainEntity source, protected final List<OaBrokerRelatedDataset> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getDatasets().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingDatasets = target final Set<String> existingDatasets = target
.getDatasets() .getDatasets()
.stream() .stream()
.filter(rel -> filterByType(rel.getRelType())) .filter(rel -> filterByType(rel.getRelType()))
.map(OaBrokerRelatedDataset::getOriginalId) .map(OaBrokerRelatedDataset::getOpenaireId)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
return source return source
.getDatasets() .getDatasets()
.stream() .stream()
.filter(rel -> filterByType(rel.getRelType())) .filter(rel -> filterByType(rel.getRelType()))
.filter(d -> !existingDatasets.contains(d.getOriginalId())) .filter(d -> !existingDatasets.contains(d.getOpenaireId()))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }

View File

@ -12,7 +12,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
public class EnrichMissingProject extends UpdateMatcher<OaBrokerProject> { public class EnrichMissingProject extends UpdateMatcher<OaBrokerProject> {
public EnrichMissingProject() { public EnrichMissingProject() {
super(true, super(20,
prj -> Topic.ENRICH_MISSING_PROJECT, prj -> Topic.ENRICH_MISSING_PROJECT,
(p, prj) -> p.getProjects().add(prj), (p, prj) -> p.getProjects().add(prj),
prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode()); prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode());

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.relatedProjects; package eu.dnetlib.dhp.broker.oa.matchers.relatedProjects;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,11 +10,12 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerProject; import eu.dnetlib.broker.objects.OaBrokerProject;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMoreProject extends UpdateMatcher<OaBrokerProject> { public class EnrichMoreProject extends UpdateMatcher<OaBrokerProject> {
public EnrichMoreProject() { public EnrichMoreProject() {
super(true, super(20,
prj -> Topic.ENRICH_MORE_PROJECT, prj -> Topic.ENRICH_MORE_PROJECT,
(p, prj) -> p.getProjects().add(prj), (p, prj) -> p.getProjects().add(prj),
prj -> projectAsString(prj)); prj -> projectAsString(prj));
@ -27,6 +29,10 @@ public class EnrichMoreProject extends UpdateMatcher<OaBrokerProject> {
protected List<OaBrokerProject> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerProject> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingProjects = target final Set<String> existingProjects = target
.getProjects() .getProjects()
.stream() .stream()

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,14 +10,15 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerRelatedPublication; import eu.dnetlib.broker.objects.OaBrokerRelatedPublication;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public abstract class AbstractEnrichMissingPublication extends UpdateMatcher<OaBrokerRelatedPublication> { public abstract class AbstractEnrichMissingPublication extends UpdateMatcher<OaBrokerRelatedPublication> {
public AbstractEnrichMissingPublication(final Topic topic) { public AbstractEnrichMissingPublication(final Topic topic) {
super(true, super(10,
rel -> topic, rel -> topic,
(p, rel) -> p.getPublications().add(rel), (p, rel) -> p.getPublications().add(rel),
rel -> rel.getOriginalId()); rel -> rel.getOpenaireId());
} }
@ -27,18 +29,22 @@ public abstract class AbstractEnrichMissingPublication extends UpdateMatcher<OaB
final OaBrokerMainEntity source, final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getPublications().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingPublications = target final Set<String> existingPublications = target
.getPublications() .getPublications()
.stream() .stream()
.filter(rel -> filterByType(rel.getRelType())) .filter(rel -> filterByType(rel.getRelType()))
.map(OaBrokerRelatedPublication::getOriginalId) .map(OaBrokerRelatedPublication::getOpenaireId)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
return source return source
.getPublications() .getPublications()
.stream() .stream()
.filter(rel -> filterByType(rel.getRelType())) .filter(rel -> filterByType(rel.getRelType()))
.filter(p -> !existingPublications.contains(p.getOriginalId())) .filter(p -> !existingPublications.contains(p.getOpenaireId()))
.collect(Collectors.toList()); .collect(Collectors.toList());
} }

View File

@ -13,10 +13,10 @@ public class EnrichMissingSoftware
extends UpdateMatcher<OaBrokerRelatedSoftware> { extends UpdateMatcher<OaBrokerRelatedSoftware> {
public EnrichMissingSoftware() { public EnrichMissingSoftware() {
super(true, super(10,
s -> Topic.ENRICH_MISSING_SOFTWARE, s -> Topic.ENRICH_MISSING_SOFTWARE,
(p, s) -> p.getSoftwares().add(s), (p, s) -> p.getSoftwares().add(s),
s -> s.getName()); s -> s.getOpenaireId());
} }
@Override @Override

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.relatedSoftware; package eu.dnetlib.dhp.broker.oa.matchers.relatedSoftware;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,14 +10,15 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware; import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMoreSoftware extends UpdateMatcher<OaBrokerRelatedSoftware> { public class EnrichMoreSoftware extends UpdateMatcher<OaBrokerRelatedSoftware> {
public EnrichMoreSoftware() { public EnrichMoreSoftware() {
super(true, super(10,
s -> Topic.ENRICH_MORE_SOFTWARE, s -> Topic.ENRICH_MORE_SOFTWARE,
(p, s) -> p.getSoftwares().add(s), (p, s) -> p.getSoftwares().add(s),
s -> s.getName()); s -> s.getOpenaireId());
} }
@Override @Override
@ -24,6 +26,10 @@ public class EnrichMoreSoftware extends UpdateMatcher<OaBrokerRelatedSoftware> {
final OaBrokerMainEntity source, final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getSoftwares().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingSoftwares = source final Set<String> existingSoftwares = source
.getSoftwares() .getSoftwares()
.stream() .stream()

View File

@ -12,7 +12,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
public class EnrichMissingAbstract extends UpdateMatcher<String> { public class EnrichMissingAbstract extends UpdateMatcher<String> {
public EnrichMissingAbstract() { public EnrichMissingAbstract() {
super(false, super(1,
s -> Topic.ENRICH_MISSING_ABSTRACT, s -> Topic.ENRICH_MISSING_ABSTRACT,
(p, s) -> p.getAbstracts().add(s), (p, s) -> p.getAbstracts().add(s),
s -> s); s -> s);

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -11,11 +12,12 @@ import eu.dnetlib.broker.objects.OaBrokerAuthor;
import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMissingAuthorOrcid extends UpdateMatcher<OaBrokerAuthor> { public class EnrichMissingAuthorOrcid extends UpdateMatcher<OaBrokerAuthor> {
public EnrichMissingAuthorOrcid() { public EnrichMissingAuthorOrcid() {
super(true, super(40,
aut -> Topic.ENRICH_MISSING_AUTHOR_ORCID, aut -> Topic.ENRICH_MISSING_AUTHOR_ORCID,
(p, aut) -> p.getCreators().add(aut), (p, aut) -> p.getCreators().add(aut),
aut -> aut.getOrcid()); aut -> aut.getOrcid());
@ -25,6 +27,10 @@ public class EnrichMissingAuthorOrcid extends UpdateMatcher<OaBrokerAuthor> {
protected List<OaBrokerAuthor> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerAuthor> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getCreators().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingOrcids = target final Set<String> existingOrcids = target
.getCreators() .getCreators()
.stream() .stream()

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -14,7 +15,7 @@ import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMissingOpenAccess extends UpdateMatcher<OaBrokerInstance> { public class EnrichMissingOpenAccess extends UpdateMatcher<OaBrokerInstance> {
public EnrichMissingOpenAccess() { public EnrichMissingOpenAccess() {
super(true, super(20,
i -> Topic.ENRICH_MISSING_OA_VERSION, i -> Topic.ENRICH_MISSING_OA_VERSION,
(p, i) -> p.getInstances().add(i), (p, i) -> p.getInstances().add(i),
OaBrokerInstance::getUrl); OaBrokerInstance::getUrl);
@ -23,6 +24,11 @@ public class EnrichMissingOpenAccess extends UpdateMatcher<OaBrokerInstance> {
@Override @Override
protected List<OaBrokerInstance> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerInstance> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getInstances().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final long count = target final long count = target
.getInstances() .getInstances()
.stream() .stream()

View File

@ -13,7 +13,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
public class EnrichMissingPid extends UpdateMatcher<OaBrokerTypedValue> { public class EnrichMissingPid extends UpdateMatcher<OaBrokerTypedValue> {
public EnrichMissingPid() { public EnrichMissingPid() {
super(true, super(10,
pid -> Topic.ENRICH_MISSING_PID, pid -> Topic.ENRICH_MISSING_PID,
(p, pid) -> p.getPids().add(pid), (p, pid) -> p.getPids().add(pid),
pid -> pid.getType() + "::" + pid.getValue()); pid -> pid.getType() + "::" + pid.getValue());
@ -22,9 +22,8 @@ public class EnrichMissingPid extends UpdateMatcher<OaBrokerTypedValue> {
@Override @Override
protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
final long count = target.getPids().size();
if (count > 0) { if (target.getPids().size() > 0) {
return Arrays.asList(); return Arrays.asList();
} }

View File

@ -12,7 +12,7 @@ import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
public class EnrichMissingPublicationDate extends UpdateMatcher<String> { public class EnrichMissingPublicationDate extends UpdateMatcher<String> {
public EnrichMissingPublicationDate() { public EnrichMissingPublicationDate() {
super(false, super(1,
date -> Topic.ENRICH_MISSING_PUBLICATION_DATE, date -> Topic.ENRICH_MISSING_PUBLICATION_DATE,
(p, date) -> p.setPublicationdate(date), (p, date) -> p.setPublicationdate(date),
s -> s); s -> s);

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,11 +10,12 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.broker.objects.OaBrokerTypedValue;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMissingSubject extends UpdateMatcher<OaBrokerTypedValue> { public class EnrichMissingSubject extends UpdateMatcher<OaBrokerTypedValue> {
public EnrichMissingSubject() { public EnrichMissingSubject() {
super(true, super(20,
s -> Topic.fromPath("ENRICH/MISSING/SUBJECT/" + s.getType()), s -> Topic.fromPath("ENRICH/MISSING/SUBJECT/" + s.getType()),
(p, s) -> p.getSubjects().add(s), (p, s) -> p.getSubjects().add(s),
s -> subjectAsString(s)); s -> subjectAsString(s));
@ -22,6 +24,11 @@ public class EnrichMissingSubject extends UpdateMatcher<OaBrokerTypedValue> {
@Override @Override
protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getSubjects().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingSubject = target final Set<String> existingSubject = target
.getSubjects() .getSubjects()
.stream() .stream()

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -14,7 +15,7 @@ import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMoreOpenAccess extends UpdateMatcher<OaBrokerInstance> { public class EnrichMoreOpenAccess extends UpdateMatcher<OaBrokerInstance> {
public EnrichMoreOpenAccess() { public EnrichMoreOpenAccess() {
super(true, super(20,
i -> Topic.ENRICH_MORE_OA_VERSION, i -> Topic.ENRICH_MORE_OA_VERSION,
(p, i) -> p.getInstances().add(i), (p, i) -> p.getInstances().add(i),
OaBrokerInstance::getUrl); OaBrokerInstance::getUrl);
@ -23,6 +24,11 @@ public class EnrichMoreOpenAccess extends UpdateMatcher<OaBrokerInstance> {
@Override @Override
protected List<OaBrokerInstance> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerInstance> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getInstances().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> urls = target final Set<String> urls = target
.getInstances() .getInstances()
.stream() .stream()

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,11 +10,12 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.broker.objects.OaBrokerTypedValue;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMorePid extends UpdateMatcher<OaBrokerTypedValue> { public class EnrichMorePid extends UpdateMatcher<OaBrokerTypedValue> {
public EnrichMorePid() { public EnrichMorePid() {
super(true, super(20,
pid -> Topic.ENRICH_MORE_PID, pid -> Topic.ENRICH_MORE_PID,
(p, pid) -> p.getPids().add(pid), (p, pid) -> p.getPids().add(pid),
pid -> pidAsString(pid)); pid -> pidAsString(pid));
@ -22,6 +24,11 @@ public class EnrichMorePid extends UpdateMatcher<OaBrokerTypedValue> {
@Override @Override
protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getPids().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingPids = target final Set<String> existingPids = target
.getPids() .getPids()
.stream() .stream()

View File

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple; package eu.dnetlib.dhp.broker.oa.matchers.simple;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -9,11 +10,12 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.broker.objects.OaBrokerTypedValue;
import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.model.Topic;
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
public class EnrichMoreSubject extends UpdateMatcher<OaBrokerTypedValue> { public class EnrichMoreSubject extends UpdateMatcher<OaBrokerTypedValue> {
public EnrichMoreSubject() { public EnrichMoreSubject() {
super(true, super(20,
s -> Topic.fromPath("ENRICH/MORE/SUBJECT/" + s.getType()), s -> Topic.fromPath("ENRICH/MORE/SUBJECT/" + s.getType()),
(p, s) -> p.getSubjects().add(s), (p, s) -> p.getSubjects().add(s),
s -> subjectAsString(s)); s -> subjectAsString(s));
@ -22,6 +24,11 @@ public class EnrichMoreSubject extends UpdateMatcher<OaBrokerTypedValue> {
@Override @Override
protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source, protected List<OaBrokerTypedValue> findDifferences(final OaBrokerMainEntity source,
final OaBrokerMainEntity target) { final OaBrokerMainEntity target) {
if (target.getSubjects().size() >= BrokerConstants.MAX_LIST_SIZE) {
return new ArrayList<>();
}
final Set<String> existingSubjects = target final Set<String> existingSubjects = target
.getSubjects() .getSubjects()
.stream() .stream()

View File

@ -17,6 +17,12 @@ public class BrokerConstants {
public static final float MIN_TRUST = 0.25f; public static final float MIN_TRUST = 0.25f;
public static final float MAX_TRUST = 1.00f; public static final float MAX_TRUST = 1.00f;
public static final int MAX_NUMBER_OF_RELS = 20;
public static final int MAX_STRING_SIZE = 3000;
public static final int MAX_LIST_SIZE = 50;
public static Class<?>[] getModelClasses() { public static Class<?>[] getModelClasses() {
final Set<Class<?>> list = new HashSet<>(); final Set<Class<?>> list = new HashSet<>();
list.addAll(Arrays.asList(ModelSupport.getOafModelClasses())); list.addAll(Arrays.asList(ModelSupport.getOafModelClasses()));

View File

@ -4,7 +4,9 @@ package eu.dnetlib.dhp.broker.oa.util;
import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
@ -32,4 +34,32 @@ public class ClusterUtils {
.map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); .map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz));
} }
public static boolean isDedupRoot(final String id) {
return id.contains("dedup_wf_");
}
public static final boolean isValidResultResultClass(final String s) {
return s.equals("isReferencedBy")
|| s.equals("isRelatedTo")
|| s.equals("references")
|| s.equals("isSupplementedBy")
|| s.equals("isSupplementedTo");
}
public static <T> T incrementAccumulator(final T o, final LongAccumulator acc) {
if (acc != null) {
acc.add(1);
}
return o;
}
public static <T> void save(final Dataset<T> dataset, final String path, final Class<T> clazz,
final LongAccumulator acc) {
dataset
.map(o -> ClusterUtils.incrementAccumulator(o, acc), Encoders.bean(clazz))
.write()
.mode(SaveMode.Overwrite)
.json(path);
}
} }

View File

@ -123,7 +123,8 @@ public class ConversionUtils {
res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue));
res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid));
res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances));
res.setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef)); res
.setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef));
return res; return res;
} }
@ -245,7 +246,13 @@ public class ConversionUtils {
private static List<String> fieldList(final List<Field<String>> fl) { private static List<String> fieldList(final List<Field<String>> fl) {
return fl != null return fl != null
? fl.stream().map(Field::getValue).filter(StringUtils::isNotBlank).collect(Collectors.toList()) ? fl
.stream()
.map(Field::getValue)
.map(s -> StringUtils.abbreviate(s, BrokerConstants.MAX_STRING_SIZE))
.filter(StringUtils::isNotBlank)
.limit(BrokerConstants.MAX_LIST_SIZE)
.collect(Collectors.toList())
: new ArrayList<>(); : new ArrayList<>();
} }
@ -255,6 +262,7 @@ public class ConversionUtils {
.stream() .stream()
.map(StructuredProperty::getValue) .map(StructuredProperty::getValue)
.filter(StringUtils::isNotBlank) .filter(StringUtils::isNotBlank)
.limit(BrokerConstants.MAX_LIST_SIZE)
.collect(Collectors.toList()) .collect(Collectors.toList())
: new ArrayList<>(); : new ArrayList<>();
} }
@ -280,6 +288,7 @@ public class ConversionUtils {
.stream() .stream()
.map(func::apply) .map(func::apply)
.filter(Objects::nonNull) .filter(Objects::nonNull)
.limit(BrokerConstants.MAX_LIST_SIZE)
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@ -293,6 +302,7 @@ public class ConversionUtils {
.map(func::apply) .map(func::apply)
.flatMap(List::stream) .flatMap(List::stream)
.filter(Objects::nonNull) .filter(Objects::nonNull)
.limit(BrokerConstants.MAX_LIST_SIZE)
.collect(Collectors.toList()); .collect(Collectors.toList());
} }

View File

@ -3,6 +3,12 @@ package eu.dnetlib.dhp.broker.oa.util;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.model.EventFactory; import eu.dnetlib.dhp.broker.model.EventFactory;
@ -35,7 +41,9 @@ import eu.dnetlib.pace.config.DedupConfig;
public class EventFinder { public class EventFinder {
private static List<UpdateMatcher<?>> matchers = new ArrayList<>(); private static final Logger log = LoggerFactory.getLogger(EventFinder.class);
private static final List<UpdateMatcher<?>> matchers = new ArrayList<>();
static { static {
matchers.add(new EnrichMissingAbstract()); matchers.add(new EnrichMissingAbstract());
matchers.add(new EnrichMissingAuthorOrcid()); matchers.add(new EnrichMissingAuthorOrcid());
@ -65,22 +73,48 @@ public class EventFinder {
matchers.add(new EnrichMissingAbstract()); matchers.add(new EnrichMissingAbstract());
} }
public static EventGroup generateEvents(final ResultGroup results, final DedupConfig dedupConfig) { public static EventGroup generateEvents(final ResultGroup results,
final Set<String> dsIdWhitelist,
final Set<String> dsIdBlacklist,
final Set<String> dsTypeWhitelist,
final DedupConfig dedupConfig,
final Map<String, LongAccumulator> accumulators) {
final List<UpdateInfo<?>> list = new ArrayList<>(); final List<UpdateInfo<?>> list = new ArrayList<>();
for (final OaBrokerMainEntity target : results.getData()) { for (final OaBrokerMainEntity target : results.getData()) {
for (final UpdateMatcher<?> matcher : matchers) { if (verifyTarget(target, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) {
list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig)); for (final UpdateMatcher<?> matcher : matchers) {
list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators));
}
} }
} }
return asEventGroup(list); return asEventGroup(list);
} }
private static boolean verifyTarget(final OaBrokerMainEntity target,
final Set<String> dsIdWhitelist,
final Set<String> dsIdBlacklist,
final Set<String> dsTypeWhitelist) {
if (dsIdWhitelist.contains(target.getCollectedFromId())) {
return true;
} else if (dsIdBlacklist.contains(target.getCollectedFromId())) {
return false;
} else {
return dsTypeWhitelist.contains(target.getCollectedFromType());
}
}
private static EventGroup asEventGroup(final List<UpdateInfo<?>> list) { private static EventGroup asEventGroup(final List<UpdateInfo<?>> list) {
final EventGroup events = new EventGroup(); final EventGroup events = new EventGroup();
list.stream().map(EventFactory::newBrokerEvent).forEach(events::addElement); list.stream().map(EventFactory::newBrokerEvent).forEach(events::addElement);
return events; return events;
} }
public static List<UpdateMatcher<?>> getMatchers() {
return matchers;
}
} }

View File

@ -14,12 +14,16 @@ public class EventGroup implements Serializable {
*/ */
private static final long serialVersionUID = 765977943803533130L; private static final long serialVersionUID = 765977943803533130L;
private final List<Event> data = new ArrayList<>(); private List<Event> data = new ArrayList<>();
public List<Event> getData() { public List<Event> getData() {
return data; return data;
} }
public void setData(final List<Event> data) {
this.data = data;
}
public EventGroup addElement(final Event elem) { public EventGroup addElement(final Event elem) {
data.add(elem); data.add(elem);
return this; return this;

View File

@ -111,8 +111,9 @@ public final class UpdateInfo<T> {
final OaBrokerMainEntity hl = new OaBrokerMainEntity(); final OaBrokerMainEntity hl = new OaBrokerMainEntity();
compileHighlight.accept(hl, getHighlightValue()); compileHighlight.accept(hl, getHighlightValue());
final String provId = getSource().getOriginalId(); final String provId = getSource().getOpenaireId();
final String provRepo = getSource().getCollectedFromName(); final String provRepo = getSource().getCollectedFromName();
final String provType = getSource().getCollectedFromType();
final String provUrl = getSource() final String provUrl = getSource()
.getInstances() .getInstances()
@ -122,7 +123,7 @@ public final class UpdateInfo<T> {
.orElse(null); .orElse(null);
; ;
final OaBrokerProvenance provenance = new OaBrokerProvenance(provId, provRepo, provUrl); final OaBrokerProvenance provenance = new OaBrokerProvenance(provId, provRepo, provType, provUrl);
final OaBrokerEventPayload res = new OaBrokerEventPayload(); final OaBrokerEventPayload res = new OaBrokerEventPayload();
res.setResult(target); res.setResult(target);

View File

@ -0,0 +1,59 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import scala.Tuple2;
public class AddDatasourceTypeAggregator
extends Aggregator<Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = 8788588975496014728L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g,
final Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo> t) {
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
if (t._2 != null && StringUtils.isNotBlank(t._2.getType())) {
res.setCollectedFromType(t._2.getType());
}
return res;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (StringUtils.isNotBlank(g1.getOpenaireId()) && StringUtils.isNotBlank(g1.getCollectedFromType())) {
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -1,71 +0,0 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import scala.Tuple2;
public class OaBrokerMainEntityAggregator<T>
extends Aggregator<Tuple2<OaBrokerMainEntity, T>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = -3687878788861013488L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, final Tuple2<OaBrokerMainEntity, T> t) {
if (g.getOriginalId() == null) {
return t._1;
} else if (t._2 instanceof RelatedSoftware) {
g.getSoftwares().add(((RelatedSoftware) t._2).getRelSoftware());
} else if (t._2 instanceof RelatedDataset) {
g.getDatasets().add(((RelatedDataset) t._2).getRelDataset());
} else if (t._2 instanceof RelatedPublication) {
g.getPublications().add(((RelatedPublication) t._2).getRelPublication());
} else if (t._2 instanceof RelatedProject) {
g.getProjects().add(((RelatedProject) t._2).getRelProject());
} else {
throw new RuntimeException("Invalid Object: " + t._2.getClass());
}
return g;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (g1.getOriginalId() != null) {
g1.getSoftwares().addAll(g2.getSoftwares());
g1.getDatasets().addAll(g2.getDatasets());
g1.getPublications().addAll(g2.getPublications());
g1.getProjects().addAll(g2.getProjects());
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -11,16 +11,15 @@ public class RelatedDataset implements Serializable {
* *
*/ */
private static final long serialVersionUID = 774487705184038324L; private static final long serialVersionUID = 774487705184038324L;
private String source; private String source;
private String relType;
private OaBrokerRelatedDataset relDataset; private OaBrokerRelatedDataset relDataset;
public RelatedDataset() { public RelatedDataset() {
} }
public RelatedDataset(final String source, final String relType, final OaBrokerRelatedDataset relDataset) { public RelatedDataset(final String source, final OaBrokerRelatedDataset relDataset) {
this.source = source; this.source = source;
this.relType = relType;
this.relDataset = relDataset; this.relDataset = relDataset;
} }
@ -32,14 +31,6 @@ public class RelatedDataset implements Serializable {
this.source = source; this.source = source;
} }
public String getRelType() {
return relType;
}
public void setRelType(final String relType) {
this.relType = relType;
}
public OaBrokerRelatedDataset getRelDataset() { public OaBrokerRelatedDataset getRelDataset() {
return relDataset; return relDataset;
} }

View File

@ -0,0 +1,68 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import scala.Tuple2;
public class RelatedDatasetAggregator
extends Aggregator<Tuple2<OaBrokerMainEntity, RelatedDataset>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = 6969761680131482557L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, final Tuple2<OaBrokerMainEntity, RelatedDataset> t) {
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
if (t._2 != null && res.getDatasets().size() < BrokerConstants.MAX_NUMBER_OF_RELS) {
res.getDatasets().add(t._2.getRelDataset());
}
return res;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (StringUtils.isNotBlank(g1.getOpenaireId())) {
final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getDatasets().size();
if (availables > 0) {
if (g2.getDatasets().size() <= availables) {
g1.getDatasets().addAll(g2.getDatasets());
} else {
g1.getDatasets().addAll(g2.getDatasets().subList(0, availables));
}
}
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -13,15 +13,13 @@ public class RelatedProject implements Serializable {
private static final long serialVersionUID = 4941437626549329870L; private static final long serialVersionUID = 4941437626549329870L;
private String source; private String source;
private String relType;
private OaBrokerProject relProject; private OaBrokerProject relProject;
public RelatedProject() { public RelatedProject() {
} }
public RelatedProject(final String source, final String relType, final OaBrokerProject relProject) { public RelatedProject(final String source, final OaBrokerProject relProject) {
this.source = source; this.source = source;
this.relType = relType;
this.relProject = relProject; this.relProject = relProject;
} }
@ -33,14 +31,6 @@ public class RelatedProject implements Serializable {
this.source = source; this.source = source;
} }
public String getRelType() {
return relType;
}
public void setRelType(final String relType) {
this.relType = relType;
}
public OaBrokerProject getRelProject() { public OaBrokerProject getRelProject() {
return relProject; return relProject;
} }

View File

@ -0,0 +1,68 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import scala.Tuple2;
public class RelatedProjectAggregator
extends Aggregator<Tuple2<OaBrokerMainEntity, RelatedProject>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = 8559808519152275763L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, final Tuple2<OaBrokerMainEntity, RelatedProject> t) {
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
if (t._2 != null && res.getProjects().size() < BrokerConstants.MAX_NUMBER_OF_RELS) {
res.getProjects().add(t._2.getRelProject());
}
return res;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (StringUtils.isNotBlank(g1.getOpenaireId())) {
final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getProjects().size();
if (availables > 0) {
if (g2.getProjects().size() <= availables) {
g1.getProjects().addAll(g2.getProjects());
} else {
g1.getProjects().addAll(g2.getProjects().subList(0, availables));
}
}
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -13,16 +13,13 @@ public class RelatedPublication implements Serializable {
private static final long serialVersionUID = 9021609640411395128L; private static final long serialVersionUID = 9021609640411395128L;
private String source; private String source;
private String relType;
private OaBrokerRelatedPublication relPublication; private OaBrokerRelatedPublication relPublication;
public RelatedPublication() { public RelatedPublication() {
} }
public RelatedPublication(final String source, final String relType, public RelatedPublication(final String source, final OaBrokerRelatedPublication relPublication) {
final OaBrokerRelatedPublication relPublication) {
this.source = source; this.source = source;
this.relType = relType;
this.relPublication = relPublication; this.relPublication = relPublication;
} }
@ -34,14 +31,6 @@ public class RelatedPublication implements Serializable {
this.source = source; this.source = source;
} }
public String getRelType() {
return relType;
}
public void setRelType(final String relType) {
this.relType = relType;
}
public OaBrokerRelatedPublication getRelPublication() { public OaBrokerRelatedPublication getRelPublication() {
return relPublication; return relPublication;
} }

View File

@ -0,0 +1,70 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import scala.Tuple2;
public class RelatedPublicationAggregator
extends Aggregator<Tuple2<OaBrokerMainEntity, RelatedPublication>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = 4656934981558135919L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g,
final Tuple2<OaBrokerMainEntity, RelatedPublication> t) {
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
if (t._2 != null && res.getPublications().size() < BrokerConstants.MAX_NUMBER_OF_RELS) {
res.getPublications().add(t._2.getRelPublication());
}
return res;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (StringUtils.isNotBlank(g1.getOpenaireId())) {
final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getPublications().size();
if (availables > 0) {
if (g2.getPublications().size() <= availables) {
g1.getPublications().addAll(g2.getPublications());
} else {
g1.getPublications().addAll(g2.getPublications().subList(0, availables));
}
}
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -11,16 +11,15 @@ public class RelatedSoftware implements Serializable {
* *
*/ */
private static final long serialVersionUID = 7573383356943300157L; private static final long serialVersionUID = 7573383356943300157L;
private String source; private String source;
private String relType;
private OaBrokerRelatedSoftware relSoftware; private OaBrokerRelatedSoftware relSoftware;
public RelatedSoftware() { public RelatedSoftware() {
} }
public RelatedSoftware(final String source, final String relType, final OaBrokerRelatedSoftware relSoftware) { public RelatedSoftware(final String source, final OaBrokerRelatedSoftware relSoftware) {
this.source = source; this.source = source;
this.relType = relType;
this.relSoftware = relSoftware; this.relSoftware = relSoftware;
} }
@ -32,14 +31,6 @@ public class RelatedSoftware implements Serializable {
this.source = source; this.source = source;
} }
public String getRelType() {
return relType;
}
public void setRelType(final String relType) {
this.relType = relType;
}
public OaBrokerRelatedSoftware getRelSoftware() { public OaBrokerRelatedSoftware getRelSoftware() {
return relSoftware; return relSoftware;
} }

View File

@ -0,0 +1,68 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.expressions.Aggregator;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
import scala.Tuple2;
public class RelatedSoftwareAggregator
extends Aggregator<Tuple2<OaBrokerMainEntity, RelatedSoftware>, OaBrokerMainEntity, OaBrokerMainEntity> {
/**
*
*/
private static final long serialVersionUID = -8987959389106443702L;
@Override
public OaBrokerMainEntity zero() {
return new OaBrokerMainEntity();
}
@Override
public OaBrokerMainEntity finish(final OaBrokerMainEntity g) {
return g;
}
@Override
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, final Tuple2<OaBrokerMainEntity, RelatedSoftware> t) {
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
if (t._2 != null && res.getSoftwares().size() < BrokerConstants.MAX_NUMBER_OF_RELS) {
res.getSoftwares().add(t._2.getRelSoftware());
}
return res;
}
@Override
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
if (StringUtils.isNotBlank(g1.getOpenaireId())) {
final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getSoftwares().size();
if (availables > 0) {
if (g2.getSoftwares().size() <= availables) {
g1.getSoftwares().addAll(g2.getSoftwares());
} else {
g1.getSoftwares().addAll(g2.getSoftwares().subList(0, availables));
}
}
return g1;
} else {
return g2;
}
}
@Override
public Encoder<OaBrokerMainEntity> bufferEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
@Override
public Encoder<OaBrokerMainEntity> outputEncoder() {
return Encoders.bean(OaBrokerMainEntity.class);
}
}

View File

@ -0,0 +1,40 @@
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
import java.io.Serializable;
public class SimpleDatasourceInfo implements Serializable {
/**
*
*/
private static final long serialVersionUID = 2996609859416024734L;
private String id;
private String type;
public SimpleDatasourceInfo() {
}
public SimpleDatasourceInfo(final String id, final String type) {
this.id = id;
this.type = type;
}
public String getId() {
return id;
}
public void setId(final String id) {
this.id = id;
}
public String getType() {
return type;
}
public void setType(final String type) {
this.type = type;
}
}

View File

@ -17,7 +17,29 @@
<name>dedupConfProfId</name> <name>dedupConfProfId</name>
<description>the id of a valid Dedup Configuration Profile</description> <description>the id of a valid Dedup Configuration Profile</description>
</property> </property>
<property>
<name>datasourceIdWhitelist</name>
<value>-</value>
<description>a white list (comma separeted, - for empty list) of datasource ids</description>
</property>
<property>
<name>datasourceTypeWhitelist</name>
<value>-</value>
<description>a white list (comma separeted, - for empty list) of datasource types</description>
</property>
<property>
<name>datasourceIdBlacklist</name>
<value>-</value>
<description>a black list (comma separeted, - for empty list) of datasource ids</description>
</property>
<property>
<name>esIndexName</name>
<description>the elasticsearch index name</description>
</property>
<property>
<name>esIndexHost</name>
<description>the elasticsearch host</description>
</property>
<property> <property>
<name>sparkDriverMemory</name> <name>sparkDriverMemory</name>
<description>memory for driver process</description> <description>memory for driver process</description>
@ -88,11 +110,12 @@
</action> </action>
<fork name="start_entities_and_rels"> <fork name="start_entities_and_rels">
<path start="prepare_simple_entities"/> <path start="prepare_simple_entities"/>
<path start="prepare_related_datasources"/>
<path start="prepare_related_softwares"/>
<path start="prepare_related_datasets"/> <path start="prepare_related_datasets"/>
<path start="prepare_related_projects"/> <path start="prepare_related_projects"/>
<path start="prepare_related_publications"/> <path start="prepare_related_publications"/>
<path start="prepare_related_softwares"/>
</fork> </fork>
<action name="prepare_simple_entities"> <action name="prepare_simple_entities">
@ -118,7 +141,32 @@
<ok to="wait_entities_and_rels"/> <ok to="wait_entities_and_rels"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<action name="prepare_related_datasources">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>PrepareRelatedDatasourcesJob</name>
<class>eu.dnetlib.dhp.broker.oa.PrepareRelatedDatasourcesJob</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=3840
</spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="wait_entities_and_rels"/>
<error to="Kill"/>
</action>
<action name="prepare_related_datasets"> <action name="prepare_related_datasets">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master> <master>yarn</master>
@ -190,7 +238,7 @@
<ok to="wait_entities_and_rels"/> <ok to="wait_entities_and_rels"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<action name="prepare_related_softwares"> <action name="prepare_related_softwares">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master> <master>yarn</master>
@ -214,15 +262,111 @@
<ok to="wait_entities_and_rels"/> <ok to="wait_entities_and_rels"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<join name="wait_entities_and_rels" to="join_entities"/> <join name="wait_entities_and_rels" to="join_entities_step0"/>
<action name="join_entities"> <action name="join_entities_step0">
<spark xmlns="uri:oozie:spark-action:0.2"> <spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master> <master>yarn</master>
<mode>cluster</mode> <mode>cluster</mode>
<name>JoinEntitiesJob</name> <name>JoinStep0</name>
<class>eu.dnetlib.dhp.broker.oa.JoinEntitiesJob</class> <class>eu.dnetlib.dhp.broker.oa.JoinStep0Job</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=3840
</spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="join_entities_step1"/>
<error to="Kill"/>
</action>
<action name="join_entities_step1">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>JoinStep1</name>
<class>eu.dnetlib.dhp.broker.oa.JoinStep1Job</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=3840
</spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="join_entities_step2"/>
<error to="Kill"/>
</action>
<action name="join_entities_step2">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>JoinStep2</name>
<class>eu.dnetlib.dhp.broker.oa.JoinStep2Job</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=3840
</spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="join_entities_step3"/>
<error to="Kill"/>
</action>
<action name="join_entities_step3">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>JoinStep3</name>
<class>eu.dnetlib.dhp.broker.oa.JoinStep3Job</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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.shuffle.partitions=3840
</spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</spark>
<ok to="join_entities_step4"/>
<error to="Kill"/>
</action>
<action name="join_entities_step4">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>JoinStep4</name>
<class>eu.dnetlib.dhp.broker.oa.JoinStep4Job</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar> <jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts> <spark-opts>
--executor-cores=${sparkExecutorCores} --executor-cores=${sparkExecutorCores}
@ -282,10 +426,37 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840 --conf spark.sql.shuffle.partitions=3840
</spark-opts> </spark-opts>
<arg>--graphPath</arg><arg>${graphInputPath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg> <arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--dedupConfProfile</arg><arg>${dedupConfProfId}</arg> <arg>--dedupConfProfile</arg><arg>${dedupConfProfId}</arg>
<arg>--datasourceIdWhitelist</arg><arg>${datasourceIdWhitelist}</arg>
<arg>--datasourceTypeWhitelist</arg><arg>${datasourceTypeWhitelist}</arg>
<arg>--datasourceIdBlacklist</arg><arg>${datasourceIdBlacklist}</arg>
</spark>
<ok to="index_es"/>
<error to="Kill"/>
</action>
<action name="index_es">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>IndexOnESJob</name>
<class>eu.dnetlib.dhp.broker.oa.IndexOnESJob</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.dynamicAllocation.maxExecutors="2"
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--index</arg><arg>${esIndexName}</arg>
<arg>--esHost</arg><arg>${esIndexHost}</arg>
</spark> </spark>
<ok to="End"/> <ok to="End"/>
<error to="Kill"/> <error to="Kill"/>

View File

@ -1,10 +1,4 @@
[ [
{
"paramName": "g",
"paramLongName": "graphPath",
"paramDescription": "the path where there the graph is stored",
"paramRequired": true
},
{ {
"paramName": "o", "paramName": "o",
"paramLongName": "workingPath", "paramLongName": "workingPath",
@ -22,5 +16,23 @@
"paramLongName": "dedupConfProfile", "paramLongName": "dedupConfProfile",
"paramDescription": "the id of a valid Dedup Configuration Profile", "paramDescription": "the id of a valid Dedup Configuration Profile",
"paramRequired": true "paramRequired": true
},
{
"paramName": "datasourceIdWhitelist",
"paramLongName": "datasourceIdWhitelist",
"paramDescription": "a white list (comma separeted, - for empty list) of datasource ids",
"paramRequired": true
},
{
"paramName": "datasourceTypeWhitelist",
"paramLongName": "datasourceTypeWhitelist",
"paramDescription": "a white list (comma separeted, - for empty list) of datasource types",
"paramRequired": true
},
{
"paramName": "datasourceIdBlacklist",
"paramLongName": "datasourceIdBlacklist",
"paramDescription": "a black list (comma separeted, - for empty list) of datasource ids",
"paramRequired": true
} }
] ]

View File

@ -0,0 +1,20 @@
[
{
"paramName": "o",
"paramLongName": "workingPath",
"paramDescription": "the workinh path",
"paramRequired": true
},
{
"paramName": "idx",
"paramLongName": "index",
"paramDescription": "the ES index",
"paramRequired": true
},
{
"paramName": "es",
"paramLongName": "esHost",
"paramDescription": "the ES host",
"paramRequired": true
}
]

View File

@ -0,0 +1,18 @@
<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>
</configuration>

View File

@ -0,0 +1,110 @@
<workflow-app name="create broker events - partial" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphInputPath</name>
<description>the path where the graph is stored</description>
</property>
<property>
<name>workingPath</name>
<description>the path where the the generated data will be stored</description>
</property>
<property>
<name>isLookupUrl</name>
<description>the address of the lookUp service</description>
</property>
<property>
<name>dedupConfProfId</name>
<description>the id of a valid Dedup Configuration Profile</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="index_es"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="index_es">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>IndexOnESJob</name>
<class>eu.dnetlib.dhp.broker.oa.IndexOnESJob</class>
<jar>dhp-broker-events-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.dynamicAllocation.maxExecutors="2"
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--index</arg><arg>${esIndexName}</arg>
<arg>--esHost</arg><arg>${esIndexHost}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,125 @@
package eu.dnetlib.dhp.broker.oa.matchers;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.Arrays;
import java.util.Collection;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingPublicationDate;
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
class UpdateMatcherTest {
UpdateMatcher<String> matcher = new EnrichMissingPublicationDate();
@BeforeEach
void setUp() throws Exception {
}
@Test
void testSearchUpdatesForRecord_1() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.isEmpty());
}
@Test
void testSearchUpdatesForRecord_2() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
res.setPublicationdate("2018");
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.isEmpty());
}
@Test
void testSearchUpdatesForRecord_3() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
p2.setPublicationdate("2018");
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.size() == 1);
}
@Test
void testSearchUpdatesForRecord_4() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
res.setPublicationdate("2018");
p2.setPublicationdate("2018");
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.isEmpty());
}
@Test
void testSearchUpdatesForRecord_5() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
res.setPublicationdate("2018");
p1.setPublicationdate("2018");
p2.setPublicationdate("2018");
p3.setPublicationdate("2018");
p4.setPublicationdate("2018");
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.isEmpty());
}
@Test
void testSearchUpdatesForRecord_6() {
final OaBrokerMainEntity res = new OaBrokerMainEntity();
final OaBrokerMainEntity p1 = new OaBrokerMainEntity();
final OaBrokerMainEntity p2 = new OaBrokerMainEntity();
final OaBrokerMainEntity p3 = new OaBrokerMainEntity();
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
p1.setPublicationdate("2018");
p2.setPublicationdate("2018");
p3.setPublicationdate("2018");
p4.setPublicationdate("2018");
final Collection<UpdateInfo<String>> list = matcher
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
assertTrue(list.size() == 1);
}
}

View File

@ -0,0 +1,57 @@
package eu.dnetlib.dhp.broker.oa.matchers.simple;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.List;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
class EnrichMissingPublicationDateTest {
final EnrichMissingPublicationDate matcher = new EnrichMissingPublicationDate();
@BeforeEach
void setUp() throws Exception {
}
@Test
void testFindDifferences_1() {
final OaBrokerMainEntity source = new OaBrokerMainEntity();
final OaBrokerMainEntity target = new OaBrokerMainEntity();
final List<String> list = matcher.findDifferences(source, target);
assertTrue(list.isEmpty());
}
@Test
void testFindDifferences_2() {
final OaBrokerMainEntity source = new OaBrokerMainEntity();
final OaBrokerMainEntity target = new OaBrokerMainEntity();
source.setPublicationdate("2018");
final List<String> list = matcher.findDifferences(source, target);
assertTrue(list.size() == 1);
}
@Test
void testFindDifferences_3() {
final OaBrokerMainEntity source = new OaBrokerMainEntity();
final OaBrokerMainEntity target = new OaBrokerMainEntity();
target.setPublicationdate("2018");
final List<String> list = matcher.findDifferences(source, target);
assertTrue(list.isEmpty());
}
@Test
void testFindDifferences_4() {
final OaBrokerMainEntity source = new OaBrokerMainEntity();
final OaBrokerMainEntity target = new OaBrokerMainEntity();
source.setPublicationdate("2018");
target.setPublicationdate("2018");
final List<String> list = matcher.findDifferences(source, target);
assertTrue(list.isEmpty());
}
}

View File

@ -271,6 +271,26 @@ object DoiBoostMappingUtil {
} }
def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String): StructuredProperty = {
val sp = new StructuredProperty
sp.setQualifier(createQualifier(classId,className, schemeId, schemeName))
sp.setValue(value)
sp
}
def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String, dataInfo: DataInfo): StructuredProperty = {
val sp = new StructuredProperty
sp.setQualifier(createQualifier(classId,className, schemeId, schemeName))
sp.setValue(value)
sp.setDataInfo(dataInfo)
sp
}
def createSP(value: String, classId: String, schemeId: String): StructuredProperty = { def createSP(value: String, classId: String, schemeId: String): StructuredProperty = {
val sp = new StructuredProperty val sp = new StructuredProperty
sp.setQualifier(createQualifier(classId, schemeId)) sp.setQualifier(createQualifier(classId, schemeId))
@ -279,6 +299,8 @@ object DoiBoostMappingUtil {
} }
def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = { def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = {
val sp = new StructuredProperty val sp = new StructuredProperty
sp.setQualifier(createQualifier(classId, schemeId)) sp.setQualifier(createQualifier(classId, schemeId))

View File

@ -129,16 +129,16 @@ case object ConversionUtil {
val fieldOfStudy = item._2 val fieldOfStudy = item._2
if (fieldOfStudy != null && fieldOfStudy.subjects != null && fieldOfStudy.subjects.nonEmpty) { if (fieldOfStudy != null && fieldOfStudy.subjects != null && fieldOfStudy.subjects.nonEmpty) {
val p: List[StructuredProperty] = fieldOfStudy.subjects.flatMap(s => { val p: List[StructuredProperty] = fieldOfStudy.subjects.flatMap(s => {
val s1 = createSP(s.DisplayName, "keyword", "dnet:subject_classification_typologies") val s1 = createSP(s.DisplayName, "MAG","Microsoft Academic Graph classification", "dnet:subject_classification_typologies", "dnet:subject_classification_typologies")
val di = DoiBoostMappingUtil.generateDataInfo(s.Score.toString) val di = DoiBoostMappingUtil.generateDataInfo(s.Score.toString)
var resList: List[StructuredProperty] = List(s1) var resList: List[StructuredProperty] = List(s1)
if (s.MainType.isDefined) { if (s.MainType.isDefined) {
val maintp = s.MainType.get val maintp = s.MainType.get
val s2 = createSP(s.MainType.get, "keyword", "dnet:subject_classification_typologies") val s2 = createSP(s.MainType.get, "MAG","Microsoft Academic Graph classification", "dnet:subject_classification_typologies", "dnet:subject_classification_typologies")
s2.setDataInfo(di) s2.setDataInfo(di)
resList = resList ::: List(s2) resList = resList ::: List(s2)
if (maintp.contains(".")) { if (maintp.contains(".")) {
val s3 = createSP(maintp.split("\\.").head, "keyword", "dnet:subject_classification_typologies") val s3 = createSP(maintp.split("\\.").head, "MAG","Microsoft Academic Graph classification", "dnet:subject_classification_typologies", "dnet:subject_classification_typologies")
s3.setDataInfo(di) s3.setDataInfo(di)
resList = resList ::: List(s3) resList = resList ::: List(s3)
} }

View File

@ -43,7 +43,7 @@ object SparkPreProcessMAG {
val distinctPaper: Dataset[MagPapers] = spark.createDataset(result) val distinctPaper: Dataset[MagPapers] = spark.createDataset(result)
distinctPaper.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/Papers_distinct") distinctPaper.write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/Papers_distinct")
logger.info("Phase 6) Enrich Publication with description") logger.info("Phase 0) Enrich Publication with description")
val pa = spark.read.load(s"${parser.get("sourcePath")}/PaperAbstractsInvertedIndex").as[MagPaperAbstract] val pa = spark.read.load(s"${parser.get("sourcePath")}/PaperAbstractsInvertedIndex").as[MagPaperAbstract]
pa.map(ConversionUtil.transformPaperAbstract).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/PaperAbstract") pa.map(ConversionUtil.transformPaperAbstract).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/PaperAbstract")

View File

@ -32,7 +32,7 @@
<start to="GenerateActionSet"/> <start to="CreateDOIBoost"/>
<kill name="Kill"> <kill name="Kill">

View File

@ -18,6 +18,9 @@ class CrossrefMappingTest {
val mapper = new ObjectMapper() val mapper = new ObjectMapper()
@Test @Test
def testFunderRelationshipsMapping(): Unit = { def testFunderRelationshipsMapping(): Unit = {
val template = Source.fromInputStream(getClass.getResourceAsStream("article_funder_template.json")).mkString val template = Source.fromInputStream(getClass.getResourceAsStream("article_funder_template.json")).mkString
@ -58,6 +61,27 @@ class CrossrefMappingTest {
} }
@Test
def testOrcidID() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("orcid_data.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
assertTrue(resultList.nonEmpty)
val items = resultList.filter(p => p.isInstanceOf[Result])
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
items.foreach(p => println(mapper.writeValueAsString(p)))
}
@Test @Test
def testEmptyTitle() :Unit = { def testEmptyTitle() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("empty_title.json")).mkString val json = Source.fromInputStream(getClass.getResourceAsStream("empty_title.json")).mkString

View File

@ -0,0 +1,271 @@
{
"DOI":"10.1016/j.carbpol.2020.115930",
"issued":{
"date-parts":[
[
2020,
4
]
]
},
"published-print":{
"date-parts":[
[
2020,
4
]
]
},
"prefix":"10.1016",
"subject":[
"Organic Chemistry",
"Materials Chemistry",
"Polymers and Plastics"
],
"author":[
{
"affiliation":[
],
"given":"Lei",
"family":"Fang",
"sequence":"first"
},
{
"affiliation":[
],
"given":"Hua",
"family":"Lin",
"sequence":"additional"
},
{
"affiliation":[
],
"given":"Zhenfeng",
"family":"Wu",
"sequence":"additional"
},
{
"affiliation":[
],
"given":"Zhen",
"family":"Wang",
"sequence":"additional"
},
{
"affiliation":[
],
"given":"Xinxin",
"family":"Fan",
"sequence":"additional"
},
{
"affiliation":[
],
"given":"Ziting",
"family":"Cheng",
"sequence":"additional"
},
{
"affiliation":[
],
"given":"Xiaoya",
"family":"Hou",
"sequence":"additional"
},
{
"authenticated-orcid":false,
"given":"Daquan",
"family":"Chen",
"sequence":"additional",
"affiliation":[
],
"ORCID":"http://orcid.org/0000-0002-6796-0204"
}
],
"reference-count":41,
"ISSN":[
"0144-8617"
],
"assertion":[
{
"name":"publisher",
"value":"Elsevier",
"label":"This article is maintained by"
},
{
"name":"articletitle",
"value":"In vitro/vivo evaluation of novel mitochondrial targeting charge-reversal polysaccharide-based antitumor nanoparticle",
"label":"Article Title"
},
{
"name":"journaltitle",
"value":"Carbohydrate Polymers",
"label":"Journal Title"
},
{
"name":"articlelink",
"value":"https://doi.org/10.1016/j.carbpol.2020.115930",
"label":"CrossRef DOI link to publisher maintained version"
},
{
"name":"content_type",
"value":"article",
"label":"Content Type"
},
{
"name":"copyright",
"value":"\\u00a9 2020 Elsevier Ltd. All rights reserved.",
"label":"Copyright"
}
],
"member":"78",
"source":"Crossref",
"score":1.0,
"deposited":{
"timestamp":1584590965000,
"date-time":"2020-03-19T04:09:25Z",
"date-parts":[
[
2020,
3,
19
]
]
},
"indexed":{
"timestamp":1584592912467,
"date-time":"2020-03-19T04:41:52Z",
"date-parts":[
[
2020,
3,
19
]
]
},
"type":"journal-article",
"URL":"http://dx.doi.org/10.1016/j.carbpol.2020.115930",
"is-referenced-by-count":0,
"volume":"234",
"issn-type":[
{
"type":"print",
"value":"0144-8617"
}
],
"link":[
{
"URL":"https://api.elsevier.com/content/article/PII:S0144861720301041?httpAccept=text/xml",
"intended-application":"text-mining",
"content-version":"vor",
"content-type":"text/xml"
},
{
"URL":"https://api.elsevier.com/content/article/PII:S0144861720301041?httpAccept=text/plain",
"intended-application":"text-mining",
"content-version":"vor",
"content-type":"text/plain"
}
],
"update-policy":"http://dx.doi.org/10.1016/elsevier_cm_policy",
"references-count":41,
"short-container-title":[
"Carbohydrate Polymers"
],
"publisher":"Elsevier BV",
"content-domain":{
"domain":[
"elsevier.com",
"sciencedirect.com"
],
"crossmark-restriction":true
},
"language":"en",
"license":[
{
"URL":"https://www.elsevier.com/tdm/userlicense/1.0/",
"start":{
"timestamp":1585699200000,
"date-time":"2020-04-01T00:00:00Z",
"date-parts":[
[
2020,
4,
1
]
]
},
"content-version":"tdm",
"delay-in-days":0
}
],
"created":{
"timestamp":1581759678000,
"date-time":"2020-02-15T09:41:18Z",
"date-parts":[
[
2020,
2,
15
]
]
},
"title":[
"In vitro/vivo evaluation of novel mitochondrial targeting charge-reversal polysaccharide-based antitumor nanoparticle"
],
"alternative-id":[
"S0144861720301041"
],
"container-title":[
"Carbohydrate Polymers"
],
"funder":[
{
"doi-asserted-by":"publisher",
"DOI":"10.13039/501100007129",
"name":"Natural Science Foundation of Shandong Province",
"award":[
"ZR2019ZD24",
"ZR2019YQ30"
]
},
{
"doi-asserted-by":"publisher",
"DOI":"10.13039/100010449",
"name":"Ministry of Education, Libya",
"award":[
]
},
{
"doi-asserted-by":"publisher",
"DOI":"10.13039/501100012249",
"name":"Jiangxi University of Traditional Chinese Medicine",
"award":[
"TCM-0906"
]
},
{
"name":"Taishan Scholar Program",
"award":[
"qnts20161035"
]
},
{
"name":"Open fund project of Key Laboratory of Modern Preparation of TCM",
"award":[
]
}
],
"page":"115930",
"article-number":"115930"
}

View File

@ -110,13 +110,13 @@ public class CommunityConfigurationFactory {
} }
private static List<ZenodoCommunity> parseZenodoCommunities(final Node node) { private static List<ZenodoCommunity> parseZenodoCommunities(final Node node) {
final Node oacommunitynode = node.selectSingleNode("./oacommunity"); // final Node oacommunitynode = node.selectSingleNode("./oacommunity");
String oacommunity = null; // String oacommunity = null;
if (oacommunitynode != null) { // if (oacommunitynode != null) {
String tmp = oacommunitynode.getText(); // String tmp = oacommunitynode.getText();
if (StringUtils.isNotBlank(tmp)) // if (StringUtils.isNotBlank(tmp))
oacommunity = tmp; // oacommunity = tmp;
} // }
final List<Node> list = node.selectNodes("./zenodocommunities/zenodocommunity"); final List<Node> list = node.selectNodes("./zenodocommunities/zenodocommunity");
final List<ZenodoCommunity> zenodoCommunityList = new ArrayList<>(); final List<ZenodoCommunity> zenodoCommunityList = new ArrayList<>();
@ -127,11 +127,11 @@ public class CommunityConfigurationFactory {
zenodoCommunityList.add(zc); zenodoCommunityList.add(zc);
} }
if (oacommunity != null) { // if (oacommunity != null) {
ZenodoCommunity zc = new ZenodoCommunity(); // ZenodoCommunity zc = new ZenodoCommunity();
zc.setZenodoCommunityId(oacommunity); // zc.setZenodoCommunityId(oacommunity);
zenodoCommunityList.add(zc); // zenodoCommunityList.add(zc);
} // }
log.info("size of the zenodo community list " + zenodoCommunityList.size()); log.info("size of the zenodo community list " + zenodoCommunityList.size());
return zenodoCommunityList; return zenodoCommunityList;
} }

View File

@ -44,7 +44,7 @@ public class Provider implements Serializable {
} }
private void setSelCriteria(String json, VerbResolver resolver) { private void setSelCriteria(String json, VerbResolver resolver) {
log.info("Selection constraints for datasource = " + json); log.debug("Selection constraints for datasource = " + json);
selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class); selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class);
selectionConstraints.setSelection(resolver); selectionConstraints.setSelection(resolver);
@ -54,7 +54,7 @@ public class Provider implements Serializable {
try { try {
setSelCriteria(n.getText(), resolver); setSelCriteria(n.getText(), resolver);
} catch (Exception e) { } catch (Exception e) {
log.info("not set selection criteria... "); log.debug("not set selection criteria... ");
selectionConstraints = null; selectionConstraints = null;
} }
} }

View File

@ -17,6 +17,8 @@ public class QueryInformationSystem {
+ " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " + " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept "
+ " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " + " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept "
+ " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " + " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept "
+
"let $zenodo := $x//param[./@name='zenodoCommunity']/text() "
+ " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] "
+ " return " + " return "
+ " <community> " + " <community> "
@ -38,8 +40,15 @@ public class QueryInformationSystem {
+ " {$d/param[./@name='selcriteria']/text()} " + " {$d/param[./@name='selcriteria']/text()} "
+ " </selcriteria> " + " </selcriteria> "
+ " </datasource> } " + " </datasource> } "
+ " </datasources> " + " </datasources> " +
+ " <zenodocommunities> " " <zenodocommunities> "+
"{for $zc in $zenodo " +
"return " +
"<zenodocommunity> " +
"<zenodoid> " +
"{$zc} " +
"</zenodoid> " +
"</zenodocommunity>}"
+ " {for $zc in $communities " + " {for $zc in $communities "
+ " return " + " return "
+ " <zenodocommunity> " + " <zenodocommunity> "

View File

@ -257,6 +257,9 @@
<zenodoid>bodhgaya</zenodoid> <zenodoid>bodhgaya</zenodoid>
<selcriteria/> <selcriteria/>
</zenodocommunity> </zenodocommunity>
<zenodocommunity>
<zenodoid>oac_dh-ch</zenodoid>
</zenodocommunity>
</zenodocommunities> </zenodocommunities>
<organizations/> <organizations/>
</community> </community>

View File

@ -147,9 +147,23 @@ public class CleanGraphSparkJob {
if (Objects.isNull(i.getHostedby()) || StringUtils.isBlank(i.getHostedby().getKey())) { if (Objects.isNull(i.getHostedby()) || StringUtils.isBlank(i.getHostedby().getKey())) {
i.setHostedby(ModelConstants.UNKNOWN_REPOSITORY); i.setHostedby(ModelConstants.UNKNOWN_REPOSITORY);
} }
if (Objects.isNull(i.getRefereed())) {
i.setRefereed(qualifier("0000", "Unknown", ModelConstants.DNET_REVIEW_LEVELS));
}
}
}
if (Objects.nonNull(r.getAuthor())) {
boolean nullRank = r
.getAuthor()
.stream()
.anyMatch(a -> Objects.isNull(a.getRank()));
if (nullRank) {
int i = 1;
for (Author author : r.getAuthor()) {
author.setRank(i++);
}
} }
} }
if (value instanceof Publication) { if (value instanceof Publication) {
} else if (value instanceof eu.dnetlib.dhp.schema.oaf.Dataset) { } else if (value instanceof eu.dnetlib.dhp.schema.oaf.Dataset) {

View File

@ -50,8 +50,6 @@ import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.common.DbClient;
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
@ -106,6 +104,9 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
final String dbPassword = parser.get("postgresPassword"); final String dbPassword = parser.get("postgresPassword");
log.info("postgresPassword: xxx"); log.info("postgresPassword: xxx");
final String dbSchema = parser.get("dbschema");
log.info("dbSchema {}: " + dbSchema);
final String isLookupUrl = parser.get("isLookupUrl"); final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl); log.info("isLookupUrl: {}", isLookupUrl);
@ -125,7 +126,11 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
smdbe.execute("queryDatasources.sql", smdbe::processDatasource); smdbe.execute("queryDatasources.sql", smdbe::processDatasource);
log.info("Processing projects..."); log.info("Processing projects...");
smdbe.execute("queryProjects.sql", smdbe::processProject); if (dbSchema.equalsIgnoreCase("beta")) {
smdbe.execute("queryProjects.sql", smdbe::processProject);
} else {
smdbe.execute("queryProjects_production.sql", smdbe::processProject);
}
log.info("Processing orgs..."); log.info("Processing orgs...");
smdbe.execute("queryOrganizations.sql", smdbe::processOrganization); smdbe.execute("queryOrganizations.sql", smdbe::processOrganization);

View File

@ -9,7 +9,15 @@ import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.ExtraInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.OAIProvenance;
import eu.dnetlib.dhp.schema.oaf.OriginDescription;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.utils.DHPUtils; import eu.dnetlib.dhp.utils.DHPUtils;
public class OafMapperUtils { public class OafMapperUtils {
@ -89,7 +97,9 @@ public class OafMapperUtils {
} }
public static StructuredProperty structuredProperty( public static StructuredProperty structuredProperty(
final String value, final Qualifier qualifier, final DataInfo dataInfo) { final String value,
final Qualifier qualifier,
final DataInfo dataInfo) {
if (value == null) { if (value == null) {
return null; return null;
} }
@ -192,8 +202,12 @@ public class OafMapperUtils {
} }
public static String createOpenaireId( public static String createOpenaireId(
final int prefix, final String originalId, final boolean to_md5) { final int prefix,
if (to_md5) { final String originalId,
final boolean to_md5) {
if (StringUtils.isBlank(originalId)) {
return null;
} else if (to_md5) {
final String nsPrefix = StringUtils.substringBefore(originalId, "::"); final String nsPrefix = StringUtils.substringBefore(originalId, "::");
final String rest = StringUtils.substringAfter(originalId, "::"); final String rest = StringUtils.substringAfter(originalId, "::");
return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest)); return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest));
@ -203,7 +217,9 @@ public class OafMapperUtils {
} }
public static String createOpenaireId( public static String createOpenaireId(
final String type, final String originalId, final boolean to_md5) { final String type,
final String originalId,
final boolean to_md5) {
switch (type) { switch (type) {
case "datasource": case "datasource":
return createOpenaireId(10, originalId, to_md5); return createOpenaireId(10, originalId, to_md5);

View File

@ -34,5 +34,11 @@
"paramLongName": "isLookupUrl", "paramLongName": "isLookupUrl",
"paramDescription": "the url of the ISLookupService", "paramDescription": "the url of the ISLookupService",
"paramRequired": true "paramRequired": true
},
{
"paramName": "dbschema",
"paramLongName": "dbschema",
"paramDescription": "the database schema according to the D-Net infrastructure (beta or production)",
"paramRequired": true
} }
] ]

View File

@ -25,6 +25,11 @@
<property> <property>
<name>postgresPassword</name> <name>postgresPassword</name>
<description>the password postgres</description> <description>the password postgres</description>
</property>
<property>
<name>dbSchema</name>
<value>beta</value>
<description>the database schema according to the D-Net infrastructure (beta or production)</description>
</property> </property>
<property> <property>
<name>mongoURL</name> <name>mongoURL</name>
@ -125,6 +130,7 @@
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg> <arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg> <arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--action</arg><arg>claims</arg> <arg>--action</arg><arg>claims</arg>
<arg>--dbschema</arg><arg>${dbSchema}</arg>
</java> </java>
<ok to="ImportODF_claims"/> <ok to="ImportODF_claims"/>
<error to="Kill"/> <error to="Kill"/>
@ -175,6 +181,7 @@
<arg>--postgresUser</arg><arg>${postgresUser}</arg> <arg>--postgresUser</arg><arg>${postgresUser}</arg>
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg> <arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg> <arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--dbschema</arg><arg>${dbSchema}</arg>
</java> </java>
<ok to="ImportODF"/> <ok to="ImportODF"/>
<error to="Kill"/> <error to="Kill"/>

View File

@ -111,6 +111,7 @@ public class MappersTest {
assertNotNull(i.getAccessright()); assertNotNull(i.getAccessright());
assertEquals("OPEN", i.getAccessright().getClassid()); assertEquals("OPEN", i.getAccessright().getClassid());
}); });
assertEquals("0001", p.getInstance().get(0).getRefereed().getClassid());
assertNotNull(p.getBestaccessright()); assertNotNull(p.getBestaccessright());
assertEquals("OPEN", p.getBestaccessright().getClassid()); assertEquals("OPEN", p.getBestaccessright().getClassid());
@ -217,6 +218,7 @@ public class MappersTest {
assertNotNull(i.getAccessright()); assertNotNull(i.getAccessright());
assertEquals("OPEN", i.getAccessright().getClassid()); assertEquals("OPEN", i.getAccessright().getClassid());
}); });
assertEquals("0001", d.getInstance().get(0).getRefereed().getClassid());
assertValidId(r1.getSource()); assertValidId(r1.getSource());
assertValidId(r1.getTarget()); assertValidId(r1.getTarget());

View File

@ -57,6 +57,7 @@
<oaf:identifier identifierType="doi">10.3897/oneeco.2.e13718</oaf:identifier> <oaf:identifier identifierType="doi">10.3897/oneeco.2.e13718</oaf:identifier>
<oaf:fulltext>https://oneecosystem.pensoft.net/article/13718/</oaf:fulltext> <oaf:fulltext>https://oneecosystem.pensoft.net/article/13718/</oaf:fulltext>
<oaf:journal eissn="2367-8194" issn="">One Ecosystem</oaf:journal> <oaf:journal eissn="2367-8194" issn="">One Ecosystem</oaf:journal>
<oaf:refereed>0001</oaf:refereed>
</metadata> </metadata>
<about xmlns:oai="http://www.openarchives.org/OAI/2.0/"> <about xmlns:oai="http://www.openarchives.org/OAI/2.0/">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd"> <provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">

View File

@ -90,6 +90,7 @@
<oaf:hostedBy id="re3data_____::r3d100010468" name="Zenodo"/> <oaf:hostedBy id="re3data_____::r3d100010468" name="Zenodo"/>
<oaf:projectid>corda_______::226852</oaf:projectid> <oaf:projectid>corda_______::226852</oaf:projectid>
<oaf:collectedFrom id="re3data_____::r3d100010468" name="Zenodo"/> <oaf:collectedFrom id="re3data_____::r3d100010468" name="Zenodo"/>
<oaf:refereed>0001</oaf:refereed>s
</metadata> </metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/" <about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:dri="http://www.driver-repository.eu/namespace/dri"

View File

@ -25,9 +25,7 @@ import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.oa.provision.model.SortableRelation;
import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2; import scala.Tuple2;
@ -109,11 +107,12 @@ public class CreateRelatedEntitiesJob_phase1 {
Class<E> clazz, Class<E> clazz,
String outputPath) { String outputPath) {
Dataset<Tuple2<String, SortableRelation>> relsByTarget = readPathRelation(spark, inputRelationsPath) Dataset<Tuple2<String, Relation>> relsByTarget = readPathRelation(spark, inputRelationsPath)
.filter("dataInfo.deletedbyinference == false") .filter("dataInfo.deletedbyinference == false")
.map( .map(
(MapFunction<SortableRelation, Tuple2<String, SortableRelation>>) r -> new Tuple2<>(r.getTarget(), r), (MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(r.getTarget(),
Encoders.tuple(Encoders.STRING(), Encoders.kryo(SortableRelation.class))) r),
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class)))
.cache(); .cache();
Dataset<Tuple2<String, RelatedEntity>> entities = readPathEntity(spark, inputEntityPath, clazz) Dataset<Tuple2<String, RelatedEntity>> entities = readPathEntity(spark, inputEntityPath, clazz)
@ -129,7 +128,7 @@ public class CreateRelatedEntitiesJob_phase1 {
relsByTarget relsByTarget
.joinWith(entities, entities.col("_1").equalTo(relsByTarget.col("_1")), "inner") .joinWith(entities, entities.col("_1").equalTo(relsByTarget.col("_1")), "inner")
.map( .map(
(MapFunction<Tuple2<Tuple2<String, SortableRelation>, Tuple2<String, RelatedEntity>>, RelatedEntityWrapper>) t -> new RelatedEntityWrapper( (MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, RelatedEntity>>, RelatedEntityWrapper>) t -> new RelatedEntityWrapper(
t._1()._2(), t._2()._2()), t._1()._2(), t._2()._2()),
Encoders.kryo(RelatedEntityWrapper.class)) Encoders.kryo(RelatedEntityWrapper.class))
.write() .write()
@ -232,11 +231,11 @@ public class CreateRelatedEntitiesJob_phase1 {
* @param relationPath * @param relationPath
* @return the Dataset<SortableRelation> containing all the relationships * @return the Dataset<SortableRelation> containing all the relationships
*/ */
private static Dataset<SortableRelation> readPathRelation( private static Dataset<Relation> readPathRelation(
SparkSession spark, final String relationPath) { SparkSession spark, final String relationPath) {
log.info("Reading relations from: {}", relationPath); log.info("Reading relations from: {}", relationPath);
return spark.read().load(relationPath).as(Encoders.bean(SortableRelation.class)); return spark.read().load(relationPath).as(Encoders.bean(Relation.class));
} }
private static void removeOutputDir(SparkSession spark, String path) { private static void removeOutputDir(SparkSession spark, String path) {

View File

@ -19,7 +19,6 @@ import org.apache.spark.sql.expressions.Aggregator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -28,8 +27,6 @@ import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.oa.provision.model.TypedRow;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2; import scala.Tuple2;
@ -305,20 +302,6 @@ public class CreateRelatedEntitiesJob_phase2 {
private static FilterFunction<JoinedEntity> filterEmptyEntityFn() { private static FilterFunction<JoinedEntity> filterEmptyEntityFn() {
return (FilterFunction<JoinedEntity>) v -> Objects.nonNull(v.getEntity()); return (FilterFunction<JoinedEntity>) v -> Objects.nonNull(v.getEntity());
/*
* return (FilterFunction<JoinedEntity>) v -> Optional .ofNullable(v.getEntity()) .map(e ->
* StringUtils.isNotBlank(e.getId())) .orElse(false);
*/
}
private static TypedRow getTypedRow(String type, OafEntity entity)
throws JsonProcessingException {
TypedRow t = new TypedRow();
t.setType(type);
t.setDeleted(entity.getDataInfo().getDeletedbyinference());
t.setId(entity.getId());
t.setOaf(OBJECT_MAPPER.writeValueAsString(entity));
return t;
} }
private static void removeOutputDir(SparkSession spark, String path) { private static void removeOutputDir(SparkSession spark, String path) {

View File

@ -4,34 +4,35 @@ package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.*; import java.util.*;
import java.util.function.Function; import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.*; import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.rdd.RDD; import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Dataset; import org.apache.spark.sql.*;
import org.apache.spark.sql.Encoders; import org.apache.spark.sql.expressions.Aggregator;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.clearspring.analytics.util.Lists;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Splitter; import com.google.common.base.Splitter;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators; import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.FunctionalInterfaceSupport;
import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.provision.model.SortableRelation; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.oa.provision.model.SortableRelationKey;
import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner;
import scala.Function1; import eu.dnetlib.dhp.schema.oaf.Relation;
import scala.Tuple2; import scala.Tuple2;
/** /**
@ -106,6 +107,8 @@ public class PrepareRelationsJob {
log.info("maxRelations: {}", maxRelations); log.info("maxRelations: {}", maxRelations);
SparkConf conf = new SparkConf(); SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
runWithSparkSession( runWithSparkSession(
conf, conf,
@ -129,31 +132,111 @@ public class PrepareRelationsJob {
* @param maxRelations maximum number of allowed outgoing edges * @param maxRelations maximum number of allowed outgoing edges
* @param relPartitions number of partitions for the output RDD * @param relPartitions number of partitions for the output RDD
*/ */
private static void prepareRelationsRDD( private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath,
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations, Set<String> relationFilter, int maxRelations, int relPartitions) {
int relPartitions) {
RDD<SortableRelation> cappedRels = readPathRelationRDD(spark, inputRelationsPath) // group by SOURCE and apply limit
.repartition(relPartitions) RDD<Relation> bySource = readPathRelationRDD(spark, inputRelationsPath)
.filter(rel -> !rel.getDataInfo().getDeletedbyinference()) .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false)
.filter(rel -> !relationFilter.contains(rel.getRelClass())) .filter(rel -> relationFilter.contains(rel.getRelClass()) == false)
// group by SOURCE and apply limit .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, r.getSource()), r))
.mapToPair(rel -> new Tuple2<>(rel.getSource(), rel)) .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions))
.groupByKey(new RelationPartitioner(relPartitions)) .groupBy(Tuple2::_1)
.flatMap(group -> Iterables.limit(group._2(), maxRelations).iterator()) .map(Tuple2::_2)
// group by TARGET and apply limit .map(t -> Iterables.limit(t, maxRelations))
.mapToPair(rel -> new Tuple2<>(rel.getTarget(), rel)) .flatMap(Iterable::iterator)
.groupByKey(new RelationPartitioner(relPartitions)) .map(Tuple2::_2)
.flatMap(group -> Iterables.limit(group._2(), maxRelations).iterator())
.rdd(); .rdd();
spark spark
.createDataset(cappedRels, Encoders.bean(SortableRelation.class)) .createDataset(bySource, Encoders.bean(Relation.class))
.repartition(relPartitions)
.write() .write()
.mode(SaveMode.Overwrite) .mode(SaveMode.Overwrite)
.parquet(outputPath); .parquet(outputPath);
} }
private static void prepareRelationsDataset(
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations,
int relPartitions) {
spark
.read()
.textFile(inputRelationsPath)
.repartition(relPartitions)
.map(
(MapFunction<String, Relation>) s -> OBJECT_MAPPER.readValue(s, Relation.class),
Encoders.kryo(Relation.class))
.filter((FilterFunction<Relation>) rel -> rel.getDataInfo().getDeletedbyinference() == false)
.filter((FilterFunction<Relation>) rel -> relationFilter.contains(rel.getRelClass()) == false)
.groupByKey(
(MapFunction<Relation, String>) Relation::getSource,
Encoders.STRING())
.agg(new RelationAggregator(maxRelations).toColumn())
.flatMap(
(FlatMapFunction<Tuple2<String, RelationList>, Relation>) t -> Iterables
.limit(t._2().getRelations(), maxRelations)
.iterator(),
Encoders.bean(Relation.class))
.repartition(relPartitions)
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
}
public static class RelationAggregator
extends Aggregator<Relation, RelationList, RelationList> {
private int maxRelations;
public RelationAggregator(int maxRelations) {
this.maxRelations = maxRelations;
}
@Override
public RelationList zero() {
return new RelationList();
}
@Override
public RelationList reduce(RelationList b, Relation a) {
b.getRelations().add(a);
return getSortableRelationList(b);
}
@Override
public RelationList merge(RelationList b1, RelationList b2) {
b1.getRelations().addAll(b2.getRelations());
return getSortableRelationList(b1);
}
@Override
public RelationList finish(RelationList r) {
return getSortableRelationList(r);
}
private RelationList getSortableRelationList(RelationList b1) {
RelationList sr = new RelationList();
sr
.setRelations(
b1
.getRelations()
.stream()
.limit(maxRelations)
.collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator()))));
return sr;
}
@Override
public Encoder<RelationList> bufferEncoder() {
return Encoders.kryo(RelationList.class);
}
@Override
public Encoder<RelationList> outputEncoder() {
return Encoders.kryo(RelationList.class);
}
}
/** /**
* Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text
* file, * file,
@ -162,10 +245,10 @@ public class PrepareRelationsJob {
* @param inputPath * @param inputPath
* @return the JavaRDD<SortableRelation> containing all the relationships * @return the JavaRDD<SortableRelation> containing all the relationships
*/ */
private static JavaRDD<SortableRelation> readPathRelationRDD( private static JavaRDD<Relation> readPathRelationRDD(
SparkSession spark, final String inputPath) { SparkSession spark, final String inputPath) {
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, SortableRelation.class)); return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, Relation.class));
} }
private static void removeOutputDir(SparkSession spark, String path) { private static void removeOutputDir(SparkSession spark, String path) {

View File

@ -0,0 +1,43 @@
package eu.dnetlib.dhp.oa.provision;
import java.util.Comparator;
import java.util.Map;
import java.util.Optional;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class RelationComparator implements Comparator<Relation> {
private static final Map<String, Integer> weights = Maps.newHashMap();
static {
weights.put("outcome", 0);
weights.put("supplement", 1);
weights.put("review", 2);
weights.put("citation", 3);
weights.put("affiliation", 4);
weights.put("relationship", 5);
weights.put("publicationDataset", 6);
weights.put("similarity", 7);
weights.put("provision", 8);
weights.put("participation", 9);
weights.put("dedup", 10);
}
private Integer getWeight(Relation o) {
return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE);
}
@Override
public int compare(Relation o1, Relation o2) {
return ComparisonChain
.start()
.compare(getWeight(o1), getWeight(o2))
.result();
}
}

View File

@ -0,0 +1,25 @@
package eu.dnetlib.dhp.oa.provision;
import java.io.Serializable;
import java.util.PriorityQueue;
import java.util.Queue;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class RelationList implements Serializable {
private Queue<Relation> relations;
public RelationList() {
this.relations = new PriorityQueue<>(new RelationComparator());
}
public Queue<Relation> getRelations() {
return relations;
}
public void setRelations(Queue<Relation> relations) {
this.relations = relations;
}
}

View File

@ -0,0 +1,80 @@
package eu.dnetlib.dhp.oa.provision;
import java.io.Serializable;
import java.util.Map;
import java.util.Optional;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class SortableRelation extends Relation implements Comparable<SortableRelation>, Serializable {
private static final Map<String, Integer> weights = Maps.newHashMap();
static {
weights.put("outcome", 0);
weights.put("supplement", 1);
weights.put("review", 2);
weights.put("citation", 3);
weights.put("affiliation", 4);
weights.put("relationship", 5);
weights.put("publicationDataset", 6);
weights.put("similarity", 7);
weights.put("provision", 8);
weights.put("participation", 9);
weights.put("dedup", 10);
}
private static final long serialVersionUID = 34753984579L;
private String groupingKey;
public static SortableRelation create(Relation r, String groupingKey) {
SortableRelation sr = new SortableRelation();
sr.setGroupingKey(groupingKey);
sr.setSource(r.getSource());
sr.setTarget(r.getTarget());
sr.setRelType(r.getRelType());
sr.setSubRelType(r.getSubRelType());
sr.setRelClass(r.getRelClass());
sr.setDataInfo(r.getDataInfo());
sr.setCollectedfrom(r.getCollectedfrom());
sr.setLastupdatetimestamp(r.getLastupdatetimestamp());
sr.setProperties(r.getProperties());
sr.setValidated(r.getValidated());
sr.setValidationDate(r.getValidationDate());
return sr;
}
@JsonIgnore
public Relation asRelation() {
return this;
}
@Override
public int compareTo(SortableRelation o) {
return ComparisonChain
.start()
.compare(getGroupingKey(), o.getGroupingKey())
.compare(getWeight(this), getWeight(o))
.result();
}
private Integer getWeight(SortableRelation o) {
return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE);
}
public String getGroupingKey() {
return groupingKey;
}
public void setGroupingKey(String groupingKey) {
this.groupingKey = groupingKey;
}
}

View File

@ -5,6 +5,8 @@ import java.util.List;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import eu.dnetlib.dhp.oa.provision.RelationList;
import eu.dnetlib.dhp.oa.provision.SortableRelation;
import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.common.ModelSupport;
public class ProvisionModelSupport { public class ProvisionModelSupport {
@ -15,11 +17,12 @@ public class ProvisionModelSupport {
.addAll( .addAll(
Lists Lists
.newArrayList( .newArrayList(
TypedRow.class,
RelatedEntityWrapper.class, RelatedEntityWrapper.class,
JoinedEntity.class, JoinedEntity.class,
RelatedEntity.class, RelatedEntity.class,
SortableRelation.class)); SortableRelationKey.class,
SortableRelation.class,
RelationList.class));
return modelClasses.toArray(new Class[] {}); return modelClasses.toArray(new Class[] {});
} }
} }

View File

@ -5,28 +5,26 @@ import java.io.Serializable;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class RelatedEntityWrapper implements Serializable { public class RelatedEntityWrapper implements Serializable {
private SortableRelation relation; private Relation relation;
private RelatedEntity target; private RelatedEntity target;
public RelatedEntityWrapper() { public RelatedEntityWrapper() {
} }
public RelatedEntityWrapper(SortableRelation relation, RelatedEntity target) { public RelatedEntityWrapper(Relation relation, RelatedEntity target) {
this(null, relation, target);
}
public RelatedEntityWrapper(TypedRow entity, SortableRelation relation, RelatedEntity target) {
this.relation = relation; this.relation = relation;
this.target = target; this.target = target;
} }
public SortableRelation getRelation() { public Relation getRelation() {
return relation; return relation;
} }
public void setRelation(SortableRelation relation) { public void setRelation(Relation relation) {
this.relation = relation; this.relation = relation;
} }

View File

@ -1,38 +0,0 @@
package eu.dnetlib.dhp.oa.provision.model;
import java.io.Serializable;
import java.util.Map;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class SortableRelation extends Relation implements Comparable<Relation>, Serializable {
private static final Map<String, Integer> weights = Maps.newHashMap();
static {
weights.put("outcome", 0);
weights.put("supplement", 1);
weights.put("affiliation", 2);
weights.put("relationship", 3);
weights.put("publicationDataset", 4);
weights.put("similarity", 5);
weights.put("provision", 6);
weights.put("participation", 7);
weights.put("dedup", 8);
}
@Override
public int compareTo(Relation o) {
return ComparisonChain
.start()
.compare(weights.get(getSubRelType()), weights.get(o.getSubRelType()))
.compare(getSource(), o.getSource())
.compare(getTarget(), o.getTarget())
.result();
}
}

View File

@ -0,0 +1,90 @@
package eu.dnetlib.dhp.oa.provision.model;
import java.io.Serializable;
import java.util.Map;
import java.util.Optional;
import com.google.common.base.Objects;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class SortableRelationKey implements Comparable<SortableRelationKey>, Serializable {
private static final Map<String, Integer> weights = Maps.newHashMap();
static {
weights.put("outcome", 0);
weights.put("supplement", 1);
weights.put("review", 2);
weights.put("citation", 3);
weights.put("affiliation", 4);
weights.put("relationship", 5);
weights.put("publicationDataset", 6);
weights.put("similarity", 7);
weights.put("provision", 8);
weights.put("participation", 9);
weights.put("dedup", 10);
}
private static final long serialVersionUID = 3232323;
private String groupingKey;
private String subRelType;
public static SortableRelationKey create(Relation r, String groupingKey) {
SortableRelationKey sr = new SortableRelationKey();
sr.setGroupingKey(groupingKey);
sr.setSubRelType(r.getSubRelType());
return sr;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
SortableRelationKey that = (SortableRelationKey) o;
return getGroupingKey().equals(that.getGroupingKey());
}
@Override
public int hashCode() {
return Objects.hashCode(getGroupingKey());
}
@Override
public int compareTo(SortableRelationKey o) {
return ComparisonChain
.start()
.compare(getGroupingKey(), o.getGroupingKey())
.compare(getWeight(this), getWeight(o))
.result();
}
private Integer getWeight(SortableRelationKey o) {
return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE);
}
public String getSubRelType() {
return subRelType;
}
public void setSubRelType(String subRelType) {
this.subRelType = subRelType;
}
public String getGroupingKey() {
return groupingKey;
}
public void setGroupingKey(String groupingKey) {
this.groupingKey = groupingKey;
}
}

View File

@ -1,64 +0,0 @@
package eu.dnetlib.dhp.oa.provision.model;
import java.io.Serializable;
import com.google.common.base.Objects;
public class TypedRow implements Serializable {
private String id;
private Boolean deleted;
private String type;
private String oaf;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public Boolean getDeleted() {
return deleted;
}
public void setDeleted(Boolean deleted) {
this.deleted = deleted;
}
public String getType() {
return type;
}
public void setType(String type) {
this.type = type;
}
public String getOaf() {
return oaf;
}
public void setOaf(String oaf) {
this.oaf = oaf;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
TypedRow typedRow2 = (TypedRow) o;
return Objects.equal(id, typedRow2.id);
}
@Override
public int hashCode() {
return Objects.hashCode(id);
}
}

View File

@ -4,12 +4,16 @@ package eu.dnetlib.dhp.oa.provision.utils;
import org.apache.spark.Partitioner; import org.apache.spark.Partitioner;
import org.apache.spark.util.Utils; import org.apache.spark.util.Utils;
import eu.dnetlib.dhp.oa.provision.model.SortableRelationKey;
/** /**
* Used in combination with SortableRelationKey, allows to partition the records by source id, therefore allowing to * Used in combination with SortableRelationKey, allows to partition the records by source id, therefore allowing to
* sort relations sharing the same source id by the ordering defined in SortableRelationKey. * sort relations sharing the same source id by the ordering defined in SortableRelationKey.
*/ */
public class RelationPartitioner extends Partitioner { public class RelationPartitioner extends Partitioner {
private static final long serialVersionUID = 343434456L;
private final int numPartitions; private final int numPartitions;
public RelationPartitioner(int numPartitions) { public RelationPartitioner(int numPartitions) {
@ -23,8 +27,18 @@ public class RelationPartitioner extends Partitioner {
@Override @Override
public int getPartition(Object key) { public int getPartition(Object key) {
String partitionKey = (String) key; SortableRelationKey partitionKey = (SortableRelationKey) key;
return Utils.nonNegativeMod(partitionKey.hashCode(), numPartitions()); return Utils.nonNegativeMod(partitionKey.getGroupingKey().hashCode(), numPartitions());
}
@Override
public boolean equals(Object obj) {
if (obj instanceof RelationPartitioner) {
RelationPartitioner p = (RelationPartitioner) obj;
if (p.numPartitions() == numPartitions())
return true;
}
return false;
} }
} }

View File

@ -121,10 +121,6 @@ public class XmlRecordFactory implements Serializable {
} }
} }
private static OafEntity toOafEntity(TypedRow typedRow) {
return parseOaf(typedRow.getOaf(), typedRow.getType());
}
private static OafEntity parseOaf(final String json, final String type) { private static OafEntity parseOaf(final String json, final String type) {
try { try {
switch (EntityType.valueOf(type)) { switch (EntityType.valueOf(type)) {
@ -276,7 +272,7 @@ public class XmlRecordFactory implements Serializable {
pidType, pidType,
pidValue pidValue
.toLowerCase() .toLowerCase()
.replaceAll("orcid", ""))); .replaceAll("^.*orcid\\.org\\/", "")));
} }
} }
}); });

View File

@ -133,6 +133,7 @@
</spark-opts> </spark-opts>
<arg>--inputRelationsPath</arg><arg>${inputGraphRootPath}/relation</arg> <arg>--inputRelationsPath</arg><arg>${inputGraphRootPath}/relation</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation</arg> <arg>--outputPath</arg><arg>${workingDir}/relation</arg>
<arg>--maxRelations</arg><arg>${maxRelations}</arg>
<arg>--relPartitions</arg><arg>5000</arg> <arg>--relPartitions</arg><arg>5000</arg>
</spark> </spark>
<ok to="fork_join_related_entities"/> <ok to="fork_join_related_entities"/>

View File

@ -0,0 +1,93 @@
package eu.dnetlib.dhp.oa.provision;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.schema.oaf.Relation;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
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.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
public class PrepareRelationsJobTest {
private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJobTest.class);
public static final String SUBRELTYPE = "subRelType";
public static final String OUTCOME = "outcome";
public static final String SUPPLEMENT = "supplement";
private static SparkSession spark;
private static Path workingDir;
@BeforeAll
public static void setUp() throws IOException {
workingDir = Files.createTempDirectory(PrepareRelationsJobTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
spark = SparkSession
.builder()
.appName(PrepareRelationsJobTest.class.getSimpleName())
.master("local[*]")
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
public void testRunPrepareRelationsJob(@TempDir Path testPath) throws Exception {
final int maxRelations = 10;
PrepareRelationsJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-inputRelationsPath", getClass().getResource("relations.gz").getPath(),
"-outputPath", testPath.toString(),
"-relPartitions", "10",
"-relationFilter", "asd",
"-maxRelations", String.valueOf(maxRelations)
});
Dataset<Relation> out = spark.read()
.parquet(testPath.toString())
.as(Encoders.bean(Relation.class))
.cache();
Assertions.assertEquals(10, out.count());
Dataset<Row> freq = out.toDF().cube(SUBRELTYPE).count().filter((FilterFunction<Row>) value -> !value.isNullAt(0));
long outcome = freq.filter(freq.col(SUBRELTYPE).equalTo(OUTCOME)).collectAsList().get(0).getAs("count");
long supplement = freq.filter(freq.col(SUBRELTYPE).equalTo(SUPPLEMENT)).collectAsList().get(0).getAs("count");
Assertions.assertTrue(outcome > supplement);
Assertions.assertEquals(7, outcome);
Assertions.assertEquals(3, supplement);
}
}

View File

@ -0,0 +1,42 @@
package eu.dnetlib.dhp.oa.provision;
import java.io.IOException;
import java.util.List;
import org.apache.commons.io.IOUtils;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.provision.model.SortableRelationKey;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class SortableRelationKeyTest {
@Test
public void doTesSorting() throws IOException {
final ObjectMapper mapper = new ObjectMapper();
final String json = IOUtils.toString(this.getClass().getResourceAsStream("relations.json"));
final List<Relation> relations = mapper.readValue(json, new TypeReference<List<Relation>>() {
});
relations
.stream()
.map(r -> SortableRelationKey.create(r, r.getSource()))
.sorted()
.forEach(
it -> {
try {
System.out.println(mapper.writeValueAsString(it));
} catch (JsonProcessingException e) {
e.printStackTrace();
}
});
}
}

View File

@ -0,0 +1,90 @@
[
{
"collectedfrom": [],
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"lastupdatetimestamp": 1592688952862,
"properties": [],
"relClass": "hasAuthorInstitution",
"relType": "resultOrganization",
"source": "1",
"subRelType": "affiliation",
"target": "2"
},
{
"collectedfrom": [],
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"lastupdatetimestamp": 1592688952862,
"properties": [],
"relClass": "isAuthorInstitutionOf",
"relType": "resultOrganization",
"source": "2",
"subRelType": "affiliation",
"target": "1"
},
{
"collectedfrom": [],
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"lastupdatetimestamp": 1592688952862,
"properties": [],
"relClass": "isProducedBy",
"relType": "resultProject",
"source": "1",
"subRelType": "outcome",
"target": "2"
},
{
"collectedfrom": [],
"dataInfo": {
"deletedbyinference": false,
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "Harvested",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"lastupdatetimestamp": 1592688952862,
"properties": [],
"relClass": "produces",
"relType": "resultProject",
"source": "2",
"subRelType": "outcome",
"target": "1"
}
]

View File

@ -0,0 +1,11 @@
# Set root logger level to DEBUG and its only appender to A1.
log4j.rootLogger=INFO, A1
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.logger.org = ERROR
log4j.logger.eu.dnetlib = DEBUG
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n