implementation of the decision tree. It takes place of the distance algos, necessaryConditions and sufficientConditions are still there. The model contains only path, type and name of the field. ignoreMissing is still in the model because it is used by the conditions.

clustering
miconis 5 years ago
parent cb51e017aa
commit 72b14ec36b

@ -53,7 +53,7 @@ a different license, not a version of the Affero GPL, but Affero has
released a new version of the Affero GPL which permits relicensing under
this license.
The precise terms and conditions for copying, distribution and
The precise terms and necessaryConditions for copying, distribution and
modification follow.
TERMS AND CONDITIONS
@ -143,14 +143,14 @@ same work.
All rights granted under this License are granted for the term of
copyright on the Program, and are irrevocable provided the stated
conditions are met. This License explicitly affirms your unlimited
necessaryConditions are met. This License explicitly affirms your unlimited
permission to run the unmodified Program. The output from running a
covered work is covered by this License only if the output, given its
content, constitutes a covered work. This License acknowledges your
rights of fair use or other equivalent, as provided by copyright law.
You may make, run and propagate covered works that you do not
convey, without conditions so long as your license otherwise remains
convey, without necessaryConditions so long as your license otherwise remains
in force. You may convey covered works to others for the sole purpose
of having them make modifications exclusively for you, or provide you
with facilities for running those works, provided that you comply with
@ -161,7 +161,7 @@ and control, on terms that prohibit them from making any copies of
your copyrighted material outside their relationship with you.
Conveying under any other circumstances is permitted solely under
the conditions stated below. Sublicensing is not allowed; section 10
the necessaryConditions stated below. Sublicensing is not allowed; section 10
makes it unnecessary.
3. Protecting Users' Legal Rights From Anti-Circumvention Law.
@ -197,13 +197,13 @@ and you may offer support or warranty protection for a fee.
You may convey a work based on the Program, or the modifications to
produce it from the Program, in the form of source code under the
terms of section 4, provided that you also meet all of these conditions:
terms of section 4, provided that you also meet all of these necessaryConditions:
a) The work must carry prominent notices stating that you modified
it, and giving a relevant date.
b) The work must carry prominent notices stating that it is
released under this License and any conditions added under section
released under this License and any necessaryConditions added under section
7. This requirement modifies the requirement in section 4 to
"keep intact all notices".
@ -331,7 +331,7 @@ unpacking, reading or copying.
7. Additional Terms.
"Additional permissions" are terms that supplement the terms of this
License by making exceptions from one or more of its conditions.
License by making exceptions from one or more of its necessaryConditions.
Additional permissions that are applicable to the entire Program shall
be treated as though they were included in this License, to the extent
that they are valid under applicable law. If additional permissions
@ -527,9 +527,9 @@ otherwise be available to you under applicable patent law.
12. No Surrender of Others' Freedom.
If conditions are imposed on you (whether by court order, agreement or
otherwise) that contradict the conditions of this License, they do not
excuse you from the conditions of this License. If you cannot convey a
If necessaryConditions are imposed on you (whether by court order, agreement or
otherwise) that contradict the necessaryConditions of this License, they do not
excuse you from the necessaryConditions of this License. If you cannot convey a
covered work so as to satisfy simultaneously your obligations under this
License and any other pertinent obligations, then as a consequence you may
not convey it at all. For example, if you agree to terms that obligate you
@ -568,7 +568,7 @@ address new problems or concerns.
Each version is given a distinguishing version number. If the
Program specifies that a certain numbered version of the GNU Affero General
Public License "or any later version" applies to it, you have the
option of following the terms and conditions either of that numbered
option of following the terms and necessaryConditions either of that numbered
version or of any later version published by the Free Software
Foundation. If the Program does not specify a version number of the
GNU Affero General Public License, you may choose any version ever published

@ -31,7 +31,7 @@ if score<\th --- negative result
In order to make the decision tree work, the BlockProcessor has been modified with the following changes:
- if the decision tree is defined into the JSON configuration the deduplication process relies on it
- if the decision tree is not defined the deduplication process is exactly like before (strict conditions, conditions, dinstance algos etc.)
- if the decision tree is not defined the deduplication process is exactly like before (strict necessaryConditions, necessaryConditions, dinstance algos etc.)
# Cities and Keyword identification for organization deduplication

@ -3,10 +3,10 @@ package eu.dnetlib;
import eu.dnetlib.graph.GraphProcessor;
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.SparkBlockProcessor;
import eu.dnetlib.reporter.SparkReporter;
import eu.dnetlib.reporter.SparkReporter;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@ -64,9 +64,9 @@ public class SparkLocalTest {
//create relations by comparing only elements in the same group
final JavaPairRDD<String, String> relationRDD = blocks.flatMapToPair(it -> {
final SparkReporter reporter = new SparkReporter();
new SparkBlockProcessor(config).process(it.getKey(), it.getElements(), reporter, accumulators);
return reporter.getReport().iterator();
final SparkReporter reporter = new SparkReporter(accumulators);
new BlockProcessor(config).process(it.getKey(), it.getElements(), reporter);
return reporter.getRelations().iterator();
});
final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd();

@ -1,118 +1,118 @@
package eu.dnetlib;
import com.google.common.collect.Lists;
import eu.dnetlib.graph.GraphProcessor;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.utils.PaceUtils;
import eu.dnetlib.reporter.SparkBlockProcessor2;
import eu.dnetlib.reporter.SparkReporter;
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 org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator;
import scala.Tuple2;
import java.math.BigInteger;
import java.net.URL;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class SparkLocalTest2 {
public static void main(String[] args) {
double startTime = System.currentTimeMillis();
final SparkSession spark = SparkSession
.builder()
.appName("Deduplication")
.master("local[*]")
.getOrCreate();
final JavaSparkContext context = new JavaSparkContext(spark.sparkContext());
final URL dataset = SparkLocalTest2.class.getResource("/eu/dnetlib/pace/softwares.huge.json");
JavaRDD<String> dataRDD = context.textFile(dataset.getPath());
//read the configuration from the classpath
final DedupConfig config = DedupConfig.load(Utility.readFromClasspath("/eu/dnetlib/pace/software.pace.conf", SparkLocalTest2.class));
Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
//create vertexes of the graph: <ID, MapDocument>
JavaPairRDD<String, MapDocument> mapDocs = dataRDD.mapToPair(it -> {
MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
});
// System.out.println("mapDocs = " + mapDocs.count());
RDD<Tuple2<Object, MapDocument>> vertexes = mapDocs.mapToPair(t -> new Tuple2<Object, MapDocument>( (long) t._1().hashCode(), t._2())).rdd();
//create relations between documents
JavaRDD<Block> 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().map(b -> new Block(b._1(), b._2())).filter(b -> b.getElements().size()>1);
// //BLOCK PURGING
// blocks = BlockUtils.blockPurging2(blocks);
//// blockPurging(blocks);
//
//// //BLOCK FILTERING
// blocks = BlockUtils.blockFiltering(blocks);
JavaPairRDD<Tuple2<MapDocument, MapDocument>, Integer> edge = blocks.flatMap(it -> {
final SparkReporter reporter = new SparkReporter();
return new SparkBlockProcessor2(config).process(it.getKey(), it.getElements(), reporter, accumulators);
}).mapToPair(candidate -> new Tuple2<>(candidate, 1))
.reduceByKey((a, b) -> a + b);
final JavaPairRDD<String, String> relationRDD = edge.filter(e -> {
final SparkReporter reporter = new SparkReporter();
return new SparkBlockProcessor2(config).isSimilar(e._1(), reporter, accumulators);
}).mapToPair(t -> new Tuple2<>(t._1()._1().getIdentifier(), t._1()._2().getIdentifier()));
System.out.println("relationRDD = " + relationRDD.count());
final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd();
JavaRDD<ConnectedComponent> ccs = GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD();
System.out.println("total time = " + (System.currentTimeMillis()-startTime));
printStatistics(ccs);
accumulators.forEach((name, acc) -> System.out.println(name + " -> " + acc.value()));
}
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);
// });
// //print nondeduped
// nonDeduplicated.foreach(cc -> {
// System.out.println(cc);
//package eu.dnetlib;
//
//import com.google.common.collect.Lists;
//import eu.dnetlib.graph.GraphProcessor;
//import eu.dnetlib.pace.config.DedupConfig;
//import eu.dnetlib.pace.model.MapDocument;
//import eu.dnetlib.pace.utils.PaceUtils;
//import eu.dnetlib.reporter.SparkBlockProcessor2;
//import eu.dnetlib.reporter.SparkReporter;
//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 org.apache.spark.sql.SparkSession;
//import org.apache.spark.util.LongAccumulator;
//import scala.Tuple2;
//
//import java.math.BigInteger;
//import java.net.URL;
//import java.util.ArrayList;
//import java.util.List;
//import java.util.Map;
//import java.util.stream.Collectors;
//
//public class SparkLocalTest2 {
//
// public static void main(String[] args) {
//
// double startTime = System.currentTimeMillis();
//
// final SparkSession spark = SparkSession
// .builder()
// .appName("Deduplication")
// .master("local[*]")
// .getOrCreate();
//
// final JavaSparkContext context = new JavaSparkContext(spark.sparkContext());
//
// final URL dataset = SparkLocalTest2.class.getResource("/eu/dnetlib/pace/softwares.huge.json");
// JavaRDD<String> dataRDD = context.textFile(dataset.getPath());
//
// //read the configuration from the classpath
// final DedupConfig config = DedupConfig.load(Utility.readFromClasspath("/eu/dnetlib/pace/software.pace.conf", SparkLocalTest2.class));
//
// Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
//
// //create vertexes of the graph: <ID, MapDocument>
// JavaPairRDD<String, MapDocument> mapDocs = dataRDD.mapToPair(it -> {
// MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
// return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
// });
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());
}
}
//
//// System.out.println("mapDocs = " + mapDocs.count());
//
// RDD<Tuple2<Object, MapDocument>> vertexes = mapDocs.mapToPair(t -> new Tuple2<Object, MapDocument>( (long) t._1().hashCode(), t._2())).rdd();
//
// //create relations between documents
// JavaRDD<Block> 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().map(b -> new Block(b._1(), b._2())).filter(b -> b.getElements().size()>1);
//
//// //BLOCK PURGING
//// blocks = BlockUtils.blockPurging2(blocks);
////// blockPurging(blocks);
////
////// //BLOCK FILTERING
//// blocks = BlockUtils.blockFiltering(blocks);
//
// JavaPairRDD<Tuple2<MapDocument, MapDocument>, Integer> edge = blocks.flatMap(it -> {
// final SparkReporter reporter = new SparkReporter(accumulators);
// return new SparkBlockProcessor2(config).process(it.getKey(), it.getElements(), reporter, accumulators);
// }).mapToPair(candidate -> new Tuple2<>(candidate, 1))
// .reduceByKey((a, b) -> a + b);
//
// final JavaPairRDD<String, String> relationRDD = edge.filter(e -> {
// final SparkReporter reporter = new SparkReporter(accumulators);
// return new SparkBlockProcessor2(config).isSimilar(e._1(), reporter, accumulators);
// }).mapToPair(t -> new Tuple2<>(t._1()._1().getIdentifier(), t._1()._2().getIdentifier()));
//
// System.out.println("relationRDD = " + relationRDD.count());
//
// final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd();
//
// JavaRDD<ConnectedComponent> ccs = GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD();
//
// System.out.println("total time = " + (System.currentTimeMillis()-startTime));
//
// printStatistics(ccs);
// accumulators.forEach((name, acc) -> System.out.println(name + " -> " + acc.value()));
//
// }
//
// 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);
//// });
//// //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());
//
// }
//
//}

@ -1,90 +1,90 @@
package eu.dnetlib;
import eu.dnetlib.graph.GraphProcessor;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.utils.PaceUtils;
import eu.dnetlib.reporter.SparkBlockProcessor;
import eu.dnetlib.reporter.SparkReporter;
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 org.apache.spark.sql.SparkSession;
import org.apache.spark.util.LongAccumulator;
import scala.Tuple2;
import java.io.IOException;
import java.util.Map;
import java.util.stream.Collectors;
public class SparkTest {
public static void main(String[] args) throws IOException {
final String inputSpacePath = args[0];
final String dedupConfigPath = args[1];
final String groupsPath = args[2] + "_groups";
final String outputPath = args[2] + "_output";
final SparkSession spark = SparkSession
.builder()
.appName("Deduplication")
.master("yarn")
.getOrCreate();
final JavaSparkContext context = new JavaSparkContext(spark.sparkContext());
final JavaRDD<String> dataRDD = Utility.loadDataFromHDFS(inputSpacePath, context);
final DedupConfig config = Utility.loadConfigFromHDFS(dedupConfigPath);
Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
//create vertexes of the graph: <ID, MapDocument>
JavaPairRDD<String, MapDocument> mapDocs = dataRDD.mapToPair(it -> {
MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
});
RDD<Tuple2<Object, MapDocument>> vertexes = mapDocs.mapToPair(t -> new Tuple2<Object, MapDocument>( (long) t._1().hashCode(), t._2())).rdd();
//group documents basing on clustering
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();
Utility.deleteIfExists(groupsPath);
blocks.map(group -> new DocumentsBlock(group._1(), group._2())).saveAsTextFile(groupsPath);
//create relations by comparing only elements in the same group
final JavaPairRDD<String, String> relationRDD = blocks.flatMapToPair(it -> {
final SparkReporter reporter = new SparkReporter();
new SparkBlockProcessor(config).process(it._1(), it._2(), reporter, accumulators);
return reporter.getReport().iterator();
});
final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd();
JavaRDD<ConnectedComponent> ccs = GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD();
//save connected components on textfile
Utility.deleteIfExists(outputPath);
ccs.saveAsTextFile(outputPath);
final JavaRDD<ConnectedComponent> connectedComponents = ccs.filter(cc -> cc.getDocs().size()>1);
final JavaRDD<ConnectedComponent> 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());
accumulators.forEach((name, acc) -> System.out.println(name + " -> " + acc.value()));
}
}
//package eu.dnetlib;
//
//import eu.dnetlib.graph.GraphProcessor;
//import eu.dnetlib.pace.config.DedupConfig;
//import eu.dnetlib.pace.model.MapDocument;
//import eu.dnetlib.pace.utils.PaceUtils;
//import eu.dnetlib.reporter.SparkBlockProcessor;
//import eu.dnetlib.reporter.SparkReporter;
//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 org.apache.spark.sql.SparkSession;
//import org.apache.spark.util.LongAccumulator;
//import scala.Tuple2;
//
//import java.io.IOException;
//import java.util.Map;
//import java.util.stream.Collectors;
//
//public class SparkTest {
//
// public static void main(String[] args) throws IOException {
//
// final String inputSpacePath = args[0];
// final String dedupConfigPath = args[1];
// final String groupsPath = args[2] + "_groups";
// final String outputPath = args[2] + "_output";
//
// final SparkSession spark = SparkSession
// .builder()
// .appName("Deduplication")
// .master("yarn")
// .getOrCreate();
//
// final JavaSparkContext context = new JavaSparkContext(spark.sparkContext());
//
// final JavaRDD<String> dataRDD = Utility.loadDataFromHDFS(inputSpacePath, context);
//
// final DedupConfig config = Utility.loadConfigFromHDFS(dedupConfigPath);
//
// Map<String, LongAccumulator> accumulators = Utility.constructAccumulator(config, context.sc());
//
// //create vertexes of the graph: <ID, MapDocument>
// JavaPairRDD<String, MapDocument> mapDocs = dataRDD.mapToPair(it -> {
// MapDocument mapDocument = PaceUtils.asMapDocument(config, it);
// return new Tuple2<>(mapDocument.getIdentifier(), mapDocument);
// });
// RDD<Tuple2<Object, MapDocument>> vertexes = mapDocs.mapToPair(t -> new Tuple2<Object, MapDocument>( (long) t._1().hashCode(), t._2())).rdd();
//
// //group documents basing on clustering
// 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();
//
// Utility.deleteIfExists(groupsPath);
// blocks.map(group -> new DocumentsBlock(group._1(), group._2())).saveAsTextFile(groupsPath);
//
// //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 SparkBlockProcessor(config).process(it._1(), it._2(), reporter, accumulators);
// return reporter.getRelations().iterator();
// });
//
// final RDD<Edge<String>> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(),it._2().hashCode(), "similarTo")).rdd();
//
// JavaRDD<ConnectedComponent> ccs = GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD();
//
// //save connected components on textfile
// Utility.deleteIfExists(outputPath);
// ccs.saveAsTextFile(outputPath);
//
// final JavaRDD<ConnectedComponent> connectedComponents = ccs.filter(cc -> cc.getDocs().size()>1);
// final JavaRDD<ConnectedComponent> 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());
// accumulators.forEach((name, acc) -> System.out.println(name + " -> " + acc.value()));
//
// }
//
//}

