From b02aa08833c47a6748558417adb13c4940929324 Mon Sep 17 00:00:00 2001 From: Michele De Bonis Date: Fri, 8 Feb 2019 12:56:47 +0100 Subject: [PATCH] implementation of the test classes and minor changes --- .../src/main/java/eu/dnetlib/SparkTest.java | 15 +- .../java/eu/dnetlib/SparkTestAuthors.java | 115 -------------- .../eu/dnetlib/proto/utils/OAFProtoUtils.java | 3 +- .../dnetlib/pace/organization.test2.pace.conf | 3 +- .../dnetlib/pace/AbstractProtoPaceTest.java | 36 ++++- .../dnetlib/pace/tree/TreeProcessingTest.java | 93 +++++++++++ .../eu/dnetlib/pace/authors.test.pace.conf | 40 +++++ dnet-pace-core/pom.xml | 1 - .../algo/JaroWinklerNormalizedName.java | 8 +- .../eu/dnetlib/pace/tree/UndefinedNode.java | 3 - .../eu/dnetlib/pace/tree/support/AggType.java | 14 +- .../eu/dnetlib/pace/util/BlockProcessor.java | 2 +- .../eu/dnetlib/pace/AbstractPaceTest.java | 23 ++- .../eu/dnetlib/pace/tree/ComparatorTest.java | 144 ++++++++++++++++++ 14 files changed, 350 insertions(+), 150 deletions(-) delete mode 100644 dnet-dedup-test/src/main/java/eu/dnetlib/SparkTestAuthors.java create mode 100644 dnet-dedup-test/src/test/java/eu/dnetlib/pace/tree/TreeProcessingTest.java create mode 100644 dnet-dedup-test/src/test/resources/eu/dnetlib/pace/authors.test.pace.conf create mode 100644 dnet-pace-core/src/test/java/eu/dnetlib/pace/tree/ComparatorTest.java diff --git a/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTest.java b/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTest.java index b321ff3..ead0503 100644 --- a/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTest.java +++ b/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTest.java @@ -1,10 +1,8 @@ package eu.dnetlib; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import eu.dnetlib.graph.GraphProcessor; import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; -import eu.dnetlib.pace.common.AbstractPaceFunctions; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.util.BlockProcessor; @@ -35,13 +33,13 @@ public class SparkTest { public static void main(String[] args) { final JavaSparkContext context = new JavaSparkContext(new SparkConf().setAppName("Deduplication").setMaster("local[*]")); - final URL dataset = SparkTest.class.getResource("/eu/dnetlib/pace/orgs2.json"); + final URL dataset = SparkTest.class.getResource("/eu/dnetlib/pace/authors.json");//"/eu/dnetlib/pace/orgs2.json"); final JavaRDD dataRDD = context.textFile(dataset.getPath()); counter = new SparkCounter(context); //read the configuration from the classpath - final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.test2.pace.conf")); + final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/authors.test.pace.conf"));//"/eu/dnetlib/pace/organization.test2.pace.conf")); BlockProcessor.constructAccumulator(config); BlockProcessor.accumulators.forEach(acc -> { @@ -63,6 +61,7 @@ public class SparkTest { //Clustering: from to List .flatMapToPair(a -> { final MapDocument currentDocument = a._2(); + return getGroupingKeys(config, currentDocument).stream() .map(it -> new Tuple2<>(it, currentDocument)).collect(Collectors.toList()).iterator(); }).groupByKey() //group documents basing on the key @@ -86,8 +85,8 @@ public class SparkTest { counter.getAccumulators().values().forEach(it-> System.out.println(it.getGroup()+" "+it.getName()+" -->"+it.value())); -// connectedComponents.foreach(cc -> System.out.println("cc = " + cc.toString() + " size =" + cc.getDocs().size())); -// nonDeduplicated.foreach(cc -> System.out.println("nd = " + cc.toString())); + connectedComponents.foreach(cc -> System.out.println("cc = " + cc.toString() + " size =" + cc.getDocs().size())); + nonDeduplicated.foreach(cc -> System.out.println("nd = " + cc.toString())); //print ids // ccs.foreach(cc -> System.out.println(cc.getId())); @@ -105,10 +104,8 @@ public class SparkTest { } } - static Set getGroupingKeys(DedupConfig conf, MapDocument doc) { return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf)); } - -} +} \ No newline at end of file diff --git a/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTestAuthors.java b/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTestAuthors.java deleted file mode 100644 index e079728..0000000 --- a/dnet-dedup-test/src/main/java/eu/dnetlib/SparkTestAuthors.java +++ /dev/null @@ -1,115 +0,0 @@ -package eu.dnetlib; - -import com.google.common.collect.Sets; -import eu.dnetlib.graph.GraphProcessor; -import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.util.BlockProcessor; -import eu.dnetlib.pace.utils.PaceUtils; -import eu.dnetlib.reporter.SparkCounter; -import eu.dnetlib.reporter.SparkReporter; -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.graphx.Edge; -import org.apache.spark.rdd.RDD; -import scala.Tuple2; - -import java.io.IOException; -import java.io.StringWriter; -import java.net.URL; -import java.util.Set; -import java.util.stream.Collectors; - -//import eu.dnetlib.data.proto.TypeProtos; - -public class SparkTestAuthors { - public static SparkCounter counter ; - private static final Log log = LogFactory.getLog(SparkTestAuthors.class); - - public static void main(String[] args) { - final JavaSparkContext context = new JavaSparkContext(new SparkConf().setAppName("Deduplication").setMaster("local[*]")); - - final URL dataset = SparkTest.class.getResource("/eu/dnetlib/pace/authors.json"); - final JavaRDD dataRDD = context.textFile(dataset.getPath()); - - counter = new SparkCounter(context); - - //read the configuration from the classpath - final DedupConfig config = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/authors.test.pace.conf")); - - BlockProcessor.constructAccumulator(config); - BlockProcessor.accumulators.forEach(acc -> { - - final String[] values = acc.split("::"); - counter.incrementCounter(values[0], values[1], 0); - - }); - - //create vertexes of the graph: - JavaPairRDD mapDocs = dataRDD.mapToPair(it -> { - MapDocument mapDocument = PaceUtils.asMapDocument(config, it); - return new Tuple2<>(mapDocument.getIdentifier(), mapDocument); - }); - - RDD> vertexes = mapDocs.mapToPair(t -> new Tuple2( (long) t._1().hashCode(), t._2())).rdd(); - - //create relations between documents - JavaPairRDD relationRDD = mapDocs.reduceByKey((a, b) -> a) //the reduce is just to be sure that we haven't document with same id - //Clustering: from to List - .flatMapToPair(a -> { - final MapDocument currentDocument = a._2(); - return getGroupingKeys(config, currentDocument).stream() - .map(it -> new Tuple2<>(it, currentDocument)).collect(Collectors.toList()).iterator(); - }).groupByKey() //group documents basing on the key - //create relations by comparing only elements in the same group - .flatMapToPair(it -> { - final SparkReporter reporter = new SparkReporter(counter); - new BlockProcessor(config).process(it._1(), it._2(), reporter); - return reporter.getReport().iterator(); - }); - - final RDD> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd(); - - JavaRDD ccs = GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD(); - - final JavaRDD connectedComponents = ccs.filter(cc -> cc.getDocs().size()>1); - final JavaRDD nonDeduplicated = ccs.filter(cc -> cc.getDocs().size()==1); - - System.out.println("Non duplicates: " + nonDeduplicated.count()); - System.out.println("Duplicates: " + connectedComponents.flatMap(cc -> cc.getDocs().iterator()).count()); - System.out.println("Connected Components: " + connectedComponents.count()); - - counter.getAccumulators().values().forEach(it-> System.out.println(it.getGroup()+" "+it.getName()+" -->"+it.value())); - - connectedComponents.foreach(cc -> System.out.println("cc = " + cc.toString() + " size =" + cc.getDocs().size())); - nonDeduplicated.foreach(cc -> System.out.println("nd = " + cc.toString())); - - //print ids -// ccs.foreach(cc -> System.out.println(cc.getId())); -// connectedComponents.saveAsTextFile("file:///Users/miconis/Downloads/dumps/organizations_dedup"); - - } - - static String readFromClasspath(final String filename) { - final StringWriter sw = new StringWriter(); - try { - IOUtils.copy(SparkTest.class.getResourceAsStream(filename), sw); - return sw.toString(); - } catch (final IOException e) { - throw new RuntimeException("cannot load resource from classpath: " + filename); - } - } - - - static Set getGroupingKeys(DedupConfig conf, MapDocument doc) { - return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf)); - } - - -} diff --git a/dnet-dedup-test/src/main/java/eu/dnetlib/proto/utils/OAFProtoUtils.java b/dnet-dedup-test/src/main/java/eu/dnetlib/proto/utils/OAFProtoUtils.java index 4a32638..b920b46 100644 --- a/dnet-dedup-test/src/main/java/eu/dnetlib/proto/utils/OAFProtoUtils.java +++ b/dnet-dedup-test/src/main/java/eu/dnetlib/proto/utils/OAFProtoUtils.java @@ -32,8 +32,7 @@ public class OAFProtoUtils { } public static FieldTypeProtos.Qualifier.Builder getQualifier(final String classname, final String schemename) { - return - FieldTypeProtos.Qualifier.newBuilder().setClassid(classname).setClassname(classname).setSchemeid(schemename).setSchemename(schemename); + return FieldTypeProtos.Qualifier.newBuilder().setClassid(classname).setClassname(classname).setSchemeid(schemename).setSchemename(schemename); } public static OafProtos.OafEntity.Builder oafEntity(final String id, final eu.dnetlib.data.proto.TypeProtos.Type type) { diff --git a/dnet-dedup-test/src/main/resources/eu/dnetlib/pace/organization.test2.pace.conf b/dnet-dedup-test/src/main/resources/eu/dnetlib/pace/organization.test2.pace.conf index d828d6f..8bde975 100644 --- a/dnet-dedup-test/src/main/resources/eu/dnetlib/pace/organization.test2.pace.conf +++ b/dnet-dedup-test/src/main/resources/eu/dnetlib/pace/organization.test2.pace.conf @@ -20,11 +20,12 @@ { "name" : "exactMatch", "fields" : [ "country" ] }, { "name" : "DomainExactMatch", "fields" : [ "websiteurl" ] } ], + "decisionTree": {}, "model" : [ { "name" : "legalname", "algo" : "Null", "type" : "String", "weight" : "0", "ignoreMissing" : "false", "path" : "organization/metadata/legalname/value" }, { "name" : "country", "algo" : "Null", "type" : "String", "weight" : "0", "ignoreMissing" : "true", "path" : "organization/metadata/country/classid" }, { "name" : "legalshortname", "algo" : "JaroWinkler", "type" : "String", "weight" : "0.3", "ignoreMissing" : "true", "path" : "organization/metadata/legalshortname/value" }, - { "name" : "legalname", "algo" : "JaroWinklerNormalizedName", "type" : "String", "weight" : "0.7", "ignoreMissing" : "false", "path" : "organization/metadata/legalname/value" }, + { "name" : "legalname", "algo" : "JaroWinklerNormalizedName", "type" : "String", "weight" : "0.7", "ignoreMissing" : "false", "path" : "organization/metadata/legalname/value", "params" : { "windowSize" : 4 } }, { "name" : "websiteurl", "algo" : "Null", "type" : "URL", "weight" : "0", "ignoreMissing" : "true", "path" : "organization/metadata/websiteurl/value", "params" : { "host" : 0.5, "path" : 0.5 } } ], "blacklists" : { } diff --git a/dnet-dedup-test/src/test/java/eu/dnetlib/pace/AbstractProtoPaceTest.java b/dnet-dedup-test/src/test/java/eu/dnetlib/pace/AbstractProtoPaceTest.java index 7adfa5c..4647d39 100644 --- a/dnet-dedup-test/src/test/java/eu/dnetlib/pace/AbstractProtoPaceTest.java +++ b/dnet-dedup-test/src/test/java/eu/dnetlib/pace/AbstractProtoPaceTest.java @@ -13,10 +13,7 @@ import eu.dnetlib.data.proto.ResultProtos.Result; import eu.dnetlib.pace.config.Config; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.config.Type; -import eu.dnetlib.pace.model.Field; -import eu.dnetlib.pace.model.FieldValueImpl; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.model.ProtoDocumentBuilder; +import eu.dnetlib.pace.model.*; import eu.dnetlib.pace.model.gt.GTAuthor; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.RandomStringUtils; @@ -25,9 +22,7 @@ import org.apache.commons.lang3.RandomUtils; import java.io.IOException; import java.io.StringWriter; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; +import java.util.*; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -49,11 +44,13 @@ public abstract class AbstractProtoPaceTest extends OafTest { return DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf")); } - protected DedupConfig getOrganizationTestConf() { return DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.test.conf")); } + protected DedupConfig getAuthorsTestConf() { + return DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/authors.test.pace.conf")); + } protected DedupConfig getResultAuthorsConf() { return DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/result.authors.pace.conf")); @@ -108,6 +105,29 @@ public abstract class AbstractProtoPaceTest extends OafTest { return result(config, id, title, date, Lists.newArrayList(pid), authors); } + protected MapDocument author(final String identifier, final String area, final String firstname, final String lastname, final String fullname, final Double[] topics, final String pubID, final String pubDOI, final int rank, final String orcid, final List coauthors) { + Map fieldMap = new HashMap<>(); + + fieldMap.put("area", new FieldValueImpl(Type.String, "area", area)); + fieldMap.put("firstname", new FieldValueImpl(Type.String, "firstname", firstname)); + fieldMap.put("lastname", new FieldValueImpl(Type.String, "lastname", lastname)); + fieldMap.put("fullname", new FieldValueImpl(Type.String, "fullname", fullname)); + fieldMap.put("pubID", new FieldValueImpl(Type.String, "pubID", pubID)); + fieldMap.put("pubDOI", new FieldValueImpl(Type.String, "pubDOI", pubDOI)); + fieldMap.put("rank", new FieldValueImpl(Type.Int, "rank", rank)); + fieldMap.put("orcid", new FieldValueImpl(Type.String, "orcid", orcid)); + + FieldListImpl ca = new FieldListImpl("coauthors", Type.String); + ca.addAll(coauthors.stream().map(s -> new FieldValueImpl(Type.String, "coauthors", s)).collect(Collectors.toList())); + fieldMap.put("coauthors", ca); + + FieldListImpl t = new FieldListImpl("topics", Type.String); + t.addAll(Arrays.asList(topics).stream().map(d -> new FieldValueImpl(Type.String, "topics", d.toString())).collect(Collectors.toList())); + fieldMap.put("topics", t); + + return new MapDocument(identifier, fieldMap); + } + static List pidTypes = Lists.newArrayList(); static { pidTypes.add("doi"); diff --git a/dnet-dedup-test/src/test/java/eu/dnetlib/pace/tree/TreeProcessingTest.java b/dnet-dedup-test/src/test/java/eu/dnetlib/pace/tree/TreeProcessingTest.java new file mode 100644 index 0000000..f232752 --- /dev/null +++ b/dnet-dedup-test/src/test/java/eu/dnetlib/pace/tree/TreeProcessingTest.java @@ -0,0 +1,93 @@ +package eu.dnetlib.pace.tree; + +import eu.dnetlib.pace.AbstractProtoPaceTest; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.tree.support.MatchType; +import eu.dnetlib.pace.util.BlockProcessor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.junit.Assert.assertTrue; + +public class TreeProcessingTest extends AbstractProtoPaceTest { + + private static final Log log = LogFactory.getLog(TreeProcessingTest.class); + + private DedupConfig config; + + @Before + public void setup(){ + config = getAuthorsTestConf(); + } + + @Test + public void testOrcidMatch (){ + + final MapDocument authorA = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5")); + final MapDocument authorB = author("id2", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID2", "pubDOI2", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5")); + final MapDocument authorC = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0001", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5")); + + log.info("Author 1 = " + authorA); + log.info("Author 2 = " + authorB); + log.info("Author 3 = " + authorC); + + MatchType matchType1 = new BlockProcessor(config).navigateTree(authorA, authorB); + MatchType matchType2 = new BlockProcessor(config).navigateTree(authorA, authorC); + + log.info("1 vs 2 Match Type = " + matchType1); + log.info("1 vs 3 Match Type = " + matchType2); + + assertTrue(matchType1 == MatchType.ORCID_MATCH); + assertTrue(matchType2 == MatchType.NO_MATCH); + } + + @Test + public void testCoauthorsMatch() { + final MapDocument authorA = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + final MapDocument authorB = author("id2", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID2", "pubDOI2", 1, "", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + + log.info("Author 1 = " + authorA); + log.info("Author 2 = " + authorB); + + MatchType matchType = new BlockProcessor(config).navigateTree(authorA, authorB); + + log.info("Match Type = " + matchType); + + assertTrue(matchType == MatchType.COAUTHORS_MATCH); + } + + @Test + public void testTopicsMatch() { + final MapDocument authorA = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + final MapDocument authorB = author("id2", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID2", "pubDOI2", 1, "", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5")); + + log.info("Author 1 = " + authorA); + log.info("Author 2 = " + authorB); + + MatchType matchType = new BlockProcessor(config).navigateTree(authorA, authorB); + + log.info("Match Type = " + matchType); + + assertTrue(matchType == MatchType.TOPICS_MATCH); + } + + @Test + public void testNoMatch() { + + final MapDocument authorA = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + final MapDocument authorB = author("id1", "1", "john", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "0000-0000-0000-0000", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + final MapDocument authorC = author("id2", "1", "jesus f.", "smith", "smith, john", new Double[]{0.0,0.5,0.0,0.5}, "pubID1", "pubDOI1", 1, "", Arrays.asList("coauthor1", "coauthor2", "coauthor3", "coauthor4", "coauthor5", "coauthor6")); + + MatchType matchType1 = new BlockProcessor(config).navigateTree(authorA,authorB); + MatchType matchType2 = new BlockProcessor(config).navigateTree(authorA,authorC); + + assertTrue(matchType1 == MatchType.NO_MATCH); //same identifier + assertTrue(matchType2 == MatchType.NO_MATCH); //not similar firstname + } + +} diff --git a/dnet-dedup-test/src/test/resources/eu/dnetlib/pace/authors.test.pace.conf b/dnet-dedup-test/src/test/resources/eu/dnetlib/pace/authors.test.pace.conf new file mode 100644 index 0000000..c804a60 --- /dev/null +++ b/dnet-dedup-test/src/test/resources/eu/dnetlib/pace/authors.test.pace.conf @@ -0,0 +1,40 @@ +{ + "wf" : { + "threshold" : "0.99", + "dedupRun" : "001", + "entityType" : "person", + "orderField" : "fullname", + "queueMaxSize" : "2000", + "groupMaxSize" : "10", + "slidingWindowSize" : "200", + "rootBuilder" : [ "person" ], + "includeChildren" : "true" + }, + "pace": { + "clustering": [ + {"name": "personClustering", "fields": ["fullname"], "params": {}} + ], + "conditions": [], + "decisionTree": { + "start": {"fields": [{"field":"pubID", "comparator":"exactMatch", "weight":1.0, "params":{}}], "threshold":1.0, "aggregation": "SUM", "positive":"NO_MATCH", "negative":"layer2", "undefined": "layer2", "ignoreMissing": "false"}, + "layer2": {"fields": [{"field":"orcid", "comparator":"exactMatch", "weight":1.0, "params":{}}], "threshold":1.0, "aggregation": "SUM", "positive":"ORCID_MATCH", "negative":"NO_MATCH", "undefined": "layer3", "ignoreMissing": "false"}, + "layer3": {"fields": [{"field":"firstname", "comparator":"similar", "weight":1.0, "params":{}}], "threshold":0.7, "aggregation": "SUM", "positive":"layer4", "negative":"NO_MATCH", "undefined": "layer4", "ignoreMissing": "false"}, + "layer4": {"fields": [{"field":"coauthors", "comparator":"coauthorsMatch", "weight":1.0, "params":{"minCoauthors":6, "maxCoauthors": 200}}], "threshold":5.0, "aggregation": "SUM", "positive":"COAUTHORS_MATCH", "negative":"NO_MATCH", "undefined": "layer5", "ignoreMissing": "false"}, + "layer5": {"fields": [{"field":"area", "comparator":"exactMatch", "weight":1.0, "params":{}}], "threshold":1.0, "aggregation": "SUM", "positive":"layer6", "negative":"NO_MATCH", "undefined": "NO_MATCH", "ignoreMissing": "false"}, + "layer6": {"fields": [{"field":"topics", "comparator":"topicsMatch", "weight":1.0, "params":{}}], "threshold":0.7, "aggregation": "SUM", "positive":"TOPICS_MATCH", "negative":"NO_MATCH", "undefined": "NO_MATCH", "ignoreMissing": "false"} + }, + "model": [ + {"name": "fullname", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/fullname"}, + {"name": "firstname", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/firstname"}, + {"name": "lastname", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/lastname"}, + {"name": "coauthors", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/coauthors"}, + {"name": "orcid", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/orcid"}, + {"name": "topics", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/topics"}, + {"name": "pubID", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/pubID"}, + {"name": "pubDOI", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/pubDOI"}, + {"name": "rank", "algo": "Null", "type": "Int", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/rank"}, + {"name": "area", "algo": "Null", "type": "String", "weight": "0", "ignoreMissing": "false", "path": "person/metadata/area"} + ], + "blacklists": {} + } +} \ No newline at end of file diff --git a/dnet-pace-core/pom.xml b/dnet-pace-core/pom.xml index 7a56668..d88dea4 100644 --- a/dnet-pace-core/pom.xml +++ b/dnet-pace-core/pom.xml @@ -74,7 +74,6 @@ commons-math3 - diff --git a/dnet-pace-core/src/main/java/eu/dnetlib/pace/distance/algo/JaroWinklerNormalizedName.java b/dnet-pace-core/src/main/java/eu/dnetlib/pace/distance/algo/JaroWinklerNormalizedName.java index 2f79493..ebaa0ea 100644 --- a/dnet-pace-core/src/main/java/eu/dnetlib/pace/distance/algo/JaroWinklerNormalizedName.java +++ b/dnet-pace-core/src/main/java/eu/dnetlib/pace/distance/algo/JaroWinklerNormalizedName.java @@ -23,8 +23,11 @@ public class JaroWinklerNormalizedName extends SecondStringDistanceAlgo { private static Map cityMap = AbstractPaceFunctions.loadMapFromClasspath("/eu/dnetlib/pace/config/city_map.csv"); + private Map params; + public JaroWinklerNormalizedName(Map params){ super(params, new com.wcohen.ss.JaroWinkler()); + this.params = params; } public JaroWinklerNormalizedName(double weight) { @@ -52,9 +55,8 @@ public class JaroWinklerNormalizedName extends SecondStringDistanceAlgo { // ca = norm.split("\\|\\|\\|")[0].trim(); // cb = norm.split("\\|\\|\\|")[1].trim(); - ca = normalizeCities2(ca, cityMap, 4); - cb = normalizeCities2(cb, cityMap, 4); - + ca = normalizeCities2(ca, cityMap, params.getOrDefault("windowSize", 4).intValue()); + cb = normalizeCities2(cb, cityMap, params.getOrDefault("windowSize", 4).intValue()); if (sameCity(ca,cb)){ if (sameKeywords(ca,cb)){ diff --git a/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/UndefinedNode.java b/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/UndefinedNode.java index cf90847..de43543 100644 --- a/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/UndefinedNode.java +++ b/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/UndefinedNode.java @@ -17,9 +17,6 @@ public class UndefinedNode implements Comparator { final List sa = ((FieldList) a).stringList(); final List sb = ((FieldList) b).stringList(); - System.out.println("sa = " + sa.size()); - System.out.println("sb = " + sb.size()); - return 0; } } diff --git a/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/support/AggType.java b/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/support/AggType.java index 71e3ad0..bd7bd9f 100644 --- a/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/support/AggType.java +++ b/dnet-pace-core/src/main/java/eu/dnetlib/pace/tree/support/AggType.java @@ -1,9 +1,21 @@ package eu.dnetlib.pace.tree.support; +import eu.dnetlib.pace.util.PaceException; + public enum AggType { AVG, SUM, MAX, - MIN + MIN; + + public static AggType getEnum(String value) { + + try { + return AggType.valueOf(value); + } + catch (IllegalArgumentException e) { + throw new PaceException("Undefined aggregation type", e); + } + } } diff --git a/dnet-pace-core/src/main/java/eu/dnetlib/pace/util/BlockProcessor.java b/dnet-pace-core/src/main/java/eu/dnetlib/pace/util/BlockProcessor.java index 19105ae..1cd0eb3 100644 --- a/dnet-pace-core/src/main/java/eu/dnetlib/pace/util/BlockProcessor.java +++ b/dnet-pace-core/src/main/java/eu/dnetlib/pace/util/BlockProcessor.java @@ -76,7 +76,7 @@ public class BlockProcessor { } } - private MatchType navigateTree(final MapDocument doc1, final MapDocument doc2){ + public MatchType navigateTree(final MapDocument doc1, final MapDocument doc2){ final Map decisionTree = dedupConf.getPace().getDecisionTree(); diff --git a/dnet-pace-core/src/test/java/eu/dnetlib/pace/AbstractPaceTest.java b/dnet-pace-core/src/test/java/eu/dnetlib/pace/AbstractPaceTest.java index 8a0c08d..3da6f0a 100644 --- a/dnet-pace-core/src/test/java/eu/dnetlib/pace/AbstractPaceTest.java +++ b/dnet-pace-core/src/test/java/eu/dnetlib/pace/AbstractPaceTest.java @@ -1,14 +1,15 @@ package eu.dnetlib.pace; -import java.io.IOException; -import java.io.StringWriter; - -import org.apache.commons.io.IOUtils; - import eu.dnetlib.pace.config.Type; import eu.dnetlib.pace.model.Field; +import eu.dnetlib.pace.model.FieldListImpl; import eu.dnetlib.pace.model.FieldValueImpl; -import org.junit.Test; +import org.apache.commons.io.IOUtils; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.List; +import java.util.stream.Collectors; public abstract class AbstractPaceTest { @@ -34,4 +35,14 @@ public abstract class AbstractPaceTest { return new FieldValueImpl(Type.URL, "url", s); } + protected Field createFieldList(List strings, String fieldName){ + + List fieldValueStream = strings.stream().map(s -> new FieldValueImpl(Type.String, fieldName, s)).collect(Collectors.toList()); + + FieldListImpl a = new FieldListImpl(); + a.addAll(fieldValueStream); + + return a; + + } } diff --git a/dnet-pace-core/src/test/java/eu/dnetlib/pace/tree/ComparatorTest.java b/dnet-pace-core/src/test/java/eu/dnetlib/pace/tree/ComparatorTest.java new file mode 100644 index 0000000..240a5d6 --- /dev/null +++ b/dnet-pace-core/src/test/java/eu/dnetlib/pace/tree/ComparatorTest.java @@ -0,0 +1,144 @@ +package eu.dnetlib.pace.tree; + +import eu.dnetlib.pace.AbstractPaceTest; +import eu.dnetlib.pace.config.Type; +import eu.dnetlib.pace.model.Field; +import eu.dnetlib.pace.model.FieldListImpl; +import eu.dnetlib.pace.model.FieldValueImpl; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static junit.framework.Assert.assertEquals; +import static junit.framework.Assert.assertTrue; + +//test class for comparators (to be used into the tree nodes) +public class ComparatorTest extends AbstractPaceTest { + + private Map params; + + @Before + public void setup() { + params = new HashMap<>(); + //to put all the needed parameters + params.put("minCoauthors", 5); + params.put("maxCoauthors", 200); + + } + + @Test + public void testCoauthorsMatch() { + + final CoauthorsMatch coauthorsMatch = new CoauthorsMatch(params); + + Field a = createFieldList(Arrays.asList("la bruzzo, sandro", "atzori, claudio", "artini, michele", "de bonis, michele", "bardi, alessia", "dell'amico, andrea", "baglioni, miriam"), "coauthors"); + Field b = createFieldList(Arrays.asList("la bruzzo, sandro"), "coauthors"); + + double result1 = coauthorsMatch.compare(a, b); + double result2 = coauthorsMatch.compare(a, a); + + System.out.println("a = " + a); + System.out.println("b = " + b); + + System.out.println("a vs b = " + result1); + System.out.println("a vs a = " + result2); + + assertEquals(result1, -1.0); + assertEquals(result2, 7.0); + } + + @Test + public void testExactMatch() { + + final ExactMatch exactMatch = new ExactMatch(params); + + Field a = new FieldValueImpl(Type.String, "doi", "10.1000/0000000000"); + Field b = new FieldValueImpl(Type.String, "doi", "10.1033/0000000000"); + Field c = new FieldValueImpl(Type.String, "doi", ""); + + double result1 = exactMatch.compare(a,a); + double result2 = exactMatch.compare(a,b); + double result3 = exactMatch.compare(a,c); + + System.out.println("a = " + a); + System.out.println("b = " + b); + System.out.println("c = " + c); + + System.out.println("a vs a = " + result1); + System.out.println("a vs b = " + result2); + System.out.println("a vs c = " + result3); + + assertEquals(result1, 1.0); + assertEquals(result2, 0.0); + assertEquals(result3, -1.0); + + } + + @Test + public void testSimilarMatch() { + + final SimilarMatch similarMatch = new SimilarMatch(params); + + Field a = new FieldValueImpl(Type.String, "firstname", "sandro"); + Field b = new FieldValueImpl(Type.String, "firstname", "s."); + Field c = new FieldValueImpl(Type.String, "firstname", "stefano"); + + double result1 = similarMatch.compare(a,b); + double result2 = similarMatch.compare(a,c); + double result3 = similarMatch.compare(b,c); + + System.out.println("a = " + a); + System.out.println("b = " + b); + System.out.println("c = " + c); + + System.out.println("a vs b = " + result1); + System.out.println("a vs c = " + result2); + System.out.println("b vs c = " + result3); + + assertEquals(result1, 1.0); + assertEquals(result3, 1.0); + assertTrue(result2<0.7); + + } + + @Test + public void testTopicsMatch() { + + final TopicsMatch topicsMatch = new TopicsMatch(params); + + Field a = createFieldList(Arrays.asList("0.0", "1.0", "0.0"), "topics"); + Field b = createFieldList(Arrays.asList("0.0", "0.0", "1.0"), "topics"); + Field c = createFieldList(Arrays.asList("0.5", "0.5", "0.0"), "topics"); + + double result1 = topicsMatch.compare(a,a); + double result2 = topicsMatch.compare(a,c); + double result3 = topicsMatch.compare(b,c); + + System.out.println("a = " + a); + System.out.println("b = " + b); + System.out.println("c = " + c); + + System.out.println("a vs a = " + result1); + System.out.println("a vs c = " + result2); + System.out.println("b vs c = " + result3); + + assertEquals(result1, 1.0); + assertEquals(result2, 0.5); + assertEquals(result3, 0.0); + + } + + @Test + public void testUndefinedNode() { + + final UndefinedNode undefinedNode = new UndefinedNode(); + double result = undefinedNode.compare(new FieldListImpl(),new FieldListImpl()); + + assertEquals(result, 0.0); + } + + +}