jarowinklernormalizedname splitted in 3 different comparators: citymatch, keywordmatch and jarowinkler. Implementation of the TreeStatistic support functions

This commit is contained in:
miconis 2019-11-20 10:45:00 +01:00
parent 676e9c8e37
commit 79e62787cf
18 changed files with 581 additions and 145 deletions

View File

@ -16,12 +16,13 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.graphx.Edge;
import org.apache.spark.rdd.RDD;
import org.apache.spark.util.LongAccumulator;
import scala.Serializable;
import scala.Tuple2;
import java.util.Map;
import java.util.stream.Collectors;
public class Deduper {
public class Deduper implements Serializable {
private static final Log log = LogFactory.getLog(Deduper.class);
@ -37,28 +38,14 @@ public class Deduper {
Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
//create vertexes of the graph: <ID, MapDocument>
JavaPairRDD<String, MapDocument> mapDocs = entities.mapToPair(it -> {
MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
});
JavaPairRDD<String, MapDocument> mapDocs = mapToVertexes(context, entities, config);
RDD<Tuple2<Object, MapDocument>> vertexes = mapDocs.mapToPair(t -> new Tuple2<Object, MapDocument>( (long) t._1().hashCode(), t._2())).rdd();
//create blocks for deduplication
JavaPairRDD<String, Iterable<MapDocument>> blocks = mapDocs.reduceByKey((a, b) -> a) //the reduce is just to be sure that we haven't document with same id
//Clustering: from <id, doc> to List<groupkey,doc>
.flatMapToPair(a -> {
final MapDocument currentDocument = a._2();
return Utility.getGroupingKeys(config, currentDocument).stream()
.map(it -> new Tuple2<>(it, currentDocument)).collect(Collectors.toList()).iterator();
}).groupByKey();
JavaPairRDD<String, Iterable<MapDocument>> blocks = createBlocks(context, mapDocs, config);
//create relations by comparing only elements in the same group
final JavaPairRDD<String, String> relationRDD = blocks.flatMapToPair(it -> {
final SparkReporter reporter = new SparkReporter(accumulators);
new BlockProcessor(config).process(it._1(), it._2(), reporter);
return reporter.getRelations().iterator();
});
final JavaPairRDD<String, String> relationRDD = computeRelations(context, blocks, config);
final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "equalTo")).rdd();
@ -67,4 +54,54 @@ public class Deduper {
return GraphProcessor.findCCs(vertexes, edgeRdd, config.getWf().getMaxIterations()).toJavaRDD();
}
/**
* @param: the spark context
* @param: list of blocks
* @param: the dedup configuration
*
* @return the list of relations generated by the deduplication
*/
public static JavaPairRDD<String, String> computeRelations(JavaSparkContext context, JavaPairRDD<String, Iterable<MapDocument>> blocks, DedupConfig config) {
Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
return blocks.flatMapToPair(it -> {
final SparkReporter reporter = new SparkReporter(accumulators);
new BlockProcessor(config).process(it._1(), it._2(), reporter);
return reporter.getRelations().iterator();
});
}
/**
* @param: the spark context
* @param: list of entities: <id, entity>
* @param: the dedup configuration
*
* @return the list of blocks based on clustering of dedup configuration
*/
public static JavaPairRDD<String, Iterable<MapDocument>> createBlocks(JavaSparkContext context, JavaPairRDD<String, MapDocument> mapDocs, DedupConfig config) {
return mapDocs.reduceByKey((a, b) -> a) //the reduce is just to be sure that we haven't document with same id
//Clustering: from <id, doc> to List<groupkey,doc>
.flatMapToPair(a -> {
final MapDocument currentDocument = a._2();
return Utility.getGroupingKeys(config, currentDocument).stream()
.map(it -> new Tuple2<>(it, currentDocument)).collect(Collectors.toList()).iterator();
}).groupByKey();
}
/**
* @param: the spark context
* @param: list of JSON entities
* @param: the dedup configuration
*
* @return the list of vertexes: <id, mapDocument>
*/
public static JavaPairRDD<String, MapDocument> mapToVertexes(JavaSparkContext context, JavaRDD<String> entities, DedupConfig config){
return entities.mapToPair(it -> {
MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
});
}
}