@ -1,189 +1,189 @@
package eu.dnetlib.reporter;
import com.google.common.collect.Lists;
import eu.dnetlib.pace.clustering.NGramUtils;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.config.WfConfig;
import eu.dnetlib.pace.distance.PaceDocumentDistance;
import eu.dnetlib.pace.distance.eval.ScoreResult;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.model.MapDocumentComparator;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.spark.util.LongAccumulator;
import java.util.*;
public class SparkBlockProcessor {
public static final List<String> accumulators= new ArrayList<>();
private static final Log log = LogFactory.getLog(SparkBlockProcessor.class);
private DedupConfig dedupConf;
public SparkBlockProcessor(DedupConfig dedupConf) {
this.dedupConf = dedupConf;
}
public void process(final String key, final Iterable<MapDocument> documents, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final Queue<MapDocument> q = prepare(documents);
if (q.size() > 1) {
process(simplifyQueue(q, key, context, accumulators), context, accumulators);
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "records per hash key = 1", 1, accumulators);
}
}
private Queue<MapDocument> prepare(final Iterable<MapDocument> documents) {
final Queue<MapDocument> queue = new PriorityQueue<>(100, new MapDocumentComparator(dedupConf.getWf().getOrderField()));
final Set<String> seen = new HashSet<String>();
final int queueMaxSize = dedupConf.getWf().getQueueMaxSize();
documents.forEach(doc -> {
if (queue.size() <= queueMaxSize) {
final String id = doc.getIdentifier();
if (!seen.contains(id)) {
seen.add(id);
queue.add(doc);
}
}
});
return queue;
}
private Queue<MapDocument> simplifyQueue(final Queue<MapDocument> queue, final String ngram, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final Queue<MapDocument> q = new LinkedList<>();
String fieldRef = "";
final List<MapDocument> tempResults = Lists.newArrayList();
while (!queue.isEmpty()) {
final MapDocument result = queue.remove();
final String orderFieldName = dedupConf.getWf().getOrderField();
final Field orderFieldValue = result.values(orderFieldName);
if (!orderFieldValue.isEmpty()) {
final String field = NGramUtils.cleanupForOrdering(orderFieldValue.stringValue());
if (field.equals(fieldRef)) {
tempResults.add(result);
} else {
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
tempResults.clear();
tempResults.add(result);
fieldRef = field;
}
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField(), 1, accumulators);
}
}
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
return q;
}
private void populateSimplifiedQueue(final Queue<MapDocument> q,
final List<MapDocument> tempResults,
final SparkReporter context,
final String fieldRef,
final String ngram,
Map<String, LongAccumulator> accumulators) {
WfConfig wf = dedupConf.getWf();
if (tempResults.size() < wf.getGroupMaxSize()) {
q.addAll(tempResults);
} else {
context.incrementCounter(wf.getEntityType(), String.format("Skipped records for count(%s) >= %s", wf.getOrderField(), wf.getGroupMaxSize()), tempResults.size(), accumulators);
// log.info("Skipped field: " + fieldRef + " - size: " + tempResults.size() + " - ngram: " + ngram);
}
}
private void process(final Queue<MapDocument> queue, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final PaceDocumentDistance algo = new PaceDocumentDistance();
while (!queue.isEmpty()) {
final MapDocument pivot = queue.remove();
final String idPivot = pivot.getIdentifier();
WfConfig wf = dedupConf.getWf();
final Field fieldsPivot = pivot.values(wf.getOrderField());
final String fieldPivot = (fieldsPivot == null) || fieldsPivot.isEmpty() ? null : fieldsPivot.stringValue();
if (fieldPivot != null) {
// System.out.println(idPivot + " --> " + fieldPivot);
int i = 0;
for (final MapDocument curr : queue) {
final String idCurr = curr.getIdentifier();
if (mustSkip(idCurr)) {
context.incrementCounter(wf.getEntityType(), "skip list", 1, accumulators);
break;
}
if (i > wf.getSlidingWindowSize()) {
break;
}
final Field fieldsCurr = curr.values(wf.getOrderField());
final String fieldCurr = (fieldsCurr == null) || fieldsCurr.isEmpty() ? null : fieldsCurr.stringValue();
if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
final ScoreResult sr = similarity(algo, pivot, curr);
// log.info(sr.toString()+"SCORE "+ sr.getScore());
emitOutput(sr, idPivot, idCurr, context, accumulators);
i++;
}
}
}
}
}
private void emitOutput(final ScoreResult sr, final String idPivot, final String idCurr, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final double d = sr.getScore();
if (d >= dedupConf.getWf().getThreshold()) {
writeSimilarity(context, idPivot, idCurr);
context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1, accumulators);
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold(), 1, accumulators);
}
}
private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
try {
return algo.between(a, b, dedupConf);
} catch(Throwable e) {
log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
throw new IllegalArgumentException(e);
}
}
private boolean mustSkip(final String idPivot) {
return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
}
private String getNsPrefix(final String id) {
return StringUtils.substringBetween(id, "|", "::");
}
private void writeSimilarity(final SparkReporter context, final String from, final String to) {
final String type = dedupConf.getWf().getEntityType();
context.emit(type, from, to);
// context.emit(type, to, from);
}
}
//package eu.dnetlib.reporter;
//import com.google.common.collect.Lists;
//import eu.dnetlib.pace.clustering.NGramUtils;
//import eu.dnetlib.pace.config.DedupConfig;
//import eu.dnetlib.pace.config.WfConfig;
//import eu.dnetlib.pace.distance.PaceDocumentDistance;
//import eu.dnetlib.pace.distance.eval.ScoreResult;
//import eu.dnetlib.pace.model.Field;
//import eu.dnetlib.pace.model.MapDocument;
//import eu.dnetlib.pace.model.MapDocumentComparator;
//import org.apache.commons.lang.StringUtils;
//import org.apache.commons.logging.Log;
//import org.apache.commons.logging.LogFactory;
//import org.apache.spark.util.LongAccumulator;
//
//import java.util.*;
//
//public class SparkBlockProcessor {
//
// public static final List<String> accumulators= new ArrayList<>();
//
// private static final Log log = LogFactory.getLog(SparkBlockProcessor.class);
//
// private DedupConfig dedupConf;
//
// public SparkBlockProcessor(DedupConfig dedupConf) {
// this.dedupConf = dedupConf;
// }
//
// public void process(final String key, final Iterable<MapDocument> documents, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
//
// final Queue<MapDocument> q = prepare(documents);
//
// if (q.size() > 1) {
// process(simplifyQueue(q, key, context, accumulators), context, accumulators);
//
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "records per hash key = 1", 1, accumulators);
// }
// }
//
// private Queue<MapDocument> prepare(final Iterable<MapDocument> documents) {
// final Queue<MapDocument> queue = new PriorityQueue<>(100, new MapDocumentComparator(dedupConf.getWf().getOrderField()));
//
// final Set<String> seen = new HashSet<String>();
// final int queueMaxSize = dedupConf.getWf().getQueueMaxSize();
//
// documents.forEach(doc -> {
// if (queue.size() <= queueMaxSize) {
// final String id = doc.getIdentifier();
//
// if (!seen.contains(id)) {
// seen.add(id);
// queue.add(doc);
// }
// }
// });
//
// return queue;
// }
//
// private Queue<MapDocument> simplifyQueue(final Queue<MapDocument> queue, final String ngram, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
// final Queue<MapDocument> q = new LinkedList<>();
//
// String fieldRef = "";
// final List<MapDocument> tempResults = Lists.newArrayList();
//
// while (!queue.isEmpty()) {
// final MapDocument result = queue.remove();
//
// final String orderFieldName = dedupConf.getWf().getOrderField();
// final Field orderFieldValue = result.values(orderFieldName);
// if (!orderFieldValue.isEmpty()) {
// final String field = NGramUtils.cleanupForOrdering(orderFieldValue.stringValue());
// if (field.equals(fieldRef)) {
// tempResults.add(result);
// } else {
// populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
// tempResults.clear();
// tempResults.add(result);
// fieldRef = field;
// }
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField(), 1, accumulators);
// }
// }
// populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
//
// return q;
// }
//
// private void populateSimplifiedQueue(final Queue<MapDocument> q,
// final List<MapDocument> tempResults,
// final SparkReporter context,
// final String fieldRef,
// final String ngram,
// Map<String, LongAccumulator> accumulators) {
// WfConfig wf = dedupConf.getWf();
// if (tempResults.size() < wf.getGroupMaxSize()) {
// q.addAll(tempResults);
// } else {
// context.incrementCounter(wf.getEntityType(), String.format("Skipped records for count(%s) >= %s", wf.getOrderField(), wf.getGroupMaxSize()), tempResults.size(), accumulators);
//// log.info("Skipped field: " + fieldRef + " - size: " + tempResults.size() + " - ngram: " + ngram);
// }
// }
//
// private void process(final Queue<MapDocument> queue, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
//
// final PaceDocumentDistance algo = new PaceDocumentDistance();
//
// while (!queue.isEmpty()) {
//
// final MapDocument pivot = queue.remove();
// final String idPivot = pivot.getIdentifier();
//
// WfConfig wf = dedupConf.getWf();
// final Field fieldsPivot = pivot.values(wf.getOrderField());
// final String fieldPivot = (fieldsPivot == null) || fieldsPivot.isEmpty() ? null : fieldsPivot.stringValue();
//
// if (fieldPivot != null) {
// // System.out.println(idPivot + " --> " + fieldPivot);
//
// int i = 0;
// for (final MapDocument curr : queue) {
// final String idCurr = curr.getIdentifier();
//
// if (mustSkip(idCurr)) {
//
// context.incrementCounter(wf.getEntityType(), "skip list", 1, accumulators);
//
// break;
// }
//
// if (i > wf.getSlidingWindowSize()) {
// break;
// }
//
// final Field fieldsCurr = curr.values(wf.getOrderField());
// final String fieldCurr = (fieldsCurr == null) || fieldsCurr.isEmpty() ? null : fieldsCurr.stringValue();
//
// if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
//
// final ScoreResult sr = similarity(algo, pivot, curr);
//// log.info(sr.toString()+"SCORE "+ sr.getScore());
// emitOutput(sr, idPivot, idCurr, context, accumulators);
// i++;
// }
// }
// }
// }
// }
//
// private void emitOutput(final ScoreResult sr, final String idPivot, final String idCurr, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
// final double d = sr.getScore();
//
// if (d >= dedupConf.getWf().getThreshold()) {
//
// writeSimilarity(context, idPivot, idCurr);
// context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1, accumulators);
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold(), 1, accumulators);
// }
// }
//
// private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
// try {
// return algo.between(a, b, dedupConf);
// } catch(Throwable e) {
// log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
// throw new IllegalArgumentException(e);
// }
// }
//
// private boolean mustSkip(final String idPivot) {
// return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
// }
//
// private String getNsPrefix(final String id) {
// return StringUtils.substringBetween(id, "|", "::");
// }
//
// private void writeSimilarity(final SparkReporter context, final String from, final String to) {
// final String type = dedupConf.getWf().getEntityType();
//
// context.emit(type, from, to);
//// context.emit(type, to, from);
// }
//
//}

