minor changes in dedup tests, bug fix in the idgenerator and pace-core version update

This commit is contained in:
miconis 2020-09-29 15:31:46 +02:00
parent 4cf79f32eb
commit e3f7798d1b
16 changed files with 751 additions and 669 deletions

View File

@ -90,6 +90,10 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
</dependencies>

View File

@ -29,6 +29,7 @@ import eu.dnetlib.pace.config.DedupConfig;
abstract class AbstractSparkAction implements Serializable {
protected static final int NUM_PARTITIONS = 1000;
protected static final int NUM_CONNECTIONS = 20;
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

View File

@ -1,12 +1,10 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.*;
import org.apache.commons.lang.StringUtils;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
@ -15,11 +13,15 @@ import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.*;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2;
public class DedupRecordFactory {
@ -80,14 +82,14 @@ public class DedupRecordFactory {
final Collection<String> dates = Lists.newArrayList();
final List<List<Author>> authors = Lists.newArrayList();
final List<Identifier> bestPids = Lists.newArrayList(); //best pids list
final List<Identifier> bestPids = Lists.newArrayList(); // best pids list
entities
.forEachRemaining(
t -> {
T duplicate = t._2();
//prepare the list of pids to use for the id generation
// prepare the list of pids to use for the id generation
bestPids.addAll(IdGenerator.bestPidtoIdentifier(duplicate));
entity.mergeFrom(duplicate);
@ -115,5 +117,4 @@ public class DedupRecordFactory {
return entity;
}
}

View File

@ -1,59 +1,79 @@
package eu.dnetlib.dhp.oa.dedup;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import org.apache.commons.lang.NullArgumentException;
import org.apache.commons.lang.StringUtils;
package eu.dnetlib.dhp.oa.dedup;
import java.io.Serializable;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.*;
import org.apache.commons.lang.NullArgumentException;
import org.apache.commons.lang.StringUtils;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class IdGenerator implements Serializable {
private static SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
public static String CROSSREF_ID = "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2";
public static String DATACITE_ID = "10|openaire____::9e3be59865b2c1c335d32dae2fe7b254";
//pick the best pid from the list (consider date and pidtype)
// pick the best pid from the list (consider date and pidtype)
public static String generate(List<Identifier> pids, String defaultID) {
if (pids == null || pids.size() == 0)
return defaultID;
Optional<Identifier> bp = pids.stream()
Optional<Identifier> bp = pids
.stream()
.max(Identifier::compareTo);
if (bp.get().isUseOriginal() || bp.get().getPid().getValue() == null) {
return bp.get().getOriginalID().split("\\|")[0] + "|dedup_wf_001::" + DedupUtility.md5(bp.get().getOriginalID());
return bp.get().getOriginalID().split("\\|")[0] + "|dedup_wf_001::"
+ DedupUtility.md5(bp.get().getOriginalID());
} else {
return bp.get().getOriginalID().split("\\|")[0] + "|" + createPrefix(bp.get().getPid().getQualifier().getClassid()) + "::" + DedupUtility.md5(bp.get().getPid().getValue());
return bp.get().getOriginalID().split("\\|")[0] + "|"
+ createPrefix(bp.get().getPid().getQualifier().getClassid()) + "::"
+ DedupUtility.md5(bp.get().getPid().getValue());
}
}
//pick the best pid from the entity. Returns a list (length 1) to save time in the call
// pick the best pid from the entity. Returns a list (length 1) to save time in the call
public static <T extends OafEntity> List<Identifier> bestPidtoIdentifier(T entity) {
if (entity.getPid() == null || entity.getPid().size() == 0)
return Lists.newArrayList(new Identifier(new StructuredProperty(), new Date(), PidType.original, entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId()));
return Lists
.newArrayList(
new Identifier(new StructuredProperty(), new Date(), PidType.original, entity.getCollectedfrom(),
EntityType.fromClass(entity.getClass()), entity.getId()));
Optional<StructuredProperty> bp = entity.getPid().stream()
Optional<StructuredProperty> bp = entity
.getPid()
.stream()
.filter(pid -> PidType.classidValueOf(pid.getQualifier().getClassid()) != PidType.undefined)
.max(Comparator.comparing(pid -> PidType.classidValueOf(pid.getQualifier().getClassid())));
return bp.map(structuredProperty ->
Lists.newArrayList(new Identifier(structuredProperty, extractDate(entity, sdf), PidType.classidValueOf(structuredProperty.getQualifier().getClassid()), entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId()))
).orElseGet(() -> Lists.newArrayList(new Identifier(new StructuredProperty(), new Date(), PidType.original, entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId())));
return bp
.map(
structuredProperty -> Lists
.newArrayList(
new Identifier(structuredProperty, extractDate(entity, new SimpleDateFormat("yyyy-MM-dd")),
PidType.classidValueOf(structuredProperty.getQualifier().getClassid()),
entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId())))
.orElseGet(
() -> Lists
.newArrayList(
new Identifier(new StructuredProperty(), new Date(), PidType.original,
entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId())));
}
//create the prefix (length = 12): dedup_+ pidType
// create the prefix (length = 12): dedup_+ pidType
public static String createPrefix(String pidType) {
StringBuilder prefix = new StringBuilder("dedup_" + pidType);
@ -65,13 +85,14 @@ public class IdGenerator implements Serializable {
}
//extracts the date from the record. If the date is not available or is not wellformed, it returns a base date: 00-01-01
public static <T extends OafEntity> Date extractDate(T duplicate, SimpleDateFormat sdf){
// extracts the date from the record. If the date is not available or is not wellformed, it returns a base date:
// 00-01-01
public static <T extends OafEntity> Date extractDate(T duplicate, SimpleDateFormat sdf) {
String date = "2000-01-01";
if (ModelSupport.isSubClass(duplicate, Result.class)) {
Result result = (Result) duplicate;
if (isWellformed(result.getDateofacceptance())){
if (isWellformed(result.getDateofacceptance())) {
date = result.getDateofacceptance().getValue();
}
}
@ -85,6 +106,7 @@ public class IdGenerator implements Serializable {
}
public static boolean isWellformed(Field<String> date) {
return date != null && StringUtils.isNotBlank(date.getValue()) && date.getValue().matches(DatePicker.DATE_PATTERN) && DatePicker.inRange(date.getValue());
return date != null && StringUtils.isNotBlank(date.getValue())
&& date.getValue().matches(DatePicker.DATE_PATTERN) && DatePicker.inRange(date.getValue());
}
}

View File

@ -1,14 +1,15 @@
package eu.dnetlib.dhp.oa.dedup;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
package eu.dnetlib.dhp.oa.dedup;
import java.io.Serializable;
import java.util.Date;
import java.util.List;
public class Identifier implements Serializable, Comparable<Identifier>{
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class Identifier implements Serializable, Comparable<Identifier> {
StructuredProperty pid;
Date date;
@ -17,9 +18,11 @@ public class Identifier implements Serializable, Comparable<Identifier>{
EntityType entityType;
String originalID;
boolean useOriginal = false; //to know if the top identifier won because of the alphabetical order of the original ID
boolean useOriginal = false; // to know if the top identifier won because of the alphabetical order of the original
// ID
public Identifier(StructuredProperty pid, Date date, PidType type, List<KeyValue> collectedFrom, EntityType entityType, String originalID) {
public Identifier(StructuredProperty pid, Date date, PidType type, List<KeyValue> collectedFrom,
EntityType entityType, String originalID) {
this.pid = pid;
this.date = date;
this.type = type;
@ -86,45 +89,48 @@ public class Identifier implements Serializable, Comparable<Identifier>{
@Override
public int compareTo(Identifier i) {
//priority in comparisons: 1) pidtype, 2) collectedfrom (depending on the entity type) , 3) date 4) alphabetical order of the originalID
if (this.getType().compareTo(i.getType()) == 0){ //same type
// priority in comparisons: 1) pidtype, 2) collectedfrom (depending on the entity type) , 3) date 4)
// alphabetical order of the originalID
if (this.getType().compareTo(i.getType()) == 0) { // same type
if (entityType == EntityType.publication) {
if (isFromDatasourceID(this.collectedFrom, IdGenerator.CROSSREF_ID) && !isFromDatasourceID(i.collectedFrom, IdGenerator.CROSSREF_ID))
if (isFromDatasourceID(this.collectedFrom, IdGenerator.CROSSREF_ID)
&& !isFromDatasourceID(i.collectedFrom, IdGenerator.CROSSREF_ID))
return 1;
if (isFromDatasourceID(i.collectedFrom, IdGenerator.CROSSREF_ID) && !isFromDatasourceID(this.collectedFrom, IdGenerator.CROSSREF_ID))
if (isFromDatasourceID(i.collectedFrom, IdGenerator.CROSSREF_ID)
&& !isFromDatasourceID(this.collectedFrom, IdGenerator.CROSSREF_ID))
return -1;
}
if (entityType == EntityType.dataset) {
if (isFromDatasourceID(this.collectedFrom, IdGenerator.DATACITE_ID) && !isFromDatasourceID(i.collectedFrom, IdGenerator.DATACITE_ID))
if (isFromDatasourceID(this.collectedFrom, IdGenerator.DATACITE_ID)
&& !isFromDatasourceID(i.collectedFrom, IdGenerator.DATACITE_ID))
return 1;
if (isFromDatasourceID(i.collectedFrom, IdGenerator.DATACITE_ID) && !isFromDatasourceID(this.collectedFrom, IdGenerator.DATACITE_ID))
if (isFromDatasourceID(i.collectedFrom, IdGenerator.DATACITE_ID)
&& !isFromDatasourceID(this.collectedFrom, IdGenerator.DATACITE_ID))
return -1;
}
if (this.getDate().compareTo(date) == 0) {//same date
if (this.getDate().compareTo(date) == 0) {// same date
if (this.originalID.compareTo(i.originalID) > 0)
this.useOriginal = true;
else
i.setUseOriginal(true);
//the minus because we need to take the alphabetically lower id
// the minus because we need to take the alphabetically lower id
return -this.originalID.compareTo(i.originalID);
}
else
//the minus is because we need to take the elder date
} else
// the minus is because we need to take the elder date
return -this.getDate().compareTo(date);
}
else {
} else {
return this.getType().compareTo(i.getType());
}
}
public boolean isFromDatasourceID(List<KeyValue> collectedFrom, String dsId){
public boolean isFromDatasourceID(List<KeyValue> collectedFrom, String dsId) {
for(KeyValue cf: collectedFrom) {
if(cf.getKey().equals(dsId))
for (KeyValue cf : collectedFrom) {
if (cf.getKey().equals(dsId))
return true;
}
return false;

View File

@ -1,3 +1,4 @@
package eu.dnetlib.dhp.oa.dedup;
import java.io.Serializable;
@ -15,7 +16,8 @@ public class OrgSimRel implements Serializable {
public OrgSimRel() {
}
public OrgSimRel(String local_id, String oa_original_id, String oa_name, String oa_acronym, String oa_country, String oa_url, String oa_collectedfrom) {
public OrgSimRel(String local_id, String oa_original_id, String oa_name, String oa_acronym, String oa_country,
String oa_url, String oa_collectedfrom) {
this.local_id = local_id;
this.oa_original_id = oa_original_id;
this.oa_name = oa_name;
@ -80,4 +82,17 @@ public class OrgSimRel implements Serializable {
public void setOa_collectedfrom(String oa_collectedfrom) {
this.oa_collectedfrom = oa_collectedfrom;
}
@Override
public String toString() {
return "OrgSimRel{" +
"local_id='" + local_id + '\'' +
", oa_original_id='" + oa_original_id + '\'' +
", oa_name='" + oa_name + '\'' +
", oa_acronym='" + oa_acronym + '\'' +
", oa_country='" + oa_country + '\'' +
", oa_url='" + oa_url + '\'' +
", oa_collectedfrom='" + oa_collectedfrom + '\'' +
'}';
}
}

View File

@ -1,23 +1,15 @@
package eu.dnetlib.dhp.oa.dedup;
public enum PidType {
//from the less to the more important
undefined,
original,
orcid,
ror,
grid,
pdb,
arXiv,
pmid,
doi;
// from the less to the more important
undefined, original, orcid, ror, grid, pdb, arXiv, pmid, doi;
public static PidType classidValueOf(String s){
public static PidType classidValueOf(String s) {
try {
return PidType.valueOf(s);
}
catch (Exception e) {
} catch (Exception e) {
return PidType.undefined;
}
}

View File

@ -1,21 +1,5 @@
package eu.dnetlib.dhp.oa.dedup;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
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 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.sql.*;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
package eu.dnetlib.dhp.oa.dedup;
import java.io.IOException;
import java.util.ArrayList;
@ -24,6 +8,24 @@ import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
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.sql.*;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
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 scala.Tuple2;
public class SparkCollectSimRels extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCollectSimRels.class);
@ -31,7 +33,8 @@ public class SparkCollectSimRels extends AbstractSparkAction {
Dataset<Row> simGroupsDS;
Dataset<Row> groupsDS;
public SparkCollectSimRels(ArgumentApplicationParser parser, SparkSession spark, Dataset<Row> simGroupsDS, Dataset<Row> groupsDS) {
public SparkCollectSimRels(ArgumentApplicationParser parser, SparkSession spark, Dataset<Row> simGroupsDS,
Dataset<Row> groupsDS) {
super(parser, spark);
this.simGroupsDS = simGroupsDS;
this.groupsDS = groupsDS;
@ -54,7 +57,8 @@ public class SparkCollectSimRels extends AbstractSparkAction {
SparkSession spark = getSparkSession(conf);
DataFrameReader readOptions = spark.read()
DataFrameReader readOptions = spark
.read()
.format("jdbc")
.option("url", dbUrl)
.option("user", dbUser)
@ -64,8 +68,8 @@ public class SparkCollectSimRels extends AbstractSparkAction {
parser,
spark,
readOptions.option("dbtable", "similarity_groups").load(),
readOptions.option("dbtable", "groups").load()
).run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
readOptions.option("dbtable", "groups").load())
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
@ -90,16 +94,16 @@ public class SparkCollectSimRels extends AbstractSparkAction {
log.info("postgresUrl: {}", dbUrl);
log.info("postgresPassword: xxx");
JavaPairRDD<String, List<String>> similarityGroup =
simGroupsDS
JavaPairRDD<String, List<String>> similarityGroup = simGroupsDS
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1)))
.groupByKey()
.mapToPair(i -> new Tuple2<>(i._1(), StreamSupport.stream(i._2().spliterator(), false)
.mapToPair(
i -> new Tuple2<>(i._1(), StreamSupport
.stream(i._2().spliterator(), false)
.collect(Collectors.toList())));
JavaPairRDD<String, String> groupIds =
groupsDS
JavaPairRDD<String, String> groupIds = groupsDS
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1)));
@ -120,23 +124,26 @@ public class SparkCollectSimRels extends AbstractSparkAction {
return rels.iterator();
});
Dataset<Relation> resultRelations = spark.createDataset(
Dataset<Relation> resultRelations = spark
.createDataset(
relations.filter(r -> r.getRelType().equals("resultResult")).rdd(),
Encoders.bean(Relation.class)
).repartition(numPartitions);
Encoders.bean(Relation.class))
.repartition(numPartitions);
Dataset<Relation> organizationRelations = spark.createDataset(
Dataset<Relation> organizationRelations = spark
.createDataset(
relations.filter(r -> r.getRelType().equals("organizationOrganization")).rdd(),
Encoders.bean(Relation.class)
).repartition(numPartitions);
Encoders.bean(Relation.class))
.repartition(numPartitions);
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
switch(dedupConf.getWf().getSubEntityValue()){
switch (dedupConf.getWf().getSubEntityValue()) {
case "organization":
savePostgresRelation(organizationRelations, workingPath, actionSetId, "organization");
break;
default:
savePostgresRelation(resultRelations, workingPath, actionSetId, dedupConf.getWf().getSubEntityValue());
savePostgresRelation(
resultRelations, workingPath, actionSetId, dedupConf.getWf().getSubEntityValue());
break;
}
}
@ -166,7 +173,8 @@ public class SparkCollectSimRels extends AbstractSparkAction {
return r;
}
private void savePostgresRelation(Dataset<Relation> newRelations, String workingPath, String actionSetId, String entityType) {
private void savePostgresRelation(Dataset<Relation> newRelations, String workingPath, String actionSetId,
String entityType) {
newRelations
.write()
.mode(SaveMode.Append)

View File

@ -100,14 +100,14 @@ public class SparkCreateSimRels extends AbstractSparkAction {
.repartition(numPartitions);
// create relations by comparing only elements in the same group
spark.createDataset(
spark
.createDataset(
Deduper
.computeRelations(sc, blocks, dedupConf)
.map(t -> createSimRel(t._1(), t._2(), entity))
.repartition(numPartitions)
.rdd(),
Encoders.bean(Relation.class)
)
Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.parquet(outputPath);

View File

@ -1,13 +1,11 @@
package eu.dnetlib.dhp.oa.dedup;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
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 static jdk.nashorn.internal.objects.NativeDebug.map;
import java.io.IOException;
import java.util.*;
import org.apache.commons.io.IOUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
@ -15,6 +13,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.Dataset;
@ -24,22 +23,21 @@ import org.apache.spark.sql.SparkSession;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
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 scala.Tuple2;
public class SparkPrepareOrgRels extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class);
public static final String ROOT_TRUST = "0.8";
public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup";
public static final String PROVENANCE_ACTIONS = "dnet:provenanceActions";
public SparkPrepareOrgRels(ArgumentApplicationParser parser, SparkSession spark) {
super(parser, spark);
}
@ -57,7 +55,7 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkCreateDedupRecord(parser, getSparkSession(conf))
new SparkPrepareOrgRels(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@ -68,7 +66,15 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
final String apiUrl = parser.get("apiUrl");
final int numConnections = Optional
.ofNullable(parser.get("numConnections"))
.map(Integer::valueOf)
.orElse(NUM_CONNECTIONS);
final String apiUrl = Optional
.ofNullable(parser.get("apiUrl"))
.orElse("");
final String dbUrl = parser.get("dbUrl");
final String dbTable = parser.get("dbTable");
final String dbUser = parser.get("dbUser");
@ -78,6 +84,7 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
log.info("numPartitions: '{}'", numConnections);
log.info("apiUrl: '{}'", apiUrl);
log.info("dbUrl: '{}'", dbUrl);
log.info("dbUser: '{}'", dbUser);
@ -93,7 +100,11 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
connectionProperties.put("user", dbUser);
connectionProperties.put("password", dbPwd);
relations.write().mode(SaveMode.Overwrite).jdbc(dbUrl, dbTable, connectionProperties);
relations
.repartition(numConnections)
.write()
.mode(SaveMode.Overwrite)
.jdbc(dbUrl, dbTable, connectionProperties);
if (!apiUrl.isEmpty())
updateSimRels(apiUrl);
@ -116,7 +127,8 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
},
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
Dataset<Tuple2<String, String>> relations = spark.createDataset(
Dataset<Tuple2<String, String>> relations = spark
.createDataset(
spark
.read()
.load(mergeRelsPath)
@ -130,33 +142,47 @@ public class SparkPrepareOrgRels extends AbstractSparkAction {
for (String id1 : g._2()) {
for (String id2 : g._2()) {
if (!id1.equals(id2))
if (id1.contains("openorgs"))
if (id1.contains("openorgs____") && !id2.contains("openorgsmesh"))
rels.add(new Tuple2<>(id1, id2));
}
}
return rels.iterator();
}).rdd(),
})
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
return relations
Dataset<Tuple2<String, OrgSimRel>> relations2 = relations // <openorgs, corda>
.joinWith(entities, relations.col("_2").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, Organization>>, OrgSimRel>)r ->
new OrgSimRel(
r._1()._2(),
(MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, Organization>>, OrgSimRel>) r -> new OrgSimRel(
r._1()._1(),
r._2()._2().getOriginalId().get(0),
r._2()._2().getLegalname().getValue(),
r._2()._2().getLegalshortname().getValue(),
r._2()._2().getCountry().getClassid(),
r._2()._2().getWebsiteurl().getValue(),
r._2()._2().getCollectedfrom().get(0).getValue()
),
Encoders.bean(OrgSimRel.class)
);
r._2()._2().getLegalname() != null ? r._2()._2().getLegalname().getValue() : "",
r._2()._2().getLegalshortname() != null ? r._2()._2().getLegalshortname().getValue() : "",
r._2()._2().getCountry() != null ? r._2()._2().getCountry().getClassid() : "",
r._2()._2().getWebsiteurl() != null ? r._2()._2().getWebsiteurl().getValue() : "",
r._2()._2().getCollectedfrom().get(0).getValue()),
Encoders.bean(OrgSimRel.class))
.map(
(MapFunction<OrgSimRel, Tuple2<String, OrgSimRel>>) o -> new Tuple2<>(o.getLocal_id(), o),
Encoders.tuple(Encoders.STRING(), Encoders.bean(OrgSimRel.class)));
return relations2
.joinWith(entities, relations2.col("_1").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple2<String, OrgSimRel>, Tuple2<String, Organization>>, OrgSimRel>) r -> {
OrgSimRel orgSimRel = r._1()._2();
orgSimRel.setLocal_id(r._2()._2().getOriginalId().get(0));
return orgSimRel;
},
Encoders.bean(OrgSimRel.class));
}
private static String updateSimRels(final String apiUrl) throws IOException {
log.info("Updating simrels on the portal");
final HttpGet req = new HttpGet(apiUrl);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {

View File

@ -1,4 +1,4 @@
<workflow-app name="Duplicate Scan" xmlns="uri:oozie:workflow:0.5">
<workflow-app name="Organization Dedup" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphBasePath</name>
@ -24,10 +24,6 @@
<name>cutConnectedComponent</name>
<description>max number of elements in a connected component</description>
</property>
<property>
<name>apiUrl</name>
<description>the url for the APIs of the openorgs service</description>
</property>
<property>
<name>dbUrl</name>
<description>the url of the database</description>
@ -109,6 +105,16 @@
<fs>
<delete path="${workingPath}"/>
</fs>
<ok to="copyRelations"/>
<error to="Kill"/>
</action>
<action name="copyRelations">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>-pb</arg>
<arg>${graphBasePath}/relation</arg>
<arg>${workingPath}/${actionSetId}/organization_simrel</arg>
</distcp>
<ok to="CreateSimRel"/>
<error to="Kill"/>
</action>
@ -136,16 +142,6 @@
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="copyRelations"/>
<error to="Kill"/>
</action>
<action name="copyRelations">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>-pb</arg>
<arg>${graphBasePath}/relation</arg>
<arg>${workingPath}/organization_simrel</arg>
</distcp>
<ok to="CreateMergeRel"/>
<error to="Kill"/>
</action>
@ -203,6 +199,7 @@
<arg>--dbTable</arg><arg>${dbTable}</arg>
<arg>--dbUser</arg><arg>${dbUser}</arg>
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
<arg>--numConnections</arg><arg>20</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -23,11 +23,17 @@
"paramDescription": "the id of the actionset (orchestrator)",
"paramRequired": true
},
{
"paramName": "nc",
"paramLongName": "numConnections",
"paramDescription": "number of connections to the postgres db (for the write operation)",
"paramRequired": false
},
{
"paramName": "au",
"paramLongName": "apiUrl",
"paramDescription": "the url for the APIs of the openorgs service",
"paramRequired": true
"paramRequired": false
},
{
"paramName": "du",

View File

@ -141,7 +141,7 @@ public class EntityMergerTest implements Serializable {
.entityMerger(dedupId, publications3.iterator(), 0, dataInfo, Publication.class);
// verify id
assertEquals( "50|dedup_doi___::0ca46ff10b2b4c756191719d85302b14", pub_merged.getId());
assertEquals("50|dedup_doi___::0ca46ff10b2b4c756191719d85302b14", pub_merged.getId());
}

View File

@ -1,11 +1,18 @@
package eu.dnetlib.dhp.oa.dedup;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.count;
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;
@ -21,19 +28,16 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.net.URISyntaxException;
import java.nio.file.Paths;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.count;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.lenient;
@ExtendWith(MockitoExtension.class)
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class SparkDedupTest implements Serializable {
@ -190,10 +194,10 @@ public class SparkDedupTest implements Serializable {
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
.count();
assertEquals(3432, orgs_simrel);
assertEquals(7152, pubs_simrel);
assertEquals(3082, orgs_simrel);
assertEquals(7036, pubs_simrel);
assertEquals(344, sw_simrel);
assertEquals(458, ds_simrel);
assertEquals(442, ds_simrel);
assertEquals(6750, orp_simrel);
}
@ -203,7 +207,7 @@ public class SparkDedupTest implements Serializable {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkCreateSimRels.class
SparkCollectSimRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json")));
parser
@ -214,7 +218,7 @@ public class SparkDedupTest implements Serializable {
"-w", testOutputBasePath,
"-np", "50",
"-purl", "jdbc:postgresql://localhost:5432/dnet_dedup",
"-pusr", "postgres_url",
"-pusr", "postgres_user",
"-ppwd", ""
});
@ -222,8 +226,8 @@ public class SparkDedupTest implements Serializable {
parser,
spark,
spark.read().load(testDedupAssertionsBasePath + "/similarity_groups"),
spark.read().load(testDedupAssertionsBasePath + "/groups")
).run(null);
spark.read().load(testDedupAssertionsBasePath + "/groups"))
.run(isLookUpService);
long orgs_simrel = spark
.read()
@ -250,10 +254,10 @@ public class SparkDedupTest implements Serializable {
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
.count();
assertEquals(4022, orgs_simrel);
assertEquals(10575, pubs_simrel);
assertEquals(3672, orgs_simrel);
assertEquals(10459, pubs_simrel);
assertEquals(3767, sw_simrel);
assertEquals(3881, ds_simrel);
assertEquals(3865, ds_simrel);
assertEquals(10173, orp_simrel);
}
@ -402,8 +406,8 @@ public class SparkDedupTest implements Serializable {
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel")
.count();
assertEquals(1276, orgs_mergerel);
assertEquals(1442, pubs_mergerel);
assertEquals(1272, orgs_mergerel);
assertEquals(1438, pubs_mergerel);
assertEquals(288, sw_mergerel);
assertEquals(472, ds_mergerel);
assertEquals(718, orp_mergerel);
@ -449,10 +453,10 @@ public class SparkDedupTest implements Serializable {
testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_deduprecord")
.count();
assertEquals(82, orgs_deduprecord);
assertEquals(66, pubs_deduprecord);
assertEquals(84, orgs_deduprecord);
assertEquals(65, pubs_deduprecord);
assertEquals(51, sw_deduprecord);
assertEquals(96, ds_deduprecord);
assertEquals(97, ds_deduprecord);
assertEquals(89, orp_deduprecord);
}
@ -532,12 +536,12 @@ public class SparkDedupTest implements Serializable {
.distinct()
.count();
assertEquals(897, publications);
assertEquals(835, organizations);
assertEquals(896, publications);
assertEquals(837, organizations);
assertEquals(100, projects);
assertEquals(100, datasource);
assertEquals(200, softwares);
assertEquals(388, dataset);
assertEquals(389, dataset);
assertEquals(517, otherresearchproduct);
long deletedOrgs = jsc
@ -592,7 +596,7 @@ public class SparkDedupTest implements Serializable {
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
assertEquals(4866, relations);
assertEquals(4858, relations);
// check deletedbyinference
final Dataset<Relation> mergeRels = spark
@ -641,11 +645,11 @@ public class SparkDedupTest implements Serializable {
assertEquals(expected_unique, rel.distinct().count());
}
// @AfterAll
// public static void finalCleanUp() throws IOException {
// FileUtils.deleteDirectory(new File(testOutputBasePath));
// FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
// }
@AfterAll
public static void finalCleanUp() throws IOException {
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
}
public boolean isDeletedByInference(String s) {
return s.contains("\"deletedbyinference\":true");

View File

@ -315,7 +315,7 @@
<dependency>
<groupId>eu.dnetlib</groupId>
<artifactId>dnet-pace-core</artifactId>
<version>4.0.4</version>
<version>4.0.5</version>
</dependency>
<dependency>
<groupId>eu.dnetlib</groupId>