forked from D-Net/dnet-hadoop
bug fix in graph-mapper, changes in the implementation of the openorgs wf to create relations and populate openorgs db
This commit is contained in:
parent
4b2124a18e
commit
1a85020572
|
@ -90,7 +90,10 @@
|
||||||
<groupId>com.fasterxml.jackson.core</groupId>
|
<groupId>com.fasterxml.jackson.core</groupId>
|
||||||
<artifactId>jackson-core</artifactId>
|
<artifactId>jackson-core</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.httpcomponents</groupId>
|
||||||
|
<artifactId>httpclient</artifactId>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -29,6 +29,7 @@ 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 int NUM_PARTITIONS = 1000;
|
||||||
|
protected static final int NUM_CONNECTIONS = 20;
|
||||||
|
|
||||||
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);
|
||||||
|
|
|
@ -95,6 +95,11 @@ public class DedupUtility {
|
||||||
return String.format("%s/%s/%s_simrel", basePath, actionSetId, entityType);
|
return String.format("%s/%s/%s_simrel", basePath, actionSetId, entityType);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static String createOpenorgsMergeRelsPath(
|
||||||
|
final String basePath, final String actionSetId, final String entityType) {
|
||||||
|
return String.format("%s/%s/%s_openorgs_mergerels", basePath, actionSetId, entityType);
|
||||||
|
}
|
||||||
|
|
||||||
public static String createMergeRelPath(
|
public static String createMergeRelPath(
|
||||||
final String basePath, final String actionSetId, final String entityType) {
|
final String basePath, final String actionSetId, final String entityType) {
|
||||||
return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType);
|
return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType);
|
||||||
|
|
|
@ -6,6 +6,7 @@ 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.JavaSparkContext;
|
||||||
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;
|
||||||
|
@ -19,6 +20,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.schema.common.EntityType;
|
import eu.dnetlib.dhp.schema.common.EntityType;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
import eu.dnetlib.dhp.schema.oaf.OafEntity;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Organization;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
@ -34,7 +36,7 @@ public class SparkCopyOpenorgs extends AbstractSparkAction {
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
IOUtils
|
IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
SparkCreateSimRels.class
|
SparkCopyOpenorgs.class
|
||||||
.getResourceAsStream(
|
.getResourceAsStream(
|
||||||
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json")));
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
@ -72,7 +74,7 @@ public class SparkCopyOpenorgs extends AbstractSparkAction {
|
||||||
|
|
||||||
final Class<OafEntity> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
|
final Class<OafEntity> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
|
||||||
|
|
||||||
filterEntities(spark, entityPath, clazz)
|
filterOpenorgs(spark, entityPath)
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.mode(SaveMode.Overwrite)
|
||||||
.option("compression", "gzip")
|
.option("compression", "gzip")
|
||||||
|
@ -80,21 +82,20 @@ public class SparkCopyOpenorgs extends AbstractSparkAction {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static <T extends OafEntity> Dataset<T> filterEntities(
|
public static Dataset<Organization> filterOpenorgs(
|
||||||
final SparkSession spark,
|
final SparkSession spark,
|
||||||
final String entitiesInputPath,
|
final String entitiesInputPath) {
|
||||||
final Class<T> clazz) {
|
|
||||||
|
|
||||||
// <id, json_entity>
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
Dataset<T> entities = spark
|
Dataset<Organization> entities = spark
|
||||||
.read()
|
.createDataset(
|
||||||
.textFile(entitiesInputPath)
|
sc
|
||||||
.map(
|
.textFile(entitiesInputPath)
|
||||||
(MapFunction<String, T>) it -> {
|
.map(it -> OBJECT_MAPPER.readValue(it, Organization.class))
|
||||||
T entity = OBJECT_MAPPER.readValue(it, clazz);
|
.rdd(),
|
||||||
return entity;
|
Encoders.bean(Organization.class));
|
||||||
},
|
|
||||||
Encoders.kryo(clazz));
|
entities.show();
|
||||||
|
|
||||||
return entities.filter(entities.col("id").contains("openorgs____"));
|
return entities.filter(entities.col("id").contains("openorgs____"));
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,14 +6,13 @@ import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
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.Encoders;
|
||||||
|
import org.apache.spark.sql.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.dom4j.DocumentException;
|
import org.dom4j.DocumentException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -21,10 +20,13 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
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.Qualifier;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
|
||||||
//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication
|
//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication
|
||||||
public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
||||||
|
@ -83,17 +85,17 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
||||||
.textFile(relationPath)
|
.textFile(relationPath)
|
||||||
.map(patchRelFn(), Encoders.bean(Relation.class))
|
.map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.filter(this::isOpenorgs) //takes only relations coming from openorgs
|
.filter(this::isOpenorgs) // takes only relations coming from openorgs
|
||||||
.filter(this::filterOpenorgsRels) //takes only isSimilarTo relations between organizations from openorgs
|
.filter(this::filterOpenorgsRels) // takes only isSimilarTo relations between organizations from openorgs
|
||||||
.filter(this::excludeOpenorgsMesh) //excludes relations between an organization and an openorgsmesh
|
.filter(this::excludeOpenorgsMesh) // excludes relations between an organization and an openorgsmesh
|
||||||
.filter(this::excludeNonOpenorgs); //excludes relations with no openorgs id involved
|
.filter(this::excludeNonOpenorgs); // excludes relations with no openorgs id involved
|
||||||
|
|
||||||
//turn openorgs isSimilarTo relations into mergerels
|
// turn openorgs isSimilarTo relations into mergerels
|
||||||
JavaRDD<Relation> mergeRels = rawRels.flatMap(rel -> {
|
JavaRDD<Relation> mergeRelsRDD = rawRels.flatMap(rel -> {
|
||||||
List<Relation> mergerels = new ArrayList<>();
|
List<Relation> mergerels = new ArrayList<>();
|
||||||
|
|
||||||
String openorgsId = rel.getSource().contains("openorgs____")? rel.getSource() : rel.getTarget();
|
String openorgsId = rel.getSource().contains("openorgs____") ? rel.getSource() : rel.getTarget();
|
||||||
String mergedId = rel.getSource().contains("openorgs____")? rel.getTarget() : rel.getSource();
|
String mergedId = rel.getSource().contains("openorgs____") ? rel.getTarget() : rel.getSource();
|
||||||
|
|
||||||
mergerels.add(rel(openorgsId, mergedId, "merges", dedupConf));
|
mergerels.add(rel(openorgsId, mergedId, "merges", dedupConf));
|
||||||
mergerels.add(rel(mergedId, openorgsId, "isMergedIn", dedupConf));
|
mergerels.add(rel(mergedId, openorgsId, "isMergedIn", dedupConf));
|
||||||
|
@ -101,7 +103,13 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
||||||
return mergerels.iterator();
|
return mergerels.iterator();
|
||||||
});
|
});
|
||||||
|
|
||||||
mergeRels.saveAsTextFile(outputPath);
|
spark
|
||||||
|
.createDataset(
|
||||||
|
mergeRelsRDD.rdd(),
|
||||||
|
Encoders.bean(Relation.class))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.parquet(outputPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static MapFunction<String, Relation> patchRelFn() {
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
|
@ -116,7 +124,8 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
||||||
|
|
||||||
private boolean filterOpenorgsRels(Relation rel) {
|
private boolean filterOpenorgsRels(Relation rel) {
|
||||||
|
|
||||||
if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup"))
|
if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization")
|
||||||
|
&& rel.getSubRelType().equals("dedup"))
|
||||||
return true;
|
return true;
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
@ -124,7 +133,7 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction {
|
||||||
private boolean isOpenorgs(Relation rel) {
|
private boolean isOpenorgs(Relation rel) {
|
||||||
|
|
||||||
if (rel.getCollectedfrom() != null) {
|
if (rel.getCollectedfrom() != null) {
|
||||||
for (KeyValue k: rel.getCollectedfrom()) {
|
for (KeyValue k : rel.getCollectedfrom()) {
|
||||||
if (k.getValue().equals("OpenOrgs Database")) {
|
if (k.getValue().equals("OpenOrgs Database")) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -6,13 +6,13 @@ import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.Qualifier;
|
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
import org.apache.spark.graphx.Edge;
|
||||||
|
import org.apache.spark.rdd.RDD;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
@ -22,97 +22,98 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
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.Qualifier;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
|
||||||
//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication
|
//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication
|
||||||
public class SparkCopyOpenorgsSimRels extends AbstractSparkAction {
|
public class SparkCopyOpenorgsSimRels extends AbstractSparkAction {
|
||||||
private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsMergeRels.class);
|
private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsSimRels.class);
|
||||||
|
|
||||||
public SparkCopyOpenorgsSimRels(ArgumentApplicationParser parser, SparkSession spark) {
|
public SparkCopyOpenorgsSimRels(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
super(parser, spark);
|
super(parser, spark);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
IOUtils
|
IOUtils
|
||||||
.toString(
|
.toString(
|
||||||
SparkCopyOpenorgsSimRels.class
|
SparkCopyOpenorgsSimRels.class
|
||||||
.getResourceAsStream(
|
.getResourceAsStream(
|
||||||
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json")));
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
new SparkCopyOpenorgsSimRels(parser, getSparkSession(conf))
|
new SparkCopyOpenorgsSimRels(parser, getSparkSession(conf))
|
||||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run(ISLookUpService isLookUpService)
|
public void run(ISLookUpService isLookUpService)
|
||||||
throws DocumentException, IOException, ISLookUpException {
|
throws DocumentException, IOException, ISLookUpException {
|
||||||
|
|
||||||
// read oozie parameters
|
// read oozie parameters
|
||||||
final String graphBasePath = parser.get("graphBasePath");
|
final String graphBasePath = parser.get("graphBasePath");
|
||||||
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
|
final int numPartitions = Optional
|
||||||
.ofNullable(parser.get("numPartitions"))
|
.ofNullable(parser.get("numPartitions"))
|
||||||
.map(Integer::valueOf)
|
.map(Integer::valueOf)
|
||||||
.orElse(NUM_PARTITIONS);
|
.orElse(NUM_PARTITIONS);
|
||||||
|
|
||||||
log.info("numPartitions: '{}'", numPartitions);
|
log.info("numPartitions: '{}'", numPartitions);
|
||||||
log.info("graphBasePath: '{}'", graphBasePath);
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
log.info("workingPath: '{}'", workingPath);
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
|
||||||
log.info("Copying OpenOrgs SimRels");
|
log.info("Copying OpenOrgs SimRels");
|
||||||
|
|
||||||
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, "organization");
|
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, "organization");
|
||||||
|
|
||||||
removeOutputDir(spark, outputPath);
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
|
|
||||||
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
Dataset<Relation> rawRels = spark
|
||||||
|
.read()
|
||||||
|
.textFile(relationPath)
|
||||||
|
.map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
|
.filter(this::filterOpenorgsRels);
|
||||||
|
|
||||||
JavaRDD<Relation> rawRels = spark
|
save(rawRels, outputPath, SaveMode.Append);
|
||||||
.read()
|
|
||||||
.textFile(relationPath)
|
|
||||||
.map(patchRelFn(), Encoders.bean(Relation.class))
|
|
||||||
.toJavaRDD()
|
|
||||||
.filter(this::isOpenorgs)
|
|
||||||
.filter(this::filterOpenorgsRels);
|
|
||||||
|
|
||||||
save(spark.createDataset(rawRels.rdd(),Encoders.bean(Relation.class)), outputPath, SaveMode.Append);
|
log.info("Copied " + rawRels.count() + " Similarity Relations");
|
||||||
}
|
}
|
||||||
|
|
||||||
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);
|
||||||
if (rel.getDataInfo() == null) {
|
if (rel.getDataInfo() == null) {
|
||||||
rel.setDataInfo(new DataInfo());
|
rel.setDataInfo(new DataInfo());
|
||||||
}
|
}
|
||||||
return rel;
|
return rel;
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean filterOpenorgsRels(Relation rel) {
|
private boolean filterOpenorgsRels(Relation rel) {
|
||||||
|
|
||||||
if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup"))
|
if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization")
|
||||||
return true;
|
&& rel.getSubRelType().equals("dedup") && isOpenorgs(rel))
|
||||||
return false;
|
return true;
|
||||||
}
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
private boolean isOpenorgs(Relation rel) {
|
private boolean isOpenorgs(Relation rel) {
|
||||||
|
|
||||||
if (rel.getCollectedfrom() != null) {
|
if (rel.getCollectedfrom() != null) {
|
||||||
for (KeyValue k: rel.getCollectedfrom()) {
|
for (KeyValue k : rel.getCollectedfrom()) {
|
||||||
if (k.getValue().equals("OpenOrgs Database")) {
|
if (k.getValue() != null && k.getValue().equals("OpenOrgs Database")) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,12 +1,9 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.oa.dedup;
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import java.io.IOException;
|
||||||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
import java.util.Optional;
|
||||||
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.util.MapDocumentUtil;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
@ -19,92 +16,95 @@ 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.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.*;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
|
||||||
import org.apache.spark.sql.Row;
|
|
||||||
import org.apache.spark.sql.SparkSession;
|
|
||||||
import org.dom4j.DocumentException;
|
import org.dom4j.DocumentException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import java.util.Optional;
|
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.util.MapDocumentUtil;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction {
|
public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class);
|
private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class);
|
||||||
|
|
||||||
private static final String IDJSONPATH = "$.id";
|
public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
|
super(parser, spark);
|
||||||
|
}
|
||||||
|
|
||||||
public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) {
|
public static void main(String[] args) throws Exception {
|
||||||
super(parser, spark);
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
}
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCopyRelationsNoOpenorgs.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
SparkConf conf = new SparkConf();
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
IOUtils
|
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||||
.toString(
|
|
||||||
SparkCopyRelationsNoOpenorgs.class
|
|
||||||
.getResourceAsStream(
|
|
||||||
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
|
||||||
parser.parseArgument(args);
|
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
new SparkUpdateEntity(parser, getSparkSession(conf))
|
||||||
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
}
|
||||||
|
|
||||||
new SparkUpdateEntity(parser, getSparkSession(conf))
|
public void run(ISLookUpService isLookUpService) throws IOException {
|
||||||
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
|
||||||
}
|
|
||||||
|
|
||||||
public void run(ISLookUpService isLookUpService) throws IOException {
|
final String graphBasePath = parser.get("graphBasePath");
|
||||||
|
final String workingPath = parser.get("workingPath");
|
||||||
|
final String dedupGraphPath = parser.get("dedupGraphPath");
|
||||||
|
|
||||||
final String graphBasePath = parser.get("graphBasePath");
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
final String workingPath = parser.get("workingPath");
|
log.info("workingPath: '{}'", workingPath);
|
||||||
final String dedupGraphPath = parser.get("dedupGraphPath");
|
log.info("dedupGraphPath: '{}'", dedupGraphPath);
|
||||||
|
|
||||||
log.info("graphBasePath: '{}'", graphBasePath);
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
log.info("workingPath: '{}'", workingPath);
|
final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
|
||||||
log.info("dedupGraphPath: '{}'", dedupGraphPath);
|
|
||||||
|
|
||||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
removeOutputDir(spark, outputPath);
|
||||||
|
|
||||||
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
JavaRDD<Relation> simRels = spark
|
||||||
final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation");
|
.read()
|
||||||
|
.textFile(relationPath)
|
||||||
|
.map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.filter(this::excludeOpenorgsRels);
|
||||||
|
|
||||||
removeOutputDir(spark, outputPath);
|
spark
|
||||||
|
.createDataset(simRels.rdd(), Encoders.bean(Relation.class))
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.json(outputPath);
|
||||||
|
|
||||||
JavaRDD<Relation> simRels = spark
|
}
|
||||||
.read()
|
|
||||||
.textFile(relationPath)
|
|
||||||
.map(patchRelFn(), Encoders.bean(Relation.class))
|
|
||||||
.toJavaRDD()
|
|
||||||
.filter(this::excludeOpenorgsRels);
|
|
||||||
|
|
||||||
simRels.saveAsTextFile(outputPath);
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
|
return value -> {
|
||||||
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
|
if (rel.getDataInfo() == null) {
|
||||||
|
rel.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
|
return rel;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
}
|
private boolean excludeOpenorgsRels(Relation rel) {
|
||||||
|
|
||||||
private static MapFunction<String, Relation> patchRelFn() {
|
if (rel.getCollectedfrom() != null) {
|
||||||
return value -> {
|
for (KeyValue k : rel.getCollectedfrom()) {
|
||||||
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
if (k.getValue().equals("OpenOrgs Database")) {
|
||||||
if (rel.getDataInfo() == null) {
|
return false;
|
||||||
rel.setDataInfo(new DataInfo());
|
}
|
||||||
}
|
}
|
||||||
return rel;
|
}
|
||||||
};
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean excludeOpenorgsRels(Relation rel) {
|
|
||||||
|
|
||||||
if (rel.getCollectedfrom() != null) {
|
|
||||||
for (KeyValue k: rel.getCollectedfrom()) {
|
|
||||||
if (k.getValue().equals("OpenOrgs Database")) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -10,6 +10,7 @@ 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.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
@ -81,7 +82,6 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
log.info("Creating simrels for: '{}'", subEntity);
|
log.info("Creating simrels for: '{}'", subEntity);
|
||||||
|
|
||||||
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
|
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
|
||||||
removeOutputDir(spark, outputPath);
|
|
||||||
|
|
||||||
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
@ -99,13 +99,19 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
.createSortedBlocks(mapDocuments, dedupConf)
|
.createSortedBlocks(mapDocuments, dedupConf)
|
||||||
.repartition(numPartitions);
|
.repartition(numPartitions);
|
||||||
|
|
||||||
// create relations by comparing only elements in the same group
|
Dataset<Relation> simRels = spark
|
||||||
Deduper
|
.createDataset(
|
||||||
.computeRelations(sc, blocks, dedupConf)
|
Deduper
|
||||||
.map(t -> createSimRel(t._1(), t._2(), entity))
|
.computeRelations(sc, blocks, dedupConf)
|
||||||
.repartition(numPartitions)
|
.map(t -> createSimRel(t._1(), t._2(), entity))
|
||||||
.map(r -> OBJECT_MAPPER.writeValueAsString(r))
|
.repartition(numPartitions)
|
||||||
.saveAsTextFile(outputPath);
|
.rdd(),
|
||||||
|
Encoders.bean(Relation.class));
|
||||||
|
|
||||||
|
save(simRels, outputPath, SaveMode.Append);
|
||||||
|
|
||||||
|
log.info("Generated " + simRels.count() + " Similarity Relations");
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,249 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Properties;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.http.client.methods.CloseableHttpResponse;
|
||||||
|
import org.apache.http.client.methods.HttpGet;
|
||||||
|
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.JavaPairRDD;
|
||||||
|
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.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 eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.OrgSimRel;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Organization;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
|
public class SparkPrepareNewOrgs extends AbstractSparkAction {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(SparkPrepareNewOrgs.class);
|
||||||
|
|
||||||
|
public SparkPrepareNewOrgs(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
|
super(parser, spark);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkPrepareNewOrgs.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||||
|
|
||||||
|
new SparkPrepareNewOrgs(parser, getSparkSession(conf))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(ISLookUpService isLookUpService) throws IOException {
|
||||||
|
|
||||||
|
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 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");
|
||||||
|
final String dbPwd = parser.get("dbPwd");
|
||||||
|
|
||||||
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
|
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);
|
||||||
|
log.info("table: '{}'", dbTable);
|
||||||
|
log.info("dbPwd: '{}'", "xxx");
|
||||||
|
|
||||||
|
final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization");
|
||||||
|
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization");
|
||||||
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
|
|
||||||
|
Dataset<OrgSimRel> newOrgs = createNewOrgs(spark, mergeRelPath, relationPath, entityPath);
|
||||||
|
|
||||||
|
final Properties connectionProperties = new Properties();
|
||||||
|
connectionProperties.put("user", dbUser);
|
||||||
|
connectionProperties.put("password", dbPwd);
|
||||||
|
|
||||||
|
log.info("Number of New Organization created: '{}'", newOrgs.count());
|
||||||
|
|
||||||
|
newOrgs
|
||||||
|
.repartition(numConnections)
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.jdbc(dbUrl, dbTable, connectionProperties);
|
||||||
|
|
||||||
|
if (!apiUrl.isEmpty())
|
||||||
|
updateSimRels(apiUrl);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Dataset<OrgSimRel> createNewOrgs(
|
||||||
|
final SparkSession spark,
|
||||||
|
final String mergeRelsPath,
|
||||||
|
final String relationPath,
|
||||||
|
final String entitiesPath) {
|
||||||
|
|
||||||
|
// collect diffrels from the raw graph relations: <other id, "diffRel">
|
||||||
|
JavaPairRDD<String, String> diffRels = spark
|
||||||
|
.read()
|
||||||
|
.textFile(relationPath)
|
||||||
|
.map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.filter(r -> filterRels(r, "organization"))
|
||||||
|
// take the worst id of the diffrel: <other id, "diffRel">
|
||||||
|
.mapToPair(rel -> {
|
||||||
|
if (compareIds(rel.getSource(), rel.getTarget()) > 0)
|
||||||
|
return new Tuple2<>(rel.getSource(), "diffRel");
|
||||||
|
else
|
||||||
|
return new Tuple2<>(rel.getTarget(), "diffRel");
|
||||||
|
})
|
||||||
|
.distinct();
|
||||||
|
log.info("Number of DiffRels collected: '{}'", diffRels.count());
|
||||||
|
|
||||||
|
// collect entities: <id, json_entity>
|
||||||
|
Dataset<Tuple2<String, Organization>> entities = spark
|
||||||
|
.read()
|
||||||
|
.textFile(entitiesPath)
|
||||||
|
.map(
|
||||||
|
(MapFunction<String, Tuple2<String, Organization>>) it -> {
|
||||||
|
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
|
||||||
|
return new Tuple2<>(entity.getId(), entity);
|
||||||
|
},
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
|
||||||
|
|
||||||
|
// collect mergerels and remove ids in the diffrels
|
||||||
|
Dataset<Tuple2<String, String>> openorgsRels = spark
|
||||||
|
.createDataset(
|
||||||
|
spark
|
||||||
|
.read()
|
||||||
|
.load(mergeRelsPath)
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.where("relClass == 'isMergedIn'")
|
||||||
|
.toJavaRDD()
|
||||||
|
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget())) // <id, dedup_id>
|
||||||
|
.leftOuterJoin(diffRels) // <target, "diffRel">
|
||||||
|
.filter(rel -> !rel._2()._2().isPresent())
|
||||||
|
.mapToPair(rel -> new Tuple2<>(rel._1(), rel._2()._1()))
|
||||||
|
.rdd(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
log.info("Number of Openorgs Relations loaded: '{}'", openorgsRels.count());
|
||||||
|
|
||||||
|
return entities
|
||||||
|
.joinWith(openorgsRels, entities.col("_1").equalTo(openorgsRels.col("_1")), "left")
|
||||||
|
.filter((FilterFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>>) t -> t._2() == null)
|
||||||
|
// take entities not in mergerels (they are single entities, therefore are new orgs)
|
||||||
|
.filter(
|
||||||
|
(FilterFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>>) t -> !t
|
||||||
|
._1()
|
||||||
|
._1()
|
||||||
|
.contains("openorgs"))
|
||||||
|
// exclude openorgs, don't need to propose them as new orgs
|
||||||
|
.map(
|
||||||
|
(MapFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>, OrgSimRel>) r -> new OrgSimRel(
|
||||||
|
"",
|
||||||
|
r._1()._2().getOriginalId().get(0),
|
||||||
|
r._1()._2().getLegalname() != null ? r._1()._2().getLegalname().getValue() : "",
|
||||||
|
r._1()._2().getLegalshortname() != null ? r._1()._2().getLegalshortname().getValue() : "",
|
||||||
|
r._1()._2().getCountry() != null ? r._1()._2().getCountry().getClassid() : "",
|
||||||
|
r._1()._2().getWebsiteurl() != null ? r._1()._2().getWebsiteurl().getValue() : "",
|
||||||
|
r._1()._2().getCollectedfrom().get(0).getValue(), ""),
|
||||||
|
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)) {
|
||||||
|
return IOUtils.toString(response.getEntity().getContent());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
|
return value -> {
|
||||||
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
|
if (rel.getDataInfo() == null) {
|
||||||
|
rel.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
|
return rel;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int compareIds(String o1, String o2) {
|
||||||
|
if (o1.contains("openorgs____") && o2.contains("openorgs____"))
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
if (o1.contains("corda") && o2.contains("corda"))
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
|
||||||
|
if (o1.contains("openorgs____"))
|
||||||
|
return -1;
|
||||||
|
if (o2.contains("openorgs____"))
|
||||||
|
return 1;
|
||||||
|
|
||||||
|
if (o1.contains("corda"))
|
||||||
|
return -1;
|
||||||
|
if (o2.contains("corda"))
|
||||||
|
return 1;
|
||||||
|
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static boolean filterRels(Relation rel, String entityType) {
|
||||||
|
|
||||||
|
switch (entityType) {
|
||||||
|
case "result":
|
||||||
|
if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult")
|
||||||
|
&& rel.getSubRelType().equals("dedup"))
|
||||||
|
return true;
|
||||||
|
break;
|
||||||
|
case "organization":
|
||||||
|
if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization")
|
||||||
|
&& rel.getSubRelType().equals("dedup"))
|
||||||
|
return true;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,341 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.*;
|
||||||
|
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.JavaRDD;
|
||||||
|
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.google.common.collect.Lists;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.OrgSimRel;
|
||||||
|
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Organization;
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
|
||||||
|
import scala.Tuple2;
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
|
public class SparkPrepareOrgRels extends AbstractSparkAction {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(SparkPrepareOrgRels.class);
|
||||||
|
|
||||||
|
public SparkPrepareOrgRels(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
|
super(parser, spark);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCreateSimRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
|
||||||
|
|
||||||
|
new SparkPrepareOrgRels(parser, getSparkSession(conf))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(ISLookUpService isLookUpService) throws IOException {
|
||||||
|
|
||||||
|
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 numConnections = Optional
|
||||||
|
.ofNullable(parser.get("numConnections"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(NUM_CONNECTIONS);
|
||||||
|
|
||||||
|
final String dbUrl = parser.get("dbUrl");
|
||||||
|
final String dbTable = parser.get("dbTable");
|
||||||
|
final String dbUser = parser.get("dbUser");
|
||||||
|
final String dbPwd = parser.get("dbPwd");
|
||||||
|
|
||||||
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
|
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
||||||
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
log.info("numPartitions: '{}'", numConnections);
|
||||||
|
log.info("dbUrl: '{}'", dbUrl);
|
||||||
|
log.info("dbUser: '{}'", dbUser);
|
||||||
|
log.info("table: '{}'", dbTable);
|
||||||
|
log.info("dbPwd: '{}'", "xxx");
|
||||||
|
|
||||||
|
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization");
|
||||||
|
final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization");
|
||||||
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
|
||||||
|
|
||||||
|
Dataset<OrgSimRel> relations = createRelations(spark, mergeRelPath, relationPath, entityPath);
|
||||||
|
|
||||||
|
final Properties connectionProperties = new Properties();
|
||||||
|
connectionProperties.put("user", dbUser);
|
||||||
|
connectionProperties.put("password", dbPwd);
|
||||||
|
|
||||||
|
relations
|
||||||
|
.repartition(numConnections)
|
||||||
|
.write()
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.jdbc(dbUrl, dbTable, connectionProperties);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static boolean filterRels(Relation rel, String entityType) {
|
||||||
|
|
||||||
|
switch (entityType) {
|
||||||
|
case "result":
|
||||||
|
if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult")
|
||||||
|
&& rel.getSubRelType().equals("dedup"))
|
||||||
|
return true;
|
||||||
|
break;
|
||||||
|
case "organization":
|
||||||
|
if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization")
|
||||||
|
&& rel.getSubRelType().equals("dedup"))
|
||||||
|
return true;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// create openorgs simrels <best id, other id> starting from mergerels, remove the diffrels
|
||||||
|
public static Dataset<OrgSimRel> createRelations(
|
||||||
|
final SparkSession spark,
|
||||||
|
final String mergeRelsPath,
|
||||||
|
final String relationPath,
|
||||||
|
final String entitiesPath) {
|
||||||
|
|
||||||
|
// collect diffrels from the raw graph relations: <<best id, other id>, "diffRel">
|
||||||
|
JavaRDD<Tuple2<Tuple2<String, String>, String>> diffRels = spark
|
||||||
|
.read()
|
||||||
|
.textFile(relationPath)
|
||||||
|
.map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.filter(r -> filterRels(r, "organization"))
|
||||||
|
// put the best id as source of the diffrel: <best id, other id>
|
||||||
|
.map(rel -> {
|
||||||
|
if (compareIds(rel.getSource(), rel.getTarget()) < 0)
|
||||||
|
return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel");
|
||||||
|
else
|
||||||
|
return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel");
|
||||||
|
})
|
||||||
|
.distinct();
|
||||||
|
log.info("Number of DiffRels collected: {}", diffRels.count());
|
||||||
|
|
||||||
|
// collect all the organizations
|
||||||
|
Dataset<Tuple2<String, Organization>> entities = spark
|
||||||
|
.read()
|
||||||
|
.textFile(entitiesPath)
|
||||||
|
.map(
|
||||||
|
(MapFunction<String, Tuple2<String, Organization>>) it -> {
|
||||||
|
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
|
||||||
|
return new Tuple2<>(entity.getId(), entity);
|
||||||
|
},
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
|
||||||
|
|
||||||
|
// relations with their group (connected component id)
|
||||||
|
JavaRDD<Tuple2<Tuple2<String, String>, String>> rawOpenorgsRels = spark
|
||||||
|
.read()
|
||||||
|
.load(mergeRelsPath)
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.where("relClass == 'merges'")
|
||||||
|
.toJavaRDD()
|
||||||
|
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget()))
|
||||||
|
.filter(t -> !t._2().contains("openorgsmesh")) // remove openorgsmesh: they are only for dedup
|
||||||
|
.groupByKey()
|
||||||
|
.map(g -> Lists.newArrayList(g._2()))
|
||||||
|
.filter(l -> l.size() > 1)
|
||||||
|
.flatMap(l -> {
|
||||||
|
String groupId = "group::" + UUID.randomUUID();
|
||||||
|
List<String> ids = sortIds(l); // sort IDs by type
|
||||||
|
List<Tuple2<Tuple2<String, String>, String>> rels = new ArrayList<>();
|
||||||
|
String source = ids.get(0);
|
||||||
|
for (String target : ids) {
|
||||||
|
rels.add(new Tuple2<>(new Tuple2<>(source, target), groupId));
|
||||||
|
}
|
||||||
|
|
||||||
|
return rels.iterator();
|
||||||
|
});
|
||||||
|
log.info("Number of Raw Openorgs Relations created: {}", rawOpenorgsRels.count());
|
||||||
|
|
||||||
|
// filter out diffRels
|
||||||
|
JavaRDD<Tuple3<String, String, String>> openorgsRels = rawOpenorgsRels
|
||||||
|
.union(diffRels)
|
||||||
|
// concatenation of source and target: <source|||target, group id> or <source|||target, "diffRel">
|
||||||
|
.mapToPair(t -> new Tuple2<>(t._1()._1() + "@@@" + t._1()._2(), t._2()))
|
||||||
|
.groupByKey()
|
||||||
|
.map(
|
||||||
|
g -> new Tuple2<>(g._1(), StreamSupport
|
||||||
|
.stream(g._2().spliterator(), false)
|
||||||
|
.collect(Collectors.toList())))
|
||||||
|
// <source|||target, list(group_id, "diffRel")>: take only relations with only the group_id, it
|
||||||
|
// means they are correct. If the diffRel is present the relation has to be removed
|
||||||
|
.filter(g -> g._2().size() == 1 && g._2().get(0).contains("group::"))
|
||||||
|
.map(
|
||||||
|
t -> new Tuple3<>(
|
||||||
|
t._1().split("@@@")[0],
|
||||||
|
t._1().split("@@@")[1],
|
||||||
|
t._2().get(0)));
|
||||||
|
log.info("Number of Openorgs Relations created: '{}'", openorgsRels.count());
|
||||||
|
|
||||||
|
// <best ID basing on priority, ID, groupID>
|
||||||
|
Dataset<Tuple3<String, String, String>> relations = spark
|
||||||
|
.createDataset(
|
||||||
|
openorgsRels.rdd(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
// create orgsimrels
|
||||||
|
Dataset<Tuple2<String, OrgSimRel>> relations2 = relations
|
||||||
|
.joinWith(entities, relations.col("_2").equalTo(entities.col("_1")), "inner")
|
||||||
|
.map(
|
||||||
|
(MapFunction<Tuple2<Tuple3<String, String, String>, Tuple2<String, Organization>>, OrgSimRel>) r -> new OrgSimRel(
|
||||||
|
r._1()._1(),
|
||||||
|
r._2()._2().getOriginalId().get(0),
|
||||||
|
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(),
|
||||||
|
r._1()._3()),
|
||||||
|
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));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int compareIds(String o1, String o2) {
|
||||||
|
if (o1.contains("openorgs____") && o2.contains("openorgs____"))
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
if (o1.contains("corda") && o2.contains("corda"))
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
|
||||||
|
if (o1.contains("openorgs____"))
|
||||||
|
return -1;
|
||||||
|
if (o2.contains("openorgs____"))
|
||||||
|
return 1;
|
||||||
|
|
||||||
|
if (o1.contains("corda"))
|
||||||
|
return -1;
|
||||||
|
if (o2.contains("corda"))
|
||||||
|
return 1;
|
||||||
|
|
||||||
|
return o1.compareTo(o2);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Sort IDs basing on the type. Priority: 1) openorgs, 2)corda, 3)alphabetic
|
||||||
|
public static List<String> sortIds(List<String> ids) {
|
||||||
|
ids.sort((o1, o2) -> compareIds(o1, o2));
|
||||||
|
return ids;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Dataset<OrgSimRel> createRelationsFromScratch(
|
||||||
|
final SparkSession spark,
|
||||||
|
final String mergeRelsPath,
|
||||||
|
final String entitiesPath) {
|
||||||
|
|
||||||
|
// <id, json_entity>
|
||||||
|
Dataset<Tuple2<String, Organization>> entities = spark
|
||||||
|
.read()
|
||||||
|
.textFile(entitiesPath)
|
||||||
|
.map(
|
||||||
|
(MapFunction<String, Tuple2<String, Organization>>) it -> {
|
||||||
|
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
|
||||||
|
return new Tuple2<>(entity.getId(), entity);
|
||||||
|
},
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, String>> relations = spark
|
||||||
|
.createDataset(
|
||||||
|
spark
|
||||||
|
.read()
|
||||||
|
.load(mergeRelsPath)
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.where("relClass == 'merges'")
|
||||||
|
.toJavaRDD()
|
||||||
|
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget()))
|
||||||
|
.groupByKey()
|
||||||
|
.flatMap(g -> {
|
||||||
|
List<Tuple2<String, String>> rels = new ArrayList<>();
|
||||||
|
for (String id1 : g._2()) {
|
||||||
|
for (String id2 : g._2()) {
|
||||||
|
if (!id1.equals(id2))
|
||||||
|
if (id1.contains("openorgs____") && !id2.contains("openorgsmesh"))
|
||||||
|
rels.add(new Tuple2<>(id1, id2));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return rels.iterator();
|
||||||
|
})
|
||||||
|
.rdd(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
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()._1(),
|
||||||
|
r._2()._2().getOriginalId().get(0),
|
||||||
|
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(),
|
||||||
|
"group::" + r._1()._1()),
|
||||||
|
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 MapFunction<String, Relation> patchRelFn() {
|
||||||
|
return value -> {
|
||||||
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
|
if (rel.getDataInfo() == null) {
|
||||||
|
rel.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
|
return rel;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -27,6 +27,8 @@ import org.dom4j.DocumentException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
|
import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent;
|
||||||
import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor;
|
import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor;
|
||||||
|
@ -95,6 +97,9 @@ public class SparkRemoveDiffRels extends AbstractSparkAction {
|
||||||
|
|
||||||
final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity);
|
final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity);
|
||||||
|
|
||||||
|
final String openorgsMergeRelsPath = DedupUtility
|
||||||
|
.createOpenorgsMergeRelsPath(workingPath, actionSetId, subEntity);
|
||||||
|
|
||||||
final int maxIterations = dedupConf.getWf().getMaxIterations();
|
final int maxIterations = dedupConf.getWf().getMaxIterations();
|
||||||
log.info("Max iterations {}", maxIterations);
|
log.info("Max iterations {}", maxIterations);
|
||||||
|
|
||||||
|
@ -105,67 +110,103 @@ public class SparkRemoveDiffRels extends AbstractSparkAction {
|
||||||
.where("relClass == 'merges'")
|
.where("relClass == 'merges'")
|
||||||
.toJavaRDD();
|
.toJavaRDD();
|
||||||
|
|
||||||
|
System.out.println("mergeRelsRDD = " + mergeRelsRDD.count());
|
||||||
|
|
||||||
|
// JavaRDD<Tuple2<Tuple2<String, String>, String>> diffRelsRDD = spark
|
||||||
|
// .read()
|
||||||
|
// .textFile(relationPath)
|
||||||
|
// .map(patchRelFn(), Encoders.bean(Relation.class))
|
||||||
|
// .toJavaRDD()
|
||||||
|
// .filter(r -> filterRels(r, entity))
|
||||||
|
// .map(rel -> {
|
||||||
|
// if (rel.getSource().compareTo(rel.getTarget()) < 0)
|
||||||
|
// return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel");
|
||||||
|
// else
|
||||||
|
// return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel");
|
||||||
|
// });
|
||||||
|
// THIS IS FOR TESTING PURPOSE
|
||||||
JavaRDD<Tuple2<Tuple2<String, String>, String>> diffRelsRDD = spark
|
JavaRDD<Tuple2<Tuple2<String, String>, String>> diffRelsRDD = spark
|
||||||
.read()
|
.read()
|
||||||
.textFile(relationPath)
|
.load(mergeRelsPath)
|
||||||
.map(patchRelFn(), Encoders.bean(Relation.class))
|
.as(Encoders.bean(Relation.class))
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.filter(r -> filterRels(r, entity))
|
|
||||||
.map(rel -> {
|
.map(rel -> {
|
||||||
if (rel.getSource().compareTo(rel.getTarget()) < 0)
|
if (rel.getSource().compareTo(rel.getTarget()) < 0)
|
||||||
return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel");
|
return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel");
|
||||||
else
|
else
|
||||||
return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel");
|
return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel");
|
||||||
|
})
|
||||||
|
.distinct();
|
||||||
|
|
||||||
|
System.out.println("diffRelsRDD = " + diffRelsRDD.count());
|
||||||
|
|
||||||
|
// JavaRDD<Tuple2<Tuple2<String, String>, String>> flatMergeRels = mergeRelsRDD
|
||||||
|
// .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget()))
|
||||||
|
// .groupByKey()
|
||||||
|
// .flatMap(g -> {
|
||||||
|
// List<Tuple2<Tuple2<String, String>, String>> rels = new ArrayList<>();
|
||||||
|
//
|
||||||
|
// List<String> ids = StreamSupport
|
||||||
|
// .stream(g._2().spliterator(), false)
|
||||||
|
// .collect(Collectors.toList());
|
||||||
|
//
|
||||||
|
// for (int i = 0; i < ids.size(); i++) {
|
||||||
|
// for (int j = i + 1; j < ids.size(); j++) {
|
||||||
|
// if (ids.get(i).compareTo(ids.get(j)) < 0)
|
||||||
|
// rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1()));
|
||||||
|
// else
|
||||||
|
// rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1()));
|
||||||
|
// }
|
||||||
|
// }
|
||||||
|
// return rels.iterator();
|
||||||
|
//
|
||||||
|
// });
|
||||||
|
JavaRDD<Tuple2<Tuple2<String, String>, String>> mergeRels = mergeRelsRDD
|
||||||
|
.map(rel -> {
|
||||||
|
if (rel.getSource().compareTo(rel.getTarget()) < 0)
|
||||||
|
return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "mergeRel");
|
||||||
|
else
|
||||||
|
return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "mergeRel");
|
||||||
});
|
});
|
||||||
|
System.out.println("mergeRelsProcessed = " + mergeRels.count());
|
||||||
|
|
||||||
JavaRDD<Tuple2<Tuple2<String, String>, String>> flatMergeRels = mergeRelsRDD
|
// JavaRDD<Relation> purgedMergeRels = flatMergeRels
|
||||||
.mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget()))
|
// .union(diffRelsRDD)
|
||||||
.groupByKey()
|
// .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2())))
|
||||||
.flatMap(g -> {
|
// .reduceByKey((a, b) -> {
|
||||||
List<Tuple2<Tuple2<String, String>, String>> rels = new ArrayList<>();
|
// List<String> list = new ArrayList<String>();
|
||||||
|
// list.addAll(a);
|
||||||
List<String> ids = StreamSupport
|
// list.addAll(b);
|
||||||
.stream(g._2().spliterator(), false)
|
// return list;
|
||||||
.collect(Collectors.toList());
|
// })
|
||||||
|
// .filter(rel -> rel._2().size() == 1)
|
||||||
for (int i = 0; i < ids.size(); i++) {
|
// .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1()))
|
||||||
for (int j = i + 1; j < ids.size(); j++) {
|
// .flatMap(rel -> {
|
||||||
if (ids.get(i).compareTo(ids.get(j)) < 0)
|
// List<Tuple2<String, String>> rels = new ArrayList<>();
|
||||||
rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1()));
|
// String source = rel._1();
|
||||||
else
|
// rels.add(new Tuple2<>(source, rel._2()._1()));
|
||||||
rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1()));
|
// rels.add(new Tuple2<>(source, rel._2()._2()));
|
||||||
}
|
// return rels.iterator();
|
||||||
}
|
// })
|
||||||
return rels.iterator();
|
// .distinct()
|
||||||
|
// .flatMap(rel -> tupleToMergeRel(rel, dedupConf));
|
||||||
});
|
JavaRDD<Relation> purgedMergeRels = mergeRels
|
||||||
|
|
||||||
JavaRDD<Relation> purgedMergeRels = flatMergeRels
|
|
||||||
.union(diffRelsRDD)
|
.union(diffRelsRDD)
|
||||||
.mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2())))
|
.mapToPair(t -> new Tuple2<>(t._1()._1() + "|||" + t._1()._2(), t._2()))
|
||||||
.reduceByKey((a, b) -> {
|
.groupByKey()
|
||||||
List<String> list = new ArrayList<String>();
|
.filter(g -> Iterables.size(g._2()) == 1)
|
||||||
list.addAll(a);
|
.flatMap(
|
||||||
list.addAll(b);
|
t -> tupleToMergeRel(
|
||||||
return list;
|
new Tuple2<>(t._1().split("\\|\\|\\|")[0], t._1().split("\\|\\|\\|")[1]),
|
||||||
})
|
dedupConf));
|
||||||
.filter(rel -> rel._2().size() == 1)
|
|
||||||
.mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1()))
|
System.out.println("purgedMergeRels = " + purgedMergeRels.count());
|
||||||
.flatMap(rel -> {
|
|
||||||
List<Tuple2<String, String>> rels = new ArrayList<>();
|
|
||||||
String source = rel._1();
|
|
||||||
rels.add(new Tuple2<>(source, rel._2()._1()));
|
|
||||||
rels.add(new Tuple2<>(source, rel._2()._2()));
|
|
||||||
return rels.iterator();
|
|
||||||
})
|
|
||||||
.distinct()
|
|
||||||
.flatMap(rel -> tupleToMergeRel(rel, dedupConf));
|
|
||||||
|
|
||||||
spark
|
spark
|
||||||
.createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class))
|
.createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class))
|
||||||
.write()
|
.write()
|
||||||
.mode(SaveMode.Overwrite)
|
.mode(SaveMode.Overwrite)
|
||||||
.json(mergeRelsPath);
|
.json(openorgsMergeRelsPath);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,108 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup.model;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
public class OrgSimRel implements Serializable {
|
||||||
|
|
||||||
|
String local_id;
|
||||||
|
String oa_original_id;
|
||||||
|
String oa_name;
|
||||||
|
String oa_acronym;
|
||||||
|
String oa_country;
|
||||||
|
String oa_url;
|
||||||
|
String oa_collectedfrom;
|
||||||
|
String group_id;
|
||||||
|
|
||||||
|
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, String group_id) {
|
||||||
|
this.local_id = local_id;
|
||||||
|
this.oa_original_id = oa_original_id;
|
||||||
|
this.oa_name = oa_name;
|
||||||
|
this.oa_acronym = oa_acronym;
|
||||||
|
this.oa_country = oa_country;
|
||||||
|
this.oa_url = oa_url;
|
||||||
|
this.oa_collectedfrom = oa_collectedfrom;
|
||||||
|
this.group_id = group_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getLocal_id() {
|
||||||
|
return local_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setLocal_id(String local_id) {
|
||||||
|
this.local_id = local_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_original_id() {
|
||||||
|
return oa_original_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_original_id(String oa_original_id) {
|
||||||
|
this.oa_original_id = oa_original_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_name() {
|
||||||
|
return oa_name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_name(String oa_name) {
|
||||||
|
this.oa_name = oa_name;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_acronym() {
|
||||||
|
return oa_acronym;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_acronym(String oa_acronym) {
|
||||||
|
this.oa_acronym = oa_acronym;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_country() {
|
||||||
|
return oa_country;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_country(String oa_country) {
|
||||||
|
this.oa_country = oa_country;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_url() {
|
||||||
|
return oa_url;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_url(String oa_url) {
|
||||||
|
this.oa_url = oa_url;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getOa_collectedfrom() {
|
||||||
|
return oa_collectedfrom;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOa_collectedfrom(String oa_collectedfrom) {
|
||||||
|
this.oa_collectedfrom = oa_collectedfrom;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getGroup_id() {
|
||||||
|
return group_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setGroup_id(String group_id) {
|
||||||
|
this.group_id = group_id;
|
||||||
|
}
|
||||||
|
|
||||||
|
@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 + '\'' +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
|
@ -79,7 +79,7 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</global>
|
</global>
|
||||||
|
|
||||||
<start to="resetWorkingPath"/>
|
<start to="resetOrgSimRels"/>
|
||||||
|
|
||||||
<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>
|
||||||
|
@ -88,8 +88,9 @@
|
||||||
<action name="resetOrgSimRels">
|
<action name="resetOrgSimRels">
|
||||||
<fs>
|
<fs>
|
||||||
<delete path="${workingPath}/${actionSetId}/organization_simrel"/>
|
<delete path="${workingPath}/${actionSetId}/organization_simrel"/>
|
||||||
|
<delete path="${workingPath}/${actionSetId}/organization_mergerel"/>
|
||||||
</fs>
|
</fs>
|
||||||
<ok to="copySimRels"/>
|
<ok to="CreateSimRels"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
@ -120,7 +121,7 @@
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<!-- copy similarity relations coming from openorgs in order to improve dedup quality -->
|
<!-- copy simrels relations coming from openorgs -->
|
||||||
<action name="CopyOpenorgsSimRels">
|
<action name="CopyOpenorgsSimRels">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
|
@ -139,6 +140,7 @@
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||||
|
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
<arg>--numPartitions</arg><arg>8000</arg>
|
<arg>--numPartitions</arg><arg>8000</arg>
|
||||||
|
@ -170,33 +172,6 @@
|
||||||
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||||
<arg>--cutConnectedComponent</arg><arg>${cutConnectedComponent}</arg>
|
<arg>--cutConnectedComponent</arg><arg>${cutConnectedComponent}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="RemoveDifferences"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="RemoveDifferences">
|
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
|
||||||
<master>yarn</master>
|
|
||||||
<mode>cluster</mode>
|
|
||||||
<name>Create Merge Relations</name>
|
|
||||||
<class>eu.dnetlib.dhp.oa.dedup.SparkRemoveDiffRels</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>${workingPath}</arg>
|
|
||||||
<arg>--numPartitions</arg><arg>8000</arg>
|
|
||||||
</spark>
|
|
||||||
<ok to="PrepareOrgRels"/>
|
<ok to="PrepareOrgRels"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
|
@ -0,0 +1,62 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "i",
|
||||||
|
"paramLongName": "graphBasePath",
|
||||||
|
"paramDescription": "the base path of raw graph",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "w",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "the working directory path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "la",
|
||||||
|
"paramLongName": "isLookUpUrl",
|
||||||
|
"paramDescription": "the url of the lookup service",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "asi",
|
||||||
|
"paramLongName": "actionSetId",
|
||||||
|
"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": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "du",
|
||||||
|
"paramLongName": "dbUrl",
|
||||||
|
"paramDescription": "the url of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dusr",
|
||||||
|
"paramLongName": "dbUser",
|
||||||
|
"paramDescription": "the user of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "t",
|
||||||
|
"paramLongName": "dbTable",
|
||||||
|
"paramDescription": "the name of the table in the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dpwd",
|
||||||
|
"paramLongName": "dbPwd",
|
||||||
|
"paramDescription": "the password for the user of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
}
|
||||||
|
]
|
|
@ -0,0 +1,56 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "i",
|
||||||
|
"paramLongName": "graphBasePath",
|
||||||
|
"paramDescription": "the base path of raw graph",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "w",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "the working directory path",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "la",
|
||||||
|
"paramLongName": "isLookUpUrl",
|
||||||
|
"paramDescription": "the url of the lookup service",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "asi",
|
||||||
|
"paramLongName": "actionSetId",
|
||||||
|
"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": "du",
|
||||||
|
"paramLongName": "dbUrl",
|
||||||
|
"paramDescription": "the url of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dusr",
|
||||||
|
"paramLongName": "dbUser",
|
||||||
|
"paramDescription": "the user of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "t",
|
||||||
|
"paramLongName": "dbTable",
|
||||||
|
"paramDescription": "the name of the table in the database",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dpwd",
|
||||||
|
"paramLongName": "dbPwd",
|
||||||
|
"paramDescription": "the password for the user of the database",
|
||||||
|
"paramRequired": true
|
||||||
|
}
|
||||||
|
]
|
|
@ -0,0 +1,296 @@
|
||||||
|
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
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;
|
||||||
|
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.MapFunction;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Encoders;
|
||||||
|
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 com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
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 jdk.nashorn.internal.ir.annotations.Ignore;
|
||||||
|
|
||||||
|
@ExtendWith(MockitoExtension.class)
|
||||||
|
public class SparkOpenorgsTest 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 String testDedupGraphBasePath;
|
||||||
|
private static final String testActionSetId = "test-orchestrator";
|
||||||
|
|
||||||
|
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
|
||||||
|
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
|
||||||
|
@BeforeAll
|
||||||
|
public static void cleanUp() throws IOException, URISyntaxException {
|
||||||
|
|
||||||
|
testGraphBasePath = Paths
|
||||||
|
.get(SparkOpenorgsTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI())
|
||||||
|
.toFile()
|
||||||
|
.getAbsolutePath();
|
||||||
|
testOutputBasePath = createTempDirectory(SparkOpenorgsTest.class.getSimpleName() + "-")
|
||||||
|
.toAbsolutePath()
|
||||||
|
.toString();
|
||||||
|
testDedupGraphBasePath = createTempDirectory(SparkOpenorgsTest.class.getSimpleName() + "-")
|
||||||
|
.toAbsolutePath()
|
||||||
|
.toString();
|
||||||
|
|
||||||
|
// FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||||
|
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||||
|
FileUtils.deleteDirectory(new File("/tmp/test-orchestrator/organization_openorgs_mergerels"));
|
||||||
|
|
||||||
|
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_openorgs.xml")));
|
||||||
|
|
||||||
|
lenient()
|
||||||
|
.when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization")))
|
||||||
|
.thenReturn(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkDedupTest.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void copyOpenorgsTest() throws Exception {
|
||||||
|
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCopyOpenorgs.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-w", testOutputBasePath,
|
||||||
|
"-np", "50"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCopyOpenorgs(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_deduprecord = jsc
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(0, orgs_deduprecord);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void copyOpenorgsMergeRels() throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCopyOpenorgsMergeRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-w", testOutputBasePath,
|
||||||
|
"-la", "lookupurl",
|
||||||
|
"-np", "50"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCopyOpenorgsMergeRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(0, orgs_mergerel);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void copyOpenorgsSimRels() throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCopyOpenorgsSimRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-w", testOutputBasePath,
|
||||||
|
"-la", "lookupurl",
|
||||||
|
"-np", "50"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCopyOpenorgsSimRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_simrel = spark
|
||||||
|
.read()
|
||||||
|
.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
System.out.println("orgs_simrel = " + orgs_simrel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void createSimRelsTest() throws Exception {
|
||||||
|
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCreateSimRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-la", "lookupurl",
|
||||||
|
"-w", "/tmp",
|
||||||
|
"-np", "50"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCreateSimRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_simrel = spark
|
||||||
|
.read()
|
||||||
|
.textFile("/tmp/" + testActionSetId + "/organization_simrel")
|
||||||
|
.count();
|
||||||
|
|
||||||
|
assertEquals(3082, orgs_simrel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void createMergeRelsTest() 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",
|
||||||
|
"/tmp"
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCreateMergeRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_mergerel = spark
|
||||||
|
.read()
|
||||||
|
.load("/tmp/" + testActionSetId + "/organization_mergerel")
|
||||||
|
.count();
|
||||||
|
assertEquals(1272, orgs_mergerel);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void copyRelationsNoOpenorgsTest() throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkCopyRelationsNoOpenorgs.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-w", testOutputBasePath,
|
||||||
|
"-o", testDedupGraphBasePath
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count();
|
||||||
|
|
||||||
|
// Dataset<Relation> relsRDD = spark.read().textFile(testDedupGraphBasePath + "/relation").map(patchRelFn(), Encoders.bean(Relation.class));
|
||||||
|
|
||||||
|
assertEquals(500, relations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@AfterAll
|
||||||
|
public static void finalCleanUp() throws IOException {
|
||||||
|
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||||
|
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MapFunction<String, Relation> patchRelFn() {
|
||||||
|
return value -> {
|
||||||
|
final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class);
|
||||||
|
if (rel.getDataInfo() == null) {
|
||||||
|
rel.setDataInfo(new DataInfo());
|
||||||
|
}
|
||||||
|
return rel;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,24 @@
|
||||||
|
<RESOURCE_PROFILE>
|
||||||
|
<HEADER>
|
||||||
|
<RESOURCE_IDENTIFIER value=""/>
|
||||||
|
<RESOURCE_TYPE value="DedupOrchestrationDSResourceType"/>
|
||||||
|
<RESOURCE_KIND value="DedupOrchestrationDSResources"/>
|
||||||
|
<RESOURCE_URI value=""/>
|
||||||
|
<DATE_OF_CREATION value="2001-12-31T12:00:00"/>
|
||||||
|
</HEADER>
|
||||||
|
<BODY>
|
||||||
|
<CONFIGURATION enabled="true">
|
||||||
|
<DEDUPLICATION>
|
||||||
|
<ENTITY code="20" label="Organization" name="organization"/>
|
||||||
|
<ACTION_SET id="test-orchestrator"/>
|
||||||
|
<SCAN_SEQUENCE>
|
||||||
|
<SCAN id="organization"/>
|
||||||
|
</SCAN_SEQUENCE>
|
||||||
|
</DEDUPLICATION>
|
||||||
|
</CONFIGURATION>
|
||||||
|
<STATUS>
|
||||||
|
<LAST_UPDATE value="2001-12-31T12:00:00"/>
|
||||||
|
</STATUS>
|
||||||
|
<SECURITY_PARAMETERS>SECURITY_PARAMETERS</SECURITY_PARAMETERS>
|
||||||
|
</BODY>
|
||||||
|
</RESOURCE_PROFILE>
|
|
@ -612,7 +612,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
|
||||||
final DataInfo info = prepareDataInfo(rs); // TODO
|
final DataInfo info = prepareDataInfo(rs); // TODO
|
||||||
|
|
||||||
final String orgId1 = createOpenaireId(20, rs.getString("id1"), true);
|
final String orgId1 = createOpenaireId(20, rs.getString("id1"), true);
|
||||||
final String orgId2 = createOpenaireId(40, rs.getString("id2"), true);
|
final String orgId2 = createOpenaireId(20, rs.getString("id2"), true);
|
||||||
final String relClass = rs.getString("relclass");
|
final String relClass = rs.getString("relclass");
|
||||||
|
|
||||||
final List<KeyValue> collectedFrom = listKeyValues(
|
final List<KeyValue> collectedFrom = listKeyValues(
|
||||||
|
|
Loading…
Reference in New Issue