@ -1,193 +1,193 @@
package eu.dnetlib.reporter;
import com.google.common.collect.Lists;
import eu.dnetlib.pace.clustering.NGramUtils;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.config.WfConfig;
import eu.dnetlib.pace.distance.PaceDocumentDistance;
import eu.dnetlib.pace.distance.eval.ScoreResult;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.model.MapDocumentComparator;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.spark.util.LongAccumulator;
import scala.Tuple2;
import java.util.*;
public class SparkBlockProcessor2 {
private static final Log log = LogFactory.getLog(SparkBlockProcessor2.class);
private DedupConfig dedupConf;
public SparkBlockProcessor2(DedupConfig dedupConf) {
this.dedupConf = dedupConf;
}
public boolean isSimilar(Tuple2<MapDocument, MapDocument> t, SparkReporter context, Map<String, LongAccumulator> accumulators) {
final PaceDocumentDistance algo = new PaceDocumentDistance();
final ScoreResult sr = similarity(algo, t._1(), t._2());
final double d = sr.getScore();
if (d >= dedupConf.getWf().getThreshold()) {
context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1, accumulators);
return true;
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold(), 1, accumulators);
return false;
}
}
public Iterator<Tuple2<MapDocument, MapDocument>> process(final String key, final Iterable<MapDocument> documents, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final Queue<MapDocument> q = prepare(documents);
if (q.size() > 1) {
return process(simplifyQueue(q, key, context, accumulators), context, accumulators);
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "records per hash key = 1", 1, accumulators);
return new ArrayList<Tuple2<MapDocument,MapDocument>>().iterator();
}
}
private Queue<MapDocument> prepare(final Iterable<MapDocument> documents) {
final Queue<MapDocument> queue = new PriorityQueue<>(100, new MapDocumentComparator(dedupConf.getWf().getOrderField()));
final Set<String> seen = new HashSet<String>();
final int queueMaxSize = dedupConf.getWf().getQueueMaxSize();
documents.forEach(doc -> {
if (queue.size() <= queueMaxSize) {
final String id = doc.getIdentifier();
if (!seen.contains(id)) {
seen.add(id);
queue.add(doc);
}
}
});
return queue;
}
private Queue<MapDocument> simplifyQueue(final Queue<MapDocument> queue, final String ngram, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final Queue<MapDocument> q = new LinkedList<>();
String fieldRef = "";
final List<MapDocument> tempResults = Lists.newArrayList();
while (!queue.isEmpty()) {
final MapDocument result = queue.remove();
final String orderFieldName = dedupConf.getWf().getOrderField();
final Field orderFieldValue = result.values(orderFieldName);
if (!orderFieldValue.isEmpty()) {
final String field = NGramUtils.cleanupForOrdering(orderFieldValue.stringValue());
if (field.equals(fieldRef)) {
tempResults.add(result);
} else {
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
tempResults.clear();
tempResults.add(result);
fieldRef = field;
}
} else {
context.incrementCounter(dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField(), 1, accumulators);
}
}
populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
return q;
}
private void populateSimplifiedQueue(final Queue<MapDocument> q,
final List<MapDocument> tempResults,
final SparkReporter context,
final String fieldRef,
final String ngram,
Map<String, LongAccumulator> accumulators) {
WfConfig wf = dedupConf.getWf();
if (tempResults.size() < wf.getGroupMaxSize()) {
q.addAll(tempResults);
} else {
context.incrementCounter(wf.getEntityType(), String.format("Skipped records for count(%s) >= %s", wf.getOrderField(), wf.getGroupMaxSize()), tempResults.size(), accumulators);
// log.info("Skipped field: " + fieldRef + " - size: " + tempResults.size() + " - ngram: " + ngram);
}
}
private Iterator<Tuple2<MapDocument, MapDocument>> process(final Queue<MapDocument> queue, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
final PaceDocumentDistance algo = new PaceDocumentDistance();
List<Tuple2<MapDocument, MapDocument>> ret = new ArrayList<>();
while (!queue.isEmpty()) {
final MapDocument pivot = queue.remove();
final String idPivot = pivot.getIdentifier();
WfConfig wf = dedupConf.getWf();
final Field fieldsPivot = pivot.values(wf.getOrderField());
final String fieldPivot = (fieldsPivot == null) || fieldsPivot.isEmpty() ? null : fieldsPivot.stringValue();
if (fieldPivot != null) {
// System.out.println(idPivot + " --> " + fieldPivot);
int i = 0;
for (final MapDocument curr : queue) {
final String idCurr = curr.getIdentifier();
if (mustSkip(idCurr)) {
context.incrementCounter(wf.getEntityType(), "skip list", 1, accumulators);
break;
}
if (i > wf.getSlidingWindowSize()) {
break;
}
final Field fieldsCurr = curr.values(wf.getOrderField());
final String fieldCurr = (fieldsCurr == null) || fieldsCurr.isEmpty() ? null : fieldsCurr.stringValue();
if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
if (pivot.getIdentifier().compareTo(curr.getIdentifier())<0){
ret.add(new Tuple2<>(pivot, curr));
} else {
ret.add(new Tuple2<>(curr, pivot));
}
i++;
}
}
}
}
return ret.iterator();
}
private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
try {
return algo.between(a, b, dedupConf);
} catch(Throwable e) {
log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
throw new IllegalArgumentException(e);
}
}
private boolean mustSkip(final String idPivot) {
return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
}
private String getNsPrefix(final String id) {
return StringUtils.substringBetween(id, "|", "::");
}
}
//package eu.dnetlib.reporter;
//import com.google.common.collect.Lists;
//import eu.dnetlib.pace.clustering.NGramUtils;
//import eu.dnetlib.pace.config.DedupConfig;
//import eu.dnetlib.pace.config.WfConfig;
//import eu.dnetlib.pace.distance.PaceDocumentDistance;
//import eu.dnetlib.pace.distance.eval.ScoreResult;
//import eu.dnetlib.pace.model.Field;
//import eu.dnetlib.pace.model.MapDocument;
//import eu.dnetlib.pace.model.MapDocumentComparator;
//import org.apache.commons.lang.StringUtils;
//import org.apache.commons.logging.Log;
//import org.apache.commons.logging.LogFactory;
//import org.apache.spark.util.LongAccumulator;
//import scala.Tuple2;
//
//import java.util.*;
//
//public class SparkBlockProcessor2 {
//
// private static final Log log = LogFactory.getLog(SparkBlockProcessor2.class);
//
// private DedupConfig dedupConf;
//
// public SparkBlockProcessor2(DedupConfig dedupConf) {
// this.dedupConf = dedupConf;
// }
//
// public boolean isSimilar(Tuple2<MapDocument, MapDocument> t, SparkReporter context, Map<String, LongAccumulator> accumulators) {
//
// final PaceDocumentDistance algo = new PaceDocumentDistance();
//
// final ScoreResult sr = similarity(algo, t._1(), t._2());
//
// final double d = sr.getScore();
//
// if (d >= dedupConf.getWf().getThreshold()) {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1, accumulators);
// return true;
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold(), 1, accumulators);
// return false;
// }
// }
//
// public Iterator<Tuple2<MapDocument, MapDocument>> process(final String key, final Iterable<MapDocument> documents, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
//
// final Queue<MapDocument> q = prepare(documents);
//
// if (q.size() > 1) {
// return process(simplifyQueue(q, key, context, accumulators), context, accumulators);
//
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "records per hash key = 1", 1, accumulators);
// return new ArrayList<Tuple2<MapDocument,MapDocument>>().iterator();
// }
// }
//
// private Queue<MapDocument> prepare(final Iterable<MapDocument> documents) {
// final Queue<MapDocument> queue = new PriorityQueue<>(100, new MapDocumentComparator(dedupConf.getWf().getOrderField()));
//
// final Set<String> seen = new HashSet<String>();
// final int queueMaxSize = dedupConf.getWf().getQueueMaxSize();
//
// documents.forEach(doc -> {
// if (queue.size() <= queueMaxSize) {
// final String id = doc.getIdentifier();
//
// if (!seen.contains(id)) {
// seen.add(id);
// queue.add(doc);
// }
// }
// });
//
// return queue;
// }
//
// private Queue<MapDocument> simplifyQueue(final Queue<MapDocument> queue, final String ngram, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
// final Queue<MapDocument> q = new LinkedList<>();
//
// String fieldRef = "";
// final List<MapDocument> tempResults = Lists.newArrayList();
//
// while (!queue.isEmpty()) {
// final MapDocument result = queue.remove();
//
// final String orderFieldName = dedupConf.getWf().getOrderField();
// final Field orderFieldValue = result.values(orderFieldName);
// if (!orderFieldValue.isEmpty()) {
// final String field = NGramUtils.cleanupForOrdering(orderFieldValue.stringValue());
// if (field.equals(fieldRef)) {
// tempResults.add(result);
// } else {
// populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
// tempResults.clear();
// tempResults.add(result);
// fieldRef = field;
// }
// } else {
// context.incrementCounter(dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField(), 1, accumulators);
// }
// }
// populateSimplifiedQueue(q, tempResults, context, fieldRef, ngram, accumulators);
//
// return q;
// }
//
// private void populateSimplifiedQueue(final Queue<MapDocument> q,
// final List<MapDocument> tempResults,
// final SparkReporter context,
// final String fieldRef,
// final String ngram,
// Map<String, LongAccumulator> accumulators) {
// WfConfig wf = dedupConf.getWf();
// if (tempResults.size() < wf.getGroupMaxSize()) {
// q.addAll(tempResults);
// } else {
// context.incrementCounter(wf.getEntityType(), String.format("Skipped records for count(%s) >= %s", wf.getOrderField(), wf.getGroupMaxSize()), tempResults.size(), accumulators);
//// log.info("Skipped field: " + fieldRef + " - size: " + tempResults.size() + " - ngram: " + ngram);
// }
// }
//
// private Iterator<Tuple2<MapDocument, MapDocument>> process(final Queue<MapDocument> queue, final SparkReporter context, Map<String, LongAccumulator> accumulators) {
//
// final PaceDocumentDistance algo = new PaceDocumentDistance();
//
// List<Tuple2<MapDocument, MapDocument>> ret = new ArrayList<>();
//
// while (!queue.isEmpty()) {
//
// final MapDocument pivot = queue.remove();
// final String idPivot = pivot.getIdentifier();
//
// WfConfig wf = dedupConf.getWf();
// final Field fieldsPivot = pivot.values(wf.getOrderField());
// final String fieldPivot = (fieldsPivot == null) || fieldsPivot.isEmpty() ? null : fieldsPivot.stringValue();
//
// if (fieldPivot != null) {
// // System.out.println(idPivot + " --> " + fieldPivot);
//
// int i = 0;
// for (final MapDocument curr : queue) {
// final String idCurr = curr.getIdentifier();
//
// if (mustSkip(idCurr)) {
//
// context.incrementCounter(wf.getEntityType(), "skip list", 1, accumulators);
//
// break;
// }
//
// if (i > wf.getSlidingWindowSize()) {
// break;
// }
//
// final Field fieldsCurr = curr.values(wf.getOrderField());
// final String fieldCurr = (fieldsCurr == null) || fieldsCurr.isEmpty() ? null : fieldsCurr.stringValue();
//
// if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
//
// if (pivot.getIdentifier().compareTo(curr.getIdentifier())<0){
// ret.add(new Tuple2<>(pivot, curr));
// } else {
// ret.add(new Tuple2<>(curr, pivot));
// }
// i++;
// }
// }
// }
// }
//
// return ret.iterator();
// }
//
// private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
// try {
// return algo.between(a, b, dedupConf);
// } catch(Throwable e) {
// log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
// throw new IllegalArgumentException(e);
// }
// }
//
// private boolean mustSkip(final String idPivot) {
// return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
// }
//
// private String getNsPrefix(final String id) {
// return StringUtils.substringBetween(id, "|", "::");
// }
//
//}

@ -1,8 +1,11 @@
package eu.dnetlib.reporter;
import eu.dnetlib.Utility;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.util.Reporter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.spark.SparkContext;
import org.apache.spark.util.LongAccumulator;
import scala.Serializable;
import scala.Tuple2;
@ -11,12 +14,15 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
public class SparkReporter implements Serializable {
public class SparkReporter implements Serializable, Reporter {
final List<Tuple2<String, String>> report = new ArrayList<>();
final List<Tuple2<String, String>> relations = new ArrayList<>();
private static final Log log = LogFactory.getLog(SparkReporter.class);
Map<String, LongAccumulator> accumulators;
public SparkReporter(){}
public SparkReporter(Map<String, LongAccumulator> accumulators){
this.accumulators = accumulators;
}
public void incrementCounter(String counterGroup, String counterName, long delta, Map<String, LongAccumulator> accumulators) {
@ -27,11 +33,18 @@ public class SparkReporter implements Serializable {
}
@Override
public void incrementCounter(String counterGroup, String counterName, long delta) {
incrementCounter(counterGroup, counterName, delta, accumulators);
}
@Override
public void emit(String type, String from, String to) {
report.add(new Tuple2<>(from, to));
relations.add(new Tuple2<>(from, to));
}
public List<Tuple2<String, String>> getReport() {
return report;
public List<Tuple2<String, String>> getRelations() {
return relations;
}
}

@ -14,7 +14,7 @@
"clustering": [
{"name": "personClustering", "fields": ["fullname"], "params": {}}
],
"conditions": [],
"necessaryConditions": [],
"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"},

@ -16,10 +16,10 @@
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } },
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "exactMatch", "fields" : [ "gridid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] },
{ "name" : "DomainExactMatch", "fields" : [ "websiteurl" ] }
],

@ -17,19 +17,22 @@
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } },
{ "name" : "keywordsclustering", "fields" : [ "legalname" ], "params" : { "max": 2, "windowSize": 4} }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "exactMatch", "fields" : [ "gridid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "DomainExactMatch", "fields" : [ "websiteurl" ] },
{ "name" : "exactMatch", "fields" : [ "country" ] }
],
"decisionTree" : {
"start": {"fields": [{"field":"legalname", "comparator":"jaroWinklerNormalizedName", "weight":0.9, "ignoreMissing":"false", "params":{"windowSize" : 4, "threshold" : 0.7}}, {"field":"legalshortname", "comparator":"jaroWinklerNormalizedName", "weight":0.1, "ignoreMissing":"true", "params":{}}], "threshold": 0.9, "aggregation": "WEIGHTED_MEAN", "positive":"MATCH", "negative":"NO_MATCH", "undefined":"NO_MATCH", "ignoreMissing": "true"}
},
"model" : [
{ "name" : "country", "algo" : "Null", "type" : "String", "weight" : "0", "ignoreMissing" : "false", "path" : "organization/metadata/country/classid" },
{ "name" : "legalshortname", "algo" : "JaroWinklerNormalizedName", "type" : "String", "weight" : "0.1", "ignoreMissing" : "true", "path" : "organization/metadata/legalshortname/value" },
{ "name" : "legalname", "algo" : "JaroWinklerNormalizedName", "type" : "String", "weight" : "0.9", "ignoreMissing" : "false", "path" : "organization/metadata/legalname/value", "params" : {"windowSize" : 4, "threshold" : 0.7} },
{ "name" : "websiteurl", "algo" : "Null", "type" : "URL", "weight" : "0", "ignoreMissing" : "true", "path" : "organization/metadata/websiteurl/value", "params" : { "host" : 0.5, "path" : 0.5 } },
{ "name" : "gridid", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "pid[qualifier#classid = {grid}]/value" }
{ "name" : "country", "type" : "String", "path" : "organization/metadata/country/classid", "ignoreMissing" : "true" },
{ "name" : "legalshortname", "type" : "String", "path" : "organization/metadata/legalshortname/value", "ignoreMissing" : "true" },
{ "name" : "legalname", "type" : "String", "path" : "organization/metadata/legalname/value", "params" : {"windowSize" : 4, "threshold" : 0.7}, "ignoreMissing" : "false" },
{ "name" : "websiteurl", "type" : "URL", "path" : "organization/metadata/websiteurl/value", "params" : { "host" : 0.5, "path" : 0.5 }, "ignoreMissing" : "true" },
{ "name" : "gridid", "type" : "String", "path" : "pid[qualifier#classid = {grid}]/value", "ignoreMissing" : "true" }
],
"blacklists" : {
"legalname" : []

@ -16,7 +16,7 @@
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : "1", "len" : "3" } },
{ "name" : "spacetrimmingfieldvalue", "fields" : [ "legalshortname" ], "params" : { "randomLength" : "5" } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] }
],
"model" : [

@ -18,10 +18,10 @@
{ "name" : "spacetrimmingfieldvalue", "fields" : [ "legalshortname" ], "params" : { "randomLength" : "5" } },
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
],
"strictConditions":[
"sufficientConditions":[
{ "name" : "exactMatch", "fields" : [ "gridid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] }
],
"model" : [

@ -15,7 +15,7 @@
{ "name" : "ngrampairs", "fields" : [ "legalname" ], "params" : { "max" : "1", "ngramLen" : "3"} },
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : "1", "len" : "3" } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] }
],
"model" : [

@ -16,7 +16,7 @@
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } },
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] },
{ "name" : "DomainExactMatch", "fields" : [ "websiteurl" ] }
],

