Merge pull request 'implementation of the whitelist for similarity relations' (#144) from dedup_whitelist into beta
Reviewed-on: #144
This commit is contained in:
commit
35619b93ee
|
@ -0,0 +1,151 @@
|
||||||
|
package eu.dnetlib.dhp.oa.dedup;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.api.java.function.FilterFunction;
|
||||||
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
|
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.SaveMode;
|
||||||
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.dom4j.DocumentException;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.xml.sax.SAXException;
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
||||||
|
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 eu.dnetlib.pace.model.MapDocument;
|
||||||
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
|
import scala.Tuple2;
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
|
public class SparkWhitelistSimRels extends AbstractSparkAction {
|
||||||
|
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class);
|
||||||
|
|
||||||
|
private static final String WHITELIST_SEPARATOR = "####";
|
||||||
|
|
||||||
|
public SparkWhitelistSimRels(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/whitelistSimRels_parameters.json")));
|
||||||
|
parser.parseArgument(args);
|
||||||
|
|
||||||
|
SparkConf conf = new SparkConf();
|
||||||
|
new SparkWhitelistSimRels(parser, getSparkSession(conf))
|
||||||
|
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run(ISLookUpService isLookUpService)
|
||||||
|
throws DocumentException, IOException, ISLookUpException, SAXException {
|
||||||
|
|
||||||
|
// read oozie parameters
|
||||||
|
final String graphBasePath = parser.get("graphBasePath");
|
||||||
|
final String isLookUpUrl = parser.get("isLookUpUrl");
|
||||||
|
final String actionSetId = parser.get("actionSetId");
|
||||||
|
final String workingPath = parser.get("workingPath");
|
||||||
|
final int numPartitions = Optional
|
||||||
|
.ofNullable(parser.get("numPartitions"))
|
||||||
|
.map(Integer::valueOf)
|
||||||
|
.orElse(NUM_PARTITIONS);
|
||||||
|
final String whiteListPath = parser.get("whiteListPath");
|
||||||
|
|
||||||
|
log.info("numPartitions: '{}'", numPartitions);
|
||||||
|
log.info("graphBasePath: '{}'", graphBasePath);
|
||||||
|
log.info("isLookUpUrl: '{}'", isLookUpUrl);
|
||||||
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
|
log.info("workingPath: '{}'", workingPath);
|
||||||
|
log.info("whiteListPath: '{}'", whiteListPath);
|
||||||
|
|
||||||
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
//file format: source####target
|
||||||
|
Dataset<Tuple2<String, String>> whiteListRels = spark.createDataset(sc
|
||||||
|
.textFile(whiteListPath)
|
||||||
|
//check if the line is in the correct format: id1####id2
|
||||||
|
.filter(s -> s.contains(WHITELIST_SEPARATOR) && s.split(WHITELIST_SEPARATOR).length == 2)
|
||||||
|
.map(s -> new Tuple2<>(s.split(WHITELIST_SEPARATOR)[0], s.split(WHITELIST_SEPARATOR)[1]))
|
||||||
|
.rdd(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
// for each dedup configuration
|
||||||
|
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
|
||||||
|
|
||||||
|
final String entity = dedupConf.getWf().getEntityType();
|
||||||
|
final String subEntity = dedupConf.getWf().getSubEntityValue();
|
||||||
|
log.info("Adding whitelist simrels for: '{}'", subEntity);
|
||||||
|
|
||||||
|
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, String>> entities = spark.createDataset(sc
|
||||||
|
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||||
|
.repartition(numPartitions)
|
||||||
|
.mapToPair(
|
||||||
|
(PairFunction<String, String, String>) s -> {
|
||||||
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
|
||||||
|
return new Tuple2<>(d.getIdentifier(), "present");
|
||||||
|
})
|
||||||
|
.rdd(),
|
||||||
|
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, String>> whiteListRels1 = whiteListRels
|
||||||
|
.joinWith(entities, whiteListRels.col("_1").equalTo(entities.col("_1")), "inner")
|
||||||
|
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, String>>, Tuple2<String, String>>) Tuple2::_1, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
Dataset<Tuple2<String, String>> whiteListRels2 = whiteListRels1
|
||||||
|
.joinWith(entities, whiteListRels1.col("_2").equalTo(entities.col("_1")), "inner")
|
||||||
|
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, String>>, Tuple2<String, String>>) Tuple2::_1, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
|
||||||
|
|
||||||
|
Dataset<Relation> whiteListSimRels = whiteListRels2
|
||||||
|
.map((MapFunction<Tuple2<String, String>, Relation>)
|
||||||
|
r -> createSimRel(r._1(), r._2(), entity),
|
||||||
|
Encoders.bean(Relation.class)
|
||||||
|
);
|
||||||
|
|
||||||
|
saveParquet(whiteListSimRels, outputPath, SaveMode.Append);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Relation createSimRel(String source, String target, String entity) {
|
||||||
|
final Relation r = new Relation();
|
||||||
|
r.setSource(source);
|
||||||
|
r.setTarget(target);
|
||||||
|
r.setSubRelType("dedupSimilarity");
|
||||||
|
r.setRelClass("isSimilarTo");
|
||||||
|
r.setDataInfo(new DataInfo());
|
||||||
|
|
||||||
|
switch (entity) {
|
||||||
|
case "result":
|
||||||
|
r.setRelType("resultResult");
|
||||||
|
break;
|
||||||
|
case "organization":
|
||||||
|
r.setRelType("organizationOrganization");
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("unmanaged entity type: " + entity);
|
||||||
|
}
|
||||||
|
return r;
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,6 +20,10 @@
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>path for the working directory</description>
|
<description>path for the working directory</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>whiteListPath</name>
|
||||||
|
<description>path for the whitelist of similarity relations</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>dedupGraphPath</name>
|
<name>dedupGraphPath</name>
|
||||||
<description>path for the output graph</description>
|
<description>path for the output graph</description>
|
||||||
|
@ -130,6 +134,34 @@
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--numPartitions</arg><arg>8000</arg>
|
<arg>--numPartitions</arg><arg>8000</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
<ok to="WhitelistSimRels"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="WhitelistSimRels">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Add Whitelist Similarity Relations</name>
|
||||||
|
<class>eu.dnetlib.dhp.oa.dedup.SparkWhitelistSimRels</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>--whiteListPath</arg><arg>${whiteListPath}</arg>
|
||||||
|
<arg>--numPartitions</arg><arg>8000</arg>
|
||||||
|
</spark>
|
||||||
<ok to="CreateMergeRel"/>
|
<ok to="CreateMergeRel"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
|
@ -0,0 +1,38 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "la",
|
||||||
|
"paramLongName": "isLookUpUrl",
|
||||||
|
"paramDescription": "address for the LookUp",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "asi",
|
||||||
|
"paramLongName": "actionSetId",
|
||||||
|
"paramDescription": "action set identifier (name of the orchestrator)",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "i",
|
||||||
|
"paramLongName": "graphBasePath",
|
||||||
|
"paramDescription": "the base path of the raw graph",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "w",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "path of the working directory",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "np",
|
||||||
|
"paramLongName": "numPartitions",
|
||||||
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "wl",
|
||||||
|
"paramLongName": "whiteListPath",
|
||||||
|
"paramDescription": "whitelist file path for the addition of custom simrels",
|
||||||
|
"paramRequired": true
|
||||||
|
}
|
||||||
|
]
|
|
@ -5,13 +5,16 @@ import static java.nio.file.Files.createTempDirectory;
|
||||||
|
|
||||||
import static org.apache.spark.sql.functions.count;
|
import static org.apache.spark.sql.functions.count;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.Mockito.lenient;
|
import static org.mockito.Mockito.lenient;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
|
@ -55,6 +58,10 @@ public class SparkDedupTest implements Serializable {
|
||||||
private static String testOutputBasePath;
|
private static String testOutputBasePath;
|
||||||
private static String testDedupGraphBasePath;
|
private static String testDedupGraphBasePath;
|
||||||
private static final String testActionSetId = "test-orchestrator";
|
private static final String testActionSetId = "test-orchestrator";
|
||||||
|
private static String whitelistPath;
|
||||||
|
private static List<String> whiteList;
|
||||||
|
|
||||||
|
private static String WHITELIST_SEPARATOR = "####";
|
||||||
|
|
||||||
@BeforeAll
|
@BeforeAll
|
||||||
public static void cleanUp() throws IOException, URISyntaxException {
|
public static void cleanUp() throws IOException, URISyntaxException {
|
||||||
|
@ -71,6 +78,12 @@ public class SparkDedupTest implements Serializable {
|
||||||
.toAbsolutePath()
|
.toAbsolutePath()
|
||||||
.toString();
|
.toString();
|
||||||
|
|
||||||
|
whitelistPath = Paths
|
||||||
|
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI())
|
||||||
|
.toFile()
|
||||||
|
.getAbsolutePath();
|
||||||
|
whiteList = IOUtils.readLines(new FileReader(whitelistPath));
|
||||||
|
|
||||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||||
|
|
||||||
|
@ -158,7 +171,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i", testGraphBasePath,
|
"-i", testGraphBasePath,
|
||||||
"-asi", testActionSetId,
|
"-asi", testActionSetId,
|
||||||
"-la", "lookupurl",
|
"-la", "lookupurl",
|
||||||
|
@ -202,6 +215,78 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(2)
|
@Order(2)
|
||||||
|
void whitelistSimRelsTest() throws Exception {
|
||||||
|
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
IOUtils
|
||||||
|
.toString(
|
||||||
|
SparkWhitelistSimRels.class
|
||||||
|
.getResourceAsStream(
|
||||||
|
"/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json")));
|
||||||
|
|
||||||
|
parser
|
||||||
|
.parseArgument(
|
||||||
|
new String[]{
|
||||||
|
"-i", testGraphBasePath,
|
||||||
|
"-asi", testActionSetId,
|
||||||
|
"-la", "lookupurl",
|
||||||
|
"-w", testOutputBasePath,
|
||||||
|
"-np", "50",
|
||||||
|
"-wl", whitelistPath
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkWhitelistSimRels(parser, spark).run(isLookUpService);
|
||||||
|
|
||||||
|
long orgs_simrel = spark
|
||||||
|
.read()
|
||||||
|
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization"))
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long pubs_simrel = spark
|
||||||
|
.read()
|
||||||
|
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication"))
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long ds_simrel = spark
|
||||||
|
.read()
|
||||||
|
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "dataset"))
|
||||||
|
.count();
|
||||||
|
|
||||||
|
long orp_simrel = spark
|
||||||
|
.read()
|
||||||
|
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "otherresearchproduct"))
|
||||||
|
.count();
|
||||||
|
|
||||||
|
//entities simrels supposed to be equal to the number of previous step (no rels in whitelist)
|
||||||
|
assertEquals(3082, orgs_simrel);
|
||||||
|
assertEquals(7036, pubs_simrel);
|
||||||
|
assertEquals(442, ds_simrel);
|
||||||
|
assertEquals(6750, orp_simrel);
|
||||||
|
|
||||||
|
//entities simrels to be different from the number of previous step (new simrels in the whitelist)
|
||||||
|
Dataset<Row> sw_simrel = spark
|
||||||
|
.read()
|
||||||
|
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "software"));
|
||||||
|
|
||||||
|
//check if the first relation in the whitelist exists
|
||||||
|
assertTrue(sw_simrel
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.filter(rel ->
|
||||||
|
rel.getSource().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[0]) && rel.getTarget().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[1])).count() > 0);
|
||||||
|
//check if the second relation in the whitelist exists
|
||||||
|
assertTrue(sw_simrel
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.filter(rel ->
|
||||||
|
rel.getSource().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[0]) && rel.getTarget().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[1])).count() > 0);
|
||||||
|
|
||||||
|
assertEquals(338, sw_simrel.count());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@Order(3)
|
||||||
void cutMergeRelsTest() throws Exception {
|
void cutMergeRelsTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -213,7 +298,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i",
|
"-i",
|
||||||
testGraphBasePath,
|
testGraphBasePath,
|
||||||
"-asi",
|
"-asi",
|
||||||
|
@ -297,7 +382,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(3)
|
@Order(4)
|
||||||
void createMergeRelsTest() throws Exception {
|
void createMergeRelsTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -309,7 +394,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i",
|
"-i",
|
||||||
testGraphBasePath,
|
testGraphBasePath,
|
||||||
"-asi",
|
"-asi",
|
||||||
|
@ -353,7 +438,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(4)
|
@Order(5)
|
||||||
void createDedupRecordTest() throws Exception {
|
void createDedupRecordTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -364,7 +449,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
"/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i",
|
"-i",
|
||||||
testGraphBasePath,
|
testGraphBasePath,
|
||||||
"-asi",
|
"-asi",
|
||||||
|
@ -394,13 +479,13 @@ public class SparkDedupTest implements Serializable {
|
||||||
|
|
||||||
assertEquals(85, orgs_deduprecord);
|
assertEquals(85, orgs_deduprecord);
|
||||||
assertEquals(65, pubs_deduprecord);
|
assertEquals(65, pubs_deduprecord);
|
||||||
assertEquals(51, sw_deduprecord);
|
assertEquals(49, sw_deduprecord);
|
||||||
assertEquals(97, ds_deduprecord);
|
assertEquals(97, ds_deduprecord);
|
||||||
assertEquals(89, orp_deduprecord);
|
assertEquals(89, orp_deduprecord);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(5)
|
@Order(6)
|
||||||
void updateEntityTest() throws Exception {
|
void updateEntityTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -411,7 +496,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||||
});
|
});
|
||||||
|
|
||||||
|
@ -479,7 +564,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
assertEquals(838, organizations);
|
assertEquals(838, organizations);
|
||||||
assertEquals(100, projects);
|
assertEquals(100, projects);
|
||||||
assertEquals(100, datasource);
|
assertEquals(100, datasource);
|
||||||
assertEquals(200, softwares);
|
assertEquals(198, softwares);
|
||||||
assertEquals(389, dataset);
|
assertEquals(389, dataset);
|
||||||
assertEquals(517, otherresearchproduct);
|
assertEquals(517, otherresearchproduct);
|
||||||
|
|
||||||
|
@ -516,7 +601,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(6)
|
@Order(7)
|
||||||
void propagateRelationTest() throws Exception {
|
void propagateRelationTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
|
@ -527,7 +612,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
||||||
parser
|
parser
|
||||||
.parseArgument(
|
.parseArgument(
|
||||||
new String[] {
|
new String[]{
|
||||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||||
});
|
});
|
||||||
|
|
||||||
|
@ -566,7 +651,7 @@ public class SparkDedupTest implements Serializable {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(7)
|
@Order(8)
|
||||||
void testRelations() throws Exception {
|
void testRelations() throws Exception {
|
||||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
|
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
|
||||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);
|
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);
|
||||||
|
|
|
@ -0,0 +1,2 @@
|
||||||
|
50|r37b0ad08687::f645b9729d1e1025a72c57883f0f2cac####50|r37b0ad08687::4c55b436743b5c49fa32cd582fd9e1aa
|
||||||
|
50|datacite____::a90f49f9fde5393c00633bea6e4e374a####50|datacite____::5f55cdee77303ba8a2bf9996c32a330c
|
Loading…
Reference in New Issue