View File

@ -11,6 +11,7 @@ import eu.dnetlib.pace.model.ProtoDocumentBuilder;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.commons.lang3.StringUtils;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
@ -21,7 +22,7 @@ import static eu.dnetlib.proto.utils.OAFProtoUtils.*;
import static eu.dnetlib.proto.utils.OAFProtoUtils.author;
import static eu.dnetlib.proto.utils.OAFProtoUtils.sp;
public class PaceUtils {
public class PaceUtils implements Serializable {
public static MapDocument result(final Config config, final String id, final String title) {
return result(config, id, title, null, new ArrayList<>(), null);

View File

@ -2,27 +2,37 @@ package eu.dnetlib.pace;
import eu.dnetlib.Deduper;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.tree.support.TreeProcessor;
import eu.dnetlib.pace.tree.support.TreeStats;
import eu.dnetlib.pace.utils.PaceUtils;
import eu.dnetlib.pace.utils.Utility;
import eu.dnetlib.support.ConnectedComponent;
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.sql.SparkSession;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import scala.Tuple2;
import java.net.URL;
import java.util.List;
import java.util.Map;
public class DedupLocalTest {
public class DedupLocalTest extends DedupTestUtils {
JavaSparkContext context;
JavaRDD<String> entities;
DedupConfig conf;
DedupConfig config;
TreeProcessor treeProcessor;
@Before
public void setup() {
conf = DedupConfig.load(Utility.readFromClasspath("/eu/dnetlib/pace/config/organization.strict.conf", DedupLocalTest.class));
config = DedupConfig.load(Utility.readFromClasspath("/eu/dnetlib/pace/config/organization.strict.conf", DedupLocalTest.class));
treeProcessor = new TreeProcessor(config);
final SparkSession spark = SparkSession
.builder()
@ -41,7 +51,7 @@ public class DedupLocalTest {
double startTime = System.currentTimeMillis();
JavaRDD<ConnectedComponent> ccs = Deduper.dedup(context, entities, conf);
JavaRDD<ConnectedComponent> ccs = Deduper.dedup(context, entities, config);
System.out.println("total time = " + (System.currentTimeMillis()-startTime));
@ -50,27 +60,49 @@ public class DedupLocalTest {
}
public static void printStatistics(JavaRDD<ConnectedComponent> ccs){
final JavaRDD<ConnectedComponent> connectedComponents = ccs.filter(cc -> cc.getDocs().size()>1);
final JavaRDD<ConnectedComponent> nonDeduplicated = ccs.filter(cc -> cc.getDocs().size()==1);
@Ignore
@Test
public void relationsTest() {
//print deduped
connectedComponents.foreach(cc -> {
System.out.println(cc);
});
// connectedComponents.foreach(cc -> {
// cc.getDocs().stream().forEach(d -> {
// System.out.println(d.getFieldMap().get("legalname") + " | " + d.getFieldMap().get("legalshortname"));
// });
// });
//print nondeduped
nonDeduplicated.foreach(cc -> {
System.out.println(cc);
});
List<String> entitiesList = entities.collect();
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());
//create vertexes of the graph: <ID, MapDocument>
JavaPairRDD<String, MapDocument> mapDocs = Deduper.mapToVertexes(context, entities, config);
Map<String, MapDocument> docsMap = mapDocs.collectAsMap();
//create blocks for deduplication
JavaPairRDD<String, Iterable<MapDocument>> blocks = Deduper.createBlocks(context, mapDocs, config);
//create relations by comparing only elements in the same group
JavaPairRDD<String, String> relationRDD = Deduper.computeRelations(context, blocks, config);
List<Tuple2<String, String>> rels = relationRDD
.filter(rel -> !getOrganizationLegalname(docsMap.get(rel._1())).equals(getOrganizationLegalname(docsMap.get(rel._2()))))
.collect();
System.out.println("Dubious relations = " + rels.size());
for (Tuple2<String,String> rel : rels) {
System.out.println(rel._1() + " ---> " + rel._2());
System.out.println(treeProcessor.evaluateTree(docsMap.get(rel._1()), docsMap.get(rel._2())));
System.out.println("---------------------------------------------");
}
}
@Ignore
@Test
public void matchTest(){
String JSONEntity1 = "{\"dateoftransformation\":\"2018-09-19\",\"originalId\":[\"doajarticles::Sociedade_Brasileira_de_Ciência_do_Solo\"],\"collectedfrom\":[{\"value\":\"DOAJ-Articles\",\"key\":\"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824\"}],\"organization\":{\"metadata\":{\"eclegalbody\":{\"value\":\"false\"},\"eclegalperson\":{\"value\":\"false\"},\"ecinternationalorganization\":{\"value\":\"false\"},\"legalshortname\":{\"value\":\"Sociedade Brasileira de Ciência do Solo\"},\"ecresearchorganization\":{\"value\":\"false\"},\"ecnonprofit\":{\"value\":\"false\"},\"ecenterprise\":{\"value\":\"false\"},\"ecnutscode\":{\"value\":\"false\"},\"ecinternationalorganizationeurinterests\":{\"value\":\"false\"},\"legalname\":{\"value\":\"Sociedade Brasileira de Ciência do Solo\"},\"country\":{\"classid\":\"BR\",\"classname\":\"Brazil\",\"schemename\":\"dnet:countries\",\"schemeid\":\"dnet:countries\"},\"echighereducation\":{\"value\":\"false\"},\"ecsmevalidated\":{\"value\":\"false\"}}},\"dateofcollection\":\"2018-09-19\",\"type\":20,\"id\":\"20|doajarticles::699ed9ecc727c90e9321e43e495e03ee\"}";
String JSONEntity2 = "{\"dateoftransformation\":\"2018-09-19\",\"originalId\":[\"doajarticles::Sociedade_Brasileira_de_Educação_Matemática\"],\"collectedfrom\":[{\"value\":\"DOAJ-Articles\",\"key\":\"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824\"}],\"organization\":{\"metadata\":{\"eclegalbody\":{\"value\":\"false\"},\"eclegalperson\":{\"value\":\"false\"},\"ecinternationalorganization\":{\"value\":\"false\"},\"legalshortname\":{\"value\":\"Sociedade Brasileira de Educação Matemática\"},\"ecresearchorganization\":{\"value\":\"false\"},\"ecnonprofit\":{\"value\":\"false\"},\"ecenterprise\":{\"value\":\"false\"},\"ecnutscode\":{\"value\":\"false\"},\"ecinternationalorganizationeurinterests\":{\"value\":\"false\"},\"legalname\":{\"value\":\"Sociedade Brasileira de Educação Matemática\"},\"country\":{\"classid\":\"BR\",\"classname\":\"Brazil\",\"schemename\":\"dnet:countries\",\"schemeid\":\"dnet:countries\"},\"echighereducation\":{\"value\":\"false\"},\"ecsmevalidated\":{\"value\":\"false\"}}},\"dateofcollection\":\"2018-09-19\",\"type\":20,\"id\":\"20|doajarticles::ec10c30a33588ad4884e042a4ea76a4a\"}";
MapDocument mapDoc1 = PaceUtils.asMapDocument(config, JSONEntity1);
MapDocument mapDoc2 = PaceUtils.asMapDocument(config, JSONEntity2);
TreeStats treeStats = treeProcessor.evaluateTree(mapDoc1, mapDoc2);
System.out.println(treeStats);
}

View File

@ -0,0 +1,53 @@
package eu.dnetlib.pace;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.tree.support.TreeNodeDef;
import eu.dnetlib.pace.tree.support.TreeNodeStats;
import eu.dnetlib.support.ConnectedComponent;
import org.apache.spark.api.java.JavaRDD;
import java.util.List;
import java.util.Map;
public abstract class DedupTestUtils {
public static void printStatistics(JavaRDD<ConnectedComponent> ccs){
final JavaRDD<ConnectedComponent> connectedComponents = ccs.filter(cc -> cc.getDocs().size()>1);
final JavaRDD<ConnectedComponent> nonDeduplicated = ccs.filter(cc -> cc.getDocs().size()==1);
//print deduped
connectedComponents.foreach(cc -> {
System.out.println(cc);
});
// connectedComponents.foreach(cc -> {
// cc.getDocs().stream().forEach(d -> {
// System.out.println(d.getFieldMap().get("legalname") + " | " + d.getFieldMap().get("legalshortname"));
// });
// });
//print nondeduped
nonDeduplicated.foreach(cc -> {
System.out.println(cc);
});
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());
}
public static String getOrganizationLegalname(MapDocument mapDocument){
return mapDocument.getFieldMap().get("legalname").stringValue();
}
public static String getJSONEntity(List<String> entities, String id){
for (String entity: entities) {
if(entity.contains(id))
return entity;
}
return "";
}
}

View File

@ -61,6 +61,46 @@
"ignoreUndefined": "true"
},
"layer3": {
"fields": [
{
"field": "legalname",
"comparator": "cityMatch",
"weight": 1.0,
"countIfUndefined": "true",
"params": {
"windowSize": 4,
"threshold": 0.0
}
}
],
"threshold": 1.0,
"aggregation": "W_MEAN",
"positive": "layer4",
"negative": "NO_MATCH",
"undefined": "NO_MATCH",
"ignoreUndefined": "true"
},
"layer4": {
"fields": [
{
"field": "legalname",
"comparator": "keywordMatch",
"weight": 1.0,
"countIfUndefined": "false",
"params": {
"windowSize": 4,
"threshold": 0.7
}
}
],
"threshold": 1.0,
"aggregation": "W_MEAN",
"positive": "layer5",
"negative": "NO_MATCH",
"undefined": "layer5",
"ignoreUndefined": "false"
},
"layer5": {
"fields": [
{
"field": "legalname",
@ -68,8 +108,7 @@
"weight": 0.9,
"countIfUndefined": "true",
"params": {
"windowSize": 4,
"threshold": 0.7
"windowSize": 4
}
},
{

View File

@ -61,6 +61,46 @@
"ignoreUndefined": "true"
},
"layer3": {
"fields": [
{
"field": "legalname",
"comparator": "cityMatch",
"weight": 1.0,
"countIfUndefined": "true",
"params": {
"windowSize": 4,
"threshold": 0.7
}
}
],
"threshold": 1.0,
"aggregation": "W_MEAN",
"positive": "layer4",
"negative": "NO_MATCH",
"undefined": "NO_MATCH",
"ignoreUndefined": "true"
},
"layer4": {
"fields": [
{
"field": "legalname",
"comparator": "keywordMatch",
"weight": 1.0,
"countIfUndefined": "false",
"params": {
"windowSize": 4,
"threshold": 0.9
}
}
],
"threshold": 1.0,
"aggregation": "W_MEAN",
"positive": "layer5",
"negative": "NO_MATCH",
"undefined": "layer5",
"ignoreUndefined": "false"
},
"layer5": {
"fields": [
{
"field": "legalname",
@ -68,8 +108,7 @@
"weight": 0.9,
"countIfUndefined": "true",
"params": {
"windowSize": 4,
"threshold": 0.9
"windowSize": 4
}
},
{

View File

@ -13,7 +13,6 @@ public class RandomClusteringFunction extends AbstractClusteringFunction {
@Override
protected Collection<String> doApply(final Config conf, String s) {
// TODO Auto-generated method stub
return null;
}

View File

@ -216,14 +216,19 @@ public abstract class AbstractPaceFunctions {
Set<String> k1 = keywordsToCodes(s1, translationMap);
Set<String> k2 = keywordsToCodes(s2, translationMap);
int longer = (k1.size()>k2.size())?k1.size():k2.size();
if (k1.isEmpty() || k2.isEmpty())
return 1.0;
else
return (double)CollectionUtils.intersection(k1,k2).size()/(double)longer;
return commonElementsPercentage(k1, k2);
}
public double commonElementsPercentage(Set<String> s1, Set<String> s2){
int longer = (s1.size()>s2.size())?s1.size():s2.size();
return (double)CollectionUtils.intersection(s1,s2).size()/(double)longer;
}
//convert the set of keywords to codes
public Set<String> toCodes(Set<String> keywords, Map<String, String> translationMap) {
return keywords.stream().map(s -> translationMap.get(s)).collect(Collectors.toSet());

View File

@ -0,0 +1,47 @@
package eu.dnetlib.pace.tree;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
import java.util.Set;
@ComparatorClass("cityMatch")
public class CityMatch extends AbstractComparator {
private Map<String, Number> params;
public CityMatch(Map<String, Number> params) {
super(params);
this.params = params;
}
@Override
public double distance(final String a, final String b, final Config conf) {
String ca = cleanup(a);
String cb = cleanup(b);
ca = normalize(ca);
cb = normalize(cb);
ca = filterAllStopWords(ca);
cb = filterAllStopWords(cb);
Set<String> cities1 = getCities(ca, params.getOrDefault("windowSize", 4).intValue());
Set<String> cities2 = getCities(cb, params.getOrDefault("windowSize", 4).intValue());
Set<String> codes1 = citiesToCodes(cities1);
Set<String> codes2 = citiesToCodes(cities2);
//if no cities are detected, the comparator gives 1.0
if (codes1.isEmpty() && codes2.isEmpty())
return 1.0;
else {
if (codes1.isEmpty() ^ codes2.isEmpty())
return -1; //undefined if one of the two has no cities
return commonElementsPercentage(codes1, codes2) > params.getOrDefault("threshold", 0).intValue() ? 1.0 : 0.0;
}
}
}

View File

@ -45,40 +45,15 @@ public class JaroWinklerNormalizedName extends AbstractComparator {
Set<String> cities1 = getCities(ca, params.getOrDefault("windowSize", 4).intValue());
Set<String> cities2 = getCities(cb, params.getOrDefault("windowSize", 4).intValue());
if (checkCities(cities1,cities2)) {
ca = removeKeywords(ca, keywords1);
ca = removeKeywords(ca, cities1);
cb = removeKeywords(cb, keywords2);
cb = removeKeywords(cb, cities2);
if (keywordsCompare(keywords1, keywords2, conf.translationMap())>params.getOrDefault("threshold", 0.5).doubleValue()) {
ca = removeKeywords(ca, keywords1);
ca = removeKeywords(ca, cities1);
cb = removeKeywords(cb, keywords2);
cb = removeKeywords(cb, cities2);
if (ca.isEmpty() && cb.isEmpty())
return 1.0;
else
return normalize(ssalgo.score(ca,cb));
}
}
return 0.0;
}
//returns true if at least 1 city is in common
//returns true if no cities are contained in names
//returns false if one of the two names have no city
public boolean checkCities(Set<String> s1, Set<String> s2){
Set<String> c1 = citiesToCodes(s1);
Set<String> c2 = citiesToCodes(s2);
if (c1.isEmpty() && c2.isEmpty())
return true;
else {
if (c1.isEmpty() ^ c2.isEmpty())
return false;
return CollectionUtils.intersection(c1, c2).size() > 0;
}
if (ca.isEmpty() && cb.isEmpty())
return 1.0;
else
return normalize(ssalgo.score(ca,cb));
}
@Override

View File

@ -0,0 +1,47 @@
package eu.dnetlib.pace.tree;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
import java.util.Set;
@ComparatorClass("keywordMatch")
public class KeywordMatch extends AbstractComparator {
Map<String, Number> params;
public KeywordMatch(Map<String, Number> params) {
super(params);
this.params = params;
}
@Override
public double distance(final String a, final String b, final Config conf) {
String ca = cleanup(a);
String cb = cleanup(b);
ca = normalize(ca);
cb = normalize(cb);
ca = filterAllStopWords(ca);
cb = filterAllStopWords(cb);
Set<String> keywords1 = getKeywords(ca, conf.translationMap(), params.getOrDefault("windowSize", 4).intValue());
Set<String> keywords2 = getKeywords(cb, conf.translationMap(), params.getOrDefault("windowSize", 4).intValue());
Set<String> codes1 = toCodes(keywords1, conf.translationMap());
Set<String> codes2 = toCodes(keywords2, conf.translationMap());
//if no cities are detected, the comparator gives 1.0
if (codes1.isEmpty() && codes2.isEmpty())
return 1.0;
else {
if (codes1.isEmpty() ^ codes2.isEmpty())
return -1; //undefined if one of the two has no keywords
return commonElementsPercentage(codes1, codes2) > params.getOrDefault("threshold", 0).intValue() ? 1.0 : 0.0;
}
}
}

View File

@ -0,0 +1,54 @@
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.util.PaceException;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
public class FieldStats implements Serializable {
private double weight; //weight for the field (to be used in the aggregation)
private double result; //the result of the comparison
private boolean countIfUndefined;
public FieldStats(double weight, double result, boolean countIfUndefined) {
this.weight = weight;
this.result = result;
this.countIfUndefined = countIfUndefined;
}
public double getWeight() {
return weight;
}
public void setWeight(double weight) {
this.weight = weight;
}
public double getResult() {
return result;
}
public void setResult(double result) {
this.result = result;
}
public boolean isCountIfUndefined() {
return countIfUndefined;
}
public void setCountIfUndefined(boolean countIfUndefined) {
this.countIfUndefined = countIfUndefined;
}
@Override
public String toString(){
try {
return new ObjectMapper().writeValueAsString(this);
} catch (IOException e) {
throw new PaceException("Impossible to convert to JSON: ", e);
}
}
}

View File

@ -39,7 +39,6 @@ public class TreeNodeDef implements Serializable {
public TreeNodeStats evaluate(MapDocument doc1, MapDocument doc2, Config conf) {
TreeNodeStats stats = new TreeNodeStats();
stats.setFieldsCount(fields.size());
for (FieldConf fieldConf : fields) {
@ -47,16 +46,7 @@ public class TreeNodeDef implements Serializable {
double result = comparator(fieldConf).compare(doc1.getFieldMap().get(fieldConf.getField()), doc2.getFieldMap().get(fieldConf.getField()), conf);
if (result == -1) { //if the comparison is undefined
stats.incrementUndefinedCount();
if (fieldConf.isCountIfUndefined()) { //if it must be taken into account, increment weights (i.e. the average would be lower)
stats.incrementWeightsSum(weight);
}
}
else { //if the field is not missing
stats.incrementScoresSum(weight * result);
stats.incrementWeightsSum(weight);
}
stats.addFieldStats(fieldConf.getComparator() + " on " + fieldConf.getField(), new FieldStats(weight, result, fieldConf.isCountIfUndefined()));
}

View File

@ -1,90 +1,108 @@
package eu.dnetlib.pace.tree.support;
import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
public class TreeNodeStats implements Serializable {
private DescriptiveStatistics stats;
private int undefinedCount = 0; //counter for the number of undefined comparisons between the fields in the tree node
private int fieldsCount = 0;
private double weightsSum = 0.0;
private Map<String, FieldStats> results; //this is an accumulator for the results of the node
public TreeNodeStats(){
this.stats = new DescriptiveStatistics();
this.results = new HashMap<>();
}
public TreeNodeStats(int undefinedCount, int fieldsCount, double weightsSum) {
this.undefinedCount = undefinedCount;
this.fieldsCount = fieldsCount;
this.weightsSum = weightsSum;
public Map<String, FieldStats> getResults() {
return results;
}
public DescriptiveStatistics getStats() {
return stats;
public void addFieldStats(String id, FieldStats fieldStats){
this.results.put(id, fieldStats);
}
public void setStats(DescriptiveStatistics stats) {
this.stats = stats;
public int fieldsCount(){
return this.results.size();
}
public int getUndefinedCount() {
public int undefinedCount(){
int undefinedCount = 0;
for(FieldStats fs: this.results.values()){
if(fs.getResult() == -1)
undefinedCount ++;
}
return undefinedCount;
}
public void setUndefinedCount(int undefinedCount) {
this.undefinedCount = undefinedCount;
public double scoreSum(){
double scoreSum = 0.0;
for(FieldStats fs: this.results.values()){
if(fs.getResult()>=0.0) {
scoreSum += fs.getResult();
}
}
return scoreSum;
}
public int getFieldsCount() {
return fieldsCount;
//return the sum of the weights without considering the fields with countIfMissing=false && result=-1
public double weightSum(){
double weightSum = 0.0;
for(FieldStats fs: this.results.values()){
if(fs.getResult()>=0.0 || (fs.getResult()<0.0 && fs.isCountIfUndefined())) {
weightSum += fs.getWeight();
}
}
return weightSum;
}
public void setFieldsCount(int fields) {
this.fieldsCount = fields;
public double weightedScoreSum(){
double weightedScoreSum = 0.0;
for(FieldStats fs: this.results.values()){
if(fs.getResult()>=0.0) {
weightedScoreSum += fs.getResult()*fs.getWeight();
}
}
return weightedScoreSum;
}
public double getWeightsSum() {
return weightsSum;
public double max(){
double max = -1.0;
for(FieldStats fs: this.results.values()){
if(fs.getResult()>max)
max = fs.getResult();
}
return max;
}
public void setWeightsSum(double weightsSum) {
this.weightsSum = weightsSum;
}
public void incrementWeightsSum(double delta){
this.weightsSum += delta;
}
public void incrementUndefinedCount(){
this.undefinedCount += 1;
}
public void incrementScoresSum(double delta){
this.stats.addValue(delta);
public double min(){
double min = 100.0; //random high value
for(FieldStats fs: this.results.values()){
if(fs.getResult()<min) {
if (fs.getResult()>=0.0 || (fs.getResult() == -1 && fs.isCountIfUndefined()))
min = fs.getResult();
}
}
return min;
}
public double getFinalScore(AggType aggregation){
switch (aggregation){
case AVG:
return stats.getMean();
return scoreSum()/fieldsCount();
case SUM:
return stats.getSum();
return scoreSum();
case SC:
case OR:
case MAX:
return stats.getMax();
return max();
case NC:
case AND:
case MIN:
return stats.getMin();
return min();
case W_MEAN:
return stats.getSum()/weightsSum;
return weightedScoreSum()/weightSum();
default:
return 0.0;
}
}
}

View File

@ -6,12 +6,13 @@ import eu.dnetlib.pace.util.PaceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.Serializable;
import java.util.Map;
/**
* The compare between two documents is given by the weighted mean of the field distances
*/
public class TreeProcessor {
public class TreeProcessor{
private static final Log log = LogFactory.getLog(TreeProcessor.class);
@ -24,10 +25,12 @@ public class TreeProcessor {
public boolean compare(final MapDocument a, final MapDocument b) {
//evaluate the decision tree
return evaluateTree(a, b) == MatchType.MATCH;
return evaluateTree(a, b).getResult() == MatchType.MATCH;
}
public MatchType evaluateTree(final MapDocument doc1, final MapDocument doc2){
public TreeStats evaluateTree(final MapDocument doc1, final MapDocument doc2){
TreeStats treeStats = new TreeStats();
String current = "start";
@ -39,9 +42,10 @@ public class TreeProcessor {
throw new PaceException("The Tree Node doesn't exist: " + current);
TreeNodeStats stats = currentNode.evaluate(doc1, doc2, config);
treeStats.addNodeStats(current, stats);
//if ignoreUndefined=false the miss is considered as undefined
if (!currentNode.isIgnoreUndefined() && stats.getUndefinedCount()>0) {
if (!currentNode.isIgnoreUndefined() && stats.undefinedCount()>0) {
current = currentNode.getUndefined();
}
//if ignoreUndefined=true the miss is ignored and the score computed anyway
@ -54,7 +58,8 @@ public class TreeProcessor {
}
return MatchType.parse(current);
treeStats.setResult(MatchType.parse(current));
return treeStats;
}
public double computeScore(final MapDocument doc1, final MapDocument doc2) {
@ -72,7 +77,7 @@ public class TreeProcessor {
score = stats.getFinalScore(currentNode.getAggregation());
//if ignoreUndefined=false the miss is considered as undefined
if (!currentNode.isIgnoreUndefined() && stats.getUndefinedCount()>0) {
if (!currentNode.isIgnoreUndefined() && stats.undefinedCount()>0) {
current = currentNode.getUndefined();
}
//if ignoreUndefined=true the miss is ignored and the score computed anyway

View File

@ -0,0 +1,51 @@
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.util.PaceException;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
public class TreeStats {
//<layer_id, <field:comparator, result>>
Map<String, TreeNodeStats> stats;
MatchType result;
public TreeStats(){
this.stats = new HashMap<>();
this.result = MatchType.NO_MATCH;
}
public MatchType getResult(){
return this.result;
}
public void setResult(MatchType result){
this.result = result;
}
public Map<String, TreeNodeStats> getStats() {
return stats;
}
public void setStats(Map<String, TreeNodeStats> stats) {
this.stats = stats;
}
public void addNodeStats(String layerID, TreeNodeStats treeNodeStats){
this.stats.put(layerID, treeNodeStats);
}
@Override
public String toString(){
try {
return new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this);
} catch (IOException e) {
throw new PaceException("Impossible to convert to JSON: ", e);
}
}
}

View File

@ -1,6 +1,7 @@
package eu.dnetlib.pace.comparators;
import eu.dnetlib.pace.clustering.NGramUtils;
import eu.dnetlib.pace.tree.CityMatch;
import eu.dnetlib.pace.tree.JaroWinklerNormalizedName;
import eu.dnetlib.pace.config.DedupConfig;
import org.junit.Before;
@ -129,4 +130,24 @@ public class ComparatorTest extends AbstractPaceFunctions {
System.out.println("result = " + result);
}
@Test
public void cityMatchTest() {
final CityMatch cityMatch = new CityMatch(params);
//both names with no cities
assertEquals(1.0, cityMatch.distance("Università", "Centro di ricerca", conf));
//one of the two names with no cities
assertEquals(-1.0, cityMatch.distance("Università di Bologna", "Centro di ricerca", conf));
//both names with cities (same)
assertEquals(1.0, cityMatch.distance("Universita di Bologna", "Biblioteca di Bologna", conf));
//both names with cities (different)
assertEquals(0.0, cityMatch.distance("Universita di Bologna", "Universita di Torino", conf));
}
}

View File

@ -0,0 +1,23 @@
package eu.dnetlib.pace.util;
import org.junit.Before;
import org.junit.Test;
import java.util.HashMap;
import java.util.Map;
public class UtilTest {
Map<String, Number> params;
@Before
public void setUp(){
params = new HashMap<String, Number>();
}
@Test
public void paceResolverTest() {
PaceResolver paceResolver = new PaceResolver();
paceResolver.getComparator("keywordMatch", params);
}
}