@ -1,12 +1,12 @@
{"dateoftransformation":"2019-07-22","originalId":["corda_______::999895789"],"collectedfrom":[{"value":"CORDA - COmmon Research DAta Warehouse","key":"10|openaire____::b30dac7baac631f3da7c2bb18dd9891f"}],"organization":{"metadata":{"eclegalbody":{"value":"true"},"eclegalperson":{"value":"true"},"ecinternationalorganization":{"value":"false"},"legalshortname":{"value":"UNIFI"},"ecresearchorganization":{"value":"true"},"ecnonprofit":{"value":"true"},"ecenterprise":{"value":"false"},"websiteurl":{"value":"http://www.unifi.it"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"UNIVERSITA DEGLI STUDI DI FIRENZE"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"true"}}},"dateofcollection":"2018-03-12","type":20,"id":"20|corda_______::19137683d6d3cd4dda5054af05081b6f"}
{"dateoftransformation":"2019-06-26","originalId":["corda__h2020::999895789"],"collectedfrom":[{"value":"CORDA - COmmon Research DAta Warehouse - Horizon 2020","key":"10|openaire____::a55eb91348674d853191f4f4fd73d078"}],"organization":{"metadata":{"eclegalbody":{"value":"true"},"eclegalperson":{"value":"true"},"ecinternationalorganization":{"value":"false"},"legalshortname":{"value":"UNIFI"},"ecresearchorganization":{"value":"true"},"ecnonprofit":{"value":"true"},"ecenterprise":{"value":"false"},"websiteurl":{"value":"http://www.unifi.it"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"UNIVERSITA DEGLI STUDI DI FIRENZE"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"true"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2018-03-12","type":20,"id":"20|corda__h2020::19137683d6d3cd4dda5054af05081b6f"}
{"dateoftransformation":"2018-09-19","originalId":["doajarticles::Firenze_University_Press"],"collectedfrom":[{"value":"DOAJ-Articles","key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"legalshortname":{"value":"Firenze University Press"},"ecresearchorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecenterprise":{"value":"false"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"Firenze University Press"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2018-09-19","type":20,"id":"20|doajarticles::b29ae16abb2343c6ffc152666b24ea95"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"University of Florence"},"websiteurl":{"value":"http://www.unifi.it/"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Università degli Studi di Firenze"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"}],"type":20,"id":"20|grid________::115715507c87ade107909750c44fbee5"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"University of Florence"},"websiteurl":{"value":"http://www.unifi.it/"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"University of Florence"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"}],"type":20,"id":"20|grid________::60e21d5264c51c62f154afa6166ba21b"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"University of Florence"},"websiteurl":{"value":"http://www.unifi.it/"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"University of Florence"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"}],"type":20,"id":"20|grid________::a6d1d3c2eb368cb2ab1ff293c625d90e"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"University of Florence"},"websiteurl":{"value":"http://www.unifi.it/"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Université de florence"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"}],"type":20,"id":"20|grid________::c8b8860f04bf3c755f4632395ea27375"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"University of Florence"},"websiteurl":{"value":"http://www.unifi.it/"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Universität Florenz"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"}],"type":20,"id":"20|grid________::ff05feef920762cbef5de7640dcb718e"}
{"originalId":["https://academic.microsoft.com/#/detail/45084792"],"pid":[{"qualifier":{"classid":"urn","classname":"urn","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"http://en.wikipedia.org/wiki/University_of_Florence"},{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.8404.8"},{"qualifier":{"classid":"mag_id","classname":"Microsoft Academic Graph Identifier","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"https://academic.microsoft.com/#/detail/45084792"}],"collectedfrom":[{"value":"Microsoft Academic Graph","key":"10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}],"organization":{"metadata":{"websiteurl":{"value":"http://www.unifi.it/"},"legalname":{"value":"University of Florence"}}},"type":20,"id":"20|microsoft___::adecd59d8ff7f5aaedac013fa0f54ffe"}
{"dateoftransformation":"2018-09-13","originalId":["openaire____::issn20381026::Università degli Studi di Firenze"],"collectedfrom":[{"value":"","key":""}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecresearchorganization":{"value":"false"},"ecenterprise":{"value":"false"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"Università degli Studi di Firenze"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2016-06-02","type":20,"id":"20|openaire____::55a8725b9d9a9a67615018901270de4b"}
{"dateoftransformation":"2018-09-13","originalId":["opendoar____::Università_degli_Studi_di_Firenze"],"collectedfrom":[{"value":"OpenDOAR","key":"10|openaire____::47ce9e9f4fad46e732cff06419ecaabb"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"ecresearchorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecenterprise":{"value":"false"},"websiteurl":{"value":"http://www.unifi.it/"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"Università degli Studi di Firenze"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2015-08-24","type":20,"id":"20|opendoar____::4f194641be797be5e5eb11227e962145"}
{"dateoftransformation":"2018-09-13","originalId":["snsf________::Università_degli_Studi_di_Firenze"],"collectedfrom":[{"value":"SNSF - Swiss National Science Foundation","key":"10|openaire____::d8f3c25e18304608ce8e816e99603d7a"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecresearchorganization":{"value":"false"},"ecenterprise":{"value":"false"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"Università degli Studi di Firenze"},"country":{"classid":"IT","classname":"Italy","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2017-09-23","type":20,"id":"20|snsf________::4f194641be797be5e5eb11227e962145"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIOK"},"websiteurl":{"value":"http://www.niok.eu/"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"NIOK"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.450158.d"}],"type":20,"id":"20|grid________::6183d331a1920dd81b8c10620a8b3a8a"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIVEL"},"websiteurl":{"value":"http://www.nivel.nl/en"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"NIVEL"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.416005.6"}],"type":20,"id":"20|grid________::8f65fd4e764086db897cc648e9cbbaed"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"SCP"},"websiteurl":{"value":"http://www.scp.nl/english/"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Netherlands Institute for Social Research"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.438038.4"}],"type":20,"id":"20|grid________::c69cffc4997b54bb2eb5ca6aebcda18b"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIVEL"},"websiteurl":{"value":"http://www.nivel.nl/en"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Netherlands Institute for Health Services Research"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.416005.6"}],"type":20,"id":"20|grid________::5b72dc608480f3d5569a7bfe3cbdaf07"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"SCP"},"websiteurl":{"value":"http://www.scp.nl/english/"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"SCP"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.438038.4"}],"type":20,"id":"20|grid________::6b7b927a3ae25f1639a6ef27b35021b5"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIOK"},"websiteurl":{"value":"http://www.niok.eu/"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Netherlands Institute for Catalysis Research"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.450158.d"}],"type":20,"id":"20|grid________::6af340f03c44041737859d3e1354d1fe"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIVEL"},"websiteurl":{"value":"http://www.nivel.nl/en"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Nederlands Instituut voor Onderzoek van de Gezondheidszorg"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.416005.6"}],"type":20,"id":"20|grid________::69ab0f5ed7da9d961355cb4eb24b8613"}
{"collectedfrom":[{"value":"GRID - Global Research Identifier Database","key":"10|openaire____::ff4a008470319a22d9cf3d14af485977"}],"organization":{"metadata":{"legalshortname":{"value":"NIOK"},"websiteurl":{"value":"http://www.niok.eu/"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"legalname":{"value":"Nederlands Instituut voor Onderzoek in de Katalyse"}}},"pid":[{"qualifier":{"classid":"grid","classname":"grid","schemename":"dnet:pid_types","schemeid":"dnet:pid_types"},"value":"grid.450158.d"}],"type":20,"id":"20|grid________::267cf3ce23903e0a8403653019ce8187"}
{"dateoftransformation":"2018-11-20","originalId":["corda_______::998294125"],"collectedfrom":[{"value":"CORDA - COmmon Research DAta Warehouse","key":"10|openaire____::b30dac7baac631f3da7c2bb18dd9891f"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"true"},"ecinternationalorganization":{"value":"false"},"legalshortname":{"value":"NIVEL"},"ecresearchorganization":{"value":"true"},"ecnonprofit":{"value":"true"},"ecenterprise":{"value":"false"},"websiteurl":{"value":"http://www.nivel.nl"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"STICHTING NEDERLANDS INSTITUUT VOOR ONDERZOEK VAN DE GEZONDHEIDSZORG"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2018-03-12","type":20,"id":"20|corda_______::755737ed505484ea374062762ef05ef6"}
{"dateoftransformation":"2019-06-26","originalId":["corda__h2020::998294125"],"collectedfrom":[{"value":"CORDA - COmmon Research DAta Warehouse - Horizon 2020","key":"10|openaire____::a55eb91348674d853191f4f4fd73d078"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"true"},"ecinternationalorganization":{"value":"false"},"legalshortname":{"value":"NIVEL"},"ecresearchorganization":{"value":"true"},"ecnonprofit":{"value":"true"},"ecenterprise":{"value":"false"},"websiteurl":{"value":"http://www.nivel.nl"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"STICHTING NEDERLANDS INSTITUUT VOOR ONDERZOEK VAN DE GEZONDHEIDSZORG"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2018-03-12","type":20,"id":"20|corda__h2020::755737ed505484ea374062762ef05ef6"}
{"dateoftransformation":"2018-09-13","originalId":["snsf________::The_Netherlands_Institute_of_Health_Services_Research_NIVEL"],"collectedfrom":[{"value":"SNSF - Swiss National Science Foundation","key":"10|openaire____::d8f3c25e18304608ce8e816e99603d7a"}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecresearchorganization":{"value":"false"},"ecenterprise":{"value":"false"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"The Netherlands Institute of Health Services Research NIVEL"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2017-09-22","type":20,"id":"20|snsf________::10653be4e9c170181486aa9782346d81"}
{"dateoftransformation":"2018-09-13","originalId":["openaire____::088a0087-4bc6-4c38-a052-b446c3b225a7::The Netherlands Institute for Social Research"],"collectedfrom":[{"value":"","key":""}],"organization":{"metadata":{"eclegalbody":{"value":"false"},"eclegalperson":{"value":"false"},"ecinternationalorganization":{"value":"false"},"ecnonprofit":{"value":"false"},"ecresearchorganization":{"value":"false"},"ecenterprise":{"value":"false"},"ecnutscode":{"value":"false"},"ecinternationalorganizationeurinterests":{"value":"false"},"legalname":{"value":"The Netherlands Institute for Social Research"},"country":{"classid":"NL","classname":"Netherlands","schemename":"dnet:countries","schemeid":"dnet:countries"},"echighereducation":{"value":"false"},"ecsmevalidated":{"value":"false"}}},"dateofcollection":"2016-03-30","type":20,"id":"20|openaire____::857b30f258c43852a2cb57875ac40892"}

@ -16,7 +16,7 @@
{ "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} },
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "yearMatch", "fields" : [ "dateofacceptance" ] },
{ "name" : "titleVersionMatch", "fields" : [ "title" ] },
{ "name" : "sizeMatch", "fields" : [ "authors" ] } ,

@ -11,7 +11,7 @@
"includeChildren" : "true"
},
"pace" : {
"conditions" : [ ],
"necessaryConditions" : [ ],
"model" : [
{ "name" : "title", "algo" : "JaroWinkler", "type" : "String", "weight" : "1.0", "ignoreMissing" : "false", "path" : "result/metadata/title[qualifier#classid = {main title}]/value" }
],

@ -18,10 +18,10 @@
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } },
{ "name" : "lowercase", "fields" : [ "doi", "url" ], "params" : { } }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "exactMatch", "fields" : [ "doi", "resulttype", "url" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "titleVersionMatch", "fields" : [ "title" ] }
],
"model" : [

@ -19,11 +19,11 @@
{ "name" : "lowercase", "fields" : [ "doi" ], "params" : { } },
{ "name" : "urlclustering", "fields": [ "url" ], "params" : { } }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "doiExactMatch", "fields": [ "doi" ] },
{ "name" : "exactMatch", "fields" : [ "url", "documentationUrl" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : ["resulttype"] }
],
"model" : [

@ -1,360 +1,360 @@
package eu.dnetlib.pace.distance;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.googlecode.protobuf.format.JsonFormat;
import eu.dnetlib.data.proto.OafProtos;
import eu.dnetlib.pace.AbstractProtoPaceTest;
import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.distance.eval.ScoreResult;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.model.ProtoDocumentBuilder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class DetectorTest extends AbstractProtoPaceTest {
private static final Log log = LogFactory.getLog(DetectorTest.class);
@Test
public void testDistanceResultSimple() {
final Config config = getResultSimpleConf();
final MapDocument resA = result(config, "A", "Recent results from CDF");
final MapDocument resB = result(config, "B", "Recent results from CDF");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d == 1.0);
}
@Test
public void testDistanceResultSimpleMissingDates() {
final Config config = getResultSimpleConf();
final MapDocument resA = result(config, "A", "Recent results from BES");
final MapDocument resB = result(config, "A", "Recent results from CES");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d > 0.97);
}
@Test
public void testDistanceResultInvalidDate() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "title title title 6BESR", "2013-01-05");
final MapDocument resB = result(config, "B", "title title title 6BESR", "qwerty");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d == 1.0);
}
@Ignore
@Test
public void testDistanceResultMissingOneDate() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "title title title 6BESR", null);
final MapDocument resB = result(config, "B", "title title title 6CLER", "2012-02");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue((d > 0.9) && (d < 1.0));
}
@Ignore
@Test
public void testDistanceResult() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "title title title BES", "");
final MapDocument resB = result(config, "B", "title title title CLEO");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue((d > 0.9) && (d < 1.0));
}
@Ignore
@Test
public void testDistanceResultMissingTwoDate() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "bellaciao");
final MapDocument resB = result(config, "B", "bellocioa");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue((d > 0.9) && (d < 1.0));
}
@Ignore
@Test
public void testDistanceOrganizationIgnoreMissing() {
final Config config = getOrganizationSimpleConf();
final MapDocument orgA = organization(config, "A", "CONSIGLIO NAZIONALE DELLE RICERCHE");
final MapDocument orgB = organization(config, "B", "CONSIGLIO NAZIONALE DELLE RICERCHE", "CNR");
final ScoreResult sr = new PaceDocumentDistance().between(orgA, orgB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d > 0.99);
}
@Test
public void testDistanceOrganizations() {
final Config config = getOrganizationTestConf();
final MapDocument orgA = organization(config, "A", "UNIVERSITA DEGLI STUDI DI VERONA");
final MapDocument orgB = organization(config, "B", "UNIVERSITY OF GENOVA");
final ScoreResult sr = new PaceDocumentDistance().between(orgA, orgB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
}
@Test
public void testDistanceResultCase1() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs boson", "2003");
final MapDocument resB = result(config, "B", "Search for the Standard Model Higgs Boson", "2003");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
System.out.println("d = " + d);
assertTrue((d >= 0.9) && (d <= 1.0));
}
@Test
public void testDistanceResultCaseDoiMatch1() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs boson", "2003", "10.1594/PANGAEA.726855");
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", "10.1594/PANGAEA.726855");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("exact DOIs will produce an exact match", d == 1.0);
}
@Test
public void testDistanceResultCaseDoiMatch2() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1594/PANGAEA.726855");
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2005", "10.1594/PANGAEA.726855");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("exact DOIs will produce an exact match, regardless of different titles or publication years", d == 1.0);
}
@Test
public void testDistanceResultCaseDoiMatch3() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2003");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("a missing DOI will casue the comparsion to continue with the following conditions", d == 1.0);
}
@Test
public void testDistanceResultCaseDoiMatch4() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2005");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("a missing DOI, comparsion continues with the following conditions, different publication years will drop the score to 0", d == 0.0);
}
@Test
public void testDistanceResultCaseDoiMatch5() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Search for the Standard Model Higgs Boson", "2003", "10.1016/j.jmb.2010.12.020");
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("a missing DOI, comparsion continues with the following conditions", (d > 0.9) && (d < 1.0));
}
@Test
public void testDistanceResultCaseDoiMatch6() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2003", "anotherDifferentDOI");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("different DOIs will NOT drop the score to 0, then evaluate other fields", d == 1.0);
}
@Test
public void testDistanceResultCaseDoiMatch7() {
final Config config = getResultConf();
final MapDocument resA = result(config, "A", "Adrenal Insufficiency asd asd", "1951", Lists.newArrayList("PMC2037944", "axdsds"));
final MapDocument resB = result(config, "B", "Adrenal Insufficiency", "1951", "PMC2037944");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue("different DOIs will drop the score to 0, regardless of the other fields", d > 0.89 & d < 1);
}
// http://dx.doi.org/10.1594/PANGAEA.726855 doi:10.1594/PANGAEA.726855
@Test
public void testDistanceResultCaseAuthor1() {
final Config config = getResultAuthorsConf();
final List<String> authorsA = Lists.newArrayList("a", "b", "c", "d");
final List<String> authorsB = Lists.newArrayList("a", "b", "c");
final List<String> pid = Lists.newArrayList();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d == 0.0);
}
@Test
public void testDistanceResultCaseAuthor2() {
final Config config = getResultAuthorsConf();
final List<String> authorsA = Lists.newArrayList("a", "b", "c");
final List<String> authorsB = Lists.newArrayList("a", "b", "c");
final List<String> pid = Lists.newArrayList();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue(d == 1.0);
}
@Test
public void testDistanceResultCaseAuthor3() {
final Config config = getResultAuthorsConf();
final List<String> authorsA = Lists.newArrayList("Bardi, A.", "Manghi, P.", "Artini, M.");
final List<String> authorsB = Lists.newArrayList("Bardi Alessia", "Manghi Paolo", "Artini Michele");
final List<String> pid = Lists.newArrayList();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
assertTrue((d > 0.9) && (d < 1.0));
}
@Test
public void testDistanceResultCaseAuthor4() {
final Config config = getResultAuthorsConf();
final List<String> authorsA = Lists.newArrayList("Bardi, Alessia", "Manghi, Paolo", "Artini, Michele", "a");
final List<String> authorsB = Lists.newArrayList("Bardi Alessia", "Manghi Paolo", "Artini Michele");
final List<String> pid = Lists.newArrayList();
final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
// assertTrue(d.getScore() == 0.0);
}
@Test
public void testDistanceResultNoPidsConf() {
final Config config = getResultFullConf();
final MapDocument resA =
result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010");
final MapDocument resB =
result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reportsX", "2010");
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double s = sr.getScore();
log.info(sr.toString());
log.info(String.format(" s ---> %s", s));
// assertTrue(d.getScore() == 0.0);
}
@Test
public void testDistanceResultPidsConf() {
final Config config = getResultFullConf();
final List<String> authorsA = Lists.newArrayList("Nagarajan Pranesh", "Guy Vautier", "Punyanganie de Silva");
final List<String> authorsB = Lists.newArrayList("Pranesh Nagarajan", "Vautier Guy", "de Silva Punyanganie");
final List<String> pidA = Lists.newArrayList("10.1186/1752-1947-4-299", "a", "b");
final MapDocument resA =
result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
pidA, authorsA);
final List<String> pidB = Lists.newArrayList("c", "a", "10.1186/1752-1947-4-299", "d");
final MapDocument resB =
result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reportsX", "2010",
pidB, authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double s = sr.getScore();
log.info(sr.toString());
log.info(String.format(" s ---> %s", s));
// assertTrue(d.getScore() == 0.0);
}
@Test
public void testDistanceResultFullConf() {
final Config config = getResultFullConf();
final List<String> authorsA = Lists.newArrayList("Nagarajan Pranesh", "Guy Vautier", "Punyanganie de Silva");
final List<String> authorsB = Lists.newArrayList("Pranesh Nagarajan", "Vautier Guy", "de Silva Punyanganie");
final MapDocument resA =
result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
"10.1186/1752-1947-4-299", authorsA);
final MapDocument resB =
result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
"10.1186/1752-1947-4-299", authorsB);
final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
final double d = sr.getScore();
log.info(String.format(" d ---> %s", d));
// assertTrue(d.getScore() == 0.0);
}
@Ignore
@Test
public void testDistance() throws IOException {
final DedupConfig conf = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/result.prod.pace.json"));
final MapDocument crossref = asMapDocument(conf, "/eu/dnetlib/pace/crossref.json");
final MapDocument alicante = asMapDocument(conf, "/eu/dnetlib/pace/alicante.json");
final ScoreResult result = new PaceDocumentDistance().between(crossref, alicante, conf);
log.info("score = " + result);
}
@Ignore
@Test
public void testDistanceOrgs() throws IOException {
final DedupConfig conf = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf"));
final MapDocument orgA = asMapDocument(conf, readFromClasspath("/eu/dnetlib/pace/organization1.json"));
final MapDocument orgB = asMapDocument(conf, readFromClasspath("/eu/dnetlib/pace/organization2.json"));
Set<String> keysA = getGroupingKeys(conf, orgA);
Set<String> keysB = getGroupingKeys(conf, orgB);
assertFalse(String.format("A: %s\nB: %s", keysA, keysB), Sets.intersection(keysA, keysB).isEmpty());
log.info("clustering keys A = " + getGroupingKeys(conf, orgA));
log.info("clustering keys B = " + getGroupingKeys(conf, orgB));
final ScoreResult result = new PaceDocumentDistance().between(orgA, orgB, conf);
log.info("score = " + result);
log.info("distance = " + result.getScore());
}
private Set<String> getGroupingKeys(DedupConfig conf, MapDocument doc) {
return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf));
}
private MapDocument asMapDocument(DedupConfig conf, final String json) {
OafProtos.OafEntity.Builder b = OafProtos.OafEntity.newBuilder();
try {
JsonFormat.merge(json, b);
} catch (JsonFormat.ParseException e) {
throw new IllegalArgumentException(e);
}
return ProtoDocumentBuilder.newInstance(b.getId(), b.build(), conf.getPace().getModel());
}
}
//package eu.dnetlib.pace.distance;
//
//import com.google.common.collect.Lists;
//import com.google.common.collect.Sets;
//import com.googlecode.protobuf.format.JsonFormat;
//import eu.dnetlib.data.proto.OafProtos;
//import eu.dnetlib.pace.AbstractProtoPaceTest;
//import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner;
//import eu.dnetlib.pace.config.Config;
//import eu.dnetlib.pace.config.DedupConfig;
//import eu.dnetlib.pace.distance.eval.ScoreResult;
//import eu.dnetlib.pace.model.MapDocument;
//import eu.dnetlib.pace.model.ProtoDocumentBuilder;
//import org.apache.commons.logging.Log;
//import org.apache.commons.logging.LogFactory;
//import org.junit.Ignore;
//import org.junit.Test;
//
//import java.io.IOException;
//import java.util.List;
//import java.util.Set;
//
//import static org.junit.Assert.assertFalse;
//import static org.junit.Assert.assertTrue;
//
//public class DetectorTest extends AbstractProtoPaceTest {
//
// private static final Log log = LogFactory.getLog(DetectorTest.class);
//
// @Test
// public void testDistanceResultSimple() {
// final Config config = getResultSimpleConf();
// final MapDocument resA = result(config, "A", "Recent results from CDF");
// final MapDocument resB = result(config, "B", "Recent results from CDF");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d == 1.0);
// }
//
// @Test
// public void testDistanceResultSimpleMissingDates() {
// final Config config = getResultSimpleConf();
// final MapDocument resA = result(config, "A", "Recent results from BES");
// final MapDocument resB = result(config, "A", "Recent results from CES");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d > 0.97);
// }
//
// @Test
// public void testDistanceResultInvalidDate() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "title title title 6BESR", "2013-01-05");
// final MapDocument resB = result(config, "B", "title title title 6BESR", "qwerty");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d == 1.0);
// }
//
// @Ignore
// @Test
// public void testDistanceResultMissingOneDate() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "title title title 6BESR", null);
// final MapDocument resB = result(config, "B", "title title title 6CLER", "2012-02");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue((d > 0.9) && (d < 1.0));
// }
//
// @Ignore
// @Test
// public void testDistanceResult() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "title title title BES", "");
// final MapDocument resB = result(config, "B", "title title title CLEO");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue((d > 0.9) && (d < 1.0));
// }
//
// @Ignore
// @Test
// public void testDistanceResultMissingTwoDate() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "bellaciao");
// final MapDocument resB = result(config, "B", "bellocioa");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue((d > 0.9) && (d < 1.0));
// }
//
// @Ignore
// @Test
// public void testDistanceOrganizationIgnoreMissing() {
// final Config config = getOrganizationSimpleConf();
// final MapDocument orgA = organization(config, "A", "CONSIGLIO NAZIONALE DELLE RICERCHE");
// final MapDocument orgB = organization(config, "B", "CONSIGLIO NAZIONALE DELLE RICERCHE", "CNR");
// final ScoreResult sr = new PaceDocumentDistance().between(orgA, orgB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d > 0.99);
// }
//
//
// @Test
// public void testDistanceOrganizations() {
// final Config config = getOrganizationTestConf();
// final MapDocument orgA = organization(config, "A", "UNIVERSITA DEGLI STUDI DI VERONA");
// final MapDocument orgB = organization(config, "B", "UNIVERSITY OF GENOVA");
// final ScoreResult sr = new PaceDocumentDistance().between(orgA, orgB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// }
//
//
// @Test
// public void testDistanceResultCase1() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs boson", "2003");
// final MapDocument resB = result(config, "B", "Search for the Standard Model Higgs Boson", "2003");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// System.out.println("d = " + d);
// assertTrue((d >= 0.9) && (d <= 1.0));
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch1() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs boson", "2003", "10.1594/PANGAEA.726855");
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", "10.1594/PANGAEA.726855");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("exact DOIs will produce an exact match", d == 1.0);
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch2() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1594/PANGAEA.726855");
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2005", "10.1594/PANGAEA.726855");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("exact DOIs will produce an exact match, regardless of different titles or publication years", d == 1.0);
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch3() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
// final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2003");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("a missing DOI will casue the comparsion to continue with the following necessaryConditions", d == 1.0);
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch4() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
// final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2005");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("a missing DOI, comparsion continues with the following necessaryConditions, different publication years will drop the score to 0", d == 0.0);
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch5() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Search for the Standard Model Higgs Boson", "2003", "10.1016/j.jmb.2010.12.020");
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("a missing DOI, comparsion continues with the following necessaryConditions", (d > 0.9) && (d < 1.0));
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch6() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Conference proceedings on X. Appendix", "2003", "10.1016/j.jmb.2010.12.024");
// final MapDocument resB = result(config, "B", "Conference proceedings on X. Appendix", "2003", "anotherDifferentDOI");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("different DOIs will NOT drop the score to 0, then evaluate other fields", d == 1.0);
// }
//
// @Test
// public void testDistanceResultCaseDoiMatch7() {
// final Config config = getResultConf();
// final MapDocument resA = result(config, "A", "Adrenal Insufficiency asd asd", "1951", Lists.newArrayList("PMC2037944", "axdsds"));
// final MapDocument resB = result(config, "B", "Adrenal Insufficiency", "1951", "PMC2037944");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue("different DOIs will drop the score to 0, regardless of the other fields", d > 0.89 & d < 1);
// }
//
// // http://dx.doi.org/10.1594/PANGAEA.726855 doi:10.1594/PANGAEA.726855
// @Test
// public void testDistanceResultCaseAuthor1() {
// final Config config = getResultAuthorsConf();
// final List<String> authorsA = Lists.newArrayList("a", "b", "c", "d");
// final List<String> authorsB = Lists.newArrayList("a", "b", "c");
// final List<String> pid = Lists.newArrayList();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d == 0.0);
// }
//
// @Test
// public void testDistanceResultCaseAuthor2() {
// final Config config = getResultAuthorsConf();
// final List<String> authorsA = Lists.newArrayList("a", "b", "c");
// final List<String> authorsB = Lists.newArrayList("a", "b", "c");
// final List<String> pid = Lists.newArrayList();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue(d == 1.0);
// }
//
// @Test
// public void testDistanceResultCaseAuthor3() {
// final Config config = getResultAuthorsConf();
// final List<String> authorsA = Lists.newArrayList("Bardi, A.", "Manghi, P.", "Artini, M.");
// final List<String> authorsB = Lists.newArrayList("Bardi Alessia", "Manghi Paolo", "Artini Michele");
// final List<String> pid = Lists.newArrayList();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// assertTrue((d > 0.9) && (d < 1.0));
// }
//
// @Test
// public void testDistanceResultCaseAuthor4() {
// final Config config = getResultAuthorsConf();
// final List<String> authorsA = Lists.newArrayList("Bardi, Alessia", "Manghi, Paolo", "Artini, Michele", "a");
// final List<String> authorsB = Lists.newArrayList("Bardi Alessia", "Manghi Paolo", "Artini Michele");
// final List<String> pid = Lists.newArrayList();
// final MapDocument resA = result(config, "A", "Search the Standard Model Higgs Boson", "2003", pid, authorsA);
// final MapDocument resB = result(config, "B", "Search the Standard Model Higgs Boson", "2003", pid, authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// // assertTrue(d.getScore() == 0.0);
// }
//
// @Test
// public void testDistanceResultNoPidsConf() {
// final Config config = getResultFullConf();
// final MapDocument resA =
// result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010");
// final MapDocument resB =
// result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reportsX", "2010");
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double s = sr.getScore();
// log.info(sr.toString());
// log.info(String.format(" s ---> %s", s));
// // assertTrue(d.getScore() == 0.0);
// }
//
// @Test
// public void testDistanceResultPidsConf() {
// final Config config = getResultFullConf();
// final List<String> authorsA = Lists.newArrayList("Nagarajan Pranesh", "Guy Vautier", "Punyanganie de Silva");
// final List<String> authorsB = Lists.newArrayList("Pranesh Nagarajan", "Vautier Guy", "de Silva Punyanganie");
// final List<String> pidA = Lists.newArrayList("10.1186/1752-1947-4-299", "a", "b");
// final MapDocument resA =
// result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
// pidA, authorsA);
// final List<String> pidB = Lists.newArrayList("c", "a", "10.1186/1752-1947-4-299", "d");
// final MapDocument resB =
// result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reportsX", "2010",
// pidB, authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double s = sr.getScore();
// log.info(sr.toString());
// log.info(String.format(" s ---> %s", s));
// // assertTrue(d.getScore() == 0.0);
// }
//
// @Test
// public void testDistanceResultFullConf() {
// final Config config = getResultFullConf();
// final List<String> authorsA = Lists.newArrayList("Nagarajan Pranesh", "Guy Vautier", "Punyanganie de Silva");
// final List<String> authorsB = Lists.newArrayList("Pranesh Nagarajan", "Vautier Guy", "de Silva Punyanganie");
// final MapDocument resA =
// result(config, "A", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
// "10.1186/1752-1947-4-299", authorsA);
// final MapDocument resB =
// result(config, "B", "Presentations of perforated colonic pathology in patients with polymyalgia rheumatica: two case reports", "2010",
// "10.1186/1752-1947-4-299", authorsB);
// final ScoreResult sr = new PaceDocumentDistance().between(resA, resB, config);
// final double d = sr.getScore();
// log.info(String.format(" d ---> %s", d));
// // assertTrue(d.getScore() == 0.0);
// }
//
// @Ignore
// @Test
// public void testDistance() throws IOException {
// final DedupConfig conf = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/result.prod.pace.json"));
// final MapDocument crossref = asMapDocument(conf, "/eu/dnetlib/pace/crossref.json");
// final MapDocument alicante = asMapDocument(conf, "/eu/dnetlib/pace/alicante.json");
// final ScoreResult result = new PaceDocumentDistance().between(crossref, alicante, conf);
// log.info("score = " + result);
// }
//
// @Ignore
// @Test
// public void testDistanceOrgs() throws IOException {
// final DedupConfig conf = DedupConfig.load(readFromClasspath("/eu/dnetlib/pace/organization.pace.conf"));
// final MapDocument orgA = asMapDocument(conf, readFromClasspath("/eu/dnetlib/pace/organization1.json"));
// final MapDocument orgB = asMapDocument(conf, readFromClasspath("/eu/dnetlib/pace/organization2.json"));
// Set<String> keysA = getGroupingKeys(conf, orgA);
// Set<String> keysB = getGroupingKeys(conf, orgB);
// assertFalse(String.format("A: %s\nB: %s", keysA, keysB), Sets.intersection(keysA, keysB).isEmpty());
// log.info("clustering keys A = " + getGroupingKeys(conf, orgA));
// log.info("clustering keys B = " + getGroupingKeys(conf, orgB));
// final ScoreResult result = new PaceDocumentDistance().between(orgA, orgB, conf);
// log.info("score = " + result);
// log.info("compare = " + result.getScore());
// }
//
// private Set<String> getGroupingKeys(DedupConfig conf, MapDocument doc) {
// return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf));
// }
//
// private MapDocument asMapDocument(DedupConfig conf, final String json) {
// OafProtos.OafEntity.Builder b = OafProtos.OafEntity.newBuilder();
// try {
// JsonFormat.merge(json, b);
// } catch (JsonFormat.ParseException e) {
// throw new IllegalArgumentException(e);
// }
// return ProtoDocumentBuilder.newInstance(b.getId(), b.build(), conf.getPace().getModel());
//
// }
//}

@ -1390,7 +1390,7 @@ public class DiffPatchMatch {
}
/**
* Compute the Levenshtein distance; the number of inserted, deleted or
* Compute the Levenshtein compare; the number of inserted, deleted or
* substituted characters.
* @param diffs List of Diff objects.
* @return Number of changes.
@ -1655,7 +1655,7 @@ public class DiffPatchMatch {
score_threshold = score;
best_loc = j - 1;
if (best_loc > loc) {
// When passing loc, don't exceed our current distance from loc.
// When passing loc, don't exceed our current compare from loc.
start = Math.max(1, 2 * loc - best_loc);
} else {
// Already passed loc, downhill from here on in.

@ -14,7 +14,7 @@
"clustering": [
{"name": "personClustering", "fields": ["fullname"], "params": {}}
],
"conditions": [],
"necessaryConditions": [],
"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"},

@ -18,7 +18,7 @@
{ "name" : "spacetrimmingfieldvalue", "fields" : [ "legalshortname" ], "params" : { "randomLength" : "5" } },
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] },
{ "name" : "mustBeDifferent", "fields" : [ "gridid" ] }
],

@ -15,7 +15,7 @@
{ "name" : "ngrampairs", "fields" : [ "legalname" ], "params" : { "max" : "1", "ngramLen" : "3"} },
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : "1", "len" : "3" } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] }
],
"model" : [

@ -11,7 +11,7 @@
"includeChildren" : "true"
},
"pace" : {
"conditions" : [
"necessaryConditions" : [
{ "name" : "sizeMatch", "fields" : [ "authors" ] },
{ "name" : "titleVersionMatch", "fields" : [ "title" ] }
],

@ -16,7 +16,7 @@
{ "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} },
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "yearMatch", "fields" : [ "dateofacceptance" ] },
{ "name" : "titleVersionMatch", "fields" : [ "title" ] },
{ "name" : "sizeMatch", "fields" : [ "authors" ] } ,

@ -11,10 +11,10 @@
"includeChildren" : "true"
},
"pace" : {
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "pidMatch", "fields" : [ "pid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "yearMatch", "fields" : [ "dateofacceptance" ] },
{ "name" : "titleVersionMatch", "fields" : [ "title" ] }
],

@ -17,10 +17,10 @@
{ "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } },
{ "name" : "lowercase", "fields" : [ "doi" ], "params" : { } }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "pidMatch", "fields" : [ "pid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "titleVersionMatch", "fields" : [ "title" ] },
{ "name" : "sizeMatch", "fields" : [ "authors" ] }
],

@ -11,7 +11,7 @@
"includeChildren" : "true"
},
"pace" : {
"conditions" : [ ],
"necessaryConditions" : [ ],
"model" : [
{ "name" : "title", "algo" : "JaroWinkler", "type" : "String", "weight" : "1.0", "ignoreMissing" : "false", "path" : "result/metadata/title[qualifier#classid = {main title}]/value" }
],

@ -9,7 +9,7 @@ import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.FieldDef;
/**
* Abstract conditions needs a list of field names.
* Abstract necessaryConditions needs a list of field names.
*
* @author claudio
*

@ -6,7 +6,7 @@ import eu.dnetlib.pace.model.Document;
import eu.dnetlib.pace.model.FieldDef;
/**
* Allows to express general conditions to be satisfied or not between two Documents.
* Allows to express general necessaryConditions to be satisfied or not between two Documents.
*
* @author claudio
*/

@ -6,6 +6,7 @@ import java.util.Map;
import eu.dnetlib.pace.condition.ConditionAlgo;
import eu.dnetlib.pace.model.ClusteringDef;
import eu.dnetlib.pace.model.FieldDef;
import eu.dnetlib.pace.tree.support.TreeNodeDef;
/**
* Interface for PACE configuration bean.
@ -21,6 +22,9 @@ public interface Config {
*/
public List<FieldDef> model();
public Map<String, TreeNodeDef> decisionTree();
/**
* Field configuration definitions.
*
@ -31,16 +35,16 @@ public interface Config {
/**
* Strict Pre-Condition definitions.
*
* @return the list of conditions
* @return the list of necessaryConditions
*/
public List<ConditionAlgo> strictConditions();
public List<ConditionAlgo> sufficientConditions();
/**
* Pre-Condition definitions.
*
* @return the list of conditions
* @return the list of necessaryConditions
*/
public List<ConditionAlgo> conditions();
public List<ConditionAlgo> necessaryConditions();
/**
* Clusterings.

@ -8,6 +8,7 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.function.BiFunction;
import eu.dnetlib.pace.tree.support.TreeNodeDef;
import eu.dnetlib.pace.util.PaceException;
import org.antlr.stringtemplate.StringTemplate;
import org.apache.commons.io.IOUtils;
@ -114,6 +115,11 @@ public class DedupConfig implements Config, Serializable {
}
}
@Override
public Map<String, TreeNodeDef> decisionTree(){
return getPace().getDecisionTree();
}
@Override
public List<FieldDef> model() {
return getPace().getModel();
@ -125,12 +131,12 @@ public class DedupConfig implements Config, Serializable {
}
@Override
public List<ConditionAlgo> strictConditions() {
public List<ConditionAlgo> sufficientConditions() {
return getPace().getStrictConditionAlgos();
}
@Override
public List<ConditionAlgo> conditions() {
public List<ConditionAlgo> necessaryConditions() {
return getPace().getConditionAlgos();
}

@ -6,6 +6,7 @@ import eu.dnetlib.pace.condition.ConditionAlgo;
import eu.dnetlib.pace.model.ClusteringDef;
import eu.dnetlib.pace.model.CondDef;
import eu.dnetlib.pace.model.FieldDef;
import eu.dnetlib.pace.tree.support.TreeNodeDef;
import eu.dnetlib.pace.util.PaceResolver;
import org.apache.commons.collections.CollectionUtils;
import org.codehaus.jackson.annotate.JsonIgnore;
@ -18,9 +19,12 @@ import java.util.stream.Collectors;
public class PaceConfig implements Serializable {
private List<FieldDef> model;
private List<CondDef> strictConditions;
private List<CondDef> conditions;
private List<CondDef> sufficientConditions;
private List<CondDef> necessaryConditions;
private List<ClusteringDef> clustering;
private Map<String, TreeNodeDef> decisionTree;
private Map<String, List<String>> blacklists;
@JsonIgnore
@ -46,30 +50,30 @@ public class PaceConfig implements Serializable {
this.model = model;
}
public List<CondDef> getStrictConditions() {
return strictConditions;
public List<CondDef> getSufficientConditions() {
return sufficientConditions;
}
public void setStrictConditions(final List<CondDef> strictConditions) {
this.strictConditions = strictConditions;
public void setSufficientConditions(final List<CondDef> sufficientConditions) {
this.sufficientConditions = sufficientConditions;
}
public List<CondDef> getConditions() {
return conditions;
public List<CondDef> getNecessaryConditions() {
return necessaryConditions;
}
@JsonIgnore
public List<ConditionAlgo> getConditionAlgos() {
return asConditionAlgos(getConditions());
return asConditionAlgos(getNecessaryConditions());
}
@JsonIgnore
public List<ConditionAlgo> getStrictConditionAlgos() {
return asConditionAlgos(getStrictConditions());
return asConditionAlgos(getSufficientConditions());
}
public void setConditions(final List<CondDef> conditions) {
this.conditions = conditions;
public void setNecessaryConditions(final List<CondDef> necessaryConditions) {
this.necessaryConditions = necessaryConditions;
}
public List<ClusteringDef> getClustering() {
@ -80,6 +84,14 @@ public class PaceConfig implements Serializable {
this.clustering = clustering;
}
public Map<String, TreeNodeDef> getDecisionTree() {
return decisionTree;
}
public void setDecisionTree(Map<String, TreeNodeDef> decisionTree) {
this.decisionTree = decisionTree;
}
public Map<String, List<String>> getBlacklists() {
return blacklists;
}

@ -1,15 +1,15 @@
package eu.dnetlib.pace.distance;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.distance.eval.ScoreResult;
import eu.dnetlib.pace.model.Document;
public abstract class AbstractDistance<A> implements Distance<A> {
protected abstract Document toDocument(A a);
@Override
public ScoreResult between(final A a, final A b, final Config config) {
return new DistanceScorer(config).distance(toDocument(a), toDocument(b));
}
}
//package eu.dnetlib.pace.distance;
//
//import eu.dnetlib.pace.config.Config;
//import eu.dnetlib.pace.distance.eval.ScoreResult;
//import eu.dnetlib.pace.model.Document;
//
//public abstract class AbstractDistance<A> implements Distance<A> {
//
// protected abstract Document toDocument(A a);
//
// @Override
// public boolean between(final A a, final A b, final Config config) {
// return new PairwiseComparison(config).compare(toDocument(a), toDocument(b));
// }
//}

@ -1,9 +1,8 @@
package eu.dnetlib.pace.distance;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.distance.eval.ScoreResult;
public interface Distance<A> {
public ScoreResult between(A a, A b, Config config);
public boolean between(A a, A b, Config config);
}

@ -5,7 +5,7 @@ import eu.dnetlib.pace.model.Field;
import java.util.Map;
/**
* Each field is configured with a distance algo which knows how to compute the distance (0-1) between the fields of two
* Each field is configured with a compare algo which knows how to compute the compare (0-1) between the fields of two
* objects.
*/
public interface DistanceAlgo {

@ -1,126 +0,0 @@
package eu.dnetlib.pace.distance;
import eu.dnetlib.pace.condition.ConditionAlgo;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.distance.eval.ConditionEvalMap;
import eu.dnetlib.pace.distance.eval.DistanceEval;
import eu.dnetlib.pace.distance.eval.DistanceEvalMap;
import eu.dnetlib.pace.distance.eval.ScoreResult;
import eu.dnetlib.pace.model.*;
import eu.dnetlib.pace.util.PaceException;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;
/**
* The distance between two documents is given by the weighted mean of the field distances
*/
public class DistanceScorer {
private static final Log log = LogFactory.getLog(DistanceScorer.class);
private Config config;
public DistanceScorer(final Config config) {
this.config = config;
}
public ScoreResult distance(final Document a, final Document b) {
final ScoreResult sr = new ScoreResult(); //to keep track of the result of the comparison
sr.setStrictConditions(verify(a, b, config.strictConditions()));
sr.setConditions(verify(a, b, config.conditions()));
final DistanceEvalMap dMap = new DistanceEvalMap(sumWeights(config.model()));
for (final FieldDef fd : config.model()) {
dMap.updateDistance(fieldDistance(a, b, fd));
}
sr.setDistances(dMap);
return sr;
}
private ConditionEvalMap verify(final Document a, final Document b, final List<ConditionAlgo> conditions) {
final ConditionEvalMap res = new ConditionEvalMap();
for (final ConditionAlgo cd : conditions) {
final ConditionEvalMap map = cd.verify(a, b);
res.mergeFrom(map);
// commented out shortcuts
/*
if (map.anyNegative()) {
return res;
}
*/
//if (strict && (res < 0)) return -1;
//cond += verify;
}
return res;
}
private DistanceEval fieldDistance(final Document a, final Document b, final FieldDef fd) {
final double w = fd.getWeight();
final Field va = getValue(a, fd);
final Field vb = getValue(b, fd);
final DistanceEval de = new DistanceEval(fd, va, vb);
if ((w == 0)) return de; // optimization for 0 weight
else {
if (va.isEmpty() || vb.isEmpty()) {
if (fd.isIgnoreMissing()) {
de.setDistance(-1);
} else {
de.setDistance(w);
}
} else {
if (va.getType().equals(vb.getType())) {
de.setDistance(w * fd.distanceAlgo().distance(va, vb));
} else {
throw new PaceException(String.format("Types are different: %s:%s - %s:%s", va, va.getType(), vb, vb.getType()));
}
}
return de;
}
}
private Field getValue(final Document d, final FieldDef fd) {
final Field v = d.values(fd.getName());
if (fd.getLength() > 0) {
if (v instanceof FieldValueImpl) {
((FieldValueImpl) v).setValue(StringUtils.substring(v.stringValue(), 0, fd.getLength()));
} else if (v instanceof FieldListImpl) {
List<String> strings = ((FieldListImpl) v).stringList();
strings = strings.stream()
.limit(fd.getSize() > 0 ? fd.getSize() : strings.size())
.map(s -> StringUtils.substring(s, 0, fd.getLength()))
.collect(Collectors.toList());
((FieldListImpl) v).clear();
((FieldListImpl) v).addAll(strings.stream()
.limit(fd.getSize() > 0 ? fd.getSize() : strings.size())
.map(s -> StringUtils.substring(s, 0, fd.getLength()))
.map(s -> new FieldValueImpl(v.getType(), v.getName(), s))
.collect(Collectors.toList()));
}
}
return v;
}
private double sumWeights(final Collection<FieldDef> fields) {
double sum = 0.0;
for (final FieldDef fd : fields) {
sum += fd.getWeight();
}
return sum;
}
}

@ -1,12 +1,12 @@
package eu.dnetlib.pace.distance;
import eu.dnetlib.pace.model.Document;
public class PaceDocumentDistance extends AbstractDistance<Document> {
@Override
protected Document toDocument(Document a) {
return a;
}
}
//package eu.dnetlib.pace.distance;
//
//import eu.dnetlib.pace.model.Document;
//
//public class PaceDocumentDistance extends AbstractDistance<Document> {
//
// @Override
// protected Document toDocument(Document a) {
// return a;
// }
//
//}

@ -0,0 +1,125 @@
package eu.dnetlib.pace.distance;
import eu.dnetlib.pace.condition.ConditionAlgo;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.distance.eval.ConditionEvalMap;
import eu.dnetlib.pace.model.*;
import eu.dnetlib.pace.tree.support.MatchType;
import eu.dnetlib.pace.tree.support.TreeNodeDef;
import eu.dnetlib.pace.util.PaceException;
import eu.dnetlib.pace.util.Reporter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.List;
import java.util.Map;
/**
* The compare between two documents is given by the weighted mean of the field distances
*/
public class PairwiseComparison {
private static final Log log = LogFactory.getLog(PairwiseComparison.class);
private Config config;
public PairwiseComparison(final Config config) {
this.config = config;
}
public boolean compare(final MapDocument a, final MapDocument b) {
//verify sufficientConditions
if (verify(a, b, config.sufficientConditions()).result() > 0)
return true;
//verify necessaryConditions
if (verify(a, b, config.necessaryConditions()).result() < 0)
return false;
//evaluate the decision tree
return evaluateTree(a, b, config.decisionTree()) == MatchType.MATCH;
}
private ConditionEvalMap verify(final Document a, final Document b, final List<ConditionAlgo> conditions) {
final ConditionEvalMap res = new ConditionEvalMap();
for (final ConditionAlgo cd : conditions) {
final ConditionEvalMap map = cd.verify(a, b);
res.mergeFrom(map);
// commented out shortcuts
/*
if (map.anyNegative()) {
return res;
}
*/
//if (strict && (res < 0)) return -1;
//cond += verify;
}
return res;
}
public MatchType evaluateTree(final MapDocument doc1, final MapDocument doc2, final Map<String, TreeNodeDef> decisionTree){
String current = "start";
double similarity;
while (MatchType.parse(current)==MatchType.UNDEFINED) {
TreeNodeDef currentNode = decisionTree.get(current);
//throw an exception if the node doesn't exist
if (currentNode == null)
throw new PaceException("The Tree Node doesn't exist: " + current);
similarity = currentNode.evaluate(doc1, doc2);
if (similarity == -1) {
current = currentNode.getUndefined();
}
else if (similarity>=currentNode.getThreshold()){
current = currentNode.getPositive();
}
else {
current = currentNode.getNegative();
}
}
return MatchType.parse(current);
}
// private Field getValue(final Document d, final FieldDef fd) {
// final Field v = d.values(fd.getName());
// if (fd.getLength() > 0) {
//
// if (v instanceof FieldValueImpl) {
// ((FieldValueImpl) v).setValue(StringUtils.substring(v.stringValue(), 0, fd.getLength()));
// } else if (v instanceof FieldListImpl) {
// List<String> strings = ((FieldListImpl) v).stringList();
// strings = strings.stream()
// .limit(fd.getSize() > 0 ? fd.getSize() : strings.size())
// .map(s -> StringUtils.substring(s, 0, fd.getLength()))
// .collect(Collectors.toList());
// ((FieldListImpl) v).clear();
// ((FieldListImpl) v).addAll(strings.stream()
// .limit(fd.getSize() > 0 ? fd.getSize() : strings.size())
// .map(s -> StringUtils.substring(s, 0, fd.getLength()))
// .map(s -> new FieldValueImpl(v.getType(), v.getName(), s))
// .collect(Collectors.toList()));
// }
// }
//
// return v;
// }
//
// private double sumWeights(final Collection<FieldDef> fields) {
// double sum = 0.0;
// for (final FieldDef fd : fields) {
// sum += fd.getWeight();
// }
// return sum;
// }
}

@ -12,7 +12,7 @@ import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.FieldList;
/**
* For the rest of the fields delegate the distance measure to the second string library.
* For the rest of the fields delegate the compare measure to the second string library.
*/
public abstract class SecondStringDistanceAlgo extends AbstractPaceFunctions implements DistanceAlgo {
@ -35,7 +35,7 @@ public abstract class SecondStringDistanceAlgo extends AbstractPaceFunctions imp
}
/**
* Instantiates a new second string distance algo.
* Instantiates a new second string compare algo.
*
* @param weight
* the weight
@ -90,7 +90,7 @@ public abstract class SecondStringDistanceAlgo extends AbstractPaceFunctions imp
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.DistanceAlgo#distance(eu.dnetlib.pace.model.Field, eu.dnetlib.pace.model.Field)
* @see eu.dnetlib.pace.compare.DistanceAlgo#compare(eu.dnetlib.pace.model.Field, eu.dnetlib.pace.model.Field)
*/
@Override
public double distance(final Field a, final Field b) {

@ -2,7 +2,6 @@ package eu.dnetlib.pace.distance.algo;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.DistanceScorer;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;

@ -7,7 +7,7 @@ import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import java.util.Map;
/**
* Compared distance between two titles, ignoring version numbers. Suitable for Software entities.
* Compared compare between two titles, ignoring version numbers. Suitable for Software entities.
*/
@DistanceClass("LevensteinTitleIgnoreVersion")
public class LevensteinTitleIgnoreVersion extends SecondStringDistanceAlgo {

@ -7,7 +7,7 @@ import eu.dnetlib.pace.model.Field;
import java.util.Map;
/**
* Not all fields of a document need to partecipate in the distance measure. We model those fields as having a
* Not all fields of a document need to partecipate in the compare measure. We model those fields as having a
* NullDistanceAlgo.
*/
@DistanceClass("Null")

@ -40,7 +40,7 @@ public class SortedJaroWinkler extends SortedSecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.DistanceAlgo#getWeight()
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
@ -50,7 +50,7 @@ public class SortedJaroWinkler extends SortedSecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.SecondStringDistanceAlgo#normalize(double)
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {

@ -40,7 +40,7 @@ public class SortedLevel2JaroWinkler extends SortedSecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.DistanceAlgo#getWeight()
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
@ -50,7 +50,7 @@ public class SortedLevel2JaroWinkler extends SortedSecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.SecondStringDistanceAlgo#normalize(double)
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {

@ -12,12 +12,12 @@ import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.FieldList;
/**
* For the rest of the fields delegate the distance measure to the second string library.
* For the rest of the fields delegate the compare measure to the second string library.
*/
public abstract class SortedSecondStringDistanceAlgo extends SecondStringDistanceAlgo {
/**
* Instantiates a new sorted second string distance algo.
* Instantiates a new sorted second string compare algo.
*
* @param weight
* the weight
@ -35,7 +35,7 @@ public abstract class SortedSecondStringDistanceAlgo extends SecondStringDistanc
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.SecondStringDistanceAlgo#toList(eu.dnetlib.pace.model.Field)
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#toList(eu.dnetlib.pace.model.Field)
*/
@Override
protected List<String> toList(final Field list) {

@ -66,7 +66,7 @@ public class SubStringLevenstein extends SecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.SecondStringDistanceAlgo#distance(eu.dnetlib.pace.model.Field, eu.dnetlib.pace.model.Field)
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#compare(eu.dnetlib.pace.model.Field, eu.dnetlib.pace.model.Field)
*/
@Override
public double distance(final Field a, final Field b) {
@ -79,7 +79,7 @@ public class SubStringLevenstein extends SecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.DistanceAlgo#getWeight()
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
@ -89,7 +89,7 @@ public class SubStringLevenstein extends SecondStringDistanceAlgo {
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.distance.SecondStringDistanceAlgo#normalize(double)
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {

@ -1,32 +0,0 @@
package eu.dnetlib.pace.distance.eval;
import java.util.HashMap;
/**
* Created by claudio on 10/03/16.
*/
public class DistanceEvalMap extends HashMap<String, DistanceEval> {
private double sumWeights;
private double sumDistances = 0.0;
public DistanceEvalMap(final double sumWeights) {
this.sumWeights = sumWeights;
}
public void updateDistance(final DistanceEval d) {
put(d.getFieldDef().getName(), d);
if (d.getDistance() >= 0) {
sumDistances += d.getDistance();
} else {
sumWeights -= d.getFieldDef().getWeight();
}
}
public double distance() {
return sumWeights == 0 ? 0 : sumDistances / sumWeights;
}
}

@ -1,62 +0,0 @@
package eu.dnetlib.pace.distance.eval;
import eu.dnetlib.pace.util.PaceException;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
/**
* Created by claudio on 09/03/16.
*/
public class ScoreResult implements Serializable {
private ConditionEvalMap strictConditions;
private ConditionEvalMap conditions;
private DistanceEvalMap distances;
public double getScore() {
if (getStrictConditions().result() > 0) return 1.0;
// if (getStrictConditions().result() < 0) return 0.0;
if (getConditions().result() < 0) return 0.0;
return getDistances().distance();
}
public ConditionEvalMap getStrictConditions() {
return strictConditions;
}
public void setStrictConditions(final ConditionEvalMap strictConditions) {
this.strictConditions = strictConditions;
}
public ConditionEvalMap getConditions() {
return conditions;
}
public void setConditions(final ConditionEvalMap conditions) {
this.conditions = conditions;
}
public DistanceEvalMap getDistances() {
return distances;
}
public void setDistances(final DistanceEvalMap distances) {
this.distances = distances;
}
@Override
public String toString() {
try {
return new ObjectMapper().writeValueAsString(this);
} catch (IOException e) {
throw new PaceException("unable to serialise " + this.getClass().getName(), e);
}
}
}

@ -14,25 +14,29 @@ import java.util.List;
import java.util.Map;
/**
* The schema is composed by field definitions (FieldDef). Each field has a type, a name, and an associated distance algorithm.
* The schema is composed by field definitions (FieldDef). Each field has a type, a name, and an associated compare algorithm.
*/
public class FieldDef implements Serializable {
public final static String PATH_SEPARATOR = "/";
private String algo;
private String name;
private String path;
private Type type;
private boolean ignoreMissing;
private Type type;
public boolean isIgnoreMissing() {
return ignoreMissing;
}
private boolean overrideMatch;
public void setIgnoreMissing(boolean ignoreMissing) {
this.ignoreMissing = ignoreMissing;
}
private double weight;
private boolean overrideMatch;
/**
* Sets maximum size for the repeatable fields in the model. -1 for unbounded size.
@ -74,20 +78,6 @@ public class FieldDef implements Serializable {
return Lists.newArrayList(Splitter.on(PATH_SEPARATOR).split(getPath()));
}
public DistanceAlgo distanceAlgo() {
if (params == null) {
params = new HashMap<>();
}
params.put("weight", getWeight());
return PaceConfig.resolver.getDistanceAlgo(getAlgo(), params);
}
public boolean isIgnoreMissing() {
return ignoreMissing;
}
public Type getType() {
return type;
}
@ -104,23 +94,6 @@ public class FieldDef implements Serializable {
this.overrideMatch = overrideMatch;
}
public double getWeight() {
return weight;
}
public void setWeight(final double weight) {
this.weight = weight;
}
public String getAlgo() {
return algo;
}
public void setAlgo(final String algo) {
this.algo = algo;
}
public int getSize() {
return size;
}
@ -153,10 +126,6 @@ public class FieldDef implements Serializable {
this.path = path;
}
public void setIgnoreMissing(boolean ignoreMissing) {
this.ignoreMissing = ignoreMissing;
}
@Override
public String toString() {
return new Gson().toJson(this);

@ -0,0 +1,42 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("alwaysMatch")
public class AlwaysMatch extends AbstractComparator {
public AlwaysMatch(final Map<String, Number> params){
super(params, new com.wcohen.ss.JaroWinkler());
}
public AlwaysMatch(final double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected AlwaysMatch(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(final String a, final String b) {
return 1.0;
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,38 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("exactMatch")
public class ExactMatch extends AbstractComparator {
public ExactMatch(Map<String, Number> params){
super(params, new com.wcohen.ss.JaroWinkler());
}
public ExactMatch(final double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected ExactMatch(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(final String a, final String b) {
return a.equals(b) ? 1.0 : 0;
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,46 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.io.Serializable;
import java.util.Map;
//case class JaroWinkler(w: Double) extends SecondStringDistanceAlgo(w, new com.wcohen.ss.JaroWinkler())
@ComparatorClass("jaroWinkler")
public class JaroWinkler extends AbstractComparator {
public JaroWinkler(Map<String, Number> params){
super(params, new com.wcohen.ss.JaroWinkler());
}
public JaroWinkler(double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected JaroWinkler(double weight, AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(String a, String b) {
String ca = cleanup(a);
String cb = cleanup(b);
return normalize(ssalgo.score(ca, cb));
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return d;
}
}

@ -0,0 +1,78 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.common.AbstractPaceFunctions;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
import java.util.Set;
@ComparatorClass("jaroWinklerNormalizedName")
public class JaroWinklerNormalizedName extends AbstractComparator {
private Map<String, Number> params;
public JaroWinklerNormalizedName(Map<String, Number> params){
super(params, new com.wcohen.ss.JaroWinkler());
this.params = params;
}
public JaroWinklerNormalizedName(double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected JaroWinklerNormalizedName(double weight, AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(String a, String b) {
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, params.getOrDefault("windowSize", 4).intValue());
Set<String> keywords2 = getKeywords(cb, params.getOrDefault("windowSize", 4).intValue());
Set<String> cities1 = getCities(ca, params.getOrDefault("windowSize", 4).intValue());
Set<String> cities2 = getCities(cb, params.getOrDefault("windowSize", 4).intValue());
if (sameCity(cities1,cities2)) {
if (keywordsCompare(keywords1, keywords2)>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;
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return d;
}
}

@ -0,0 +1,46 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
//case class JaroWinkler(w: Double) extends SecondStringDistanceAlgo(w, new com.wcohen.ss.JaroWinkler())
@ComparatorClass("jaroWinklerTitle")
public class JaroWinklerTitle extends AbstractComparator {
public JaroWinklerTitle(Map<String, Number> params){
super(params, new com.wcohen.ss.JaroWinkler());
}
public JaroWinklerTitle(double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected JaroWinklerTitle(double weight, AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(String a, String b) {
String ca = cleanup(a);
String cb = cleanup(b);
boolean check = checkNumbers(ca, cb);
return check ? 0.5 : normalize(ssalgo.score(ca, cb));
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return d;
}
}

@ -0,0 +1,36 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("level2JaroWinkler")
public class Level2JaroWinkler extends AbstractComparator {
public Level2JaroWinkler(Map<String, Number> params){
super(params, new com.wcohen.ss.Level2JaroWinkler());
}
public Level2JaroWinkler(double w) {
super(w, new com.wcohen.ss.Level2JaroWinkler());
}
protected Level2JaroWinkler(double w, AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return d;
}
}

@ -0,0 +1,51 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("level2JaroWinklerTitle")
public class Level2JaroWinklerTitle extends AbstractComparator {
public Level2JaroWinklerTitle(Map<String,Number> params){
super(params, new com.wcohen.ss.Level2JaroWinkler());
}
public Level2JaroWinklerTitle(final double w) {
super(w, new com.wcohen.ss.Level2JaroWinkler());
}
protected Level2JaroWinklerTitle(final double w, final AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double distance(final String a, final String b) {
final String ca = cleanup(a);
final String cb = cleanup(b);
final boolean check = checkNumbers(ca, cb);
if (check) return 0.5;
final String cca = finalCleanup(ca);
final String ccb = finalCleanup(cb);
return ssalgo.score(cca, ccb);
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,36 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("level2Levenstein")
public class Level2Levenstein extends AbstractComparator {
public Level2Levenstein(Map<String,Number> params){
super(params, new com.wcohen.ss.Level2Levenstein());
}
public Level2Levenstein(double w) {
super(w, new com.wcohen.ss.Level2Levenstein());
}
protected Level2Levenstein(double w, AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return 1 / Math.pow(Math.abs(d) + 1, 0.1);
}
}

@ -0,0 +1,36 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("levenstein")
public class Levenstein extends AbstractComparator {
public Levenstein(Map<String,Number> params){
super(params, new com.wcohen.ss.Levenstein());
}
public Levenstein(double w) {
super(w, new com.wcohen.ss.Levenstein());
}
protected Levenstein(double w, AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(double d) {
return 1 / Math.pow(Math.abs(d) + 1, 0.1);
}
}

@ -0,0 +1,59 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.Map;
@ComparatorClass("levensteinTitle")
public class LevensteinTitle extends AbstractComparator {
private static final Log log = LogFactory.getLog(LevensteinTitle.class);
public LevensteinTitle(Map<String,Number> params){
super(params, new com.wcohen.ss.Levenstein());
}
public LevensteinTitle(final double w) {
super(w, new com.wcohen.ss.Levenstein());
}
protected LevensteinTitle(final double w, final AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double distance(final String a, final String b) {
final String ca = cleanup(a);
final String cb = cleanup(b);
final boolean check = checkNumbers(ca, cb);
if (check) return 0.5;
final String cca = finalCleanup(ca);
final String ccb = finalCleanup(cb);
return normalize(ssalgo.score(cca, ccb), cca.length(), ccb.length());
}
private double normalize(final double score, final int la, final int lb) {
return 1 - (Math.abs(score) / Math.max(la, lb));
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return 1 / Math.pow(Math.abs(d) + 1, 0.1);
}
}

@ -0,0 +1,60 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
/**
* Compared compare between two titles, ignoring version numbers. Suitable for Software entities.
*/
@ComparatorClass("levensteinTitleIgnoreVersion")
public class LevensteinTitleIgnoreVersion extends AbstractComparator {
public LevensteinTitleIgnoreVersion(Map<String,Number> params){
super(params, new com.wcohen.ss.Levenstein());
}
public LevensteinTitleIgnoreVersion(final double w) {
super(w, new com.wcohen.ss.Levenstein());
}
protected LevensteinTitleIgnoreVersion(final double w, final AbstractStringDistance ssalgo) {
super(w, ssalgo);
}
@Override
public double distance(final String a, final String b) {
String ca = cleanup(a);
String cb = cleanup(b);
ca = ca.replaceAll("\\d", "").replaceAll(getRomans(ca), "").trim();
cb = cb.replaceAll("\\d", "").replaceAll(getRomans(cb), "").trim();
ca = filterAllStopWords(ca);
cb = filterAllStopWords(cb);
final String cca = finalCleanup(ca);
final String ccb = finalCleanup(cb);
return normalize(ssalgo.score(cca, ccb), cca.length(), ccb.length());
}
private double normalize(final double score, final int la, final int lb) {
return 1 - (Math.abs(score) / Math.max(la, lb));
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return 1 / Math.pow(Math.abs(d) + 1, 0.1);
}
}

@ -0,0 +1,41 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
@ComparatorClass("mustBeDifferent")
public class MustBeDifferent extends AbstractComparator {
public MustBeDifferent(Map<String,Number> params){
super(params, new com.wcohen.ss.Levenstein());
}
public MustBeDifferent(final double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected MustBeDifferent(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
@Override
public double distance(final String a, final String b) {
return !a.equals(b) ? 1.0 : 0;
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,25 @@
package eu.dnetlib.pace.tree;
import eu.dnetlib.pace.distance.DistanceAlgo;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.tree.support.Comparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
/**
* Not all fields of a document need to partecipate in the compare measure. We model those fields as having a
* NullDistanceAlgo.
*/
@ComparatorClass("null")
public class NullDistanceAlgo implements Comparator {
public NullDistanceAlgo(Map<String, Number> params){
}
@Override
public double compare(Field a, Field b) {
return 0;
}
}

@ -0,0 +1,63 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.algo.SortedSecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractSortedComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
/**
* The Class SortedJaroWinkler.
*/
@ComparatorClass("sortedJaroWinkler")
public class SortedJaroWinkler extends AbstractSortedComparator {
public SortedJaroWinkler(Map<String,Number> params){
super(params, new com.wcohen.ss.Levenstein());
}
/**
* Instantiates a new sorted jaro winkler.
*
* @param weight
* the weight
*/
public SortedJaroWinkler(final double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
/**
* Instantiates a new sorted jaro winkler.
*
* @param weight
* the weight
* @param ssalgo
* the ssalgo
*/
protected SortedJaroWinkler(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
return super.weight;
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,63 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.algo.SortedSecondStringDistanceAlgo;
import eu.dnetlib.pace.tree.support.AbstractSortedComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import java.util.Map;
/**
* The Class SortedJaroWinkler.
*/
@ComparatorClass("sortedLevel2JaroWinkler")
public class SortedLevel2JaroWinkler extends AbstractSortedComparator {
/**
* Instantiates a new sorted jaro winkler.
*
* @param weight
* the weight
*/
public SortedLevel2JaroWinkler(final double weight) {
super(weight, new com.wcohen.ss.Level2JaroWinkler());
}
public SortedLevel2JaroWinkler(final Map<String, Number> params){
super(params, new com.wcohen.ss.Level2JaroWinkler());
}
/**
* Instantiates a new sorted jaro winkler.
*
* @param weight
* the weight
* @param ssalgo
* the ssalgo
*/
protected SortedLevel2JaroWinkler(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
return super.weight;
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {
return d;
}
}

@ -0,0 +1,99 @@
package eu.dnetlib.pace.tree;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.config.Type;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.distance.SecondStringDistanceAlgo;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.tree.support.AbstractComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import org.apache.commons.lang.StringUtils;
import java.util.Map;
/**
* The Class SubStringLevenstein.
*/
@ComparatorClass("subStringLevenstein")
public class SubStringLevenstein extends AbstractComparator {
/** The limit. */
protected int limit;
/**
* Instantiates a new sub string levenstein.
*
* @param w
* the w
*/
public SubStringLevenstein(final double w) {
super(w, new com.wcohen.ss.Levenstein());
}
public SubStringLevenstein(Map<String, Number> params){
super(params, new com.wcohen.ss.Levenstein());
this.limit = params.get("limit").intValue();
}
/**
* Instantiates a new sub string levenstein.
*
* @param w
* the w
* @param limit
* the limit
*/
public SubStringLevenstein(final double w, final int limit) {
super(w, new com.wcohen.ss.Levenstein());
this.limit = limit;
}
/**
* Instantiates a new sub string levenstein.
*
* @param w
* the w
* @param limit
* the limit
* @param ssalgo
* the ssalgo
*/
protected SubStringLevenstein(final double w, final int limit, final AbstractStringDistance ssalgo) {
super(w, ssalgo);
this.limit = limit;
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#compare(eu.dnetlib.pace.model.Field, eu.dnetlib.pace.model.Field)
*/
@Override
public double compare(final Field a, final Field b) {
if (a.getType().equals(Type.String) && b.getType().equals(Type.String))
return distance(StringUtils.left(a.stringValue(), limit), StringUtils.left(b.stringValue(), limit));
throw new IllegalArgumentException("invalid types\n- A: " + a.toString() + "\n- B: " + b.toString());
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.DistanceAlgo#getWeight()
*/
@Override
public double getWeight() {
return super.weight;
}
/*
* (non-Javadoc)
*
* @see eu.dnetlib.pace.compare.SecondStringDistanceAlgo#normalize(double)
*/
@Override
protected double normalize(final double d) {
return 1 / Math.pow(Math.abs(d) + 1, 0.1);
}
}

@ -0,0 +1,60 @@
package eu.dnetlib.pace.tree;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import org.apache.commons.lang.StringUtils;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.Map;
@ComparatorClass("urlMatcher")
public class UrlMatcher extends Levenstein {
private Map<String, Number> params;
public UrlMatcher(Map<String, Number> params){
super(params);
this.params = params;
}
public UrlMatcher(double weight, Map<String, Number> params) {
super(weight);
this.params = params;
}
public void setParams(Map<String, Number> params) {
this.params = params;
}
@Override
public double compare(Field a, Field b) {
final URL urlA = asUrl(getFirstValue(a));
final URL urlB = asUrl(getFirstValue(b));
if (!urlA.getHost().equalsIgnoreCase(urlB.getHost())) {
return 0.0;
}
Double hostW = params.get("host").doubleValue();
Double pathW = params.get("path").doubleValue();
if (StringUtils.isBlank(urlA.getPath()) || StringUtils.isBlank(urlB.getPath())) {
return hostW * 0.5;
}
return hostW + pathW * super.distance(urlA.getPath(), urlB.getPath());
}
private URL asUrl(final String value) {
try {
return new URL(value);
} catch (MalformedURLException e) {
// should not happen as checked by pace typing
throw new IllegalStateException("invalid URL: " + value);
}
}
}

@ -0,0 +1,110 @@
package eu.dnetlib.pace.tree.support;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.common.AbstractPaceFunctions;
import eu.dnetlib.pace.config.Type;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.FieldList;
import java.util.List;
import java.util.Map;
public abstract class AbstractComparator extends AbstractPaceFunctions implements Comparator {
/** The ssalgo. */
protected AbstractStringDistance ssalgo;
/** The weight. */
protected double weight = 0.0;
private Map<String, Number> params;
protected AbstractComparator(Map<String, Number> params, final AbstractStringDistance ssalgo){
this.params = params;
this.weight = 1.0;
this.ssalgo = ssalgo;
}
/**
* Instantiates a new second string compare algo.
*
* @param weight
* the weight
* @param ssalgo
* the ssalgo
*/
protected AbstractComparator(final double weight, final AbstractStringDistance ssalgo) {
this.ssalgo = ssalgo;
this.weight = weight;
}
protected AbstractComparator(final AbstractStringDistance ssalgo){
this.ssalgo = ssalgo;
}
/**
* Normalize.
*
* @param d
* the d
* @return the double
*/
protected abstract double normalize(double d);
/**
* Distance.
*
* @param a
* the a
* @param b
* the b
* @return the double
*/
public double distance(final String a, final String b) {
if (a.isEmpty() || b.isEmpty()) {
return -1; //return -1 if a field is missing
}
double score = ssalgo.score(a, b);
return normalize(score);
}
/**
* Distance.
*
* @param a
* the a
* @param b
* the b
* @return the double
*/
protected double distance(final List<String> a, final List<String> b) {
return distance(concat(a), concat(b));
}
@Override
public double compare(final Field a, final Field b) {
if (a.isEmpty() || b.isEmpty())
return -1;
if (a.getType().equals(Type.String) && b.getType().equals(Type.String)) return distance(a.stringValue(), b.stringValue());
if (a.getType().equals(Type.List) && b.getType().equals(Type.List)) return distance(toList(a), toList(b));
throw new IllegalArgumentException("invalid types\n- A: " + a.toString() + "\n- B: " + b.toString());
}
/**
* To list.
*
* @param list
* the list
* @return the list
*/
protected List<String> toList(final Field list) {
return ((FieldList) list).stringList();
}
public double getWeight(){
return this.weight;
}
}

@ -0,0 +1,38 @@
package eu.dnetlib.pace.tree.support;
import com.google.common.collect.Lists;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.FieldList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public abstract class AbstractSortedComparator extends AbstractComparator {
/**
* Instantiates a new sorted second string compare algo.
*
* @param weight
* the weight
* @param ssalgo
* the ssalgo
*/
protected AbstractSortedComparator(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
protected AbstractSortedComparator(final Map<String, Number> params, final AbstractStringDistance ssalgo){
super(params.get("weight").doubleValue(), ssalgo);
}
@Override
protected List<String> toList(final Field list) {
FieldList fl = (FieldList) list;
List<String> values = Lists.newArrayList(fl.stringList());
Collections.sort(values);
return values;
}
}

@ -0,0 +1,22 @@
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.util.PaceException;
public enum AggType {
WEIGHTED_MEAN,
AVG,
SUM,
MAX,
MIN;
public static AggType getEnum(String value) {
try {
return AggType.valueOf(value);
}
catch (IllegalArgumentException e) {
throw new PaceException("Undefined aggregation type", e);
}
}
}

@ -0,0 +1,9 @@
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.model.Field;
public interface Comparator {
public double compare(Field a, Field b);
}

@ -0,0 +1,13 @@
package eu.dnetlib.pace.tree.support;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
public @interface ComparatorClass {
public String value();
}

@ -1,4 +1,4 @@
package eu.dnetlib.pace.model;
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.util.PaceException;
import org.codehaus.jackson.map.ObjectMapper;
@ -14,14 +14,25 @@ public class FieldConf implements Serializable {
private double weight = 1.0; //weight for the field (to be used in the aggregation)
private Map<String,Number> params; //parameters
private boolean ignoreMissing;
public boolean isIgnoreMissing() {
return ignoreMissing;
}
public void setIgnoreMissing(boolean ignoreMissing) {
this.ignoreMissing = ignoreMissing;
}
public FieldConf() {
}
public FieldConf(String field, String comparator, double weight, Map<String, Number> params) {
public FieldConf(String field, String comparator, double weight, Map<String, Number> params, boolean ignoreMissing) {
this.field = field;
this.comparator = comparator;
this.weight = weight;
this.params = params;
this.ignoreMissing = ignoreMissing;
}
public String getField() {

@ -0,0 +1,18 @@
package eu.dnetlib.pace.tree.support;
public enum MatchType {
MATCH,
NO_MATCH,
UNDEFINED;
public static MatchType parse(String value) {
try {
return MatchType.valueOf(value);
}
catch (IllegalArgumentException e) {
return MatchType.UNDEFINED; //return UNDEFINED if the enum is not parsable
}
}
}

@ -0,0 +1,157 @@
package eu.dnetlib.pace.tree.support;
import eu.dnetlib.pace.config.PaceConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.util.PaceException;
import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
import java.util.List;
public class TreeNodeDef implements Serializable {
private List<FieldConf> fields;
private AggType aggregation;
private double threshold;
private String positive;
private String negative;
private String undefined;
boolean ignoreMissing;
public TreeNodeDef(List<FieldConf> fields, AggType aggregation, double threshold, String positive, String negative, String undefined, boolean ignoreMissing) {
this.fields = fields;
this.aggregation = aggregation;
this.threshold = threshold;
this.positive = positive;
this.negative = negative;
this.undefined = undefined;
this.ignoreMissing = ignoreMissing;
}
public TreeNodeDef() {
}
public double evaluate(MapDocument doc1, MapDocument doc2) {
DescriptiveStatistics stats = new DescriptiveStatistics();
double sumWeights = 0.0; //for the weighted mean
int missCount = 0; //counter for the number of misses
for (FieldConf fieldConf : fields) {
double weight = fieldConf.getWeight();
double result = comparator(fieldConf).compare(doc1.getFieldMap().get(fieldConf.getField()), doc2.getFieldMap().get(fieldConf.getField()));
if (result >= 0.0) { //if the field is not missing
stats.addValue(weight * result);
sumWeights += weight; //sum weights, to be used in case of weighted mean
}
else { //if the field is missing
missCount += 1;
if (!fieldConf.isIgnoreMissing()){ //if the miss has not to be ignored
stats.addValue(weight * 0);
sumWeights += weight;
}
}
}
//global ignoremissing (if one of the field is missing, return undefined)
if (!ignoreMissing && missCount>0) {
return -1;
}
switch (aggregation){
case AVG:
return stats.getMean();
case SUM:
return stats.getSum();
case MAX:
return stats.getMax();
case MIN:
return stats.getMin();
case WEIGHTED_MEAN:
return stats.getSum()/sumWeights;
default:
return 0.0;
}
}
private Comparator comparator(final FieldConf field){
return PaceConfig.resolver.getComparator(field.getComparator(), field.getParams());
}
public List<FieldConf> getFields() {
return fields;
}
public void setFields(List<FieldConf> fields) {
this.fields = fields;
}
public AggType getAggregation() {
return aggregation;
}
public void setAggregation(AggType aggregation) {
this.aggregation = aggregation;
}
public double getThreshold() {
return threshold;
}
public void setThreshold(double threshold) {
this.threshold = threshold;
}
public String getPositive() {
return positive;
}
public void setPositive(String positive) {
this.positive = positive;
}
public String getNegative() {
return negative;
}
public void setNegative(String negative) {
this.negative = negative;
}
public String getUndefined() {
return undefined;
}
public void setUndefined(String undefined) {
this.undefined = undefined;
}
public boolean isIgnoreMissing() {
return ignoreMissing;
}
public void setIgnoreMissing(boolean ignoreMissing) {
this.ignoreMissing = ignoreMissing;
}
@Override
public String toString() {
try {
return new ObjectMapper().writeValueAsString(this);
} catch (IOException e) {
throw new PaceException("Impossible to convert to JSON: ", e);
}
}
}

@ -4,8 +4,8 @@ import com.google.common.collect.Lists;
import eu.dnetlib.pace.clustering.NGramUtils;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.config.WfConfig;
import eu.dnetlib.pace.distance.PaceDocumentDistance;
import eu.dnetlib.pace.distance.eval.ScoreResult;
//import eu.dnetlib.pace.distance.PaceDocumentDistance;
import eu.dnetlib.pace.distance.PairwiseComparison;
import eu.dnetlib.pace.model.Field;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.model.MapDocumentComparator;
@ -116,7 +116,7 @@ public class BlockProcessor {
private void process(final Queue<MapDocument> queue, final Reporter context) {
final PaceDocumentDistance algo = new PaceDocumentDistance();
// final PaceDocumentDistance algo = new PaceDocumentDistance();
while (!queue.isEmpty()) {
@ -150,21 +150,23 @@ public class BlockProcessor {
if (!idCurr.equals(idPivot) && (fieldCurr != null)) {
final ScoreResult sr = similarity(algo, pivot, curr);
// log.info(sr.toString()+"SCORE "+ sr.getScore());
emitOutput(sr, idPivot, idCurr, context);
i++;
final PairwiseComparison pairwiseComparison = new PairwiseComparison(dedupConf);
emitOutput(pairwiseComparison.compare(pivot, curr), idPivot, idCurr, context);
// final ScoreResult sr = similarity(algo, pivot, curr);
//// log.info(sr.toString()+"SCORE "+ sr.getScore());
// emitOutput(sr, idPivot, idCurr, context);
// i++;
}
}
}
}
}
private void emitOutput(final ScoreResult sr, final String idPivot, final String idCurr, final Reporter context) {
final double d = sr.getScore();
if (d >= dedupConf.getWf().getThreshold()) {
private void emitOutput(final boolean result, final String idPivot, final String idCurr, final Reporter context) {
if (result) {
writeSimilarity(context, idPivot, idCurr);
context.incrementCounter(dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)", 1);
} else {
@ -172,15 +174,6 @@ public class BlockProcessor {
}
}
private ScoreResult similarity(final PaceDocumentDistance algo, final MapDocument a, final MapDocument b) {
try {
return algo.between(a, b, dedupConf);
} catch(Throwable e) {
log.error(String.format("\nA: %s\n----------------------\nB: %s", a, b), e);
throw new IllegalArgumentException(e);
}
}
private boolean mustSkip(final String idPivot) {
return dedupConf.getWf().getSkipList().contains(getNsPrefix(idPivot));
}

@ -1390,7 +1390,7 @@ public class DiffPatchMatch {
}
/**
* Compute the Levenshtein distance; the number of inserted, deleted or
* Compute the Levenshtein compare; the number of inserted, deleted or
* substituted characters.
* @param diffs List of Diff objects.
* @return Number of changes.
@ -1655,7 +1655,7 @@ public class DiffPatchMatch {
score_threshold = score;
best_loc = j - 1;
if (best_loc > loc) {
// When passing loc, don't exceed our current distance from loc.
// When passing loc, don't exceed our current compare from loc.
start = Math.max(1, 2 * loc - best_loc);
} else {
// Already passed loc, downhill from here on in.

@ -7,6 +7,8 @@ import eu.dnetlib.pace.condition.ConditionClass;
import eu.dnetlib.pace.distance.DistanceAlgo;
import eu.dnetlib.pace.distance.DistanceClass;
import eu.dnetlib.pace.model.FieldDef;
import eu.dnetlib.pace.tree.support.Comparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
import org.reflections.Reflections;
import java.io.Serializable;
@ -19,11 +21,13 @@ public class PaceResolver implements Serializable {
public static final Reflections CLUSTERING_RESOLVER = new Reflections("eu.dnetlib.pace.clustering");
public static final Reflections CONDITION_RESOLVER = new Reflections("eu.dnetlib.pace.condition");
public static final Reflections DISTANCE_RESOLVER = new Reflections("eu.dnetlib.pace.distance.algo");
public static final Reflections DISTANCE_RESOLVER = new Reflections("eu.dnetlib.pace.compare.algo");
public static final Reflections COMPARATOR_RESOLVER = new Reflections("eu.dnetlib.pace.tree");
private final Map<String, Class<ClusteringFunction>> clusteringFunctions;
private final Map<String, Class<ConditionAlgo>> conditionAlgos;
private final Map<String, Class<DistanceAlgo>> distanceAlgos;
private final Map<String, Class<Comparator>> comparators;
public PaceResolver() {
@ -38,6 +42,10 @@ public class PaceResolver implements Serializable {
this.distanceAlgos = DISTANCE_RESOLVER.getTypesAnnotatedWith(DistanceClass.class).stream()
.filter(DistanceAlgo.class::isAssignableFrom)
.collect(Collectors.toMap(cl -> cl.getAnnotation(DistanceClass.class).value(), cl -> (Class<DistanceAlgo>)cl));
this.comparators = COMPARATOR_RESOLVER.getTypesAnnotatedWith(ComparatorClass.class).stream()
.filter(Comparator.class::isAssignableFrom)
.collect(Collectors.toMap(cl -> cl.getAnnotation(ComparatorClass.class).value(), cl -> (Class<Comparator>)cl));
}
public ClusteringFunction getClusteringFunction(String name, Map<String, Integer> params) throws PaceException {
@ -64,4 +72,12 @@ public class PaceResolver implements Serializable {
}
}
public Comparator getComparator(String name, Map<String, Number> params) throws PaceException {
try {
return comparators.get(name).getDeclaredConstructor(Map.class).newInstance(params);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException | NullPointerException e) {
throw new PaceException(name + " not found ", e);
}
}
}

@ -16,9 +16,9 @@
"pace" : {
"clustering" : [
],
"strictConditions" : [
"sufficientConditions" : [
],
"conditions" : [
"necessaryConditions" : [
],
"model" : [
],

@ -16,10 +16,10 @@
{ "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } },
{ "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }
],
"strictConditions" : [
"sufficientConditions" : [
{ "name" : "exactMatch", "fields" : [ "gridid" ] }
],
"conditions" : [
"necessaryConditions" : [
{ "name" : "exactMatch", "fields" : [ "country" ] },
{ "name" : "DomainExactMatch", "fields" : [ "websiteurl" ] }
],

Loading…
Cancel
Save