forked from D-Net/dnet-hadoop
merge with master fork
This commit is contained in:
commit
40bbe94f7c
|
@ -14,6 +14,37 @@
|
||||||
|
|
||||||
<description>This module contains common schema classes meant to be used across the dnet-hadoop submodules</description>
|
<description>This module contains common schema classes meant to be used across the dnet-hadoop submodules</description>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>net.alchim31.maven</groupId>
|
||||||
|
<artifactId>scala-maven-plugin</artifactId>
|
||||||
|
<version>4.0.1</version>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<id>scala-compile-first</id>
|
||||||
|
<phase>initialize</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>add-source</goal>
|
||||||
|
<goal>compile</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
<execution>
|
||||||
|
<id>scala-test-compile</id>
|
||||||
|
<phase>process-test-resources</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>testCompile</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
<configuration>
|
||||||
|
<scalaVersion>${scala.version}</scalaVersion>
|
||||||
|
</configuration>
|
||||||
|
</plugin>
|
||||||
|
|
||||||
|
</plugins>
|
||||||
|
</build>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|
|
@ -1,8 +1,6 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.schema.common;
|
package eu.dnetlib.dhp.schema.common;
|
||||||
|
|
||||||
import java.security.Key;
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
||||||
|
|
|
@ -0,0 +1,90 @@
|
||||||
|
package eu.dnetlib.dhp.schema.scholexplorer
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.{DataInfo, Field, KeyValue, Qualifier, StructuredProperty}
|
||||||
|
|
||||||
|
object OafUtils {
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
def generateKeyValue(key: String, value: String): KeyValue = {
|
||||||
|
val kv: KeyValue = new KeyValue()
|
||||||
|
kv.setKey(key)
|
||||||
|
kv.setValue(value)
|
||||||
|
kv.setDataInfo(generateDataInfo("0.9"))
|
||||||
|
kv
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def generateDataInfo(trust: String = "0.9", invisibile: Boolean = false): DataInfo = {
|
||||||
|
val di = new DataInfo
|
||||||
|
di.setDeletedbyinference(false)
|
||||||
|
di.setInferred(false)
|
||||||
|
di.setInvisible(false)
|
||||||
|
di.setTrust(trust)
|
||||||
|
di.setProvenanceaction(createQualifier("sysimport:actionset", "dnet:provenanceActions"))
|
||||||
|
di
|
||||||
|
}
|
||||||
|
|
||||||
|
def createQualifier(cls: String, sch: String): Qualifier = {
|
||||||
|
createQualifier(cls, cls, sch, sch)
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def createQualifier(classId: String, className: String, schemeId: String, schemeName: String): Qualifier = {
|
||||||
|
val q: Qualifier = new Qualifier
|
||||||
|
q.setClassid(classId)
|
||||||
|
q.setClassname(className)
|
||||||
|
q.setSchemeid(schemeId)
|
||||||
|
q.setSchemename(schemeName)
|
||||||
|
q
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def asField[T](value: T): Field[T] = {
|
||||||
|
val tmp = new Field[T]
|
||||||
|
tmp.setValue(value)
|
||||||
|
tmp
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
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 = {
|
||||||
|
val sp = new StructuredProperty
|
||||||
|
sp.setQualifier(createQualifier(classId, schemeId))
|
||||||
|
sp.setValue(value)
|
||||||
|
sp
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = {
|
||||||
|
val sp = new StructuredProperty
|
||||||
|
sp.setQualifier(createQualifier(classId, schemeId))
|
||||||
|
sp.setValue(value)
|
||||||
|
sp.setDataInfo(dataInfo)
|
||||||
|
sp
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -57,9 +57,8 @@
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>eu.dnetlib</groupId>
|
<groupId>eu.dnetlib.dhp</groupId>
|
||||||
<artifactId>dnet-openaire-broker-common</artifactId>
|
<artifactId>dnet-openaire-broker-common</artifactId>
|
||||||
<version>[3.0.4,4.0.0)</version>
|
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
|
@ -11,6 +11,8 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.commons.lang3.time.DateUtils;
|
import org.apache.commons.lang3.time.DateUtils;
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
||||||
|
|
||||||
public class EventFactory {
|
public class EventFactory {
|
||||||
|
@ -32,7 +34,10 @@ public class EventFactory {
|
||||||
final MappedFields map = createMapFromResult(updateInfo);
|
final MappedFields map = createMapFromResult(updateInfo);
|
||||||
|
|
||||||
final String eventId = calculateEventId(
|
final String eventId = calculateEventId(
|
||||||
updateInfo.getTopicPath(), updateInfo.getTarget().getOpenaireId(), updateInfo.getHighlightValueAsString());
|
updateInfo.getTopicPath(), updateInfo.getTargetDs().getOpenaireId(), updateInfo
|
||||||
|
.getTarget()
|
||||||
|
.getOpenaireId(),
|
||||||
|
updateInfo.getHighlightValueAsString());
|
||||||
|
|
||||||
res.setEventId(eventId);
|
res.setEventId(eventId);
|
||||||
res.setProducerId(PRODUCER_ID);
|
res.setProducerId(PRODUCER_ID);
|
||||||
|
@ -52,9 +57,11 @@ public class EventFactory {
|
||||||
final OaBrokerMainEntity source = updateInfo.getSource();
|
final OaBrokerMainEntity source = updateInfo.getSource();
|
||||||
final OaBrokerMainEntity target = updateInfo.getTarget();
|
final OaBrokerMainEntity target = updateInfo.getTarget();
|
||||||
|
|
||||||
map.setTargetDatasourceId(target.getCollectedFromId());
|
final OaBrokerRelatedDatasource targetDs = updateInfo.getTargetDs();
|
||||||
map.setTargetDatasourceName(target.getCollectedFromName());
|
|
||||||
map.setTargetDatasourceType(target.getCollectedFromType());
|
map.setTargetDatasourceId(targetDs.getOpenaireId());
|
||||||
|
map.setTargetDatasourceName(targetDs.getName());
|
||||||
|
map.setTargetDatasourceType(targetDs.getType());
|
||||||
|
|
||||||
map.setTargetResultId(target.getOpenaireId());
|
map.setTargetResultId(target.getOpenaireId());
|
||||||
|
|
||||||
|
@ -73,19 +80,29 @@ public class EventFactory {
|
||||||
|
|
||||||
// PROVENANCE INFO
|
// PROVENANCE INFO
|
||||||
map.setTrust(updateInfo.getTrust());
|
map.setTrust(updateInfo.getTrust());
|
||||||
map.setProvenanceDatasourceId(source.getCollectedFromId());
|
|
||||||
map.setProvenanceDatasourceName(source.getCollectedFromName());
|
|
||||||
map.setProvenanceDatasourceType(source.getCollectedFromType());
|
|
||||||
map.setProvenanceResultId(source.getOpenaireId());
|
map.setProvenanceResultId(source.getOpenaireId());
|
||||||
|
|
||||||
|
source
|
||||||
|
.getDatasources()
|
||||||
|
.stream()
|
||||||
|
.filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL))
|
||||||
|
.findFirst()
|
||||||
|
.ifPresent(ds -> {
|
||||||
|
map.setProvenanceDatasourceId(ds.getOpenaireId());
|
||||||
|
map.setProvenanceDatasourceName(ds.getName());
|
||||||
|
map.setProvenanceDatasourceType(ds.getType());
|
||||||
|
});
|
||||||
|
|
||||||
return map;
|
return map;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String calculateEventId(final String topic, final String publicationId, final String value) {
|
private static String calculateEventId(final String topic, final String dsId, final String publicationId,
|
||||||
|
final String value) {
|
||||||
return "event-"
|
return "event-"
|
||||||
+ DigestUtils.md5Hex(topic).substring(0, 6) + "-"
|
+ DigestUtils.md5Hex(topic).substring(0, 4) + "-"
|
||||||
+ DigestUtils.md5Hex(publicationId).substring(0, 8) + "-"
|
+ DigestUtils.md5Hex(dsId).substring(0, 4) + "-"
|
||||||
+ DigestUtils.md5Hex(value).substring(0, 8);
|
+ DigestUtils.md5Hex(publicationId).substring(0, 7) + "-"
|
||||||
|
+ DigestUtils.md5Hex(value).substring(0, 5);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static long calculateExpiryDate(final long now) {
|
private static long calculateExpiryDate(final long now) {
|
||||||
|
|
|
@ -0,0 +1,112 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.commons.lang.StringUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.apache.spark.sql.TypedColumn;
|
||||||
|
import org.apache.spark.sql.expressions.Aggregator;
|
||||||
|
import org.apache.spark.util.LongAccumulator;
|
||||||
|
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;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
|
public class CheckDuplictedIdsJob {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(CheckDuplictedIdsJob.class);
|
||||||
|
|
||||||
|
public static void main(final String[] args) throws Exception {
|
||||||
|
|
||||||
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
CheckDuplictedIdsJob.class
|
||||||
|
.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
final String eventsPath = parser.get("workingPath") + "/events";
|
||||||
|
log.info("eventsPath: {}", eventsPath);
|
||||||
|
|
||||||
|
final String countPath = parser.get("workingPath") + "/counts";
|
||||||
|
log.info("countPath: {}", countPath);
|
||||||
|
|
||||||
|
final SparkSession spark = SparkSession.builder().config(conf).getOrCreate();
|
||||||
|
|
||||||
|
final LongAccumulator total = spark.sparkContext().longAccumulator("invaild_event_id");
|
||||||
|
|
||||||
|
final TypedColumn<Tuple2<String, Long>, Tuple2<String, Long>> agg = new CountAggregator().toColumn();
|
||||||
|
|
||||||
|
ClusterUtils
|
||||||
|
.readPath(spark, eventsPath, Event.class)
|
||||||
|
.map(e -> new Tuple2<>(e.getEventId(), 1l), Encoders.tuple(Encoders.STRING(), Encoders.LONG()))
|
||||||
|
.groupByKey(t -> t._1, Encoders.STRING())
|
||||||
|
.agg(agg)
|
||||||
|
.map(t -> t._2, Encoders.tuple(Encoders.STRING(), Encoders.LONG()))
|
||||||
|
.filter(t -> t._2 > 1)
|
||||||
|
.map(o -> ClusterUtils.incrementAccumulator(o, total), Encoders.tuple(Encoders.STRING(), Encoders.LONG()))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.json(countPath);
|
||||||
|
;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static String eventAsJsonString(final Event f) throws JsonProcessingException {
|
||||||
|
return new ObjectMapper().writeValueAsString(f);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
class CountAggregator extends Aggregator<Tuple2<String, Long>, Tuple2<String, Long>, Tuple2<String, Long>> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = 1395935985734672538L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<Tuple2<String, Long>> bufferEncoder() {
|
||||||
|
return Encoders.tuple(Encoders.STRING(), Encoders.LONG());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Tuple2<String, Long> finish(final Tuple2<String, Long> arg0) {
|
||||||
|
return arg0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Tuple2<String, Long> merge(final Tuple2<String, Long> arg0, final Tuple2<String, Long> arg1) {
|
||||||
|
final String s = StringUtils.defaultIfBlank(arg0._1, arg1._1);
|
||||||
|
return new Tuple2<>(s, arg0._2 + arg1._2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<Tuple2<String, Long>> outputEncoder() {
|
||||||
|
return Encoders.tuple(Encoders.STRING(), Encoders.LONG());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Tuple2<String, Long> reduce(final Tuple2<String, Long> arg0, final Tuple2<String, Long> arg1) {
|
||||||
|
final String s = StringUtils.defaultIfBlank(arg0._1, arg1._1);
|
||||||
|
return new Tuple2<>(s, arg0._2 + arg1._2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Tuple2<String, Long> zero() {
|
||||||
|
return new Tuple2<>(null, 0l);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -3,8 +3,6 @@ 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.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -20,8 +18,6 @@ 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.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.matchers.UpdateMatcher;
|
||||||
|
@ -29,9 +25,6 @@ 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;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup;
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
|
|
||||||
public class GenerateEventsJob {
|
public class GenerateEventsJob {
|
||||||
|
|
||||||
|
@ -54,30 +47,20 @@ public class GenerateEventsJob {
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
log.info("workingPath: {}", workingPath);
|
log.info("workingPath: {}", workingPath);
|
||||||
|
|
||||||
final String isLookupUrl = parser.get("isLookupUrl");
|
|
||||||
log.info("isLookupUrl: {}", isLookupUrl);
|
|
||||||
|
|
||||||
final String dedupConfigProfileId = parser.get("dedupConfProfile");
|
|
||||||
log.info("dedupConfigProfileId: {}", dedupConfigProfileId);
|
|
||||||
|
|
||||||
final String eventsPath = workingPath + "/events";
|
final String eventsPath = workingPath + "/events";
|
||||||
log.info("eventsPath: {}", eventsPath);
|
log.info("eventsPath: {}", eventsPath);
|
||||||
|
|
||||||
final Set<String> dsIdWhitelist = parseParamAsList(parser, "datasourceIdWhitelist");
|
final Set<String> dsIdWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceIdWhitelist");
|
||||||
log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ","));
|
log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ","));
|
||||||
|
|
||||||
final Set<String> dsTypeWhitelist = parseParamAsList(parser, "datasourceTypeWhitelist");
|
final Set<String> dsTypeWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceTypeWhitelist");
|
||||||
log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ","));
|
log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ","));
|
||||||
|
|
||||||
final Set<String> dsIdBlacklist = parseParamAsList(parser, "datasourceIdBlacklist");
|
final Set<String> dsIdBlacklist = ClusterUtils.parseParamAsList(parser, "datasourceIdBlacklist");
|
||||||
log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ","));
|
log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ","));
|
||||||
|
|
||||||
final SparkConf conf = new SparkConf();
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
// TODO UNCOMMENT
|
|
||||||
// final DedupConfig dedupConfig = loadDedupConfig(isLookupUrl, dedupConfigProfileId);
|
|
||||||
final DedupConfig dedupConfig = null;
|
|
||||||
|
|
||||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||||
|
|
||||||
ClusterUtils.removeDir(spark, eventsPath);
|
ClusterUtils.removeDir(spark, eventsPath);
|
||||||
|
@ -92,7 +75,7 @@ public class GenerateEventsJob {
|
||||||
final Dataset<Event> dataset = groups
|
final Dataset<Event> dataset = groups
|
||||||
.map(
|
.map(
|
||||||
g -> EventFinder
|
g -> EventFinder
|
||||||
.generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, dedupConfig, accumulators),
|
.generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, accumulators),
|
||||||
Encoders
|
Encoders
|
||||||
.bean(EventGroup.class))
|
.bean(EventGroup.class))
|
||||||
.flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class));
|
.flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class));
|
||||||
|
@ -103,22 +86,6 @@ public class GenerateEventsJob {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
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) {
|
public static Map<String, LongAccumulator> prepareAccumulators(final SparkContext sc) {
|
||||||
|
|
||||||
return EventFinder
|
return EventFinder
|
||||||
|
@ -130,23 +97,4 @@ public class GenerateEventsJob {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception {
|
|
||||||
|
|
||||||
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
|
||||||
|
|
||||||
final String conf = isLookUpService
|
|
||||||
.getResourceProfileByQuery(
|
|
||||||
String
|
|
||||||
.format(
|
|
||||||
"for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()",
|
|
||||||
profId));
|
|
||||||
|
|
||||||
final DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class);
|
|
||||||
dedupConfig.getPace().initModel();
|
|
||||||
dedupConfig.getPace().initTranslationMap();
|
|
||||||
// dedupConfig.getWf().setConfigurationId("???");
|
|
||||||
|
|
||||||
return dedupConfig;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,63 @@
|
||||||
|
|
||||||
|
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.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.broker.model.Event;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.ClusterUtils;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.stats.DatasourceStats;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.stats.StatsAggregator;
|
||||||
|
|
||||||
|
public class GenerateStatsJob {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(GenerateStatsJob.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/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 SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
final String eventsPath = parser.get("workingPath") + "/events";
|
||||||
|
log.info("eventsPath: {}", eventsPath);
|
||||||
|
|
||||||
|
final String statsPath = parser.get("workingPath") + "/stats";
|
||||||
|
log.info("stats: {}", statsPath);
|
||||||
|
|
||||||
|
final TypedColumn<Event, DatasourceStats> aggr = new StatsAggregator().toColumn();
|
||||||
|
|
||||||
|
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||||
|
|
||||||
|
final Dataset<DatasourceStats> stats = ClusterUtils
|
||||||
|
.readPath(spark, eventsPath, Event.class)
|
||||||
|
.groupByKey(e -> e.getMap().getTargetDatasourceId(), Encoders.STRING())
|
||||||
|
.agg(aggr)
|
||||||
|
.map(t -> t._2, Encoders.bean(DatasourceStats.class));
|
||||||
|
|
||||||
|
ClusterUtils.save(stats, statsPath, DatasourceStats.class, null);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -48,7 +48,7 @@ public class IndexOnESJob {
|
||||||
|
|
||||||
final JavaRDD<String> inputRdd = ClusterUtils
|
final JavaRDD<String> inputRdd = ClusterUtils
|
||||||
.readPath(spark, eventsPath, Event.class)
|
.readPath(spark, eventsPath, Event.class)
|
||||||
.limit(10000) // TODO REMOVE
|
// .limit(10000) // TODO REMOVE
|
||||||
.map(IndexOnESJob::eventAsJsonString, Encoders.STRING())
|
.map(IndexOnESJob::eventAsJsonString, Encoders.STRING())
|
||||||
.javaRDD();
|
.javaRDD();
|
||||||
|
|
||||||
|
|
|
@ -17,8 +17,8 @@ import org.slf4j.LoggerFactory;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
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.aggregators.withRels.AddDatasourceTypeAggregator;
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasource;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo;
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasourceAggregator;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class JoinStep0Job {
|
public class JoinStep0Job {
|
||||||
|
@ -45,33 +45,33 @@ public class JoinStep0Job {
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
log.info("workingPath: {}", workingPath);
|
log.info("workingPath: {}", workingPath);
|
||||||
|
|
||||||
final String outputPath = workingPath + "/joinedEntities_step0";
|
final String joinedEntitiesPath = workingPath + "/joinedEntities_step0";
|
||||||
log.info("outputPath: {}", outputPath);
|
log.info("joinedEntitiesPath: {}", joinedEntitiesPath);
|
||||||
|
|
||||||
final SparkConf conf = new SparkConf();
|
final SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
|
||||||
|
|
||||||
ClusterUtils.removeDir(spark, outputPath);
|
ClusterUtils.removeDir(spark, joinedEntitiesPath);
|
||||||
|
|
||||||
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
|
final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities");
|
||||||
|
|
||||||
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
|
final Dataset<OaBrokerMainEntity> sources = ClusterUtils
|
||||||
.readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class);
|
.readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class);
|
||||||
|
|
||||||
final Dataset<SimpleDatasourceInfo> datasources = ClusterUtils
|
final Dataset<RelatedDatasource> typedRels = ClusterUtils
|
||||||
.readPath(spark, workingPath + "/datasources", SimpleDatasourceInfo.class);
|
.readPath(spark, workingPath + "/relatedDatasources", RelatedDatasource.class);
|
||||||
|
|
||||||
final TypedColumn<Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo>, OaBrokerMainEntity> aggr = new AddDatasourceTypeAggregator()
|
final TypedColumn<Tuple2<OaBrokerMainEntity, RelatedDatasource>, OaBrokerMainEntity> aggr = new RelatedDatasourceAggregator()
|
||||||
.toColumn();
|
.toColumn();
|
||||||
|
|
||||||
final Dataset<OaBrokerMainEntity> dataset = sources
|
final Dataset<OaBrokerMainEntity> dataset = sources
|
||||||
.joinWith(datasources, sources.col("collectedFromId").equalTo(datasources.col("id")), "inner")
|
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
|
||||||
.groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING())
|
.groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING())
|
||||||
.agg(aggr)
|
.agg(aggr)
|
||||||
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
|
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
|
||||||
|
|
||||||
ClusterUtils.save(dataset, outputPath, OaBrokerMainEntity.class, total);
|
ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total);
|
||||||
|
|
||||||
});
|
});
|
||||||
|
|
||||||
|
|
|
@ -7,7 +7,6 @@ import java.util.Optional;
|
||||||
|
|
||||||
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.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.TypedColumn;
|
import org.apache.spark.sql.TypedColumn;
|
||||||
|
@ -65,9 +64,7 @@ public class JoinStep2Job {
|
||||||
|
|
||||||
final Dataset<OaBrokerMainEntity> dataset = sources
|
final Dataset<OaBrokerMainEntity> dataset = sources
|
||||||
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
|
.joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer")
|
||||||
.groupByKey(
|
.groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING())
|
||||||
(MapFunction<Tuple2<OaBrokerMainEntity, RelatedSoftware>, String>) t -> t._1.getOpenaireId(),
|
|
||||||
Encoders.STRING())
|
|
||||||
.agg(aggr)
|
.agg(aggr)
|
||||||
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
|
.map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class));
|
||||||
|
|
||||||
|
|
|
@ -9,14 +9,23 @@ 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.SparkSession;
|
||||||
import org.apache.spark.util.LongAccumulator;
|
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.OaBrokerRelatedDatasource;
|
||||||
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.aggregators.withRels.SimpleDatasourceInfo;
|
import eu.dnetlib.dhp.broker.oa.util.ConversionUtils;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.DatasourceRelationsAccumulator;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasource;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Software;
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
public class PrepareRelatedDatasourcesJob {
|
public class PrepareRelatedDatasourcesJob {
|
||||||
|
|
||||||
|
@ -42,7 +51,7 @@ public class PrepareRelatedDatasourcesJob {
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
log.info("workingPath: {}", workingPath);
|
log.info("workingPath: {}", workingPath);
|
||||||
|
|
||||||
final String relsPath = workingPath + "/datasources";
|
final String relsPath = workingPath + "/relatedDatasources";
|
||||||
log.info("relsPath: {}", relsPath);
|
log.info("relsPath: {}", relsPath);
|
||||||
|
|
||||||
final SparkConf conf = new SparkConf();
|
final SparkConf conf = new SparkConf();
|
||||||
|
@ -53,16 +62,46 @@ public class PrepareRelatedDatasourcesJob {
|
||||||
|
|
||||||
final LongAccumulator total = spark.sparkContext().longAccumulator("total_datasources");
|
final LongAccumulator total = spark.sparkContext().longAccumulator("total_datasources");
|
||||||
|
|
||||||
final Dataset<SimpleDatasourceInfo> dataset = ClusterUtils
|
final Dataset<Tuple3<String, String, String>> rels = prepareResultTuples(
|
||||||
.readPath(spark, graphPath + "/datasource", Datasource.class)
|
spark, graphPath, Publication.class)
|
||||||
.map(
|
.union(prepareResultTuples(spark, graphPath, eu.dnetlib.dhp.schema.oaf.Dataset.class))
|
||||||
ds -> new SimpleDatasourceInfo(ds.getId(), ds.getDatasourcetype().getClassid()),
|
.union(prepareResultTuples(spark, graphPath, Software.class))
|
||||||
Encoders.bean(SimpleDatasourceInfo.class));
|
.union(prepareResultTuples(spark, graphPath, OtherResearchProduct.class));
|
||||||
|
|
||||||
ClusterUtils.save(dataset, relsPath, SimpleDatasourceInfo.class, total);
|
final Dataset<OaBrokerRelatedDatasource> datasources = ClusterUtils
|
||||||
|
.readPath(spark, graphPath + "/datasource", Datasource.class)
|
||||||
|
.map(ConversionUtils::oafDatasourceToBrokerDatasource, Encoders.bean(OaBrokerRelatedDatasource.class));
|
||||||
|
|
||||||
|
final Dataset<RelatedDatasource> dataset = rels
|
||||||
|
.joinWith(datasources, datasources.col("openaireId").equalTo(rels.col("_2")), "inner")
|
||||||
|
.map(t -> {
|
||||||
|
final RelatedDatasource r = new RelatedDatasource();
|
||||||
|
r.setSource(t._1._1());
|
||||||
|
r.setRelDatasource(t._2);
|
||||||
|
r.getRelDatasource().setRelType(t._1._3());
|
||||||
|
return r;
|
||||||
|
}, Encoders.bean(RelatedDatasource.class));
|
||||||
|
|
||||||
|
ClusterUtils.save(dataset, relsPath, RelatedDatasource.class, total);
|
||||||
|
|
||||||
});
|
});
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static final Dataset<Tuple3<String, String, String>> prepareResultTuples(final SparkSession spark,
|
||||||
|
final String graphPath,
|
||||||
|
final Class<? extends Result> sourceClass) {
|
||||||
|
|
||||||
|
return ClusterUtils
|
||||||
|
.readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), sourceClass)
|
||||||
|
.filter(r -> !ClusterUtils.isDedupRoot(r.getId()))
|
||||||
|
.filter(r -> r.getDataInfo().getDeletedbyinference())
|
||||||
|
.map(
|
||||||
|
r -> DatasourceRelationsAccumulator.calculateTuples(r),
|
||||||
|
Encoders.bean(DatasourceRelationsAccumulator.class))
|
||||||
|
.flatMap(
|
||||||
|
acc -> acc.getRels().iterator(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.STRING()));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,9 +15,9 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.spark.util.LongAccumulator;
|
import org.apache.spark.util.LongAccumulator;
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
import eu.dnetlib.dhp.broker.model.Topic;
|
import eu.dnetlib.dhp.broker.model.Topic;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
|
|
||||||
public abstract class UpdateMatcher<T> {
|
public abstract class UpdateMatcher<T> {
|
||||||
|
|
||||||
|
@ -35,21 +35,21 @@ public abstract class UpdateMatcher<T> {
|
||||||
this.highlightToStringFunction = highlightToStringFunction;
|
this.highlightToStringFunction = highlightToStringFunction;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final OaBrokerMainEntity res,
|
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final OaBrokerMainEntity target,
|
||||||
|
final OaBrokerRelatedDatasource targetDs,
|
||||||
final Collection<OaBrokerMainEntity> others,
|
final Collection<OaBrokerMainEntity> others,
|
||||||
final DedupConfig dedupConfig,
|
|
||||||
final Map<String, LongAccumulator> accumulators) {
|
final Map<String, LongAccumulator> accumulators) {
|
||||||
|
|
||||||
final Map<String, UpdateInfo<T>> infoMap = new HashMap<>();
|
final Map<String, UpdateInfo<T>> infoMap = new HashMap<>();
|
||||||
|
|
||||||
for (final OaBrokerMainEntity source : others) {
|
for (final OaBrokerMainEntity source : others) {
|
||||||
if (source != res) {
|
if (source != target) {
|
||||||
for (final T hl : findDifferences(source, res)) {
|
for (final T hl : findDifferences(source, target)) {
|
||||||
final Topic topic = getTopicFunction().apply(hl);
|
final Topic topic = getTopicFunction().apply(hl);
|
||||||
if (topic != null) {
|
if (topic != null) {
|
||||||
final UpdateInfo<T> info = new UpdateInfo<>(topic, hl, source, res,
|
final UpdateInfo<T> info = new UpdateInfo<>(topic, hl, source, target, targetDs,
|
||||||
getCompileHighlightFunction(),
|
getCompileHighlightFunction(),
|
||||||
getHighlightToStringFunction(), dedupConfig);
|
getHighlightToStringFunction());
|
||||||
|
|
||||||
final String s = DigestUtils.md5Hex(info.getHighlightValueAsString());
|
final String s = DigestUtils.md5Hex(info.getHighlightValueAsString());
|
||||||
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {
|
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {
|
||||||
|
|
|
@ -25,7 +25,9 @@ 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<>(); }
|
if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
final Set<String> existingProjects = target
|
final Set<String> existingProjects = target
|
||||||
.getProjects()
|
.getProjects()
|
||||||
|
|
|
@ -14,6 +14,10 @@ public class BrokerConstants {
|
||||||
public static final String OPEN_ACCESS = "OPEN";
|
public static final String OPEN_ACCESS = "OPEN";
|
||||||
public static final String IS_MERGED_IN_CLASS = "isMergedIn";
|
public static final String IS_MERGED_IN_CLASS = "isMergedIn";
|
||||||
|
|
||||||
|
public static final String COLLECTED_FROM_REL = "collectedFrom";
|
||||||
|
|
||||||
|
public static final String HOSTED_BY_REL = "hostedBy";
|
||||||
|
|
||||||
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;
|
||||||
|
|
||||||
|
|
|
@ -1,6 +1,11 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.broker.oa.util;
|
package eu.dnetlib.dhp.broker.oa.util;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
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;
|
||||||
|
@ -10,6 +15,7 @@ import org.apache.spark.util.LongAccumulator;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
|
||||||
public class ClusterUtils {
|
public class ClusterUtils {
|
||||||
|
@ -53,7 +59,9 @@ public class ClusterUtils {
|
||||||
return o;
|
return o;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T> void save(final Dataset<T> dataset, final String path, final Class<T> clazz,
|
public static <T> void save(final Dataset<T> dataset,
|
||||||
|
final String path,
|
||||||
|
final Class<T> clazz,
|
||||||
final LongAccumulator acc) {
|
final LongAccumulator acc) {
|
||||||
dataset
|
dataset
|
||||||
.map(o -> ClusterUtils.incrementAccumulator(o, acc), Encoders.bean(clazz))
|
.map(o -> ClusterUtils.incrementAccumulator(o, acc), Encoders.bean(clazz))
|
||||||
|
@ -62,4 +70,20 @@ public class ClusterUtils {
|
||||||
.json(path);
|
.json(path);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public 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;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,11 +22,13 @@ import eu.dnetlib.broker.objects.OaBrokerJournal;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerProject;
|
import eu.dnetlib.broker.objects.OaBrokerProject;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerRelatedDataset;
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDataset;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerRelatedPublication;
|
import eu.dnetlib.broker.objects.OaBrokerRelatedPublication;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware;
|
import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerTypedValue;
|
import eu.dnetlib.broker.objects.OaBrokerTypedValue;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
import eu.dnetlib.dhp.schema.oaf.Author;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Datasource;
|
||||||
import eu.dnetlib.dhp.schema.oaf.ExternalReference;
|
import eu.dnetlib.dhp.schema.oaf.ExternalReference;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Field;
|
import eu.dnetlib.dhp.schema.oaf.Field;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Instance;
|
import eu.dnetlib.dhp.schema.oaf.Instance;
|
||||||
|
@ -44,7 +46,9 @@ public class ConversionUtils {
|
||||||
private static final Logger log = LoggerFactory.getLogger(ConversionUtils.class);
|
private static final Logger log = LoggerFactory.getLogger(ConversionUtils.class);
|
||||||
|
|
||||||
public static List<OaBrokerInstance> oafInstanceToBrokerInstances(final Instance i) {
|
public static List<OaBrokerInstance> oafInstanceToBrokerInstances(final Instance i) {
|
||||||
if (i == null) { return new ArrayList<>(); }
|
if (i == null) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
return mappedList(i.getUrl(), url -> {
|
return mappedList(i.getUrl(), url -> {
|
||||||
final OaBrokerInstance res = new OaBrokerInstance();
|
final OaBrokerInstance res = new OaBrokerInstance();
|
||||||
|
@ -65,7 +69,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final OaBrokerRelatedDataset oafDatasetToBrokerDataset(final Dataset d) {
|
public static final OaBrokerRelatedDataset oafDatasetToBrokerDataset(final Dataset d) {
|
||||||
if (d == null) { return null; }
|
if (d == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerRelatedDataset res = new OaBrokerRelatedDataset();
|
final OaBrokerRelatedDataset res = new OaBrokerRelatedDataset();
|
||||||
res.setOpenaireId(d.getId());
|
res.setOpenaireId(d.getId());
|
||||||
|
@ -78,7 +84,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static OaBrokerRelatedPublication oafPublicationToBrokerPublication(final Publication p) {
|
public static OaBrokerRelatedPublication oafPublicationToBrokerPublication(final Publication p) {
|
||||||
if (p == null) { return null; }
|
if (p == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerRelatedPublication res = new OaBrokerRelatedPublication();
|
final OaBrokerRelatedPublication res = new OaBrokerRelatedPublication();
|
||||||
res.setOpenaireId(p.getId());
|
res.setOpenaireId(p.getId());
|
||||||
|
@ -92,7 +100,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final OaBrokerMainEntity oafResultToBrokerResult(final Result result) {
|
public static final OaBrokerMainEntity oafResultToBrokerResult(final Result result) {
|
||||||
if (result == null) { return null; }
|
if (result == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerMainEntity res = new OaBrokerMainEntity();
|
final OaBrokerMainEntity res = new OaBrokerMainEntity();
|
||||||
|
|
||||||
|
@ -109,9 +119,8 @@ public class ConversionUtils {
|
||||||
res.setEmbargoenddate(fieldValue(result.getEmbargoenddate()));
|
res.setEmbargoenddate(fieldValue(result.getEmbargoenddate()));
|
||||||
res.setContributor(fieldList(result.getContributor()));
|
res.setContributor(fieldList(result.getContributor()));
|
||||||
res
|
res
|
||||||
.setJournal(result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null);
|
.setJournal(
|
||||||
res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey));
|
result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null);
|
||||||
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
|
res
|
||||||
|
@ -121,7 +130,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OaBrokerAuthor oafAuthorToBrokerAuthor(final Author author) {
|
private static OaBrokerAuthor oafAuthorToBrokerAuthor(final Author author) {
|
||||||
if (author == null) { return null; }
|
if (author == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final String pids = author.getPid() != null ? author
|
final String pids = author.getPid() != null ? author
|
||||||
.getPid()
|
.getPid()
|
||||||
|
@ -145,7 +156,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OaBrokerJournal oafJournalToBrokerJournal(final Journal journal) {
|
private static OaBrokerJournal oafJournalToBrokerJournal(final Journal journal) {
|
||||||
if (journal == null) { return null; }
|
if (journal == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerJournal res = new OaBrokerJournal();
|
final OaBrokerJournal res = new OaBrokerJournal();
|
||||||
res.setName(journal.getName());
|
res.setName(journal.getName());
|
||||||
|
@ -157,7 +170,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static OaBrokerExternalReference oafExtRefToBrokerExtRef(final ExternalReference ref) {
|
private static OaBrokerExternalReference oafExtRefToBrokerExtRef(final ExternalReference ref) {
|
||||||
if (ref == null) { return null; }
|
if (ref == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerExternalReference res = new OaBrokerExternalReference();
|
final OaBrokerExternalReference res = new OaBrokerExternalReference();
|
||||||
res.setRefidentifier(ref.getRefidentifier());
|
res.setRefidentifier(ref.getRefidentifier());
|
||||||
|
@ -168,7 +183,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final OaBrokerProject oafProjectToBrokerProject(final Project p) {
|
public static final OaBrokerProject oafProjectToBrokerProject(final Project p) {
|
||||||
if (p == null) { return null; }
|
if (p == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerProject res = new OaBrokerProject();
|
final OaBrokerProject res = new OaBrokerProject();
|
||||||
res.setOpenaireId(p.getId());
|
res.setOpenaireId(p.getId());
|
||||||
|
@ -192,7 +209,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final OaBrokerRelatedSoftware oafSoftwareToBrokerSoftware(final Software sw) {
|
public static final OaBrokerRelatedSoftware oafSoftwareToBrokerSoftware(final Software sw) {
|
||||||
if (sw == null) { return null; }
|
if (sw == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
final OaBrokerRelatedSoftware res = new OaBrokerRelatedSoftware();
|
final OaBrokerRelatedSoftware res = new OaBrokerRelatedSoftware();
|
||||||
res.setOpenaireId(sw.getId());
|
res.setOpenaireId(sw.getId());
|
||||||
|
@ -204,6 +223,18 @@ public class ConversionUtils {
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static final OaBrokerRelatedDatasource oafDatasourceToBrokerDatasource(final Datasource ds) {
|
||||||
|
if (ds == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final OaBrokerRelatedDatasource res = new OaBrokerRelatedDatasource();
|
||||||
|
res.setName(StringUtils.defaultIfBlank(fieldValue(ds.getOfficialname()), fieldValue(ds.getEnglishname())));
|
||||||
|
res.setOpenaireId(ds.getId());
|
||||||
|
res.setType(classId(ds.getDatasourcetype()));
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
private static String first(final List<String> list) {
|
private static String first(final List<String> list) {
|
||||||
return list != null && list.size() > 0 ? list.get(0) : null;
|
return list != null && list.size() > 0 ? list.get(0) : null;
|
||||||
}
|
}
|
||||||
|
@ -255,7 +286,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static List<OaBrokerTypedValue> structPropTypedList(final List<StructuredProperty> list) {
|
private static List<OaBrokerTypedValue> structPropTypedList(final List<StructuredProperty> list) {
|
||||||
if (list == null) { return new ArrayList<>(); }
|
if (list == null) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
return list
|
return list
|
||||||
.stream()
|
.stream()
|
||||||
|
@ -265,7 +298,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <F, T> List<T> mappedList(final List<F> list, final Function<F, T> func) {
|
private static <F, T> List<T> mappedList(final List<F> list, final Function<F, T> func) {
|
||||||
if (list == null) { return new ArrayList<>(); }
|
if (list == null) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
return list
|
return list
|
||||||
.stream()
|
.stream()
|
||||||
|
@ -276,7 +311,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <F, T> List<T> flatMappedList(final List<F> list, final Function<F, List<T>> func) {
|
private static <F, T> List<T> flatMappedList(final List<F> list, final Function<F, List<T>> func) {
|
||||||
if (list == null) { return new ArrayList<>(); }
|
if (list == null) {
|
||||||
|
return new ArrayList<>();
|
||||||
|
}
|
||||||
|
|
||||||
return list
|
return list
|
||||||
.stream()
|
.stream()
|
||||||
|
@ -288,7 +325,9 @@ public class ConversionUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <F, T> T mappedFirst(final List<F> list, final Function<F, T> func) {
|
private static <F, T> T mappedFirst(final List<F> list, final Function<F, T> func) {
|
||||||
if (list == null) { return null; }
|
if (list == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
return list
|
return list
|
||||||
.stream()
|
.stream()
|
||||||
|
|
|
@ -0,0 +1,68 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa.util;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Result;
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
|
public class DatasourceRelationsAccumulator implements Serializable {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = 3256220670651218957L;
|
||||||
|
|
||||||
|
private List<Tuple3<String, String, String>> rels = new ArrayList<>();
|
||||||
|
|
||||||
|
public List<Tuple3<String, String, String>> getRels() {
|
||||||
|
return rels;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRels(final List<Tuple3<String, String, String>> rels) {
|
||||||
|
this.rels = rels;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void addTuple(final Tuple3<String, String, String> t) {
|
||||||
|
rels.add(t);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final DatasourceRelationsAccumulator calculateTuples(final Result r) {
|
||||||
|
|
||||||
|
final Set<String> collectedFromSet = r
|
||||||
|
.getCollectedfrom()
|
||||||
|
.stream()
|
||||||
|
.map(kv -> kv.getKey())
|
||||||
|
.filter(StringUtils::isNotBlank)
|
||||||
|
.distinct()
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
final Set<String> hostedBySet = r
|
||||||
|
.getInstance()
|
||||||
|
.stream()
|
||||||
|
.map(i -> i.getHostedby())
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.filter(kv -> !StringUtils.equalsIgnoreCase(kv.getValue(), "Unknown Repository"))
|
||||||
|
.map(kv -> kv.getKey())
|
||||||
|
.filter(StringUtils::isNotBlank)
|
||||||
|
.distinct()
|
||||||
|
.filter(id -> !collectedFromSet.contains(id))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
final DatasourceRelationsAccumulator res = new DatasourceRelationsAccumulator();
|
||||||
|
collectedFromSet
|
||||||
|
.stream()
|
||||||
|
.map(s -> new Tuple3<>(r.getId(), s, BrokerConstants.COLLECTED_FROM_REL))
|
||||||
|
.forEach(res::addTuple);
|
||||||
|
hostedBySet.stream().map(s -> new Tuple3<>(r.getId(), s, BrokerConstants.HOSTED_BY_REL)).forEach(res::addTuple);
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -11,6 +11,7 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
import eu.dnetlib.dhp.broker.model.EventFactory;
|
import eu.dnetlib.dhp.broker.model.EventFactory;
|
||||||
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
|
import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher;
|
||||||
import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsReferencedBy;
|
import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsReferencedBy;
|
||||||
|
@ -37,7 +38,6 @@ import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreOpenAccess;
|
||||||
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMorePid;
|
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMorePid;
|
||||||
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreSubject;
|
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreSubject;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup;
|
import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
|
|
||||||
public class EventFinder {
|
public class EventFinder {
|
||||||
|
|
||||||
|
@ -70,22 +70,22 @@ public class EventFinder {
|
||||||
matchers.add(new EnrichMissingDatasetReferences());
|
matchers.add(new EnrichMissingDatasetReferences());
|
||||||
matchers.add(new EnrichMissingDatasetIsSupplementedTo());
|
matchers.add(new EnrichMissingDatasetIsSupplementedTo());
|
||||||
matchers.add(new EnrichMissingDatasetIsSupplementedBy());
|
matchers.add(new EnrichMissingDatasetIsSupplementedBy());
|
||||||
matchers.add(new EnrichMissingAbstract());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static EventGroup generateEvents(final ResultGroup results,
|
public static EventGroup generateEvents(final ResultGroup results,
|
||||||
final Set<String> dsIdWhitelist,
|
final Set<String> dsIdWhitelist,
|
||||||
final Set<String> dsIdBlacklist,
|
final Set<String> dsIdBlacklist,
|
||||||
final Set<String> dsTypeWhitelist,
|
final Set<String> dsTypeWhitelist,
|
||||||
final DedupConfig dedupConfig,
|
|
||||||
final Map<String, LongAccumulator> accumulators) {
|
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()) {
|
||||||
if (verifyTarget(target, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) {
|
for (final OaBrokerRelatedDatasource targetDs : target.getDatasources()) {
|
||||||
for (final UpdateMatcher<?> matcher : matchers) {
|
if (verifyTarget(targetDs, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) {
|
||||||
list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators));
|
for (final UpdateMatcher<?> matcher : matchers) {
|
||||||
|
list.addAll(matcher.searchUpdatesForRecord(target, targetDs, results.getData(), accumulators));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -93,17 +93,17 @@ public class EventFinder {
|
||||||
return asEventGroup(list);
|
return asEventGroup(list);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean verifyTarget(final OaBrokerMainEntity target,
|
private static boolean verifyTarget(final OaBrokerRelatedDatasource target,
|
||||||
final Set<String> dsIdWhitelist,
|
final Set<String> dsIdWhitelist,
|
||||||
final Set<String> dsIdBlacklist,
|
final Set<String> dsIdBlacklist,
|
||||||
final Set<String> dsTypeWhitelist) {
|
final Set<String> dsTypeWhitelist) {
|
||||||
|
|
||||||
if (dsIdWhitelist.contains(target.getCollectedFromId())) {
|
if (dsIdWhitelist.contains(target.getOpenaireId())) {
|
||||||
return true;
|
return true;
|
||||||
} else if (dsIdBlacklist.contains(target.getCollectedFromId())) {
|
} else if (dsIdBlacklist.contains(target.getOpenaireId())) {
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
return dsTypeWhitelist.contains(target.getCollectedFromType());
|
return dsTypeWhitelist.contains(target.getType());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,8 +1,62 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.broker.oa.util;
|
package eu.dnetlib.dhp.broker.oa.util;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
|
import eu.dnetlib.pace.tree.support.TreeProcessor;
|
||||||
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
|
|
||||||
public class TrustUtils {
|
public class TrustUtils {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(TrustUtils.class);
|
||||||
|
|
||||||
|
private static DedupConfig dedupConfig;
|
||||||
|
|
||||||
|
static {
|
||||||
|
final ObjectMapper mapper = new ObjectMapper();
|
||||||
|
try {
|
||||||
|
dedupConfig = mapper
|
||||||
|
.readValue(
|
||||||
|
DedupConfig.class.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json"),
|
||||||
|
DedupConfig.class);
|
||||||
|
} catch (final IOException e) {
|
||||||
|
log.error("Error loading dedupConfig, e");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static float calculateTrust(final OaBrokerMainEntity r1, final OaBrokerMainEntity r2) {
|
||||||
|
|
||||||
|
if (dedupConfig == null) {
|
||||||
|
return BrokerConstants.MIN_TRUST;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
final ObjectMapper objectMapper = new ObjectMapper();
|
||||||
|
final MapDocument doc1 = MapDocumentUtil
|
||||||
|
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1));
|
||||||
|
final MapDocument doc2 = MapDocumentUtil
|
||||||
|
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2));
|
||||||
|
|
||||||
|
final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2);
|
||||||
|
|
||||||
|
final double threshold = dedupConfig.getWf().getThreshold();
|
||||||
|
|
||||||
|
return TrustUtils.rescale(score, threshold);
|
||||||
|
} catch (final Exception e) {
|
||||||
|
log.error("Error computing score between results", e);
|
||||||
|
return BrokerConstants.MIN_TRUST;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static float rescale(final double score, final double threshold) {
|
public static float rescale(final double score, final double threshold) {
|
||||||
if (score >= BrokerConstants.MAX_TRUST) {
|
if (score >= BrokerConstants.MAX_TRUST) {
|
||||||
return BrokerConstants.MAX_TRUST;
|
return BrokerConstants.MAX_TRUST;
|
||||||
|
|
|
@ -4,20 +4,12 @@ package eu.dnetlib.dhp.broker.oa.util;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerEventPayload;
|
import eu.dnetlib.broker.objects.OaBrokerEventPayload;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerInstance;
|
import eu.dnetlib.broker.objects.OaBrokerInstance;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
import eu.dnetlib.broker.objects.OaBrokerProvenance;
|
import eu.dnetlib.broker.objects.OaBrokerProvenance;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
import eu.dnetlib.dhp.broker.model.Topic;
|
import eu.dnetlib.dhp.broker.model.Topic;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
|
||||||
import eu.dnetlib.pace.tree.support.TreeProcessor;
|
|
||||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
|
||||||
|
|
||||||
public final class UpdateInfo<T> {
|
public final class UpdateInfo<T> {
|
||||||
|
|
||||||
|
@ -29,26 +21,27 @@ public final class UpdateInfo<T> {
|
||||||
|
|
||||||
private final OaBrokerMainEntity target;
|
private final OaBrokerMainEntity target;
|
||||||
|
|
||||||
|
private final OaBrokerRelatedDatasource targetDs;
|
||||||
|
|
||||||
private final BiConsumer<OaBrokerMainEntity, T> compileHighlight;
|
private final BiConsumer<OaBrokerMainEntity, T> compileHighlight;
|
||||||
|
|
||||||
private final Function<T, String> highlightToString;
|
private final Function<T, String> highlightToString;
|
||||||
|
|
||||||
private final float trust;
|
private final float trust;
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(UpdateInfo.class);
|
|
||||||
|
|
||||||
public UpdateInfo(final Topic topic, final T highlightValue, final OaBrokerMainEntity source,
|
public UpdateInfo(final Topic topic, final T highlightValue, final OaBrokerMainEntity source,
|
||||||
final OaBrokerMainEntity target,
|
final OaBrokerMainEntity target,
|
||||||
|
final OaBrokerRelatedDatasource targetDs,
|
||||||
final BiConsumer<OaBrokerMainEntity, T> compileHighlight,
|
final BiConsumer<OaBrokerMainEntity, T> compileHighlight,
|
||||||
final Function<T, String> highlightToString,
|
final Function<T, String> highlightToString) {
|
||||||
final DedupConfig dedupConfig) {
|
|
||||||
this.topic = topic;
|
this.topic = topic;
|
||||||
this.highlightValue = highlightValue;
|
this.highlightValue = highlightValue;
|
||||||
this.source = source;
|
this.source = source;
|
||||||
this.target = target;
|
this.target = target;
|
||||||
|
this.targetDs = targetDs;
|
||||||
this.compileHighlight = compileHighlight;
|
this.compileHighlight = compileHighlight;
|
||||||
this.highlightToString = highlightToString;
|
this.highlightToString = highlightToString;
|
||||||
this.trust = calculateTrust(dedupConfig, source, target);
|
this.trust = TrustUtils.calculateTrust(source, target);
|
||||||
}
|
}
|
||||||
|
|
||||||
public T getHighlightValue() {
|
public T getHighlightValue() {
|
||||||
|
@ -63,29 +56,8 @@ public final class UpdateInfo<T> {
|
||||||
return target;
|
return target;
|
||||||
}
|
}
|
||||||
|
|
||||||
private float calculateTrust(final DedupConfig dedupConfig,
|
public OaBrokerRelatedDatasource getTargetDs() {
|
||||||
final OaBrokerMainEntity r1,
|
return targetDs;
|
||||||
final OaBrokerMainEntity r2) {
|
|
||||||
|
|
||||||
if (dedupConfig == null) {
|
|
||||||
return BrokerConstants.MIN_TRUST;
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
final ObjectMapper objectMapper = new ObjectMapper();
|
|
||||||
final MapDocument doc1 = MapDocumentUtil
|
|
||||||
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1));
|
|
||||||
final MapDocument doc2 = MapDocumentUtil
|
|
||||||
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2));
|
|
||||||
|
|
||||||
final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2);
|
|
||||||
final double threshold = dedupConfig.getWf().getThreshold();
|
|
||||||
|
|
||||||
return TrustUtils.rescale(score, threshold);
|
|
||||||
} catch (final Exception e) {
|
|
||||||
log.error("Error computing score between results", e);
|
|
||||||
return BrokerConstants.MIN_TRUST;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Topic getTopic() {
|
protected Topic getTopic() {
|
||||||
|
@ -112,8 +84,20 @@ public final class UpdateInfo<T> {
|
||||||
compileHighlight.accept(hl, getHighlightValue());
|
compileHighlight.accept(hl, getHighlightValue());
|
||||||
|
|
||||||
final String provId = getSource().getOpenaireId();
|
final String provId = getSource().getOpenaireId();
|
||||||
final String provRepo = getSource().getCollectedFromName();
|
final String provRepo = getSource()
|
||||||
final String provType = getSource().getCollectedFromType();
|
.getDatasources()
|
||||||
|
.stream()
|
||||||
|
.filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL))
|
||||||
|
.map(ds -> ds.getName())
|
||||||
|
.findFirst()
|
||||||
|
.orElse("");
|
||||||
|
final String provType = getSource()
|
||||||
|
.getDatasources()
|
||||||
|
.stream()
|
||||||
|
.filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL))
|
||||||
|
.map(ds -> ds.getType())
|
||||||
|
.findFirst()
|
||||||
|
.orElse("");
|
||||||
|
|
||||||
final String provUrl = getSource()
|
final String provUrl = getSource()
|
||||||
.getInstances()
|
.getInstances()
|
||||||
|
|
|
@ -0,0 +1,61 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa.util.aggregators.stats;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DatasourceStats implements Serializable {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = -282112564184047677L;
|
||||||
|
|
||||||
|
private String id;
|
||||||
|
private String name;
|
||||||
|
private String type;
|
||||||
|
private Map<String, Long> topics = new HashMap<>();
|
||||||
|
|
||||||
|
public String getId() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setId(final String id) {
|
||||||
|
this.id = id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getName() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setName(final String name) {
|
||||||
|
this.name = name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getType() {
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setType(final String type) {
|
||||||
|
this.type = type;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Map<String, Long> getTopics() {
|
||||||
|
return topics;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTopics(final Map<String, Long> topics) {
|
||||||
|
this.topics = topics;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void incrementTopic(final String topic, final long inc) {
|
||||||
|
if (topics.containsKey(topic)) {
|
||||||
|
topics.put(topic, topics.get(topic) + inc);
|
||||||
|
} else {
|
||||||
|
topics.put(topic, inc);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,59 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa.util.aggregators.stats;
|
||||||
|
|
||||||
|
import org.apache.commons.lang.StringUtils;
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.expressions.Aggregator;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.broker.model.Event;
|
||||||
|
|
||||||
|
public class StatsAggregator extends Aggregator<Event, DatasourceStats, DatasourceStats> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = 6652105853037330529L;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatasourceStats zero() {
|
||||||
|
return new DatasourceStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatasourceStats reduce(final DatasourceStats stats, final Event e) {
|
||||||
|
stats.setId(e.getMap().getTargetDatasourceId());
|
||||||
|
stats.setName(e.getMap().getTargetDatasourceName());
|
||||||
|
stats.setType(e.getMap().getTargetDatasourceType());
|
||||||
|
stats.incrementTopic(e.getTopic(), 1l);
|
||||||
|
return stats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatasourceStats merge(final DatasourceStats stats0, final DatasourceStats stats1) {
|
||||||
|
if (StringUtils.isBlank(stats0.getId())) {
|
||||||
|
stats0.setId(stats1.getId());
|
||||||
|
stats0.setName(stats1.getName());
|
||||||
|
stats0.setType(stats1.getType());
|
||||||
|
}
|
||||||
|
stats1.getTopics().entrySet().forEach(e -> stats0.incrementTopic(e.getKey(), e.getValue()));
|
||||||
|
return stats0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<DatasourceStats> bufferEncoder() {
|
||||||
|
return Encoders.bean(DatasourceStats.class);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DatasourceStats finish(final DatasourceStats stats) {
|
||||||
|
return stats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<DatasourceStats> outputEncoder() {
|
||||||
|
return Encoders.bean(DatasourceStats.class);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,42 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
|
|
||||||
|
public class RelatedDatasource implements Serializable {
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
private static final long serialVersionUID = 3015550240920424010L;
|
||||||
|
|
||||||
|
private String source;
|
||||||
|
private OaBrokerRelatedDatasource relDatasource;
|
||||||
|
|
||||||
|
public RelatedDatasource() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public RelatedDatasource(final String source, final OaBrokerRelatedDatasource relDatasource) {
|
||||||
|
this.source = source;
|
||||||
|
this.relDatasource = relDatasource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getSource() {
|
||||||
|
return source;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSource(final String source) {
|
||||||
|
this.source = source;
|
||||||
|
}
|
||||||
|
|
||||||
|
public OaBrokerRelatedDatasource getRelDatasource() {
|
||||||
|
return relDatasource;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRelDatasource(final OaBrokerRelatedDatasource relDatasource) {
|
||||||
|
this.relDatasource = relDatasource;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -7,15 +7,16 @@ import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.expressions.Aggregator;
|
import org.apache.spark.sql.expressions.Aggregator;
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.dhp.broker.oa.util.BrokerConstants;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class AddDatasourceTypeAggregator
|
public class RelatedDatasourceAggregator
|
||||||
extends Aggregator<Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo>, OaBrokerMainEntity, OaBrokerMainEntity> {
|
extends Aggregator<Tuple2<OaBrokerMainEntity, RelatedDatasource>, OaBrokerMainEntity, OaBrokerMainEntity> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
private static final long serialVersionUID = 8788588975496014728L;
|
private static final long serialVersionUID = -7212121913834713672L;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OaBrokerMainEntity zero() {
|
public OaBrokerMainEntity zero() {
|
||||||
|
@ -29,10 +30,10 @@ public class AddDatasourceTypeAggregator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g,
|
public OaBrokerMainEntity reduce(final OaBrokerMainEntity g,
|
||||||
final Tuple2<OaBrokerMainEntity, SimpleDatasourceInfo> t) {
|
final Tuple2<OaBrokerMainEntity, RelatedDatasource> t) {
|
||||||
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
|
final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1;
|
||||||
if (t._2 != null && StringUtils.isNotBlank(t._2.getType())) {
|
if (t._2 != null && res.getDatasources().size() < BrokerConstants.MAX_NUMBER_OF_RELS) {
|
||||||
res.setCollectedFromType(t._2.getType());
|
res.getDatasources().add(t._2.getRelDatasource());
|
||||||
}
|
}
|
||||||
return res;
|
return res;
|
||||||
|
|
||||||
|
@ -40,7 +41,15 @@ public class AddDatasourceTypeAggregator
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
|
public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) {
|
||||||
if (StringUtils.isNotBlank(g1.getOpenaireId()) && StringUtils.isNotBlank(g1.getCollectedFromType())) {
|
if (StringUtils.isNotBlank(g1.getOpenaireId())) {
|
||||||
|
final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getDatasources().size();
|
||||||
|
if (availables > 0) {
|
||||||
|
if (g2.getDatasources().size() <= availables) {
|
||||||
|
g1.getDatasources().addAll(g2.getDatasources());
|
||||||
|
} else {
|
||||||
|
g1.getDatasources().addAll(g2.getDatasources().subList(0, availables));
|
||||||
|
}
|
||||||
|
}
|
||||||
return g1;
|
return g1;
|
||||||
} else {
|
} else {
|
||||||
return g2;
|
return g2;
|
||||||
|
@ -56,4 +65,5 @@ public class AddDatasourceTypeAggregator
|
||||||
public Encoder<OaBrokerMainEntity> outputEncoder() {
|
public Encoder<OaBrokerMainEntity> outputEncoder() {
|
||||||
return Encoders.bean(OaBrokerMainEntity.class);
|
return Encoders.bean(OaBrokerMainEntity.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -1,40 +0,0 @@
|
||||||
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -0,0 +1,122 @@
|
||||||
|
{
|
||||||
|
"wf": {
|
||||||
|
|
||||||
|
},
|
||||||
|
"pace": {
|
||||||
|
"clustering": [
|
||||||
|
{
|
||||||
|
"name": "wordssuffixprefix",
|
||||||
|
"fields": [
|
||||||
|
"title"
|
||||||
|
],
|
||||||
|
"params": {
|
||||||
|
"max": "2",
|
||||||
|
"len": "3"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "lowercase",
|
||||||
|
"fields": [
|
||||||
|
"doi"
|
||||||
|
],
|
||||||
|
"params": {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"decisionTree": {
|
||||||
|
"start": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "doi",
|
||||||
|
"comparator": "exactMatch",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "false",
|
||||||
|
"params": {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 0.5,
|
||||||
|
"aggregation": "AVG",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "layer1",
|
||||||
|
"undefined": "layer1",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"layer1": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "title",
|
||||||
|
"comparator": "titleVersionMatch",
|
||||||
|
"weight": 0.9,
|
||||||
|
"countIfUndefined": "false",
|
||||||
|
"params": {
|
||||||
|
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"field": "authors",
|
||||||
|
"comparator": "sizeMatch",
|
||||||
|
"weight": 0.9,
|
||||||
|
"countIfUndefined": "false",
|
||||||
|
"params": {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 0.5,
|
||||||
|
"aggregation": "AVG",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "layer2",
|
||||||
|
"undefined": "layer2",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"layer2": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "title",
|
||||||
|
"comparator": "levensteinTitle",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 0.99,
|
||||||
|
"aggregation": "AVG",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "NO_MATCH",
|
||||||
|
"undefined": "NO_MATCH",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"model": [
|
||||||
|
{
|
||||||
|
"name": "doi",
|
||||||
|
"type": "String",
|
||||||
|
"path": "$.pids[?(@.type == 'doi')].value"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "title",
|
||||||
|
"type": "String",
|
||||||
|
"path": "$.titles",
|
||||||
|
"length": 250,
|
||||||
|
"size": 5
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "authors",
|
||||||
|
"type": "List",
|
||||||
|
"path": "$.creators[*].fullname",
|
||||||
|
"size": 200
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"blacklists": {
|
||||||
|
|
||||||
|
},
|
||||||
|
"synonyms": {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -8,14 +8,6 @@
|
||||||
<property>
|
<property>
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>the path where the the generated data will be stored</description>
|
<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>
|
||||||
<property>
|
<property>
|
||||||
<name>datasourceIdWhitelist</name>
|
<name>datasourceIdWhitelist</name>
|
||||||
|
@ -427,8 +419,6 @@
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
|
||||||
<arg>--dedupConfProfile</arg><arg>${dedupConfProfId}</arg>
|
|
||||||
<arg>--datasourceIdWhitelist</arg><arg>${datasourceIdWhitelist}</arg>
|
<arg>--datasourceIdWhitelist</arg><arg>${datasourceIdWhitelist}</arg>
|
||||||
<arg>--datasourceTypeWhitelist</arg><arg>${datasourceTypeWhitelist}</arg>
|
<arg>--datasourceTypeWhitelist</arg><arg>${datasourceTypeWhitelist}</arg>
|
||||||
<arg>--datasourceIdBlacklist</arg><arg>${datasourceIdBlacklist}</arg>
|
<arg>--datasourceIdBlacklist</arg><arg>${datasourceIdBlacklist}</arg>
|
||||||
|
@ -447,7 +437,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.dynamicAllocation.maxExecutors="2"
|
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
@ -458,6 +448,30 @@
|
||||||
<arg>--index</arg><arg>${esIndexName}</arg>
|
<arg>--index</arg><arg>${esIndexName}</arg>
|
||||||
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
<ok to="stats"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="stats">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>GenerateStatsJob</name>
|
||||||
|
<class>eu.dnetlib.dhp.broker.oa.GenerateStatsJob</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="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
|
@ -5,18 +5,6 @@
|
||||||
"paramDescription": "the path where the generated events will be stored",
|
"paramDescription": "the path where the generated events will be stored",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
{
|
|
||||||
"paramName": "lu",
|
|
||||||
"paramLongName": "isLookupUrl",
|
|
||||||
"paramDescription": "the address of the ISLookUpService",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"paramName": "d",
|
|
||||||
"paramLongName": "dedupConfProfile",
|
|
||||||
"paramDescription": "the id of a valid Dedup Configuration Profile",
|
|
||||||
"paramRequired": true
|
|
||||||
},
|
|
||||||
{
|
{
|
||||||
"paramName": "datasourceIdWhitelist",
|
"paramName": "datasourceIdWhitelist",
|
||||||
"paramLongName": "datasourceIdWhitelist",
|
"paramLongName": "datasourceIdWhitelist",
|
||||||
|
|
|
@ -9,15 +9,6 @@
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>the path where the the generated data will be stored</description>
|
<description>the path where the the generated data will be stored</description>
|
||||||
</property>
|
</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>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -73,14 +64,39 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
<start to="index_es"/>
|
<start to="generate_events"/>
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
|
<action name="generate_events">
|
||||||
<action name="index_es">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>GenerateEventsJob</name>
|
||||||
|
<class>eu.dnetlib.dhp.broker.oa.GenerateEventsJob</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>--workingPath</arg><arg>${workingPath}</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">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
|
@ -90,7 +106,7 @@
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.dynamicAllocation.maxExecutors="2"
|
--conf spark.dynamicAllocation.maxExecutors="8"
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
@ -101,6 +117,30 @@
|
||||||
<arg>--index</arg><arg>${esIndexName}</arg>
|
<arg>--index</arg><arg>${esIndexName}</arg>
|
||||||
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
<arg>--esHost</arg><arg>${esIndexHost}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
<ok to="stats"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="stats">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>GenerateStatsJob</name>
|
||||||
|
<class>eu.dnetlib.dhp.broker.oa.GenerateStatsJob</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="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
|
@ -8,15 +8,23 @@ import java.util.Collection;
|
||||||
|
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
|
||||||
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource;
|
||||||
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingPublicationDate;
|
import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingPublicationDate;
|
||||||
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
import eu.dnetlib.dhp.broker.oa.util.UpdateInfo;
|
||||||
|
|
||||||
|
@ExtendWith(MockitoExtension.class)
|
||||||
class UpdateMatcherTest {
|
class UpdateMatcherTest {
|
||||||
|
|
||||||
UpdateMatcher<String> matcher = new EnrichMissingPublicationDate();
|
UpdateMatcher<String> matcher = new EnrichMissingPublicationDate();
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private OaBrokerRelatedDatasource targetDs;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
void setUp() throws Exception {
|
void setUp() throws Exception {
|
||||||
}
|
}
|
||||||
|
@ -30,7 +38,7 @@ class UpdateMatcherTest {
|
||||||
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
|
final OaBrokerMainEntity p4 = new OaBrokerMainEntity();
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.isEmpty());
|
assertTrue(list.isEmpty());
|
||||||
}
|
}
|
||||||
|
@ -46,7 +54,7 @@ class UpdateMatcherTest {
|
||||||
res.setPublicationdate("2018");
|
res.setPublicationdate("2018");
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.isEmpty());
|
assertTrue(list.isEmpty());
|
||||||
}
|
}
|
||||||
|
@ -62,7 +70,7 @@ class UpdateMatcherTest {
|
||||||
p2.setPublicationdate("2018");
|
p2.setPublicationdate("2018");
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.size() == 1);
|
assertTrue(list.size() == 1);
|
||||||
}
|
}
|
||||||
|
@ -79,7 +87,7 @@ class UpdateMatcherTest {
|
||||||
p2.setPublicationdate("2018");
|
p2.setPublicationdate("2018");
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.isEmpty());
|
assertTrue(list.isEmpty());
|
||||||
}
|
}
|
||||||
|
@ -98,7 +106,7 @@ class UpdateMatcherTest {
|
||||||
p4.setPublicationdate("2018");
|
p4.setPublicationdate("2018");
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.isEmpty());
|
assertTrue(list.isEmpty());
|
||||||
}
|
}
|
||||||
|
@ -117,7 +125,7 @@ class UpdateMatcherTest {
|
||||||
p4.setPublicationdate("2018");
|
p4.setPublicationdate("2018");
|
||||||
|
|
||||||
final Collection<UpdateInfo<String>> list = matcher
|
final Collection<UpdateInfo<String>> list = matcher
|
||||||
.searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null);
|
.searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null);
|
||||||
|
|
||||||
assertTrue(list.size() == 1);
|
assertTrue(list.size() == 1);
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,6 +5,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerAuthor;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerMainEntity;
|
||||||
|
import eu.dnetlib.broker.objects.OaBrokerTypedValue;
|
||||||
|
|
||||||
public class TrustUtilsTest {
|
public class TrustUtilsTest {
|
||||||
|
|
||||||
private static final double THRESHOLD = 0.95;
|
private static final double THRESHOLD = 0.95;
|
||||||
|
@ -64,6 +68,23 @@ public class TrustUtilsTest {
|
||||||
verifyValue(2.00, BrokerConstants.MAX_TRUST);
|
verifyValue(2.00, BrokerConstants.MAX_TRUST);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void test() throws Exception {
|
||||||
|
final OaBrokerMainEntity r1 = new OaBrokerMainEntity();
|
||||||
|
r1.getTitles().add("D-NET Service Package: Data Import");
|
||||||
|
r1.getPids().add(new OaBrokerTypedValue("doi", "123"));
|
||||||
|
r1.getCreators().add(new OaBrokerAuthor("Michele Artini", null));
|
||||||
|
r1.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null));
|
||||||
|
|
||||||
|
final OaBrokerMainEntity r2 = new OaBrokerMainEntity();
|
||||||
|
r2.getTitles().add("D-NET Service Package: Data Import");
|
||||||
|
// r2.getPids().add(new OaBrokerTypedValue("doi", "123"));
|
||||||
|
r2.getCreators().add(new OaBrokerAuthor("Michele Artini", null));
|
||||||
|
// r2.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null));
|
||||||
|
|
||||||
|
System.out.println("TRUST: " + TrustUtils.calculateTrust(r1, r2));
|
||||||
|
}
|
||||||
|
|
||||||
private void verifyValue(final double originalScore, final float expectedTrust) {
|
private void verifyValue(final double originalScore, final float expectedTrust) {
|
||||||
final float trust = TrustUtils.rescale(originalScore, THRESHOLD);
|
final float trust = TrustUtils.rescale(originalScore, THRESHOLD);
|
||||||
System.out.println(trust);
|
System.out.println(trust);
|
||||||
|
|
|
@ -28,6 +28,8 @@ import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
|
||||||
abstract class AbstractSparkAction implements Serializable {
|
abstract class AbstractSparkAction implements Serializable {
|
||||||
|
|
||||||
|
protected static final int NUM_PARTITIONS = 1000;
|
||||||
|
|
||||||
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||||
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
|
||||||
|
|
|
@ -100,6 +100,11 @@ public class DedupUtility {
|
||||||
return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType);
|
return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static String createBlockStatsPath(
|
||||||
|
final String basePath, final String actionSetId, final String entityType) {
|
||||||
|
return String.format("%s/%s/%s_blockstats", basePath, actionSetId, entityType);
|
||||||
|
}
|
||||||
|
|
||||||
public static List<DedupConfig> getConfigurations(String isLookUpUrl, String orchestrator)
|
public static List<DedupConfig> getConfigurations(String isLookUpUrl, String orchestrator)
|
||||||
throws ISLookUpException, DocumentException {
|
throws ISLookUpException, DocumentException {
|
||||||
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookUpUrl);
|
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookUpUrl);
|
||||||
|
|
|
@ -52,6 +52,7 @@ public class Deduper implements Serializable {
|
||||||
.collect(Collectors.toList())
|
.collect(Collectors.toList())
|
||||||
.iterator())
|
.iterator())
|
||||||
.mapToPair(block -> new Tuple2<>(block.getKey(), block))
|
.mapToPair(block -> new Tuple2<>(block.getKey(), block))
|
||||||
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize));
|
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize))
|
||||||
|
.filter(b -> b._2().getDocuments().size() > 1);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,57 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.expressions.Aggregator;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
|
||||||
|
public class RelationAggregator extends Aggregator<Relation, Relation, Relation> {
|
||||||
|
|
||||||
|
private static Relation ZERO = new Relation();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Relation zero() {
|
||||||
|
return ZERO;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Relation reduce(Relation b, Relation a) {
|
||||||
|
return mergeRel(b, a);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Relation merge(Relation b, Relation a) {
|
||||||
|
return mergeRel(b, a);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Relation finish(Relation r) {
|
||||||
|
return r;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Relation mergeRel(Relation b, Relation a) {
|
||||||
|
if (Objects.equals(b, ZERO)) {
|
||||||
|
return a;
|
||||||
|
}
|
||||||
|
if (Objects.equals(a, ZERO)) {
|
||||||
|
return b;
|
||||||
|
}
|
||||||
|
|
||||||
|
b.mergeFrom(a);
|
||||||
|
return b;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<Relation> bufferEncoder() {
|
||||||
|
return Encoders.kryo(Relation.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Encoder<Relation> outputEncoder() {
|
||||||
|
return Encoders.kryo(Relation.class);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,126 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.dom4j.DocumentException;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.BlockStats;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
|
public class SparkBlockStats extends AbstractSparkAction {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(SparkBlockStats.class);
|
||||||
|
|
||||||
|
public SparkBlockStats(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
|
super(parser, spark);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkBlockStats.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
|
new SparkBlockStats(parser, getSparkSession(conf))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
|
}
|
||||||
|
|
||||||
|
public Long computeComparisons(Long blockSize, Long slidingWindowSize) {
|
||||||
|
|
||||||
|
if (slidingWindowSize >= blockSize)
|
||||||
|
return (slidingWindowSize * (slidingWindowSize - 1)) / 2;
|
||||||
|
else {
|
||||||
|
return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(ISLookUpService isLookUpService)
|
||||||
|
throws DocumentException, IOException, ISLookUpException {
|
||||||
|
|
||||||
|
// read oozie parameters
|
||||||
|
final String graphBasePath = parser.get("graphBasePath");
|
||||||
|
final String isLookUpUrl = parser.get("isLookUpUrl");
|
||||||
|
final String actionSetId = parser.get("actionSetId");
|
||||||
|
final String workingPath = parser.get("workingPath");
|
||||||
|
final int numPartitions = Optional
|
||||||
|
.ofNullable(parser.get("numPartitions"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(NUM_PARTITIONS);
|
||||||
|
|
||||||
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
|
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
||||||
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
|
||||||
|
// for each dedup configuration
|
||||||
|
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
|
||||||
|
|
||||||
|
final String subEntity = dedupConf.getWf().getSubEntityValue();
|
||||||
|
log.info("Creating blockstats for: '{}'", subEntity);
|
||||||
|
|
||||||
|
final String outputPath = DedupUtility.createBlockStatsPath(workingPath, actionSetId, subEntity);
|
||||||
|
removeOutputDir(spark, outputPath);
|
||||||
|
|
||||||
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
||||||
|
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||||
|
.repartition(numPartitions)
|
||||||
|
.mapToPair(
|
||||||
|
(PairFunction<String, String, MapDocument>) s -> {
|
||||||
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||||
|
return new Tuple2<>(d.getIdentifier(), d);
|
||||||
|
});
|
||||||
|
|
||||||
|
// create blocks for deduplication
|
||||||
|
JavaRDD<BlockStats> blockStats = Deduper
|
||||||
|
.createSortedBlocks(mapDocuments, dedupConf)
|
||||||
|
.repartition(numPartitions)
|
||||||
|
.map(b -> asBlockStats(dedupConf, b));
|
||||||
|
|
||||||
|
// save the blockstats in the workingdir
|
||||||
|
spark
|
||||||
|
.createDataset(blockStats.rdd(), Encoders.bean(BlockStats.class))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.save(outputPath);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private BlockStats asBlockStats(DedupConfig dedupConf, Tuple2<String, Block> b) {
|
||||||
|
return new BlockStats(
|
||||||
|
b._1(),
|
||||||
|
(long) b._2().getDocuments().size(),
|
||||||
|
computeComparisons(
|
||||||
|
(long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize()));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -5,11 +5,13 @@ import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
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.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
import org.apache.spark.graphx.Edge;
|
import org.apache.spark.graphx.Edge;
|
||||||
import org.apache.spark.rdd.RDD;
|
import org.apache.spark.rdd.RDD;
|
||||||
|
@ -75,7 +77,11 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
final String isLookUpUrl = parser.get("isLookUpUrl");
|
final String isLookUpUrl = parser.get("isLookUpUrl");
|
||||||
final String actionSetId = parser.get("actionSetId");
|
final String actionSetId = parser.get("actionSetId");
|
||||||
|
int cut = Optional
|
||||||
|
.ofNullable(parser.get("cutConnectedComponent"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(0);
|
||||||
|
log.info("connected component cut: '{}'", cut);
|
||||||
log.info("graphBasePath: '{}'", graphBasePath);
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
|
@ -100,8 +106,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
|
|
||||||
final RDD<Edge<String>> edgeRdd = spark
|
final RDD<Edge<String>> edgeRdd = spark
|
||||||
.read()
|
.read()
|
||||||
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
.textFile(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
|
||||||
.as(Encoders.bean(Relation.class))
|
.map(
|
||||||
|
(MapFunction<String, Relation>) r -> OBJECT_MAPPER.readValue(r, Relation.class),
|
||||||
|
Encoders.bean(Relation.class))
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
|
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
|
||||||
.rdd();
|
.rdd();
|
||||||
|
@ -109,7 +117,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
final Dataset<Relation> mergeRels = spark
|
final Dataset<Relation> mergeRels = spark
|
||||||
.createDataset(
|
.createDataset(
|
||||||
GraphProcessor
|
GraphProcessor
|
||||||
.findCCs(vertexes.rdd(), edgeRdd, maxIterations)
|
.findCCs(vertexes.rdd(), edgeRdd, maxIterations, cut)
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.filter(k -> k.getDocIds().size() > 1)
|
.filter(k -> k.getDocIds().size() > 1)
|
||||||
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
||||||
|
@ -117,6 +125,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
|
||||||
Encoders.bean(Relation.class));
|
Encoders.bean(Relation.class));
|
||||||
|
|
||||||
mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath);
|
mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath);
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -2,6 +2,7 @@
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
|
@ -48,13 +49,6 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
|
||||||
conf
|
|
||||||
.registerKryoClasses(
|
|
||||||
new Class[] {
|
|
||||||
MapDocument.class, FieldListImpl.class, FieldValueImpl.class, Block.class
|
|
||||||
});
|
|
||||||
|
|
||||||
new SparkCreateSimRels(parser, getSparkSession(conf))
|
new SparkCreateSimRels(parser, getSparkSession(conf))
|
||||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
}
|
}
|
||||||
|
@ -68,7 +62,12 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
final String isLookUpUrl = parser.get("isLookUpUrl");
|
final String isLookUpUrl = parser.get("isLookUpUrl");
|
||||||
final String actionSetId = parser.get("actionSetId");
|
final String actionSetId = parser.get("actionSetId");
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
|
final int numPartitions = Optional
|
||||||
|
.ofNullable(parser.get("numPartitions"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(NUM_PARTITIONS);
|
||||||
|
|
||||||
|
log.info("numPartitions: '{}'", numPartitions);
|
||||||
log.info("graphBasePath: '{}'", graphBasePath);
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
|
@ -88,6 +87,7 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
|
|
||||||
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
||||||
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||||
|
.repartition(numPartitions)
|
||||||
.mapToPair(
|
.mapToPair(
|
||||||
(PairFunction<String, String, MapDocument>) s -> {
|
(PairFunction<String, String, MapDocument>) s -> {
|
||||||
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||||
|
@ -95,19 +95,17 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
});
|
});
|
||||||
|
|
||||||
// create blocks for deduplication
|
// create blocks for deduplication
|
||||||
JavaPairRDD<String, Block> blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf);
|
JavaPairRDD<String, Block> blocks = Deduper
|
||||||
|
.createSortedBlocks(mapDocuments, dedupConf)
|
||||||
|
.repartition(numPartitions);
|
||||||
|
|
||||||
// create relations by comparing only elements in the same group
|
// create relations by comparing only elements in the same group
|
||||||
JavaRDD<Relation> relations = Deduper
|
Deduper
|
||||||
.computeRelations(sc, blocks, dedupConf)
|
.computeRelations(sc, blocks, dedupConf)
|
||||||
.map(t -> createSimRel(t._1(), t._2(), entity));
|
.map(t -> createSimRel(t._1(), t._2(), entity))
|
||||||
|
.repartition(numPartitions)
|
||||||
// save the simrel in the workingdir
|
.map(r -> OBJECT_MAPPER.writeValueAsString(r))
|
||||||
spark
|
.saveAsTextFile(outputPath);
|
||||||
.createDataset(relations.rdd(), Encoders.bean(Relation.class))
|
|
||||||
.write()
|
|
||||||
.mode(SaveMode.Append)
|
|
||||||
.save(outputPath);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -4,7 +4,9 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
import static org.apache.spark.sql.functions.col;
|
import static org.apache.spark.sql.functions.col;
|
||||||
|
|
||||||
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.FilterFunction;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.*;
|
import org.apache.spark.sql.*;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -95,7 +97,24 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
FieldType.TARGET,
|
FieldType.TARGET,
|
||||||
getDeletedFn());
|
getDeletedFn());
|
||||||
|
|
||||||
save(newRels.union(updated).union(mergeRels), outputRelationPath, SaveMode.Overwrite);
|
save(
|
||||||
|
distinctRelations(
|
||||||
|
newRels
|
||||||
|
.union(updated)
|
||||||
|
.union(mergeRels)
|
||||||
|
.map((MapFunction<Relation, Relation>) r -> r, Encoders.kryo(Relation.class))),
|
||||||
|
outputRelationPath, SaveMode.Overwrite);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Dataset<Relation> distinctRelations(Dataset<Relation> rels) {
|
||||||
|
return rels
|
||||||
|
.filter(getRelationFilterFunction())
|
||||||
|
.groupByKey(
|
||||||
|
(MapFunction<Relation, String>) r -> String
|
||||||
|
.join(r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()),
|
||||||
|
Encoders.STRING())
|
||||||
|
.agg(new RelationAggregator().toColumn())
|
||||||
|
.map((MapFunction<Tuple2<String, Relation>, Relation>) t -> t._2(), Encoders.bean(Relation.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Dataset<Relation> processDataset(
|
private static Dataset<Relation> processDataset(
|
||||||
|
@ -112,6 +131,14 @@ public class SparkPropagateRelation extends AbstractSparkAction {
|
||||||
.map(mapFn, Encoders.bean(Relation.class));
|
.map(mapFn, Encoders.bean(Relation.class));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private FilterFunction<Relation> getRelationFilterFunction() {
|
||||||
|
return (FilterFunction<Relation>) r -> StringUtils.isNotBlank(r.getSource()) ||
|
||||||
|
StringUtils.isNotBlank(r.getTarget()) ||
|
||||||
|
StringUtils.isNotBlank(r.getRelClass()) ||
|
||||||
|
StringUtils.isNotBlank(r.getSubRelType()) ||
|
||||||
|
StringUtils.isNotBlank(r.getRelClass());
|
||||||
|
}
|
||||||
|
|
||||||
private static MapFunction<String, Relation> patchRelFn() {
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
return value -> {
|
return value -> {
|
||||||
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
|
|
|
@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.dedup.graph;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||||
|
@ -18,12 +19,17 @@ public class ConnectedComponent implements Serializable {
|
||||||
private Set<String> docIds;
|
private Set<String> docIds;
|
||||||
private String ccId;
|
private String ccId;
|
||||||
|
|
||||||
public ConnectedComponent() {
|
public ConnectedComponent(Set<String> docIds, final int cut) {
|
||||||
}
|
|
||||||
|
|
||||||
public ConnectedComponent(Set<String> docIds) {
|
|
||||||
this.docIds = docIds;
|
this.docIds = docIds;
|
||||||
createID();
|
createID();
|
||||||
|
if (cut > 0 && docIds.size() > cut) {
|
||||||
|
this.docIds = docIds
|
||||||
|
.stream()
|
||||||
|
.filter(s -> !ccId.equalsIgnoreCase(s))
|
||||||
|
.limit(cut - 1)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
this.docIds.add(ccId);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public String createID() {
|
public String createID() {
|
||||||
|
@ -41,6 +47,7 @@ public class ConnectedComponent implements Serializable {
|
||||||
public String getMin() {
|
public String getMin() {
|
||||||
|
|
||||||
final StringBuilder min = new StringBuilder();
|
final StringBuilder min = new StringBuilder();
|
||||||
|
|
||||||
docIds
|
docIds
|
||||||
.forEach(
|
.forEach(
|
||||||
i -> {
|
i -> {
|
||||||
|
|
|
@ -7,7 +7,7 @@ import scala.collection.JavaConversions;
|
||||||
|
|
||||||
object GraphProcessor {
|
object GraphProcessor {
|
||||||
|
|
||||||
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int): RDD[ConnectedComponent] = {
|
def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int, cut:Int): RDD[ConnectedComponent] = {
|
||||||
val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
|
val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby
|
||||||
val cc = graph.connectedComponents(maxIterations).vertices
|
val cc = graph.connectedComponents(maxIterations).vertices
|
||||||
|
|
||||||
|
@ -22,15 +22,15 @@ object GraphProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
val connectedComponents = joinResult.groupByKey()
|
val connectedComponents = joinResult.groupByKey()
|
||||||
.map[ConnectedComponent](cc => asConnectedComponent(cc))
|
.map[ConnectedComponent](cc => asConnectedComponent(cc, cut))
|
||||||
connectedComponents
|
connectedComponents
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
def asConnectedComponent(group: (VertexId, Iterable[String])): ConnectedComponent = {
|
def asConnectedComponent(group: (VertexId, Iterable[String]), cut:Int): ConnectedComponent = {
|
||||||
val docs = group._2.toSet[String]
|
val docs = group._2.toSet[String]
|
||||||
val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs));
|
val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs), cut);
|
||||||
connectedComponent
|
connectedComponent
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,45 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup.model;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
public class BlockStats implements Serializable {
|
||||||
|
|
||||||
|
private String key; // key of the block
|
||||||
|
private Long size; // number of elements in the block
|
||||||
|
private Long comparisons; // number of comparisons in the block
|
||||||
|
|
||||||
|
public BlockStats() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public BlockStats(String key, Long size, Long comparisons) {
|
||||||
|
this.key = key;
|
||||||
|
this.size = size;
|
||||||
|
this.comparisons = comparisons;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getKey() {
|
||||||
|
return key;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setKey(String key) {
|
||||||
|
this.key = key;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Long getSize() {
|
||||||
|
return size;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSize(Long size) {
|
||||||
|
this.size = size;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Long getComparisons() {
|
||||||
|
return comparisons;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setComparisons(Long comparisons) {
|
||||||
|
this.comparisons = comparisons;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,32 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "la",
|
||||||
|
"paramLongName": "isLookUpUrl",
|
||||||
|
"paramDescription": "address for the LookUp",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "asi",
|
||||||
|
"paramLongName": "actionSetId",
|
||||||
|
"paramDescription": "action set identifier (name of the orchestrator)",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "i",
|
||||||
|
"paramLongName": "graphBasePath",
|
||||||
|
"paramDescription": "the base path of the raw graph",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "w",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "path of the working directory",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "np",
|
||||||
|
"paramLongName": "numPartitions",
|
||||||
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
|
"paramRequired": false
|
||||||
|
}
|
||||||
|
]
|
|
@ -17,6 +17,12 @@
|
||||||
"paramDescription": "the url for the lookup service",
|
"paramDescription": "the url for the lookup service",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
"paramName": "cc",
|
||||||
|
"paramLongName": "cutConnectedComponent",
|
||||||
|
"paramDescription": "the number of maximum elements that belongs to a connected components",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
{
|
{
|
||||||
"paramName": "w",
|
"paramName": "w",
|
||||||
"paramLongName": "workingPath",
|
"paramLongName": "workingPath",
|
||||||
|
|
|
@ -22,5 +22,11 @@
|
||||||
"paramLongName": "workingPath",
|
"paramLongName": "workingPath",
|
||||||
"paramDescription": "path of the working directory",
|
"paramDescription": "path of the working directory",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "np",
|
||||||
|
"paramLongName": "numPartitions",
|
||||||
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
|
"paramRequired": false
|
||||||
}
|
}
|
||||||
]
|
]
|
|
@ -20,6 +20,10 @@
|
||||||
<name>dedupGraphPath</name>
|
<name>dedupGraphPath</name>
|
||||||
<description>path for the output graph</description>
|
<description>path for the output graph</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>cutConnectedComponent</name>
|
||||||
|
<description>max number of elements in a connected component</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -106,10 +110,11 @@
|
||||||
--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>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||||
<arg>--asi</arg><arg>${actionSetId}</arg>
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
|
<arg>--numPartitions</arg><arg>8000</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="CreateMergeRel"/>
|
<ok to="CreateMergeRel"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -132,10 +137,11 @@
|
||||||
--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>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||||
<arg>--asi</arg><arg>${actionSetId}</arg>
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
|
<arg>--cutConnectedComponent</arg><arg>${cutConnectedComponent}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="CreateDedupRecord"/>
|
<ok to="CreateDedupRecord"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -158,10 +164,10 @@
|
||||||
--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>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--la</arg><arg>${isLookUpUrl}</arg>
|
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||||
<arg>--asi</arg><arg>${actionSetId}</arg>
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="UpdateEntity"/>
|
<ok to="UpdateEntity"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -184,9 +190,9 @@
|
||||||
--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>--i</arg><arg>${graphBasePath}</arg>
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
<arg>--w</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--o</arg><arg>${dedupGraphPath}</arg>
|
<arg>--dedupGraphPath</arg><arg>${dedupGraphPath}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="copyRelations"/>
|
<ok to="copyRelations"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
|
@ -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>
|
|
@ -0,0 +1,108 @@
|
||||||
|
<workflow-app name="Create dedup blocks" xmlns="uri:oozie:workflow:0.5">
|
||||||
|
<parameters>
|
||||||
|
<property>
|
||||||
|
<name>graphBasePath</name>
|
||||||
|
<description>the raw graph base path</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>isLookUpUrl</name>
|
||||||
|
<description>the address of the lookUp service</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>actionSetId</name>
|
||||||
|
<description>id of the actionSet</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>numPartitions</name>
|
||||||
|
<description>number of partitions for the similarity relations intermediate phases</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="CreateBlockStats"/>
|
||||||
|
|
||||||
|
<kill name="Kill">
|
||||||
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
|
</kill>
|
||||||
|
|
||||||
|
<action name="CreateBlockStats">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Create deduplication blocks</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.dedup.SparkBlockStats</class>
|
||||||
|
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
|
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||||
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
|
<arg>--workingPath</arg><arg>${workingDir}</arg>
|
||||||
|
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<end name="End"/>
|
||||||
|
</workflow-app>
|
|
@ -1,17 +1,17 @@
|
||||||
[
|
[
|
||||||
{
|
{
|
||||||
"paramName": "i",
|
"paramName": "i",
|
||||||
"paramLongName": "graphBasePath",
|
"paramLongName": "graphBasePath",
|
||||||
"paramDescription": "the base path of raw graph",
|
"paramDescription": "the base path of raw graph",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "w",
|
"paramName": "w",
|
||||||
"paramLongName": "workingPath",
|
"paramLongName": "workingPath",
|
||||||
"paramDescription": "the working directory path",
|
"paramDescription": "the working directory path",
|
||||||
"paramRequired": true
|
"paramRequired": true
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"paramName": "o",
|
"paramName": "o",
|
||||||
"paramLongName": "dedupGraphPath",
|
"paramLongName": "dedupGraphPath",
|
||||||
"paramDescription": "the path of the dedup graph",
|
"paramDescription": "the path of the dedup graph",
|
||||||
|
|
|
@ -45,6 +45,17 @@ public class EntityMergerTest implements Serializable {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void softwareMergerTest() throws InstantiationException, IllegalAccessException {
|
||||||
|
List<Tuple2<String, Software>> softwares = readSample(
|
||||||
|
testEntityBasePath + "/software_merge.json", Software.class);
|
||||||
|
|
||||||
|
Software merged = DedupRecordFactory
|
||||||
|
.entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class);
|
||||||
|
|
||||||
|
assertEquals(merged.getBestaccessright().getClassid(), "OPEN SOURCE");
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void publicationMergerTest() throws InstantiationException, IllegalAccessException {
|
public void publicationMergerTest() throws InstantiationException, IllegalAccessException {
|
||||||
|
|
||||||
|
|
|
@ -3,6 +3,8 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import static java.nio.file.Files.createTempDirectory;
|
import static java.nio.file.Files.createTempDirectory;
|
||||||
|
|
||||||
|
import static org.apache.spark.sql.functions.col;
|
||||||
|
import static org.apache.spark.sql.functions.count;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.mockito.Mockito.lenient;
|
import static org.mockito.Mockito.lenient;
|
||||||
|
|
||||||
|
@ -11,6 +13,9 @@ import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
@ -18,6 +23,7 @@ import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
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.FilterFunction;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
|
@ -71,11 +77,13 @@ public class SparkDedupTest implements Serializable {
|
||||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||||
|
|
||||||
|
final SparkConf conf = new SparkConf();
|
||||||
|
conf.set("spark.sql.shuffle.partitions", "200");
|
||||||
spark = SparkSession
|
spark = SparkSession
|
||||||
.builder()
|
.builder()
|
||||||
.appName(SparkDedupTest.class.getSimpleName())
|
.appName(SparkDedupTest.class.getSimpleName())
|
||||||
.master("local[*]")
|
.master("local[*]")
|
||||||
.config(new SparkConf())
|
.config(conf)
|
||||||
.getOrCreate();
|
.getOrCreate();
|
||||||
|
|
||||||
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
@ -152,33 +160,38 @@ public class SparkDedupTest implements Serializable {
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[] {
|
||||||
"-i",
|
"-i", testGraphBasePath,
|
||||||
testGraphBasePath,
|
"-asi", testActionSetId,
|
||||||
"-asi",
|
"-la", "lookupurl",
|
||||||
testActionSetId,
|
"-w", testOutputBasePath,
|
||||||
"-la",
|
"-np", "50"
|
||||||
"lookupurl",
|
|
||||||
"-w",
|
|
||||||
testOutputBasePath
|
|
||||||
});
|
});
|
||||||
|
|
||||||
new SparkCreateSimRels(parser, spark).run(isLookUpService);
|
new SparkCreateSimRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
long orgs_simrel = spark
|
long orgs_simrel = spark
|
||||||
.read()
|
.read()
|
||||||
.load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
|
||||||
.count();
|
.count();
|
||||||
|
|
||||||
long pubs_simrel = spark
|
long pubs_simrel = spark
|
||||||
.read()
|
.read()
|
||||||
.load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel")
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/publication_simrel")
|
||||||
.count();
|
.count();
|
||||||
long sw_simrel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/software_simrel").count();
|
|
||||||
|
|
||||||
long ds_simrel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel").count();
|
long sw_simrel = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/software_simrel")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long ds_simrel = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel")
|
||||||
|
.count();
|
||||||
|
|
||||||
long orp_simrel = spark
|
long orp_simrel = spark
|
||||||
.read()
|
.read()
|
||||||
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
|
||||||
.count();
|
.count();
|
||||||
|
|
||||||
assertEquals(3432, orgs_simrel);
|
assertEquals(3432, orgs_simrel);
|
||||||
|
@ -190,6 +203,101 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(2)
|
@Order(2)
|
||||||
|
public void cutMergeRelsTest() throws Exception {
|
||||||
|
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCreateMergeRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i",
|
||||||
|
testGraphBasePath,
|
||||||
|
"-asi",
|
||||||
|
testActionSetId,
|
||||||
|
"-la",
|
||||||
|
"lookupurl",
|
||||||
|
"-w",
|
||||||
|
testOutputBasePath,
|
||||||
|
"-cc",
|
||||||
|
"3"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCreateMergeRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.filter((FilterFunction<Relation>) r -> r.getRelClass().equalsIgnoreCase("merges"))
|
||||||
|
.groupBy("source")
|
||||||
|
.agg(count("target").alias("cnt"))
|
||||||
|
.select("source", "cnt")
|
||||||
|
.where("cnt > 3")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long pubs_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.filter((FilterFunction<Relation>) r -> r.getRelClass().equalsIgnoreCase("merges"))
|
||||||
|
.groupBy("source")
|
||||||
|
.agg(count("target").alias("cnt"))
|
||||||
|
.select("source", "cnt")
|
||||||
|
.where("cnt > 3")
|
||||||
|
.count();
|
||||||
|
long sw_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.filter((FilterFunction<Relation>) r -> r.getRelClass().equalsIgnoreCase("merges"))
|
||||||
|
.groupBy("source")
|
||||||
|
.agg(count("target").alias("cnt"))
|
||||||
|
.select("source", "cnt")
|
||||||
|
.where("cnt > 3")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long ds_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.filter((FilterFunction<Relation>) r -> r.getRelClass().equalsIgnoreCase("merges"))
|
||||||
|
.groupBy("source")
|
||||||
|
.agg(count("target").alias("cnt"))
|
||||||
|
.select("source", "cnt")
|
||||||
|
.where("cnt > 3")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long orp_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.filter((FilterFunction<Relation>) r -> r.getRelClass().equalsIgnoreCase("merges"))
|
||||||
|
.groupBy("source")
|
||||||
|
.agg(count("target").alias("cnt"))
|
||||||
|
.select("source", "cnt")
|
||||||
|
.where("cnt > 3")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(0, orgs_mergerel);
|
||||||
|
assertEquals(0, pubs_mergerel);
|
||||||
|
assertEquals(0, sw_mergerel);
|
||||||
|
assertEquals(0, ds_mergerel);
|
||||||
|
assertEquals(0, orp_mergerel);
|
||||||
|
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel"));
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel"));
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/software_mergerel"));
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel"));
|
||||||
|
FileUtils
|
||||||
|
.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@Order(3)
|
||||||
public void createMergeRelsTest() throws Exception {
|
public void createMergeRelsTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -225,8 +333,10 @@ public class SparkDedupTest implements Serializable {
|
||||||
.read()
|
.read()
|
||||||
.load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel")
|
.load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel")
|
||||||
.count();
|
.count();
|
||||||
|
long ds_mergerel = spark
|
||||||
long ds_mergerel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel").count();
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel")
|
||||||
|
.count();
|
||||||
|
|
||||||
long orp_mergerel = spark
|
long orp_mergerel = spark
|
||||||
.read()
|
.read()
|
||||||
|
@ -241,7 +351,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(3)
|
@Order(4)
|
||||||
public void createDedupRecordTest() throws Exception {
|
public void createDedupRecordTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -288,7 +398,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(4)
|
@Order(5)
|
||||||
public void updateEntityTest() throws Exception {
|
public void updateEntityTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -404,7 +514,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(5)
|
@Order(6)
|
||||||
public void propagateRelationTest() throws Exception {
|
public void propagateRelationTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -423,7 +533,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
|
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
|
||||||
|
|
||||||
assertEquals(4975, relations);
|
assertEquals(4866, relations);
|
||||||
|
|
||||||
// check deletedbyinference
|
// check deletedbyinference
|
||||||
final Dataset<Relation> mergeRels = spark
|
final Dataset<Relation> mergeRels = spark
|
||||||
|
@ -454,7 +564,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(6)
|
@Order(7)
|
||||||
public void testRelations() throws Exception {
|
public void testRelations() throws Exception {
|
||||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
|
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
|
||||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);
|
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);
|
||||||
|
|
|
@ -0,0 +1,177 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import static java.nio.file.Files.createTempDirectory;
|
||||||
|
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.mockito.Mockito.lenient;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
|
import java.nio.file.Paths;
|
||||||
|
|
||||||
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.junit.jupiter.api.*;
|
||||||
|
import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
|
import org.mockito.Mock;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
|
||||||
|
@ExtendWith(MockitoExtension.class)
|
||||||
|
public class SparkStatsTest implements Serializable {
|
||||||
|
|
||||||
|
@Mock(serializable = true)
|
||||||
|
ISLookUpService isLookUpService;
|
||||||
|
|
||||||
|
private static SparkSession spark;
|
||||||
|
private static JavaSparkContext jsc;
|
||||||
|
|
||||||
|
private static String testGraphBasePath;
|
||||||
|
private static String testOutputBasePath;
|
||||||
|
private static final String testActionSetId = "test-orchestrator";
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void cleanUp() throws IOException, URISyntaxException {
|
||||||
|
|
||||||
|
testGraphBasePath = Paths
|
||||||
|
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI())
|
||||||
|
.toFile()
|
||||||
|
.getAbsolutePath();
|
||||||
|
testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
|
||||||
|
.toAbsolutePath()
|
||||||
|
.toString();
|
||||||
|
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||||
|
|
||||||
|
final SparkConf conf = new SparkConf();
|
||||||
|
conf.set("spark.sql.shuffle.partitions", "200");
|
||||||
|
spark = SparkSession
|
||||||
|
.builder()
|
||||||
|
.appName(SparkDedupTest.class.getSimpleName())
|
||||||
|
.master("local[*]")
|
||||||
|
.config(conf)
|
||||||
|
.getOrCreate();
|
||||||
|
|
||||||
|
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
}
|
||||||
|
|
||||||
|
@BeforeEach
|
||||||
|
public void setUp() throws IOException, ISLookUpException {
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId)))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator.xml")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("software")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("dataset")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("otherresearchproduct")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void createBlockStatsTest() throws Exception {
|
||||||
|
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCreateSimRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-la", "lookupurl",
|
||||||
|
"-w", testOutputBasePath
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkBlockStats(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_blocks = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_blockstats")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long pubs_blocks = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/publication_blockstats")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long sw_blocks = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/software_blockstats")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long ds_blocks = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_blockstats")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long orp_blocks = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_blockstats")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(121, orgs_blocks);
|
||||||
|
assertEquals(110, pubs_blocks);
|
||||||
|
assertEquals(21, sw_blocks);
|
||||||
|
assertEquals(67, ds_blocks);
|
||||||
|
assertEquals(55, orp_blocks);
|
||||||
|
}
|
||||||
|
}
|
|
@ -6,10 +6,10 @@
|
||||||
"subEntityType" : "resulttype",
|
"subEntityType" : "resulttype",
|
||||||
"subEntityValue" : "dataset",
|
"subEntityValue" : "dataset",
|
||||||
"orderField" : "title",
|
"orderField" : "title",
|
||||||
"queueMaxSize" : "2000",
|
"queueMaxSize" : "100",
|
||||||
"groupMaxSize" : "100",
|
"groupMaxSize" : "100",
|
||||||
"maxChildren" : "100",
|
"maxChildren" : "100",
|
||||||
"slidingWindowSize" : "200",
|
"slidingWindowSize" : "100",
|
||||||
"rootBuilder" : ["result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
"rootBuilder" : ["result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
||||||
"includeChildren" : "true",
|
"includeChildren" : "true",
|
||||||
"idPath" : "$.id",
|
"idPath" : "$.id",
|
||||||
|
|
|
@ -6,10 +6,10 @@
|
||||||
"subEntityType" : "resulttype",
|
"subEntityType" : "resulttype",
|
||||||
"subEntityValue" : "otherresearchproduct",
|
"subEntityValue" : "otherresearchproduct",
|
||||||
"orderField" : "title",
|
"orderField" : "title",
|
||||||
"queueMaxSize" : "2000",
|
"queueMaxSize" : "100",
|
||||||
"groupMaxSize" : "100",
|
"groupMaxSize" : "100",
|
||||||
"maxChildren" : "100",
|
"maxChildren" : "100",
|
||||||
"slidingWindowSize" : "200",
|
"slidingWindowSize" : "100",
|
||||||
"rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
"rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
||||||
"includeChildren" : "true",
|
"includeChildren" : "true",
|
||||||
"idPath" : "$.id",
|
"idPath" : "$.id",
|
||||||
|
|
|
@ -6,10 +6,10 @@
|
||||||
"subEntityType": "resulttype",
|
"subEntityType": "resulttype",
|
||||||
"subEntityValue": "publication",
|
"subEntityValue": "publication",
|
||||||
"orderField": "title",
|
"orderField": "title",
|
||||||
"queueMaxSize": "2000",
|
"queueMaxSize": "100",
|
||||||
"groupMaxSize": "100",
|
"groupMaxSize": "100",
|
||||||
"maxChildren": "100",
|
"maxChildren": "100",
|
||||||
"slidingWindowSize": "200",
|
"slidingWindowSize": "100",
|
||||||
"rootBuilder": [
|
"rootBuilder": [
|
||||||
"result",
|
"result",
|
||||||
"resultProject_outcome_isProducedBy",
|
"resultProject_outcome_isProducedBy",
|
||||||
|
|
|
@ -6,10 +6,10 @@
|
||||||
"subEntityType" : "resulttype",
|
"subEntityType" : "resulttype",
|
||||||
"subEntityValue" : "software",
|
"subEntityValue" : "software",
|
||||||
"orderField" : "title",
|
"orderField" : "title",
|
||||||
"queueMaxSize" : "2000",
|
"queueMaxSize" : "100",
|
||||||
"groupMaxSize" : "100",
|
"groupMaxSize" : "100",
|
||||||
"maxChildren" : "100",
|
"maxChildren" : "100",
|
||||||
"slidingWindowSize" : "200",
|
"slidingWindowSize" : "100",
|
||||||
"rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
"rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ],
|
||||||
"includeChildren" : "true",
|
"includeChildren" : "true",
|
||||||
"idPath" : "$.id",
|
"idPath" : "$.id",
|
||||||
|
@ -19,7 +19,7 @@
|
||||||
"clustering" : [
|
"clustering" : [
|
||||||
{ "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} },
|
{ "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} },
|
||||||
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } },
|
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } },
|
||||||
{ "name" : "lowercase", "fields" : [ "doi", "url" ], "params" : { } }
|
{ "name" : "lowercase", "fields" : [ "doi" ], "params" : { } }
|
||||||
],
|
],
|
||||||
"decisionTree": {
|
"decisionTree": {
|
||||||
"start": {
|
"start": {
|
||||||
|
|
File diff suppressed because one or more lines are too long
|
@ -41,7 +41,7 @@ public class QueryInformationSystem {
|
||||||
+ " </selcriteria> "
|
+ " </selcriteria> "
|
||||||
+ " </datasource> } "
|
+ " </datasource> } "
|
||||||
+ " </datasources> " +
|
+ " </datasources> " +
|
||||||
" <zenodocommunities> "+
|
" <zenodocommunities> " +
|
||||||
"{for $zc in $zenodo " +
|
"{for $zc in $zenodo " +
|
||||||
"return " +
|
"return " +
|
||||||
"<zenodocommunity> " +
|
"<zenodocommunity> " +
|
||||||
|
|
|
@ -71,10 +71,10 @@ public class ResultTagger implements Serializable {
|
||||||
|
|
||||||
// tagging for Subject
|
// tagging for Subject
|
||||||
final Set<String> subjects = new HashSet<>();
|
final Set<String> subjects = new HashSet<>();
|
||||||
Optional<List<StructuredProperty>> oresultsubj = Optional.ofNullable(result.getSubject());
|
|
||||||
if (oresultsubj.isPresent()) {
|
if (Objects.nonNull(result.getSubject())) {
|
||||||
oresultsubj
|
result
|
||||||
.get()
|
.getSubject()
|
||||||
.stream()
|
.stream()
|
||||||
.map(subject -> subject.getValue())
|
.map(subject -> subject.getValue())
|
||||||
.filter(StringUtils::isNotBlank)
|
.filter(StringUtils::isNotBlank)
|
||||||
|
@ -90,15 +90,23 @@ public class ResultTagger implements Serializable {
|
||||||
final Set<String> datasources = new HashSet<>();
|
final Set<String> datasources = new HashSet<>();
|
||||||
final Set<String> tmp = new HashSet<>();
|
final Set<String> tmp = new HashSet<>();
|
||||||
|
|
||||||
Optional<List<Instance>> oresultinstance = Optional.ofNullable(result.getInstance());
|
if (Objects.nonNull(result.getInstance())) {
|
||||||
if (oresultinstance.isPresent()) {
|
for (Instance i : result.getInstance()) {
|
||||||
for (Instance i : oresultinstance.get()) {
|
if (Objects.nonNull(i.getCollectedfrom())) {
|
||||||
tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|"));
|
if (Objects.nonNull(i.getCollectedfrom().getKey())) {
|
||||||
tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|"));
|
tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (Objects.nonNull(i.getHostedby())) {
|
||||||
|
if (Objects.nonNull(i.getHostedby().getKey())) {
|
||||||
|
tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
oresultinstance
|
result
|
||||||
.get()
|
.getInstance()
|
||||||
.stream()
|
.stream()
|
||||||
.map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey()))
|
.map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey()))
|
||||||
.flatMap(p -> Stream.of(p.getFst(), p.getSnd()))
|
.flatMap(p -> Stream.of(p.getFst(), p.getSnd()))
|
||||||
|
|
|
@ -9,6 +9,37 @@
|
||||||
|
|
||||||
<artifactId>dhp-graph-mapper</artifactId>
|
<artifactId>dhp-graph-mapper</artifactId>
|
||||||
|
|
||||||
|
<build>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>net.alchim31.maven</groupId>
|
||||||
|
<artifactId>scala-maven-plugin</artifactId>
|
||||||
|
<version>4.0.1</version>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<id>scala-compile-first</id>
|
||||||
|
<phase>initialize</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>add-source</goal>
|
||||||
|
<goal>compile</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
<execution>
|
||||||
|
<id>scala-test-compile</id>
|
||||||
|
<phase>process-test-resources</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>testCompile</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
<configuration>
|
||||||
|
<scalaVersion>${scala.version}</scalaVersion>
|
||||||
|
</configuration>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
|
||||||
|
</build>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
|
@ -68,6 +99,7 @@
|
||||||
<artifactId>postgresql</artifactId>
|
<artifactId>postgresql</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|
||||||
<groupId>org.apache.httpcomponents</groupId>
|
<groupId>org.apache.httpcomponents</groupId>
|
||||||
<artifactId>httpmime</artifactId>
|
<artifactId>httpmime</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
@ -82,6 +114,12 @@
|
||||||
<version>4.13.0</version>
|
<version>4.13.0</version>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.json4s</groupId>
|
||||||
|
<artifactId>json4s-jackson_2.11</artifactId>
|
||||||
|
<version>3.5.3</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.common.HdfsSupport;
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.oa.graph.raw.AbstractMdRecordToOafMapper;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils;
|
import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
|
@ -97,7 +98,7 @@ public class CleanGraphSparkJob {
|
||||||
.json(outputPath);
|
.json(outputPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T extends Oaf> T fixDefaults(T value) {
|
protected static <T extends Oaf> T fixDefaults(T value) {
|
||||||
if (value instanceof Datasource) {
|
if (value instanceof Datasource) {
|
||||||
// nothing to clean here
|
// nothing to clean here
|
||||||
} else if (value instanceof Project) {
|
} else if (value instanceof Project) {
|
||||||
|
@ -134,11 +135,6 @@ public class CleanGraphSparkJob {
|
||||||
.setResourcetype(
|
.setResourcetype(
|
||||||
qualifier("UNKNOWN", "Unknown", ModelConstants.DNET_DATA_CITE_RESOURCE));
|
qualifier("UNKNOWN", "Unknown", ModelConstants.DNET_DATA_CITE_RESOURCE));
|
||||||
}
|
}
|
||||||
if (Objects.isNull(r.getBestaccessright()) || StringUtils.isBlank(r.getBestaccessright().getClassid())) {
|
|
||||||
r
|
|
||||||
.setBestaccessright(
|
|
||||||
qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES));
|
|
||||||
}
|
|
||||||
if (Objects.nonNull(r.getInstance())) {
|
if (Objects.nonNull(r.getInstance())) {
|
||||||
for (Instance i : r.getInstance()) {
|
for (Instance i : r.getInstance()) {
|
||||||
if (Objects.isNull(i.getAccessright()) || StringUtils.isBlank(i.getAccessright().getClassid())) {
|
if (Objects.isNull(i.getAccessright()) || StringUtils.isBlank(i.getAccessright().getClassid())) {
|
||||||
|
@ -152,6 +148,16 @@ public class CleanGraphSparkJob {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if (Objects.isNull(r.getBestaccessright()) || StringUtils.isBlank(r.getBestaccessright().getClassid())) {
|
||||||
|
Qualifier bestaccessrights = AbstractMdRecordToOafMapper.createBestAccessRights(r.getInstance());
|
||||||
|
if (Objects.isNull(bestaccessrights)) {
|
||||||
|
r
|
||||||
|
.setBestaccessright(
|
||||||
|
qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES));
|
||||||
|
} else {
|
||||||
|
r.setBestaccessright(bestaccessrights);
|
||||||
|
}
|
||||||
|
}
|
||||||
if (Objects.nonNull(r.getAuthor())) {
|
if (Objects.nonNull(r.getAuthor())) {
|
||||||
boolean nullRank = r
|
boolean nullRank = r
|
||||||
.getAuthor()
|
.getAuthor()
|
||||||
|
|
|
@ -0,0 +1,162 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.graph.merge;
|
||||||
|
|
||||||
|
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
import org.apache.spark.sql.SaveMode;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.common.HdfsSupport;
|
||||||
|
import eu.dnetlib.dhp.oa.graph.clean.CleanGraphSparkJob;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Combines the content from two aggregator graph tables of the same type, entities (or relationships) with the same ids
|
||||||
|
* are picked preferring those from the BETA aggregator rather then from PROD. The identity of a relationship is defined
|
||||||
|
* by eu.dnetlib.dhp.schema.common.ModelSupport#idFn()
|
||||||
|
*/
|
||||||
|
public class MergeGraphSparkJob {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(CleanGraphSparkJob.class);
|
||||||
|
|
||||||
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
|
|
||||||
|
private static final String PRIORITY_DEFAULT = "BETA"; // BETA | PROD
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
|
String jsonConfiguration = IOUtils
|
||||||
|
.toString(
|
||||||
|
CleanGraphSparkJob.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json"));
|
||||||
|
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
String priority = Optional
|
||||||
|
.ofNullable(parser.get("priority"))
|
||||||
|
.orElse(PRIORITY_DEFAULT);
|
||||||
|
log.info("priority: {}", priority);
|
||||||
|
|
||||||
|
Boolean isSparkSessionManaged = Optional
|
||||||
|
.ofNullable(parser.get("isSparkSessionManaged"))
|
||||||
|
.map(Boolean::valueOf)
|
||||||
|
.orElse(Boolean.TRUE);
|
||||||
|
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
|
||||||
|
|
||||||
|
String betaInputPath = parser.get("betaInputPath");
|
||||||
|
log.info("betaInputPath: {}", betaInputPath);
|
||||||
|
|
||||||
|
String prodInputPath = parser.get("prodInputPath");
|
||||||
|
log.info("prodInputPath: {}", prodInputPath);
|
||||||
|
|
||||||
|
String outputPath = parser.get("outputPath");
|
||||||
|
log.info("outputPath: {}", outputPath);
|
||||||
|
|
||||||
|
String graphTableClassName = parser.get("graphTableClassName");
|
||||||
|
log.info("graphTableClassName: {}", graphTableClassName);
|
||||||
|
|
||||||
|
Class<? extends OafEntity> entityClazz = (Class<? extends OafEntity>) Class.forName(graphTableClassName);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||||
|
|
||||||
|
runWithSparkSession(
|
||||||
|
conf,
|
||||||
|
isSparkSessionManaged,
|
||||||
|
spark -> {
|
||||||
|
removeOutputDir(spark, outputPath);
|
||||||
|
mergeGraphTable(spark, priority, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <P extends Oaf, B extends Oaf> void mergeGraphTable(
|
||||||
|
SparkSession spark,
|
||||||
|
String priority,
|
||||||
|
String betaInputPath,
|
||||||
|
String prodInputPath,
|
||||||
|
Class<P> p_clazz,
|
||||||
|
Class<B> b_clazz,
|
||||||
|
String outputPath) {
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, B>> beta = readTableFromPath(spark, betaInputPath, b_clazz);
|
||||||
|
Dataset<Tuple2<String, P>> prod = readTableFromPath(spark, prodInputPath, p_clazz);
|
||||||
|
|
||||||
|
prod
|
||||||
|
.joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer")
|
||||||
|
.map((MapFunction<Tuple2<Tuple2<String, P>, Tuple2<String, B>>, P>) value -> {
|
||||||
|
Optional<P> p = Optional.ofNullable(value._1()).map(Tuple2::_2);
|
||||||
|
Optional<B> b = Optional.ofNullable(value._2()).map(Tuple2::_2);
|
||||||
|
switch (priority) {
|
||||||
|
default:
|
||||||
|
case "BETA":
|
||||||
|
return mergeWithPriorityToBETA(p, b);
|
||||||
|
case "PROD":
|
||||||
|
return mergeWithPriorityToPROD(p, b);
|
||||||
|
}
|
||||||
|
}, Encoders.bean(p_clazz))
|
||||||
|
.filter((FilterFunction<P>) Objects::nonNull)
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.option("compression", "gzip")
|
||||||
|
.json(outputPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <P extends Oaf, B extends Oaf> P mergeWithPriorityToPROD(Optional<P> p, Optional<B> b) {
|
||||||
|
if (b.isPresent() & !p.isPresent()) {
|
||||||
|
return (P) b.get();
|
||||||
|
}
|
||||||
|
if (p.isPresent()) {
|
||||||
|
return p.get();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <P extends Oaf, B extends Oaf> P mergeWithPriorityToBETA(Optional<P> p, Optional<B> b) {
|
||||||
|
if (p.isPresent() & !b.isPresent()) {
|
||||||
|
return p.get();
|
||||||
|
}
|
||||||
|
if (b.isPresent()) {
|
||||||
|
return (P) b.get();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <T extends Oaf> Dataset<Tuple2<String, T>> readTableFromPath(
|
||||||
|
SparkSession spark, String inputEntityPath, Class<T> clazz) {
|
||||||
|
|
||||||
|
log.info("Reading Graph table from: {}", inputEntityPath);
|
||||||
|
return spark
|
||||||
|
.read()
|
||||||
|
.textFile(inputEntityPath)
|
||||||
|
.map(
|
||||||
|
(MapFunction<String, Tuple2<String, T>>) value -> {
|
||||||
|
final T t = OBJECT_MAPPER.readValue(value, clazz);
|
||||||
|
final String id = ModelSupport.idFn().apply(t);
|
||||||
|
return new Tuple2<>(id, t);
|
||||||
|
},
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz)));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void removeOutputDir(SparkSession spark, String path) {
|
||||||
|
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1,36 +1,10 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.graph.raw;
|
package eu.dnetlib.dhp.oa.graph.raw;
|
||||||
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.createOpenaireId;
|
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.*;
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.dataInfo;
|
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.field;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.journal;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.keyValue;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.listFields;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.oaiIProvenance;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.qualifier;
|
|
||||||
import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.structuredProperty;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.DATASET_DEFAULT_RESULTTYPE;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_ACCESS_MODES;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PID_TYPES;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.IS_PRODUCED_BY;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.NOT_AVAILABLE;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.ORP_DEFAULT_RESULTTYPE;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.OUTCOME;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.PRODUCES;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.REPOSITORY_PROVENANCE_ACTIONS;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_PROJECT;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE;
|
|
||||||
import static eu.dnetlib.dhp.schema.common.ModelConstants.UNKNOWN;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.dom4j.Document;
|
import org.dom4j.Document;
|
||||||
|
@ -40,24 +14,8 @@ import org.dom4j.Node;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
||||||
import eu.dnetlib.dhp.schema.common.LicenseComparator;
|
import eu.dnetlib.dhp.schema.common.LicenseComparator;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Author;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Context;
|
import eu.dnetlib.dhp.schema.oaf.*;
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Field;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.GeoLocation;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Instance;
|
|
||||||
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.Oaf;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Result;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Software;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
|
||||||
|
|
||||||
public abstract class AbstractMdRecordToOafMapper {
|
public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
|
@ -99,7 +57,6 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final Document doc = DocumentHelper
|
final Document doc = DocumentHelper
|
||||||
.parseText(xml.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3));
|
.parseText(xml.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3));
|
||||||
|
|
||||||
final String type = doc.valueOf("//dr:CobjCategory/@type");
|
|
||||||
final KeyValue collectedFrom = getProvenanceDatasource(
|
final KeyValue collectedFrom = getProvenanceDatasource(
|
||||||
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
|
||||||
|
|
||||||
|
@ -118,12 +75,32 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
final DataInfo info = prepareDataInfo(doc, invisible);
|
final DataInfo info = prepareDataInfo(doc, invisible);
|
||||||
final long lastUpdateTimestamp = new Date().getTime();
|
final long lastUpdateTimestamp = new Date().getTime();
|
||||||
|
|
||||||
return createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp);
|
final List<Instance> instances = prepareInstances(doc, info, collectedFrom, hostedBy);
|
||||||
|
|
||||||
|
final String type = getResultType(doc, instances);
|
||||||
|
|
||||||
|
return createOafs(doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected String getResultType(final Document doc, final List<Instance> instances) {
|
||||||
|
String type = doc.valueOf("//dr:CobjCategory/@type");
|
||||||
|
|
||||||
|
if (StringUtils.isBlank(type) & vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
|
||||||
|
String instanceType = instances
|
||||||
|
.stream()
|
||||||
|
.map(i -> i.getInstancetype().getClassid())
|
||||||
|
.findFirst()
|
||||||
|
.orElse("0000"); // Unknown
|
||||||
|
Qualifier resultType = vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType);
|
||||||
|
return resultType.getClassid();
|
||||||
|
}
|
||||||
|
|
||||||
|
return type;
|
||||||
|
}
|
||||||
|
|
||||||
private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) {
|
private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) {
|
||||||
final String dsId = doc.valueOf(xpathId);
|
final String dsId = doc.valueOf(xpathId);
|
||||||
final String dsName = doc.valueOf(xpathName);
|
final String dsName = doc.valueOf(xpathName);
|
||||||
|
@ -138,8 +115,8 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
protected List<Oaf> createOafs(
|
protected List<Oaf> createOafs(
|
||||||
final Document doc,
|
final Document doc,
|
||||||
final String type,
|
final String type,
|
||||||
|
final List<Instance> instances,
|
||||||
final KeyValue collectedFrom,
|
final KeyValue collectedFrom,
|
||||||
final KeyValue hostedBy,
|
|
||||||
final DataInfo info,
|
final DataInfo info,
|
||||||
final long lastUpdateTimestamp) {
|
final long lastUpdateTimestamp) {
|
||||||
|
|
||||||
|
@ -148,14 +125,14 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
switch (type.toLowerCase()) {
|
switch (type.toLowerCase()) {
|
||||||
case "publication":
|
case "publication":
|
||||||
final Publication p = new Publication();
|
final Publication p = new Publication();
|
||||||
populateResultFields(p, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp);
|
populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
p.setResulttype(PUBLICATION_DEFAULT_RESULTTYPE);
|
p.setResulttype(PUBLICATION_DEFAULT_RESULTTYPE);
|
||||||
p.setJournal(prepareJournal(doc, info));
|
p.setJournal(prepareJournal(doc, info));
|
||||||
oafs.add(p);
|
oafs.add(p);
|
||||||
break;
|
break;
|
||||||
case "dataset":
|
case "dataset":
|
||||||
final Dataset d = new Dataset();
|
final Dataset d = new Dataset();
|
||||||
populateResultFields(d, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp);
|
populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
d.setResulttype(DATASET_DEFAULT_RESULTTYPE);
|
d.setResulttype(DATASET_DEFAULT_RESULTTYPE);
|
||||||
d.setStoragedate(prepareDatasetStorageDate(doc, info));
|
d.setStoragedate(prepareDatasetStorageDate(doc, info));
|
||||||
d.setDevice(prepareDatasetDevice(doc, info));
|
d.setDevice(prepareDatasetDevice(doc, info));
|
||||||
|
@ -168,7 +145,7 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
break;
|
break;
|
||||||
case "software":
|
case "software":
|
||||||
final Software s = new Software();
|
final Software s = new Software();
|
||||||
populateResultFields(s, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp);
|
populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
s.setResulttype(SOFTWARE_DEFAULT_RESULTTYPE);
|
s.setResulttype(SOFTWARE_DEFAULT_RESULTTYPE);
|
||||||
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
|
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
|
||||||
s.setLicense(prepareSoftwareLicenses(doc, info));
|
s.setLicense(prepareSoftwareLicenses(doc, info));
|
||||||
|
@ -180,7 +157,7 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
case "otherresearchproducts":
|
case "otherresearchproducts":
|
||||||
default:
|
default:
|
||||||
final OtherResearchProduct o = new OtherResearchProduct();
|
final OtherResearchProduct o = new OtherResearchProduct();
|
||||||
populateResultFields(o, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp);
|
populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp);
|
||||||
o.setResulttype(ORP_DEFAULT_RESULTTYPE);
|
o.setResulttype(ORP_DEFAULT_RESULTTYPE);
|
||||||
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
|
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
|
||||||
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
|
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
|
||||||
|
@ -259,14 +236,16 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
private void populateResultFields(
|
private void populateResultFields(
|
||||||
final Result r,
|
final Result r,
|
||||||
final Document doc,
|
final Document doc,
|
||||||
|
final List<Instance> instances,
|
||||||
final KeyValue collectedFrom,
|
final KeyValue collectedFrom,
|
||||||
final KeyValue hostedBy,
|
|
||||||
final DataInfo info,
|
final DataInfo info,
|
||||||
final long lastUpdateTimestamp) {
|
final long lastUpdateTimestamp) {
|
||||||
r.setDataInfo(info);
|
r.setDataInfo(info);
|
||||||
r.setLastupdatetimestamp(lastUpdateTimestamp);
|
r.setLastupdatetimestamp(lastUpdateTimestamp);
|
||||||
r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
|
r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
|
||||||
r.setOriginalId(Arrays.asList(doc.valueOf("//dri:objIdentifier")));
|
|
||||||
|
r.setOriginalId(Arrays.asList(findOriginalId(doc)));
|
||||||
|
|
||||||
r.setCollectedfrom(Arrays.asList(collectedFrom));
|
r.setCollectedfrom(Arrays.asList(collectedFrom));
|
||||||
r.setPid(prepareResultPids(doc, info));
|
r.setPid(prepareResultPids(doc, info));
|
||||||
r.setDateofcollection(doc.valueOf("//dr:dateOfCollection"));
|
r.setDateofcollection(doc.valueOf("//dr:dateOfCollection"));
|
||||||
|
@ -291,7 +270,7 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
r.setCoverage(prepareCoverages(doc, info));
|
r.setCoverage(prepareCoverages(doc, info));
|
||||||
r.setContext(prepareContexts(doc, info));
|
r.setContext(prepareContexts(doc, info));
|
||||||
r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES
|
r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES
|
||||||
final List<Instance> instances = prepareInstances(doc, info, collectedFrom, hostedBy);
|
|
||||||
r.setInstance(instances);
|
r.setInstance(instances);
|
||||||
r.setBestaccessright(getBestAccessRights(instances));
|
r.setBestaccessright(getBestAccessRights(instances));
|
||||||
}
|
}
|
||||||
|
@ -378,6 +357,10 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
|
|
||||||
protected abstract Field<String> prepareDatasetStorageDate(Document doc, DataInfo info);
|
protected abstract Field<String> prepareDatasetStorageDate(Document doc, DataInfo info);
|
||||||
|
|
||||||
|
public static Qualifier createBestAccessRights(final List<Instance> instanceList) {
|
||||||
|
return getBestAccessRights(instanceList);
|
||||||
|
}
|
||||||
|
|
||||||
protected static Qualifier getBestAccessRights(final List<Instance> instanceList) {
|
protected static Qualifier getBestAccessRights(final List<Instance> instanceList) {
|
||||||
if (instanceList != null) {
|
if (instanceList != null) {
|
||||||
final Optional<Qualifier> min = instanceList
|
final Optional<Qualifier> min = instanceList
|
||||||
|
@ -425,6 +408,18 @@ public abstract class AbstractMdRecordToOafMapper {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private String findOriginalId(final Document doc) {
|
||||||
|
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
|
||||||
|
if (n != null) {
|
||||||
|
final String id = n.valueOf("./*[local-name()='identifier']");
|
||||||
|
if (StringUtils.isNotBlank(id)) {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return doc.valueOf("//*[local-name()='header']/*[local-name()='identifier']");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
protected Qualifier prepareQualifier(final Node node, final String xpath, final String schemeId) {
|
protected Qualifier prepareQualifier(final Node node, final String xpath, final String schemeId) {
|
||||||
return prepareQualifier(node.valueOf(xpath).trim(), schemeId);
|
return prepareQualifier(node.valueOf(xpath).trim(), schemeId);
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,7 +4,11 @@ package eu.dnetlib.dhp.oa.graph.raw.common;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.function.Predicate;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
|
@ -57,6 +61,7 @@ public class OafMapperUtils {
|
||||||
.stream(values)
|
.stream(values)
|
||||||
.map(v -> field(v, info))
|
.map(v -> field(v, info))
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
|
.filter(distinctByKey(f -> f.getValue()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -65,6 +70,7 @@ public class OafMapperUtils {
|
||||||
.stream()
|
.stream()
|
||||||
.map(v -> field(v, info))
|
.map(v -> field(v, info))
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
|
.filter(distinctByKey(f -> f.getValue()))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -237,4 +243,10 @@ public class OafMapperUtils {
|
||||||
public static String asString(final Object o) {
|
public static String asString(final Object o) {
|
||||||
return o == null ? "" : o.toString();
|
return o == null ? "" : o.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static <T> Predicate<T> distinctByKey(
|
||||||
|
final Function<? super T, ?> keyExtractor) {
|
||||||
|
final Map<Object, Boolean> seen = new ConcurrentHashMap<>();
|
||||||
|
return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,89 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
||||||
|
import org.apache.spark.sql.{Encoder, Encoders}
|
||||||
|
import org.apache.spark.sql.expressions.Aggregator
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
object EBIAggregator {
|
||||||
|
|
||||||
|
def getDatasetAggregator(): Aggregator[(String, OafDataset), OafDataset, OafDataset] = new Aggregator[(String, OafDataset), OafDataset, OafDataset]{
|
||||||
|
|
||||||
|
override def zero: OafDataset = new OafDataset()
|
||||||
|
|
||||||
|
override def reduce(b: OafDataset, a: (String, OafDataset)): OafDataset = {
|
||||||
|
b.mergeFrom(a._2)
|
||||||
|
if (b.getId == null)
|
||||||
|
b.setId(a._2.getId)
|
||||||
|
b
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
override def merge(wx: OafDataset, wy: OafDataset): OafDataset = {
|
||||||
|
wx.mergeFrom(wy)
|
||||||
|
if(wx.getId == null && wy.getId.nonEmpty)
|
||||||
|
wx.setId(wy.getId)
|
||||||
|
wx
|
||||||
|
}
|
||||||
|
override def finish(reduction: OafDataset): OafDataset = reduction
|
||||||
|
|
||||||
|
override def bufferEncoder: Encoder[OafDataset] =
|
||||||
|
Encoders.kryo(classOf[OafDataset])
|
||||||
|
|
||||||
|
override def outputEncoder: Encoder[OafDataset] =
|
||||||
|
Encoders.kryo(classOf[OafDataset])
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def getPublicationAggregator(): Aggregator[(String, Publication), Publication, Publication] = new Aggregator[(String, Publication), Publication, Publication]{
|
||||||
|
|
||||||
|
override def zero: Publication = new Publication()
|
||||||
|
|
||||||
|
override def reduce(b: Publication, a: (String, Publication)): Publication = {
|
||||||
|
b.mergeFrom(a._2)
|
||||||
|
if (b.getId == null)
|
||||||
|
b.setId(a._2.getId)
|
||||||
|
b
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
override def merge(wx: Publication, wy: Publication): Publication = {
|
||||||
|
wx.mergeFrom(wy)
|
||||||
|
if(wx.getId == null && wy.getId.nonEmpty)
|
||||||
|
wx.setId(wy.getId)
|
||||||
|
wx
|
||||||
|
}
|
||||||
|
override def finish(reduction: Publication): Publication = reduction
|
||||||
|
|
||||||
|
override def bufferEncoder: Encoder[Publication] =
|
||||||
|
Encoders.kryo(classOf[Publication])
|
||||||
|
|
||||||
|
override def outputEncoder: Encoder[Publication] =
|
||||||
|
Encoders.kryo(classOf[Publication])
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def getRelationAggregator(): Aggregator[(String, Relation), Relation, Relation] = new Aggregator[(String, Relation), Relation, Relation]{
|
||||||
|
|
||||||
|
override def zero: Relation = new Relation()
|
||||||
|
|
||||||
|
override def reduce(b: Relation, a: (String, Relation)): Relation = {
|
||||||
|
a._2
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
override def merge(a: Relation, b: Relation): Relation = {
|
||||||
|
if(b!= null) b else a
|
||||||
|
}
|
||||||
|
override def finish(reduction: Relation): Relation = reduction
|
||||||
|
|
||||||
|
override def bufferEncoder: Encoder[Relation] =
|
||||||
|
Encoders.kryo(classOf[Relation])
|
||||||
|
|
||||||
|
override def outputEncoder: Encoder[Relation] =
|
||||||
|
Encoders.kryo(classOf[Relation])
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,138 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.{Instance, KeyValue, Oaf}
|
||||||
|
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils.createQualifier
|
||||||
|
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIRelation, OafUtils, ProvenaceInfo}
|
||||||
|
import eu.dnetlib.dhp.utils.DHPUtils
|
||||||
|
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||||
|
import org.apache.commons.io.IOUtils
|
||||||
|
import org.apache.spark.SparkConf
|
||||||
|
import org.apache.spark.sql._
|
||||||
|
import org.json4s
|
||||||
|
import org.json4s.DefaultFormats
|
||||||
|
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||||
|
import org.json4s.jackson.JsonMethods.parse
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
object SparkAddLinkUpdates {
|
||||||
|
|
||||||
|
val relationMapper = RelationMapper.load
|
||||||
|
|
||||||
|
|
||||||
|
case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:String, turl:String, title:String, publisher:String) {}
|
||||||
|
|
||||||
|
|
||||||
|
def generatePubmedDLICollectedFrom(): KeyValue = {
|
||||||
|
OafUtils.generateKeyValue("dli_________::europe_pmc__", "Europe PMC")
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def ebiLinksToOaf(input:(String, String)):List[Oaf] = {
|
||||||
|
val pmid :String = input._1
|
||||||
|
val input_json :String = input._2
|
||||||
|
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||||
|
lazy val json: json4s.JValue = parse(input_json)
|
||||||
|
|
||||||
|
|
||||||
|
val targets:List[EBILinks] = for {
|
||||||
|
JObject(link) <- json \\ "Category" \\ "Link"
|
||||||
|
JField("PublicationDate", JString(pubdate)) <- link
|
||||||
|
JField("RelationshipType", JObject(relationshipType)) <- link
|
||||||
|
JField("Name", JString(relname)) <- relationshipType
|
||||||
|
JField("Target", JObject(target)) <- link
|
||||||
|
JField("Identifier", JObject(identifier)) <- target
|
||||||
|
JField("ID", JString(tpid)) <- identifier
|
||||||
|
JField("IDScheme", JString(tpidtype)) <- identifier
|
||||||
|
JField("IDURL", JString(turl)) <- identifier
|
||||||
|
JField("Title", JString(title)) <- target
|
||||||
|
JField("Publisher", JObject(pub)) <- target
|
||||||
|
JField("Name", JString(publisher)) <- pub
|
||||||
|
} yield EBILinks(relname, pubdate, tpid, tpidtype, turl,title, publisher)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
val dnetPublicationId = s"50|${DHPUtils.md5(s"$pmid::pmid")}"
|
||||||
|
|
||||||
|
targets.flatMap(l => {
|
||||||
|
val relation = new DLIRelation
|
||||||
|
val inverseRelation = new DLIRelation
|
||||||
|
val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}"
|
||||||
|
val relInfo = relationMapper.get(l.relation.toLowerCase)
|
||||||
|
val relationSemantic = relInfo.getOriginal
|
||||||
|
val inverseRelationSemantic = relInfo.getInverse
|
||||||
|
|
||||||
|
relation.setSource(dnetPublicationId)
|
||||||
|
relation.setTarget(targetDnetId)
|
||||||
|
relation.setRelClass("datacite")
|
||||||
|
relation.setRelType(relationSemantic)
|
||||||
|
relation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||||
|
|
||||||
|
inverseRelation.setSource(targetDnetId)
|
||||||
|
inverseRelation.setTarget(dnetPublicationId)
|
||||||
|
inverseRelation.setRelClass("datacite")
|
||||||
|
inverseRelation.setRelType(inverseRelationSemantic)
|
||||||
|
inverseRelation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
val d = new DLIDataset
|
||||||
|
d.setId(targetDnetId)
|
||||||
|
d.setDataInfo(OafUtils.generateDataInfo())
|
||||||
|
d.setPid(List(OafUtils.createSP(l.tpid.toLowerCase.trim, l.tpidType.toLowerCase.trim, "dnet:pid_types")).asJava)
|
||||||
|
d.setCompletionStatus("complete")
|
||||||
|
val pi = new ProvenaceInfo
|
||||||
|
pi.setId("dli_________::europe_pmc__")
|
||||||
|
pi.setName( "Europe PMC")
|
||||||
|
pi.setCompletionStatus("complete")
|
||||||
|
pi.setCollectionMode("collected")
|
||||||
|
d.setDlicollectedfrom(List(pi).asJava)
|
||||||
|
d.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava)
|
||||||
|
d.setPublisher(OafUtils.asField(l.publisher))
|
||||||
|
d.setTitle(List(OafUtils.createSP(l.title, "main title", "dnet:dataCite_title")).asJava)
|
||||||
|
d.setDateofacceptance(OafUtils.asField(l.pubdate))
|
||||||
|
val i = new Instance
|
||||||
|
i.setCollectedfrom(generatePubmedDLICollectedFrom())
|
||||||
|
i.setDateofacceptance(d.getDateofacceptance)
|
||||||
|
i.setUrl(List(l.turl).asJava)
|
||||||
|
i.setInstancetype(createQualifier("0021", "Dataset", "dnet:publication_resource", "dnet:publication_resource"))
|
||||||
|
d.setInstance(List(i).asJava)
|
||||||
|
List(relation, inverseRelation, d)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def main(args: Array[String]): Unit = {
|
||||||
|
val conf: SparkConf = new SparkConf()
|
||||||
|
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val spark: SparkSession =
|
||||||
|
SparkSession
|
||||||
|
.builder()
|
||||||
|
.config(conf)
|
||||||
|
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||||
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
|
||||||
|
val workingPath = parser.get("workingPath")
|
||||||
|
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
|
implicit val relEncoder: Encoder[DLIRelation] = Encoders.kryo(classOf[DLIRelation])
|
||||||
|
implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset])
|
||||||
|
|
||||||
|
val ds:Dataset[(String,String)] = spark.read.load(s"$workingPath/baseline_links_updates").as[(String,String)](Encoders.tuple(Encoders.STRING, Encoders.STRING))
|
||||||
|
|
||||||
|
ds.flatMap(l =>ebiLinksToOaf(l)).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_oaf")
|
||||||
|
|
||||||
|
ds.filter(s => s.isInstanceOf)
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
val oDataset:Dataset[Oaf] = spark.read.load(s"$workingPath/baseline_links_updates_oaf").as[Oaf]
|
||||||
|
|
||||||
|
oDataset.filter(p =>p.isInstanceOf[DLIRelation]).map(p => p.asInstanceOf[DLIRelation]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_relation")
|
||||||
|
oDataset.filter(p =>p.isInstanceOf[DLIDataset]).map(p => p.asInstanceOf[DLIDataset]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_dataset")
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,49 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import org.apache.commons.io.IOUtils
|
||||||
|
import org.apache.spark.SparkConf
|
||||||
|
import org.apache.spark.rdd.RDD
|
||||||
|
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||||
|
import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal, PMParser}
|
||||||
|
|
||||||
|
|
||||||
|
import scala.io.Source
|
||||||
|
import scala.xml.pull.XMLEventReader
|
||||||
|
|
||||||
|
object SparkCreateBaselineDataFrame {
|
||||||
|
|
||||||
|
|
||||||
|
def main(args: Array[String]): Unit = {
|
||||||
|
val conf: SparkConf = new SparkConf()
|
||||||
|
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val spark: SparkSession =
|
||||||
|
SparkSession
|
||||||
|
.builder()
|
||||||
|
.config(conf)
|
||||||
|
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||||
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
val sc = spark.sparkContext
|
||||||
|
|
||||||
|
val workingPath = parser.get("workingPath")
|
||||||
|
|
||||||
|
implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle])
|
||||||
|
implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal])
|
||||||
|
implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor])
|
||||||
|
val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline",2000)
|
||||||
|
|
||||||
|
val ds:Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i =>{
|
||||||
|
val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes()))
|
||||||
|
new PMParser(xml)
|
||||||
|
|
||||||
|
} ))
|
||||||
|
|
||||||
|
ds.write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset")
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,87 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
||||||
|
import eu.dnetlib.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser}
|
||||||
|
import eu.dnetlib.scholexplorer.relation.RelationMapper
|
||||||
|
import org.apache.commons.io.IOUtils
|
||||||
|
import org.apache.spark.SparkConf
|
||||||
|
import org.apache.spark.rdd.RDD
|
||||||
|
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||||
|
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
|
||||||
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
object SparkCreateEBIDataFrame {
|
||||||
|
|
||||||
|
|
||||||
|
def main(args: Array[String]): Unit = {
|
||||||
|
val logger: Logger = LoggerFactory.getLogger(SparkCreateEBIDataFrame.getClass)
|
||||||
|
val conf: SparkConf = new SparkConf()
|
||||||
|
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json")))
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val spark: SparkSession =
|
||||||
|
SparkSession
|
||||||
|
.builder()
|
||||||
|
.config(conf)
|
||||||
|
.appName(SparkCreateEBIDataFrame.getClass.getSimpleName)
|
||||||
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
val sc = spark.sparkContext
|
||||||
|
|
||||||
|
|
||||||
|
val workingPath = parser.get("workingPath")
|
||||||
|
val relationMapper = RelationMapper.load
|
||||||
|
|
||||||
|
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
|
implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset])
|
||||||
|
implicit val pubEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication])
|
||||||
|
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
|
||||||
|
|
||||||
|
logger.info("Extract Publication and relation from publication_xml")
|
||||||
|
val oafPubsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/publication_xml").map(s =>
|
||||||
|
{
|
||||||
|
new ObjectMapper().readValue(s, classOf[String])
|
||||||
|
}).flatMap(s => {
|
||||||
|
val d = new PublicationScholexplorerParser
|
||||||
|
d.parseObject(s, relationMapper).asScala.iterator})
|
||||||
|
|
||||||
|
val mapper = new ObjectMapper()
|
||||||
|
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
||||||
|
spark.createDataset(oafPubsRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/oaf")
|
||||||
|
|
||||||
|
logger.info("Extract Publication and relation from dataset_xml")
|
||||||
|
val oafDatsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/dataset_xml").map(s =>
|
||||||
|
{
|
||||||
|
new ObjectMapper().readValue(s, classOf[String])
|
||||||
|
}).flatMap(s => {
|
||||||
|
val d = new DatasetScholexplorerParser
|
||||||
|
d.parseObject(s, relationMapper).asScala.iterator})
|
||||||
|
|
||||||
|
spark.createDataset(oafDatsRDD).write.mode(SaveMode.Append).save(s"$workingPath/oaf")
|
||||||
|
val dataset: Dataset[OafDataset] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[OafDataset]).map(d => d.asInstanceOf[OafDataset])
|
||||||
|
val publication: Dataset[Publication] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Publication]).map(d => d.asInstanceOf[Publication])
|
||||||
|
val relations: Dataset[Relation] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Relation]).map(d => d.asInstanceOf[Relation])
|
||||||
|
publication.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder))
|
||||||
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
|
.agg(EBIAggregator.getPublicationAggregator().toColumn)
|
||||||
|
.map(p => p._2)
|
||||||
|
.write.mode(SaveMode.Overwrite).save(s"$workingPath/publication")
|
||||||
|
|
||||||
|
dataset.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, datasetEncoder))
|
||||||
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
|
.agg(EBIAggregator.getDatasetAggregator().toColumn)
|
||||||
|
.map(p => p._2)
|
||||||
|
.write.mode(SaveMode.Overwrite).save(s"$workingPath/dataset")
|
||||||
|
|
||||||
|
relations.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder))
|
||||||
|
.groupByKey(_._1)(Encoders.STRING)
|
||||||
|
.agg(EBIAggregator.getRelationAggregator().toColumn)
|
||||||
|
.map(p => p._2)
|
||||||
|
.write.mode(SaveMode.Overwrite).save(s"$workingPath/relation")
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
relations.map(r => (r.getSource, r.getTarget))(Encoders.tuple(Encoders.STRING,Encoders.STRING))
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,64 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.sx.ebi.model;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class PMArticle implements Serializable {
|
||||||
|
|
||||||
|
private String pmid;
|
||||||
|
private String date;
|
||||||
|
private PMJournal journal;
|
||||||
|
private String title;
|
||||||
|
private String description;
|
||||||
|
private List<PMAuthor> authors = new ArrayList<>();
|
||||||
|
|
||||||
|
public String getPmid() {
|
||||||
|
return pmid;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setPmid(String pmid) {
|
||||||
|
this.pmid = pmid;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDate() {
|
||||||
|
return date;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDate(String date) {
|
||||||
|
this.date = date;
|
||||||
|
}
|
||||||
|
|
||||||
|
public PMJournal getJournal() {
|
||||||
|
return journal;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setJournal(PMJournal journal) {
|
||||||
|
this.journal = journal;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTitle() {
|
||||||
|
return title;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTitle(String title) {
|
||||||
|
this.title = title;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDescription() {
|
||||||
|
return description;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDescription(String description) {
|
||||||
|
this.description = description;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<PMAuthor> getAuthors() {
|
||||||
|
return authors;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setAuthors(List<PMAuthor> authors) {
|
||||||
|
this.authors = authors;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,31 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.sx.ebi.model;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
public class PMAuthor implements Serializable {
|
||||||
|
|
||||||
|
private String lastName;
|
||||||
|
private String foreName;
|
||||||
|
|
||||||
|
public String getLastName() {
|
||||||
|
return lastName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setLastName(String lastName) {
|
||||||
|
this.lastName = lastName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getForeName() {
|
||||||
|
return foreName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setForeName(String foreName) {
|
||||||
|
this.foreName = foreName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getFullName() {
|
||||||
|
return String.format("%s, %s", this.foreName, this.lastName);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,53 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.sx.ebi.model;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
public class PMJournal implements Serializable {
|
||||||
|
|
||||||
|
private String issn;
|
||||||
|
private String volume;
|
||||||
|
private String issue;
|
||||||
|
private String date;
|
||||||
|
private String title;
|
||||||
|
|
||||||
|
public String getIssn() {
|
||||||
|
return issn;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIssn(String issn) {
|
||||||
|
this.issn = issn;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getVolume() {
|
||||||
|
return volume;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setVolume(String volume) {
|
||||||
|
this.volume = volume;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getIssue() {
|
||||||
|
return issue;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setIssue(String issue) {
|
||||||
|
this.issue = issue;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getDate() {
|
||||||
|
return date;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setDate(String date) {
|
||||||
|
this.date = date;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getTitle() {
|
||||||
|
return title;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setTitle(String title) {
|
||||||
|
this.title = title;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,92 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi.model
|
||||||
|
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
|
||||||
|
class PMParser(xml:XMLEventReader) extends Iterator[PMArticle] {
|
||||||
|
|
||||||
|
var currentArticle:PMArticle = generateNextArticle()
|
||||||
|
|
||||||
|
override def hasNext: Boolean = currentArticle!= null
|
||||||
|
|
||||||
|
override def next(): PMArticle = {
|
||||||
|
val tmp = currentArticle
|
||||||
|
currentArticle = generateNextArticle()
|
||||||
|
tmp
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def generateNextArticle():PMArticle = {
|
||||||
|
|
||||||
|
var currentAuthor: PMAuthor = null
|
||||||
|
var currentJournal: PMJournal = null
|
||||||
|
var currNode: String = null
|
||||||
|
var currentYear = "0"
|
||||||
|
var currentMonth = "01"
|
||||||
|
var currentDay = "01"
|
||||||
|
|
||||||
|
while (xml.hasNext) {
|
||||||
|
xml.next match {
|
||||||
|
case EvElemStart(_, label, _, _) =>
|
||||||
|
currNode = label
|
||||||
|
label match {
|
||||||
|
case "PubmedArticle" => currentArticle = new PMArticle
|
||||||
|
case "Author" => currentAuthor = new PMAuthor
|
||||||
|
case "Journal" => currentJournal = new PMJournal
|
||||||
|
case _ =>
|
||||||
|
}
|
||||||
|
case EvElemEnd(_, label) =>
|
||||||
|
label match {
|
||||||
|
case "PubmedArticle" => return currentArticle
|
||||||
|
case "Author" => currentArticle.getAuthors.add(currentAuthor)
|
||||||
|
case "Journal" => currentArticle.setJournal(currentJournal)
|
||||||
|
case "DateCompleted" => currentArticle.setDate(s"$currentYear-$currentMonth-$currentDay")
|
||||||
|
case "PubDate" => currentJournal.setDate(s"$currentYear-$currentMonth-$currentDay")
|
||||||
|
case _ =>
|
||||||
|
}
|
||||||
|
case EvText(text) =>
|
||||||
|
if (currNode!= null && text.trim.nonEmpty)
|
||||||
|
currNode match {
|
||||||
|
case "ArticleTitle" => {
|
||||||
|
if (currentArticle.getTitle==null)
|
||||||
|
currentArticle.setTitle(text.trim)
|
||||||
|
else
|
||||||
|
currentArticle.setTitle(currentArticle.getTitle + text.trim)
|
||||||
|
}
|
||||||
|
case "AbstractText" => {
|
||||||
|
if (currentArticle.getDescription==null)
|
||||||
|
currentArticle.setDescription(text.trim)
|
||||||
|
else
|
||||||
|
currentArticle.setDescription(currentArticle.getDescription + text.trim)
|
||||||
|
}
|
||||||
|
case "PMID" => currentArticle.setPmid(text.trim)
|
||||||
|
case "ISSN" => currentJournal.setIssn(text.trim)
|
||||||
|
case "Year" => currentYear = text.trim
|
||||||
|
case "Month" => currentMonth = text.trim
|
||||||
|
case "Day" => currentDay = text.trim
|
||||||
|
case "Volume" => currentJournal.setVolume( text.trim)
|
||||||
|
case "Issue" => currentJournal.setIssue (text.trim)
|
||||||
|
case "LastName" => {
|
||||||
|
if (currentAuthor != null)
|
||||||
|
currentAuthor.setLastName(text.trim)
|
||||||
|
|
||||||
|
}
|
||||||
|
case "ForeName" => if (currentAuthor != null)
|
||||||
|
currentAuthor.setForeName(text.trim)
|
||||||
|
case "Title" =>
|
||||||
|
if (currentJournal.getTitle==null)
|
||||||
|
currentJournal.setTitle(text.trim)
|
||||||
|
else
|
||||||
|
currentJournal.setTitle(currentJournal.getTitle + text.trim)
|
||||||
|
case _ =>
|
||||||
|
|
||||||
|
}
|
||||||
|
case _ =>
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
null
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -150,6 +150,17 @@ public abstract class AbstractScholexplorerParser {
|
||||||
return uk;
|
return uk;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected Qualifier generateQualifier(final String classId, final String className, final String schemeId,
|
||||||
|
final String schemeName) {
|
||||||
|
final Qualifier q = new Qualifier();
|
||||||
|
q.setClassid(classId);
|
||||||
|
q.setClassid(className);
|
||||||
|
q.setSchemeid(schemeId);
|
||||||
|
q.setSchemename(schemeName);
|
||||||
|
return q;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
protected void generateRelations(
|
protected void generateRelations(
|
||||||
RelationMapper relationMapper,
|
RelationMapper relationMapper,
|
||||||
Result parsedObject,
|
Result parsedObject,
|
||||||
|
|
|
@ -64,7 +64,6 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
|
||||||
currentDate.setQualifier(dateQualifier);
|
currentDate.setQualifier(dateQualifier);
|
||||||
parsedObject.setRelevantdate(Collections.singletonList(currentDate));
|
parsedObject.setRelevantdate(Collections.singletonList(currentDate));
|
||||||
}
|
}
|
||||||
|
|
||||||
final String completionStatus = VtdUtilityParser
|
final String completionStatus = VtdUtilityParser
|
||||||
.getSingleValue(ap, vn, "//*[local-name()='completionStatus']");
|
.getSingleValue(ap, vn, "//*[local-name()='completionStatus']");
|
||||||
final String provisionMode = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']");
|
final String provisionMode = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']");
|
||||||
|
@ -149,6 +148,37 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
|
||||||
inferPid(currentPid);
|
inferPid(currentPid);
|
||||||
parsedObject.setPid(Collections.singletonList(currentPid));
|
parsedObject.setPid(Collections.singletonList(currentPid));
|
||||||
|
|
||||||
|
String resolvedURL = null;
|
||||||
|
|
||||||
|
switch (currentPid.getQualifier().getClassname().toLowerCase()) {
|
||||||
|
case "uniprot":
|
||||||
|
resolvedURL = "https://www.uniprot.org/uniprot/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
case "ena":
|
||||||
|
if (StringUtils.isNotBlank(currentPid.getValue()) && currentPid.getValue().length() > 7)
|
||||||
|
resolvedURL = "https://www.ebi.ac.uk/ena/data/view/" + currentPid.getValue().substring(0, 8);
|
||||||
|
break;
|
||||||
|
case "chembl":
|
||||||
|
resolvedURL = "https://www.ebi.ac.uk/chembl/compound_report_card/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
|
||||||
|
case "ncbi-n":
|
||||||
|
resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
case "ncbi-p":
|
||||||
|
resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
case "genbank":
|
||||||
|
resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
case "pdb":
|
||||||
|
resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue();
|
||||||
|
break;
|
||||||
|
case "url":
|
||||||
|
resolvedURL = currentPid.getValue();
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
final String sourceId = generateId(
|
final String sourceId = generateId(
|
||||||
currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset");
|
currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset");
|
||||||
parsedObject.setId(sourceId);
|
parsedObject.setId(sourceId);
|
||||||
|
@ -251,6 +281,11 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
|
||||||
t -> {
|
t -> {
|
||||||
final StructuredProperty st = new StructuredProperty();
|
final StructuredProperty st = new StructuredProperty();
|
||||||
st.setValue(t);
|
st.setValue(t);
|
||||||
|
st
|
||||||
|
.setQualifier(
|
||||||
|
generateQualifier(
|
||||||
|
"main title", "main title", "dnet:dataCite_title",
|
||||||
|
"dnet:dataCite_title"));
|
||||||
return st;
|
return st;
|
||||||
})
|
})
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
|
@ -282,6 +317,13 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (StringUtils.isNotBlank(resolvedURL)) {
|
||||||
|
Instance i = new Instance();
|
||||||
|
i.setCollectedfrom(parsedObject.getCollectedfrom().get(0));
|
||||||
|
i.setUrl(Collections.singletonList(resolvedURL));
|
||||||
|
parsedObject.setInstance(Collections.singletonList(i));
|
||||||
|
}
|
||||||
|
|
||||||
result.add(parsedObject);
|
result.add(parsedObject);
|
||||||
return result;
|
return result;
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
|
|
|
@ -202,6 +202,11 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
|
||||||
t -> {
|
t -> {
|
||||||
final StructuredProperty st = new StructuredProperty();
|
final StructuredProperty st = new StructuredProperty();
|
||||||
st.setValue(t);
|
st.setValue(t);
|
||||||
|
st
|
||||||
|
.setQualifier(
|
||||||
|
generateQualifier(
|
||||||
|
"main title", "main title", "dnet:dataCite_title",
|
||||||
|
"dnet:dataCite_title"));
|
||||||
return st;
|
return st;
|
||||||
})
|
})
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toList()));
|
||||||
|
|
|
@ -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>
|
|
@ -0,0 +1,293 @@
|
||||||
|
<workflow-app name="merge graphs" xmlns="uri:oozie:workflow:0.5">
|
||||||
|
|
||||||
|
<parameters>
|
||||||
|
<property>
|
||||||
|
<name>betaInputGgraphPath</name>
|
||||||
|
<description>the beta graph root path</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>prodInputGgraphPath</name>
|
||||||
|
<description>the production graph root path</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>graphOutputPath</name>
|
||||||
|
<description>the output merged graph root path</description>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>priority</name>
|
||||||
|
<description>decides from which infrastructure the content must win in case of ID clash</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>
|
||||||
|
|
||||||
|
<start to="fork_merge_graph"/>
|
||||||
|
|
||||||
|
<kill name="Kill">
|
||||||
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
|
</kill>
|
||||||
|
|
||||||
|
<fork name="fork_merge_graph">
|
||||||
|
<path start="merge_publication"/>
|
||||||
|
<path start="merge_dataset"/>
|
||||||
|
<path start="merge_otherresearchproduct"/>
|
||||||
|
<path start="merge_software"/>
|
||||||
|
<path start="merge_datasource"/>
|
||||||
|
<path start="merge_organization"/>
|
||||||
|
<path start="merge_project"/>
|
||||||
|
<path start="merge_relation"/>
|
||||||
|
</fork>
|
||||||
|
|
||||||
|
<action name="merge_publication">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge publications</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/publication</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/publication</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/publication</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_dataset">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge datasets</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/dataset</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/dataset</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/dataset</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_otherresearchproduct">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge otherresearchproducts</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/otherresearchproduct</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/otherresearchproduct</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_software">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge softwares</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/software</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/software</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/software</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_datasource">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge datasources</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/datasource</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/datasource</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/datasource</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_organization">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge organizations</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/organization</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/organization</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/organization</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_project">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge projects</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/project</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/project</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/project</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="merge_relation">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Merge relations</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob</class>
|
||||||
|
<jar>dhp-graph-mapper-${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=7680
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--betaInputPath</arg><arg>${betaInputGgraphPath}/relation</arg>
|
||||||
|
<arg>--prodInputPath</arg><arg>${prodInputGgraphPath}/relation</arg>
|
||||||
|
<arg>--outputPath</arg><arg>${graphOutputPath}/relation</arg>
|
||||||
|
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>
|
||||||
|
<arg>--priority</arg><arg>${priority}</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="wait_merge"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<join name="wait_merge" to="End"/>
|
||||||
|
|
||||||
|
<end name="End"/>
|
||||||
|
</workflow-app>
|
|
@ -0,0 +1,38 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "issm",
|
||||||
|
"paramLongName": "isSparkSessionManaged",
|
||||||
|
"paramDescription": "when true will stop SparkSession after job execution",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "bin",
|
||||||
|
"paramLongName": "betaInputPath",
|
||||||
|
"paramDescription": "the beta graph root path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "pin",
|
||||||
|
"paramLongName": "prodInputPath",
|
||||||
|
"paramDescription": "the production graph root path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "out",
|
||||||
|
"paramLongName": "outputPath",
|
||||||
|
"paramDescription": "the output merged graph root path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "class",
|
||||||
|
"paramLongName": "graphTableClassName",
|
||||||
|
"paramDescription": "class name moelling the graph table",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "pr",
|
||||||
|
"paramLongName": "priority",
|
||||||
|
"paramDescription": "decides from which infrastructure the content must win in case of ID clash",
|
||||||
|
"paramRequired": false
|
||||||
|
}
|
||||||
|
]
|
|
@ -16,6 +16,11 @@
|
||||||
<name>postgresPassword</name>
|
<name>postgresPassword</name>
|
||||||
<description>the password postgres</description>
|
<description>the password postgres</description>
|
||||||
</property>
|
</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>
|
||||||
<name>isLookupUrl</name>
|
<name>isLookupUrl</name>
|
||||||
<description>the address of the lookUp service</description>
|
<description>the address of the lookUp service</description>
|
||||||
|
@ -93,6 +98,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="ImportDB_claims"/>
|
<ok to="ImportDB_claims"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -109,6 +115,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>
|
||||||
<arg>--action</arg><arg>claims</arg>
|
<arg>--action</arg><arg>claims</arg>
|
||||||
</java>
|
</java>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
|
|
|
@ -0,0 +1,4 @@
|
||||||
|
[
|
||||||
|
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||||
|
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true}
|
||||||
|
]
|
|
@ -0,0 +1,68 @@
|
||||||
|
<configuration>
|
||||||
|
|
||||||
|
<!-- OCEAN -->
|
||||||
|
<!--
|
||||||
|
<property>
|
||||||
|
<name>jobTracker</name>
|
||||||
|
<value>yarnRM</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>nameNode</name>
|
||||||
|
<value>hdfs://nameservice1</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hive_metastore_uris</name>
|
||||||
|
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2YarnHistoryServerAddress</name>
|
||||||
|
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
|
||||||
|
</property>
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!-- GARR -->
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>jobTracker</name>
|
||||||
|
<value>yarn</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>nameNode</name>
|
||||||
|
<value>hdfs://hadoop-rm1.garr-pa1.d4science.org:8020</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>hive_metastore_uris</name>
|
||||||
|
<value>thrift://hadoop-edge3.garr-pa1.d4science.org:9083</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2YarnHistoryServerAddress</name>
|
||||||
|
<value>http://hadoop-rm2.garr-pa1.d4science.org:19888</value>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||||
|
<value>true</value>
|
||||||
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>oozie.use.system.libpath</name>
|
||||||
|
<value>true</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>oozie.action.sharelib.for.spark</name>
|
||||||
|
<value>spark2</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2EventLogDir</name>
|
||||||
|
<value>/user/spark/spark2ApplicationHistory</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2ExtraListeners</name>
|
||||||
|
<value>"com.cloudera.spark.lineage.NavigatorAppListener"</value>
|
||||||
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>spark2SqlQueryExecutionListeners</name>
|
||||||
|
<value>"com.cloudera.spark.lineage.NavigatorQueryListener"</value>
|
||||||
|
</property>
|
||||||
|
</configuration>
|
|
@ -0,0 +1,97 @@
|
||||||
|
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
|
||||||
|
<parameters>
|
||||||
|
<property>
|
||||||
|
<name>workingPath</name>
|
||||||
|
<description>the Working Path</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>
|
||||||
|
</parameters>
|
||||||
|
|
||||||
|
<start to="GenerateUpdates"/>
|
||||||
|
|
||||||
|
|
||||||
|
<kill name="Kill">
|
||||||
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
|
</kill>
|
||||||
|
|
||||||
|
|
||||||
|
<action name="GenerateBaselineDataset">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn-cluster</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Create Baselnie DataSet</name>
|
||||||
|
|
||||||
|
<class>eu.dnetlib.dhp.sx.ebi.SparkCreateBaselineDataFrame</class>
|
||||||
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=1
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
${sparkExtraOPT}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="GenerateUpdates">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn-cluster</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Create Baselnie DataSet</name>
|
||||||
|
|
||||||
|
<class>eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates</class>
|
||||||
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=1
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
${sparkExtraOPT}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
|
||||||
|
<action name="CreateEBIDataSet">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn-cluster</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Create EBI DataSet</name>
|
||||||
|
|
||||||
|
<class>eu.dnetlib.dhp.sx.ebi.SparkCreateEBIDataFrame</class>
|
||||||
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.sql.shuffle.partitions=1000
|
||||||
|
${sparkExtraOPT}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
|
</spark>
|
||||||
|
<ok to="End"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<end name="End"/>
|
||||||
|
</workflow-app>
|
|
@ -57,6 +57,8 @@ public class CleaningFunctionTest {
|
||||||
String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result.json"));
|
String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result.json"));
|
||||||
Publication p_in = MAPPER.readValue(json, Publication.class);
|
Publication p_in = MAPPER.readValue(json, Publication.class);
|
||||||
|
|
||||||
|
assertNull(p_in.getBestaccessright());
|
||||||
|
|
||||||
assertTrue(p_in instanceof Result);
|
assertTrue(p_in instanceof Result);
|
||||||
assertTrue(p_in instanceof Publication);
|
assertTrue(p_in instanceof Publication);
|
||||||
|
|
||||||
|
@ -84,6 +86,9 @@ public class CleaningFunctionTest {
|
||||||
.map(p -> p.getQualifier())
|
.map(p -> p.getQualifier())
|
||||||
.allMatch(q -> pidTerms.contains(q.getClassid())));
|
.allMatch(q -> pidTerms.contains(q.getClassid())));
|
||||||
|
|
||||||
|
Publication p_defaults = CleanGraphSparkJob.fixDefaults(p_out);
|
||||||
|
assertEquals("CLOSED", p_defaults.getBestaccessright().getClassid());
|
||||||
|
|
||||||
// TODO add more assertions to verity the cleaned values
|
// TODO add more assertions to verity the cleaned values
|
||||||
System.out.println(MAPPER.writeValueAsString(p_out));
|
System.out.println(MAPPER.writeValueAsString(p_out));
|
||||||
|
|
||||||
|
|
|
@ -6,6 +6,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.anyString;
|
import static org.mockito.ArgumentMatchers.anyString;
|
||||||
|
import static org.mockito.Mockito.lenient;
|
||||||
import static org.mockito.Mockito.when;
|
import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -20,6 +21,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
|
||||||
import org.mockito.Mock;
|
import org.mockito.Mock;
|
||||||
import org.mockito.junit.jupiter.MockitoExtension;
|
import org.mockito.junit.jupiter.MockitoExtension;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils;
|
import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils;
|
||||||
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
import eu.dnetlib.dhp.schema.common.ModelConstants;
|
||||||
|
@ -31,24 +33,25 @@ import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||||
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;
|
||||||
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
|
||||||
@ExtendWith(MockitoExtension.class)
|
@ExtendWith(MockitoExtension.class)
|
||||||
public class MappersTest {
|
public class MappersTest {
|
||||||
|
|
||||||
|
@Mock
|
||||||
|
private ISLookUpService isLookUpService;
|
||||||
|
|
||||||
@Mock
|
@Mock
|
||||||
private VocabularyGroup vocs;
|
private VocabularyGroup vocs;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
when(vocs.getTermAsQualifier(anyString(), anyString()))
|
lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs());
|
||||||
.thenAnswer(
|
lenient()
|
||||||
invocation -> OafMapperUtils
|
.when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY))
|
||||||
.qualifier(
|
.thenReturn(synonyms());
|
||||||
invocation.getArgument(1), invocation.getArgument(1), invocation.getArgument(0),
|
|
||||||
invocation.getArgument(0)));
|
|
||||||
|
|
||||||
when(vocs.termExists(anyString(), anyString())).thenReturn(true);
|
|
||||||
|
|
||||||
|
vocs = VocabularyGroup.loadVocsFromIS(isLookUpService);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -68,9 +71,14 @@ public class MappersTest {
|
||||||
final Relation r2 = (Relation) list.get(2);
|
final Relation r2 = (Relation) list.get(2);
|
||||||
|
|
||||||
assertValidId(p.getId());
|
assertValidId(p.getId());
|
||||||
|
|
||||||
|
assertTrue(p.getOriginalId().size() == 1);
|
||||||
|
assertEquals("10.3897/oneeco.2.e13718", p.getOriginalId().get(0));
|
||||||
|
|
||||||
assertValidId(p.getCollectedfrom().get(0).getKey());
|
assertValidId(p.getCollectedfrom().get(0).getKey());
|
||||||
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
|
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
|
||||||
assertFalse(p.getDataInfo().getInvisible());
|
assertFalse(p.getDataInfo().getInvisible());
|
||||||
|
assertTrue(p.getSource().size() == 1);
|
||||||
|
|
||||||
assertTrue(p.getAuthor().size() > 0);
|
assertTrue(p.getAuthor().size() > 0);
|
||||||
final Optional<Author> author = p
|
final Optional<Author> author = p
|
||||||
|
@ -79,6 +87,7 @@ public class MappersTest {
|
||||||
.filter(a -> a.getPid() != null && !a.getPid().isEmpty())
|
.filter(a -> a.getPid() != null && !a.getPid().isEmpty())
|
||||||
.findFirst();
|
.findFirst();
|
||||||
assertTrue(author.isPresent());
|
assertTrue(author.isPresent());
|
||||||
|
|
||||||
final StructuredProperty pid = author
|
final StructuredProperty pid = author
|
||||||
.get()
|
.get()
|
||||||
.getPid()
|
.getPid()
|
||||||
|
@ -169,6 +178,8 @@ public class MappersTest {
|
||||||
final Relation r2 = (Relation) list.get(2);
|
final Relation r2 = (Relation) list.get(2);
|
||||||
|
|
||||||
assertValidId(d.getId());
|
assertValidId(d.getId());
|
||||||
|
assertTrue(d.getOriginalId().size() == 1);
|
||||||
|
assertEquals("oai:zenodo.org:3234526", d.getOriginalId().get(0));
|
||||||
assertValidId(d.getCollectedfrom().get(0).getKey());
|
assertValidId(d.getCollectedfrom().get(0).getKey());
|
||||||
assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue()));
|
assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue()));
|
||||||
assertTrue(d.getAuthor().size() > 0);
|
assertTrue(d.getAuthor().size() > 0);
|
||||||
|
@ -261,4 +272,15 @@ public class MappersTest {
|
||||||
assertEquals(':', id.charAt(15));
|
assertEquals(':', id.charAt(15));
|
||||||
assertEquals(':', id.charAt(16));
|
assertEquals(':', id.charAt(16));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private List<String> vocs() throws IOException {
|
||||||
|
return IOUtils
|
||||||
|
.readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<String> synonyms() throws IOException {
|
||||||
|
return IOUtils
|
||||||
|
.readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt"));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,20 @@
|
||||||
|
package eu.dnetlib.dhp.sx.ebi
|
||||||
|
|
||||||
|
import org.junit.jupiter.api.Test
|
||||||
|
|
||||||
|
class TestEBI {
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testEBIData() = {
|
||||||
|
SparkAddLinkUpdates.main("-mt local[*] -w /home/sandro/Downloads".split(" "))
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -185,12 +185,7 @@
|
||||||
"surname": ""
|
"surname": ""
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"bestaccessright": {
|
"bestaccessright": null,
|
||||||
"classid": "CLOSED",
|
|
||||||
"classname": "Closed Access",
|
|
||||||
"schemeid": "dnet:access_modes",
|
|
||||||
"schemename": "dnet:access_modes"
|
|
||||||
},
|
|
||||||
"collectedfrom": [
|
"collectedfrom": [
|
||||||
{
|
{
|
||||||
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
|
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
|
||||||
|
|
|
@ -34,6 +34,8 @@
|
||||||
<dc:relation>info:eu-repo/semantics/altIdentifier/eissn/2367-8194</dc:relation>
|
<dc:relation>info:eu-repo/semantics/altIdentifier/eissn/2367-8194</dc:relation>
|
||||||
<dc:relation>info:eu-repo/grantAgreement/EC/FP7/226852</dc:relation>
|
<dc:relation>info:eu-repo/grantAgreement/EC/FP7/226852</dc:relation>
|
||||||
<dc:source>One Ecosystem 2: e13718</dc:source>
|
<dc:source>One Ecosystem 2: e13718</dc:source>
|
||||||
|
<dc:source>One Ecosystem 2: e13718</dc:source>
|
||||||
|
<dc:source>One Ecosystem 2: e13718</dc:source>
|
||||||
<dc:subject>Ecosystem Services hotspots</dc:subject>
|
<dc:subject>Ecosystem Services hotspots</dc:subject>
|
||||||
<dc:subject>Natura 2000</dc:subject>
|
<dc:subject>Natura 2000</dc:subject>
|
||||||
<dc:subject>Quiet Protected Areas</dc:subject>
|
<dc:subject>Quiet Protected Areas</dc:subject>
|
||||||
|
@ -47,7 +49,8 @@
|
||||||
<dc:subject>regulating services</dc:subject>
|
<dc:subject>regulating services</dc:subject>
|
||||||
<dc:subject>supporting services</dc:subject>
|
<dc:subject>supporting services</dc:subject>
|
||||||
<dc:type>Research Article</dc:type>
|
<dc:type>Research Article</dc:type>
|
||||||
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
|
<!--<dr:CobjCategory type="publication">0001</dr:CobjCategory>-->
|
||||||
|
<dr:CobjCategory>0001</dr:CobjCategory>
|
||||||
<oaf:dateAccepted>2017-01-01</oaf:dateAccepted>
|
<oaf:dateAccepted>2017-01-01</oaf:dateAccepted>
|
||||||
<oaf:projectid>corda_______::226852</oaf:projectid>
|
<oaf:projectid>corda_______::226852</oaf:projectid>
|
||||||
<oaf:accessrights>OPEN</oaf:accessrights>
|
<oaf:accessrights>OPEN</oaf:accessrights>
|
||||||
|
|
|
@ -82,7 +82,8 @@
|
||||||
<p>All files are in MATLAB .mat format.</p></description>
|
<p>All files are in MATLAB .mat format.</p></description>
|
||||||
</descriptions>
|
</descriptions>
|
||||||
</resource>
|
</resource>
|
||||||
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
|
<!--<dr:CobjCategory type="dataset">0021</dr:CobjCategory>-->
|
||||||
|
<dr:CobjCategory>0021</dr:CobjCategory>
|
||||||
<oaf:dateAccepted>2019-01-01</oaf:dateAccepted>
|
<oaf:dateAccepted>2019-01-01</oaf:dateAccepted>
|
||||||
<oaf:accessrights>OPEN</oaf:accessrights>
|
<oaf:accessrights>OPEN</oaf:accessrights>
|
||||||
<oaf:language>und</oaf:language>
|
<oaf:language>und</oaf:language>
|
||||||
|
|
|
@ -52,7 +52,8 @@
|
||||||
subjectScheme="EDAM Ontology" valueURI="http://edamontology.org/topic_3534">Protein binding sites</datacite:subject>
|
subjectScheme="EDAM Ontology" valueURI="http://edamontology.org/topic_3534">Protein binding sites</datacite:subject>
|
||||||
</datacite:subjects>
|
</datacite:subjects>
|
||||||
</datacite:resource>
|
</datacite:resource>
|
||||||
<dr:CobjCategory type="software">0029</dr:CobjCategory>
|
<!--<dr:CobjCategory type="software">0029</dr:CobjCategory>-->
|
||||||
|
<dr:CobjCategory>0029</dr:CobjCategory>
|
||||||
<oaf:hostedBy id="rest________::bioTools" name="bio.tools"/>
|
<oaf:hostedBy id="rest________::bioTools" name="bio.tools"/>
|
||||||
<oaf:collectedFrom id="rest________::bioTools" name="bio.tools"/>
|
<oaf:collectedFrom id="rest________::bioTools" name="bio.tools"/>
|
||||||
<oaf:dateAccepted>2018-06-06</oaf:dateAccepted>
|
<oaf:dateAccepted>2018-06-06</oaf:dateAccepted>
|
||||||
|
|
|
@ -0,0 +1,55 @@
|
||||||
|
{
|
||||||
|
"Category": [
|
||||||
|
{
|
||||||
|
"Section": [
|
||||||
|
{
|
||||||
|
"Linklist": {
|
||||||
|
"Link": [
|
||||||
|
{
|
||||||
|
"LinkProvider": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
},
|
||||||
|
"Target": {
|
||||||
|
"Publisher": {
|
||||||
|
"Name": "Altmetric"
|
||||||
|
},
|
||||||
|
"ImageURL": "https://api.altmetric.com/v1/donut/58578459_64.png",
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "https://www.altmetric.com/details/58578459",
|
||||||
|
"IDScheme": "URL",
|
||||||
|
"IDURL": "https://www.altmetric.com/details/58578459"
|
||||||
|
},
|
||||||
|
"Type": {
|
||||||
|
"Name": "dataset"
|
||||||
|
},
|
||||||
|
"Title": "Optical clumped isotope thermometry of carbon dioxide"
|
||||||
|
},
|
||||||
|
"Source": {
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "30886173",
|
||||||
|
"IDScheme": "PMID"
|
||||||
|
},
|
||||||
|
"Type": {
|
||||||
|
"Name": "literature"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"PublicationDate": "06-04-2019",
|
||||||
|
"RelationshipType": {
|
||||||
|
"Name": "IsReferencedBy"
|
||||||
|
},
|
||||||
|
"ObtainedBy": "ext_links"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
},
|
||||||
|
"ObtainedBy": "ext_links",
|
||||||
|
"SectionLinkCount": 1,
|
||||||
|
"Tags": [
|
||||||
|
"altmetrics"
|
||||||
|
]
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"CategoryLinkCount": 1,
|
||||||
|
"Name": "Altmetric"
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
|
@ -0,0 +1,191 @@
|
||||||
|
{
|
||||||
|
"version": "6.3",
|
||||||
|
"hitCount": 4,
|
||||||
|
"request": {
|
||||||
|
"id": "28818901",
|
||||||
|
"source": "MED"
|
||||||
|
},
|
||||||
|
"dataLinkList": {
|
||||||
|
"Category": [
|
||||||
|
{
|
||||||
|
"Name": "Nucleotide Sequences",
|
||||||
|
"CategoryLinkCount": 3,
|
||||||
|
"Section": [
|
||||||
|
{
|
||||||
|
"ObtainedBy": "tm_accession",
|
||||||
|
"Tags": [
|
||||||
|
"supporting_data"
|
||||||
|
],
|
||||||
|
"SectionLinkCount": 1,
|
||||||
|
"Linklist": {
|
||||||
|
"Link": [
|
||||||
|
{
|
||||||
|
"ObtainedBy": "tm_accession",
|
||||||
|
"PublicationDate": "27-02-2020",
|
||||||
|
"LinkProvider": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
},
|
||||||
|
"RelationshipType": {
|
||||||
|
"Name": "References"
|
||||||
|
},
|
||||||
|
"Source": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "literature"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "28818901",
|
||||||
|
"IDScheme": "MED"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"Target": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "dataset"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "AP008937",
|
||||||
|
"IDScheme": "ENA",
|
||||||
|
"IDURL": "http://identifiers.org/ena.embl/AP008937"
|
||||||
|
},
|
||||||
|
"Title": "AP008937",
|
||||||
|
"Publisher": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"Frequency": 1
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"ObtainedBy": "submission",
|
||||||
|
"Tags": [
|
||||||
|
"related_data"
|
||||||
|
],
|
||||||
|
"SectionLinkCount": 2,
|
||||||
|
"CollectionURL": "http://www.ebi.ac.uk/ena/data/search?query=28818901",
|
||||||
|
"Linklist": {
|
||||||
|
"Link": [
|
||||||
|
{
|
||||||
|
"ObtainedBy": "submission",
|
||||||
|
"PublicationDate": "25-06-2018",
|
||||||
|
"LinkProvider": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
},
|
||||||
|
"RelationshipType": {
|
||||||
|
"Name": "IsReferencedBy"
|
||||||
|
},
|
||||||
|
"Source": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "literature"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "28818901",
|
||||||
|
"IDScheme": "PMID"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"Target": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "dataset"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "NIWV01000000",
|
||||||
|
"IDScheme": "ENA",
|
||||||
|
"IDURL": "http://www.ebi.ac.uk/ena/data/view/NIWV01000000"
|
||||||
|
},
|
||||||
|
"Title": "Nucleotide sequences",
|
||||||
|
"Publisher": {
|
||||||
|
"Name": "ENA"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"ObtainedBy": "submission",
|
||||||
|
"PublicationDate": "25-06-2018",
|
||||||
|
"LinkProvider": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
},
|
||||||
|
"RelationshipType": {
|
||||||
|
"Name": "IsReferencedBy"
|
||||||
|
},
|
||||||
|
"Source": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "literature"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "28818901",
|
||||||
|
"IDScheme": "PMID"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"Target": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "dataset"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "PRJNA390617",
|
||||||
|
"IDScheme": "ENA",
|
||||||
|
"IDURL": "http://www.ebi.ac.uk/ena/data/view/PRJNA390617"
|
||||||
|
},
|
||||||
|
"Title": "Lactobacillus fermentum strain:BFE 6620",
|
||||||
|
"Publisher": {
|
||||||
|
"Name": "ENA"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"Name": "BioStudies: supplemental material and supporting data",
|
||||||
|
"CategoryLinkCount": 1,
|
||||||
|
"Section": [
|
||||||
|
{
|
||||||
|
"ObtainedBy": "ext_links",
|
||||||
|
"Tags": [
|
||||||
|
"supporting_data"
|
||||||
|
],
|
||||||
|
"SectionLinkCount": 1,
|
||||||
|
"Linklist": {
|
||||||
|
"Link": [
|
||||||
|
{
|
||||||
|
"ObtainedBy": "ext_links",
|
||||||
|
"PublicationDate": "24-07-2018",
|
||||||
|
"LinkProvider": {
|
||||||
|
"Name": "Europe PMC"
|
||||||
|
},
|
||||||
|
"RelationshipType": {
|
||||||
|
"Name": "IsReferencedBy"
|
||||||
|
},
|
||||||
|
"Source": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "literature"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "28818901",
|
||||||
|
"IDScheme": "PMID"
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"Target": {
|
||||||
|
"Type": {
|
||||||
|
"Name": "dataset"
|
||||||
|
},
|
||||||
|
"Identifier": {
|
||||||
|
"ID": "http://www.ebi.ac.uk/biostudies/studies/S-EPMC5604774?xr=true",
|
||||||
|
"IDScheme": "URL",
|
||||||
|
"IDURL": "http://www.ebi.ac.uk/biostudies/studies/S-EPMC5604774?xr=true"
|
||||||
|
},
|
||||||
|
"Title": "Draft Genome Sequence of Lactobacillus fermentum BFE 6620, a Potential Starter Culture for African Vegetable Foods, Isolated from Fermented Cassava.",
|
||||||
|
"Publisher": {
|
||||||
|
"Name": "BioStudies: supplemental material and supporting data"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
|
@ -5,11 +5,12 @@ import java.time.format.DateTimeFormatter
|
||||||
|
|
||||||
import eu.dnetlib.dhp.common.PacePerson
|
import eu.dnetlib.dhp.common.PacePerson
|
||||||
import eu.dnetlib.dhp.schema.action.AtomicAction
|
import eu.dnetlib.dhp.schema.action.AtomicAction
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, StructuredProperty}
|
import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty}
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||||
import eu.dnetlib.dhp.utils.DHPUtils
|
import eu.dnetlib.dhp.utils.DHPUtils
|
||||||
import org.apache.commons.lang3.StringUtils
|
import org.apache.commons.lang3.StringUtils
|
||||||
import org.codehaus.jackson.map.ObjectMapper
|
import org.codehaus.jackson.map.ObjectMapper
|
||||||
|
import eu.dnetlib.dhp.schema.scholexplorer.OafUtils._
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
@ -99,6 +100,20 @@ object DLIToOAF {
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
def fixInstance(r:Publication) :Publication = {
|
||||||
|
val collectedFrom = r.getCollectedfrom.asScala.head
|
||||||
|
r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom))
|
||||||
|
r
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def fixInstanceDataset(r:Dataset) :Dataset = {
|
||||||
|
val collectedFrom = r.getCollectedfrom.asScala.head
|
||||||
|
r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom))
|
||||||
|
r
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
def toActionSet(item: Oaf): (String, String) = {
|
def toActionSet(item: Oaf): (String, String) = {
|
||||||
val mapper = new ObjectMapper()
|
val mapper = new ObjectMapper()
|
||||||
|
|
||||||
|
@ -412,46 +427,6 @@ object DLIToOAF {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def generateKeyValue(key: String, value: String): KeyValue = {
|
|
||||||
val kv: KeyValue = new KeyValue()
|
|
||||||
kv.setKey(key)
|
|
||||||
kv.setValue(value)
|
|
||||||
kv.setDataInfo(generateDataInfo("0.9"))
|
|
||||||
kv
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def generateDataInfo(trust: String = "0.9", invisibile: Boolean = false): DataInfo = {
|
|
||||||
val di = new DataInfo
|
|
||||||
di.setDeletedbyinference(false)
|
|
||||||
di.setInferred(false)
|
|
||||||
di.setInvisible(false)
|
|
||||||
di.setTrust(trust)
|
|
||||||
di.setProvenanceaction(createQualifier("sysimport:actionset", "dnet:provenanceActions"))
|
|
||||||
di
|
|
||||||
}
|
|
||||||
|
|
||||||
def createQualifier(cls: String, sch: String): Qualifier = {
|
|
||||||
createQualifier(cls, cls, sch, sch)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def createQualifier(classId: String, className: String, schemeId: String, schemeName: String): Qualifier = {
|
|
||||||
val q: Qualifier = new Qualifier
|
|
||||||
q.setClassid(classId)
|
|
||||||
q.setClassname(className)
|
|
||||||
q.setSchemeid(schemeId)
|
|
||||||
q.setSchemename(schemeName)
|
|
||||||
q
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def asField[T](value: T): Field[T] = {
|
|
||||||
val tmp = new Field[T]
|
|
||||||
tmp.setValue(value)
|
|
||||||
tmp
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,7 +1,7 @@
|
||||||
package eu.dnetlib.dhp.`export`
|
package eu.dnetlib.dhp.`export`
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
import eu.dnetlib.dhp.schema.oaf.{Instance, Publication, Relation, Dataset => OafDataset}
|
||||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
import org.apache.hadoop.io.Text
|
import org.apache.hadoop.io.Text
|
||||||
|
@ -166,10 +166,13 @@ object SparkExportContentForOpenAire {
|
||||||
}).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationAS")
|
}).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationAS")
|
||||||
|
|
||||||
|
|
||||||
val fRels:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationAS").as[Relation].map(DLIToOAF.toActionSet)
|
|
||||||
val fpubs:Dataset[(String,String)] = spark.read.load(s"$workingPath/publicationAS").as[Publication].map(DLIToOAF.toActionSet)
|
|
||||||
val fdats:Dataset[(String,String)] = spark.read.load(s"$workingPath/datasetAS").as[OafDataset].map(DLIToOAF.toActionSet)
|
|
||||||
|
|
||||||
|
spark.read.load(s"$workingPath/publicationAS").as[Publication].map(DLIToOAF.fixInstance).write.mode(SaveMode.Overwrite).save(s"$workingPath/publicationAS_fixed")
|
||||||
|
spark.read.load(s"$workingPath/datasetAS").as[OafDataset].map(DLIToOAF.fixInstanceDataset).write.mode(SaveMode.Overwrite).save(s"$workingPath/datasetAS_fixed")
|
||||||
|
|
||||||
|
val fRels:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationAS").as[Relation].map(DLIToOAF.toActionSet)
|
||||||
|
val fpubs:Dataset[(String,String)] = spark.read.load(s"$workingPath/publicationAS_fixed").as[Publication].map(DLIToOAF.toActionSet)
|
||||||
|
val fdats:Dataset[(String,String)] = spark.read.load(s"$workingPath/datasetAS_fixed").as[OafDataset].map(DLIToOAF.toActionSet)
|
||||||
|
|
||||||
fRels.union(fpubs).union(fdats).rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingPath/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
|
fRels.union(fpubs).union(fdats).rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingPath/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
|
||||||
}
|
}
|
||||||
|
|
|
@ -9,6 +9,7 @@ import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
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.api.java.function.MapFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
|
@ -118,10 +119,7 @@ public class CreateRelatedEntitiesJob_phase1 {
|
||||||
Dataset<Tuple2<String, RelatedEntity>> entities = readPathEntity(spark, inputEntityPath, clazz)
|
Dataset<Tuple2<String, RelatedEntity>> entities = readPathEntity(spark, inputEntityPath, clazz)
|
||||||
.filter("dataInfo.invisible == false")
|
.filter("dataInfo.invisible == false")
|
||||||
.map(
|
.map(
|
||||||
(MapFunction<E, RelatedEntity>) value -> asRelatedEntity(value, clazz),
|
(MapFunction<E, Tuple2<String, RelatedEntity>>) e -> new Tuple2<>(e.getId(), asRelatedEntity(e, clazz)),
|
||||||
Encoders.kryo(RelatedEntity.class))
|
|
||||||
.map(
|
|
||||||
(MapFunction<RelatedEntity, Tuple2<String, RelatedEntity>>) e -> new Tuple2<>(e.getId(), e),
|
|
||||||
Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class)))
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class)))
|
||||||
.cache();
|
.cache();
|
||||||
|
|
||||||
|
@ -165,13 +163,24 @@ public class CreateRelatedEntitiesJob_phase1 {
|
||||||
Result result = (Result) entity;
|
Result result = (Result) entity;
|
||||||
|
|
||||||
if (result.getTitle() != null && !result.getTitle().isEmpty()) {
|
if (result.getTitle() != null && !result.getTitle().isEmpty()) {
|
||||||
re.setTitle(result.getTitle().stream().findFirst().get());
|
final StructuredProperty title = result.getTitle().stream().findFirst().get();
|
||||||
|
title.setValue(StringUtils.left(title.getValue(), ProvisionConstants.MAX_TITLE_LENGTH));
|
||||||
|
re.setTitle(title);
|
||||||
}
|
}
|
||||||
|
|
||||||
re.setDateofacceptance(getValue(result.getDateofacceptance()));
|
re.setDateofacceptance(getValue(result.getDateofacceptance()));
|
||||||
re.setPublisher(getValue(result.getPublisher()));
|
re.setPublisher(getValue(result.getPublisher()));
|
||||||
re.setResulttype(result.getResulttype());
|
re.setResulttype(result.getResulttype());
|
||||||
re.setInstances(result.getInstance());
|
if (Objects.nonNull(result.getInstance())) {
|
||||||
|
re
|
||||||
|
.setInstances(
|
||||||
|
result
|
||||||
|
.getInstance()
|
||||||
|
.stream()
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.limit(ProvisionConstants.MAX_INSTANCES)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
|
||||||
// TODO still to be mapped
|
// TODO still to be mapped
|
||||||
// re.setCodeRepositoryUrl(j.read("$.coderepositoryurl"));
|
// re.setCodeRepositoryUrl(j.read("$.coderepositoryurl"));
|
||||||
|
|
|
@ -61,12 +61,6 @@ public class CreateRelatedEntitiesJob_phase2 {
|
||||||
|
|
||||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||||
|
|
||||||
private static final int MAX_EXTERNAL_ENTITIES = 50;
|
|
||||||
private static final int MAX_AUTHORS = 200;
|
|
||||||
private static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000;
|
|
||||||
private static final int MAX_TITLE_LENGTH = 5000;
|
|
||||||
private static final int MAX_ABSTRACT_LENGTH = 100000;
|
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
String jsonConfiguration = IOUtils
|
String jsonConfiguration = IOUtils
|
||||||
|
@ -246,15 +240,15 @@ public class CreateRelatedEntitiesJob_phase2 {
|
||||||
List<ExternalReference> refs = r
|
List<ExternalReference> refs = r
|
||||||
.getExternalReference()
|
.getExternalReference()
|
||||||
.stream()
|
.stream()
|
||||||
.limit(MAX_EXTERNAL_ENTITIES)
|
.limit(ProvisionConstants.MAX_EXTERNAL_ENTITIES)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
r.setExternalReference(refs);
|
r.setExternalReference(refs);
|
||||||
}
|
}
|
||||||
if (r.getAuthor() != null) {
|
if (r.getAuthor() != null) {
|
||||||
List<Author> authors = Lists.newArrayList();
|
List<Author> authors = Lists.newArrayList();
|
||||||
for (Author a : r.getAuthor()) {
|
for (Author a : r.getAuthor()) {
|
||||||
a.setFullname(StringUtils.left(a.getFullname(), MAX_AUTHOR_FULLNAME_LENGTH));
|
a.setFullname(StringUtils.left(a.getFullname(), ProvisionConstants.MAX_AUTHOR_FULLNAME_LENGTH));
|
||||||
if (authors.size() < MAX_AUTHORS || hasORCID(a)) {
|
if (authors.size() < ProvisionConstants.MAX_AUTHORS || hasORCID(a)) {
|
||||||
authors.add(a);
|
authors.add(a);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -266,7 +260,7 @@ public class CreateRelatedEntitiesJob_phase2 {
|
||||||
.stream()
|
.stream()
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
.map(d -> {
|
.map(d -> {
|
||||||
d.setValue(StringUtils.left(d.getValue(), MAX_ABSTRACT_LENGTH));
|
d.setValue(StringUtils.left(d.getValue(), ProvisionConstants.MAX_ABSTRACT_LENGTH));
|
||||||
return d;
|
return d;
|
||||||
})
|
})
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
@ -278,9 +272,10 @@ public class CreateRelatedEntitiesJob_phase2 {
|
||||||
.stream()
|
.stream()
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
.map(t -> {
|
.map(t -> {
|
||||||
t.setValue(StringUtils.left(t.getValue(), MAX_TITLE_LENGTH));
|
t.setValue(StringUtils.left(t.getValue(), ProvisionConstants.MAX_TITLE_LENGTH));
|
||||||
return t;
|
return t;
|
||||||
})
|
})
|
||||||
|
.limit(ProvisionConstants.MAX_TITLES)
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
r.setTitle(titles);
|
r.setTitle(titles);
|
||||||
}
|
}
|
||||||
|
|
|
@ -100,11 +100,17 @@ public class PrepareRelationsJob {
|
||||||
.orElse(new HashSet<>());
|
.orElse(new HashSet<>());
|
||||||
log.info("relationFilter: {}", relationFilter);
|
log.info("relationFilter: {}", relationFilter);
|
||||||
|
|
||||||
int maxRelations = Optional
|
int sourceMaxRelations = Optional
|
||||||
.ofNullable(parser.get("maxRelations"))
|
.ofNullable(parser.get("sourceMaxRelations"))
|
||||||
.map(Integer::valueOf)
|
.map(Integer::valueOf)
|
||||||
.orElse(MAX_RELS);
|
.orElse(MAX_RELS);
|
||||||
log.info("maxRelations: {}", maxRelations);
|
log.info("sourceMaxRelations: {}", sourceMaxRelations);
|
||||||
|
|
||||||
|
int targetMaxRelations = Optional
|
||||||
|
.ofNullable(parser.get("targetMaxRelations"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(MAX_RELS);
|
||||||
|
log.info("targetMaxRelations: {}", targetMaxRelations);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
@ -116,7 +122,8 @@ public class PrepareRelationsJob {
|
||||||
spark -> {
|
spark -> {
|
||||||
removeOutputDir(spark, outputPath);
|
removeOutputDir(spark, outputPath);
|
||||||
prepareRelationsRDD(
|
prepareRelationsRDD(
|
||||||
spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions);
|
spark, inputRelationsPath, outputPath, relationFilter, sourceMaxRelations, targetMaxRelations,
|
||||||
|
relPartitions);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -129,33 +136,43 @@ public class PrepareRelationsJob {
|
||||||
* @param inputRelationsPath source path for the graph relations
|
* @param inputRelationsPath source path for the graph relations
|
||||||
* @param outputPath output path for the processed relations
|
* @param outputPath output path for the processed relations
|
||||||
* @param relationFilter set of relation filters applied to the `relClass` field
|
* @param relationFilter set of relation filters applied to the `relClass` field
|
||||||
* @param maxRelations maximum number of allowed outgoing edges
|
* @param sourceMaxRelations maximum number of allowed outgoing edges grouping by relation.source
|
||||||
|
* @param targetMaxRelations maximum number of allowed outgoing edges grouping by relation.target
|
||||||
* @param relPartitions number of partitions for the output RDD
|
* @param relPartitions number of partitions for the output RDD
|
||||||
*/
|
*/
|
||||||
private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath,
|
private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath,
|
||||||
Set<String> relationFilter, int maxRelations, int relPartitions) {
|
Set<String> relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) {
|
||||||
|
|
||||||
// group by SOURCE and apply limit
|
JavaRDD<Relation> rels = readPathRelationRDD(spark, inputRelationsPath)
|
||||||
RDD<Relation> bySource = readPathRelationRDD(spark, inputRelationsPath)
|
|
||||||
.filter(rel -> rel.getDataInfo().getDeletedbyinference() == false)
|
.filter(rel -> rel.getDataInfo().getDeletedbyinference() == false)
|
||||||
.filter(rel -> relationFilter.contains(rel.getRelClass()) == false)
|
.filter(rel -> relationFilter.contains(rel.getRelClass()) == false);
|
||||||
.mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, r.getSource()), r))
|
|
||||||
.repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions))
|
|
||||||
.groupBy(Tuple2::_1)
|
|
||||||
.map(Tuple2::_2)
|
|
||||||
.map(t -> Iterables.limit(t, maxRelations))
|
|
||||||
.flatMap(Iterable::iterator)
|
|
||||||
.map(Tuple2::_2)
|
|
||||||
.rdd();
|
|
||||||
|
|
||||||
|
JavaRDD<Relation> pruned = pruneRels(
|
||||||
|
pruneRels(
|
||||||
|
rels,
|
||||||
|
sourceMaxRelations, relPartitions, (Function<Relation, String>) r -> r.getSource()),
|
||||||
|
targetMaxRelations, relPartitions, (Function<Relation, String>) r -> r.getTarget());
|
||||||
spark
|
spark
|
||||||
.createDataset(bySource, Encoders.bean(Relation.class))
|
.createDataset(pruned.rdd(), Encoders.bean(Relation.class))
|
||||||
.repartition(relPartitions)
|
.repartition(relPartitions)
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.mode(SaveMode.Overwrite)
|
||||||
.parquet(outputPath);
|
.parquet(outputPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static JavaRDD<Relation> pruneRels(JavaRDD<Relation> rels, int maxRelations,
|
||||||
|
int relPartitions, Function<Relation, String> idFn) {
|
||||||
|
return rels
|
||||||
|
.mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r))
|
||||||
|
.repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions))
|
||||||
|
.groupBy(Tuple2::_1)
|
||||||
|
.map(Tuple2::_2)
|
||||||
|
.map(t -> Iterables.limit(t, maxRelations))
|
||||||
|
.flatMap(Iterable::iterator)
|
||||||
|
.map(Tuple2::_2);
|
||||||
|
}
|
||||||
|
|
||||||
|
// experimental
|
||||||
private static void prepareRelationsDataset(
|
private static void prepareRelationsDataset(
|
||||||
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations,
|
SparkSession spark, String inputRelationsPath, String outputPath, Set<String> relationFilter, int maxRelations,
|
||||||
int relPartitions) {
|
int relPartitions) {
|
||||||
|
|
|
@ -0,0 +1,14 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.provision;
|
||||||
|
|
||||||
|
public class ProvisionConstants {
|
||||||
|
|
||||||
|
public static final int MAX_EXTERNAL_ENTITIES = 50;
|
||||||
|
public static final int MAX_AUTHORS = 200;
|
||||||
|
public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000;
|
||||||
|
public static final int MAX_TITLE_LENGTH = 5000;
|
||||||
|
public static final int MAX_TITLES = 10;
|
||||||
|
public static final int MAX_ABSTRACT_LENGTH = 100000;
|
||||||
|
public static final int MAX_INSTANCES = 10;
|
||||||
|
|
||||||
|
}
|
|
@ -16,18 +16,18 @@ public class SortableRelationKey implements Comparable<SortableRelationKey>, Ser
|
||||||
private static final Map<String, Integer> weights = Maps.newHashMap();
|
private static final Map<String, Integer> weights = Maps.newHashMap();
|
||||||
|
|
||||||
static {
|
static {
|
||||||
weights.put("outcome", 0);
|
weights.put("participation", 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("outcome", 1);
|
||||||
weights.put("participation", 9);
|
weights.put("affiliation", 2);
|
||||||
weights.put("dedup", 10);
|
weights.put("dedup", 3);
|
||||||
|
weights.put("publicationDataset", 4);
|
||||||
|
weights.put("citation", 5);
|
||||||
|
weights.put("supplement", 6);
|
||||||
|
weights.put("review", 7);
|
||||||
|
weights.put("relationship", 8);
|
||||||
|
weights.put("provision", 9);
|
||||||
|
weights.put("similarity", 10);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final long serialVersionUID = 3232323;
|
private static final long serialVersionUID = 3232323;
|
||||||
|
|
|
@ -329,7 +329,7 @@ public class XmlRecordFactory implements Serializable {
|
||||||
.stream()
|
.stream()
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
.map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue()))
|
.map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue()))
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toCollection(HashSet::new)));
|
||||||
}
|
}
|
||||||
if (r.getEmbargoenddate() != null) {
|
if (r.getEmbargoenddate() != null) {
|
||||||
metadata
|
metadata
|
||||||
|
@ -370,7 +370,7 @@ public class XmlRecordFactory implements Serializable {
|
||||||
.stream()
|
.stream()
|
||||||
.filter(Objects::nonNull)
|
.filter(Objects::nonNull)
|
||||||
.map(c -> XmlSerializationUtils.asXmlElement("source", c.getValue()))
|
.map(c -> XmlSerializationUtils.asXmlElement("source", c.getValue()))
|
||||||
.collect(Collectors.toList()));
|
.collect(Collectors.toCollection(HashSet::new)));
|
||||||
}
|
}
|
||||||
if (r.getFormat() != null) {
|
if (r.getFormat() != null) {
|
||||||
metadata
|
metadata
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue