implementation of the support for authors deduplication: cosinesimilarity comparator and double array json parser
This commit is contained in:
parent
b5584f084a
commit
7e2e7dcdcd
|
@ -10,6 +10,7 @@
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<artifactId>dhp-build-properties-maven-plugin</artifactId>
|
<artifactId>dhp-build-properties-maven-plugin</artifactId>
|
||||||
|
<version>4.1.13-SNAPSHOT</version>
|
||||||
<packaging>maven-plugin</packaging>
|
<packaging>maven-plugin</packaging>
|
||||||
|
|
||||||
<description>This module is a maven plugin implementing custom properties substitutions in the build lifecycle</description>
|
<description>This module is a maven plugin implementing custom properties substitutions in the build lifecycle</description>
|
||||||
|
@ -19,16 +20,19 @@
|
||||||
<groupId>org.apache.maven</groupId>
|
<groupId>org.apache.maven</groupId>
|
||||||
<artifactId>maven-plugin-api</artifactId>
|
<artifactId>maven-plugin-api</artifactId>
|
||||||
<version>3.6.3</version>
|
<version>3.6.3</version>
|
||||||
|
<scope>provided</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.maven</groupId>
|
<groupId>org.apache.maven</groupId>
|
||||||
<artifactId>maven-project</artifactId>
|
<artifactId>maven-project</artifactId>
|
||||||
<version>2.2.1</version>
|
<version>2.2.1</version>
|
||||||
|
<scope>provided</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.maven</groupId>
|
<groupId>org.apache.maven</groupId>
|
||||||
<artifactId>maven-artifact</artifactId>
|
<artifactId>maven-artifact</artifactId>
|
||||||
<version>2.2.1</version>
|
<version>2.2.1</version>
|
||||||
|
<scope>provided</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
|
@ -100,6 +104,29 @@
|
||||||
</configuration>
|
</configuration>
|
||||||
</plugin>
|
</plugin>
|
||||||
</plugins>
|
</plugins>
|
||||||
|
|
||||||
|
<pluginManagement>
|
||||||
|
<plugins>
|
||||||
|
<plugin>
|
||||||
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
|
<artifactId>maven-plugin-plugin</artifactId>
|
||||||
|
<version>3.2</version>
|
||||||
|
<configuration>
|
||||||
|
<skipErrorNoDescriptorsFound>true</skipErrorNoDescriptorsFound>
|
||||||
|
</configuration>
|
||||||
|
<executions>
|
||||||
|
<execution>
|
||||||
|
<id>mojo-descriptor</id>
|
||||||
|
<phase>process-classes</phase>
|
||||||
|
<goals>
|
||||||
|
<goal>descriptor</goal>
|
||||||
|
</goals>
|
||||||
|
</execution>
|
||||||
|
</executions>
|
||||||
|
</plugin>
|
||||||
|
</plugins>
|
||||||
|
</pluginManagement>
|
||||||
|
|
||||||
</build>
|
</build>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -1 +1,2 @@
|
||||||
# Tue Apr 19 15:27:59 CEST 2022
|
# Sat Apr 15 10:38:57 CEST 2023
|
||||||
|
projectPropertyKey=projectPropertyValue
|
||||||
|
|
|
@ -1,12 +1,6 @@
|
||||||
#entitiesPath = /tmp/publications_test_dump
|
|
||||||
#entitiesPath = /user/michele.debonis/raw_graph_for_testing/publication
|
|
||||||
#workingPath = /user/michele.debonis/new_dedup_test/workingdirtree
|
|
||||||
#dedupConfPath = /user/michele.debonis/new_dedup_test/pubs.tree.conf.json
|
|
||||||
#numPartitions = 8000
|
|
||||||
#useTree = false
|
|
||||||
|
|
||||||
useTree = true
|
useTree = true
|
||||||
numPartitions = 1
|
entitiesPath = /user/michele.debonis/lda_experiments/authors_pubmed
|
||||||
dedupConfPath = /user/michele.debonis/authors_dedup_test/auth.tree.conf.json
|
workingPath = /user/michele.debonis/authors_dedup/gt2_dedup
|
||||||
workingPath = /user/michele.debonis/authors_dedup_test/workingdir
|
numPartitions = 1000
|
||||||
entitiesPath = /user/michele.debonis/authors_dedup_test/authors-scad-zbmath-1.json
|
dedupConfPath = /user/michele.debonis/lda_experiments/authors.fdup.gt2.conf.json
|
||||||
|
groundTruthFieldJPath = $.orcid
|
|
@ -57,14 +57,13 @@ public class Deduper implements Serializable {
|
||||||
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize));
|
.reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Iterator<Tuple2<String, String>> ccToMergeRel(ConnectedComponent cc, DedupConfig dedupConf) {
|
public static Iterator<Tuple2<String, String>> ccToMergeRel(Tuple2<String, List<String>> cc, DedupConfig dedupConf) {
|
||||||
return cc
|
return cc._2()
|
||||||
.getDocs()
|
|
||||||
.stream()
|
.stream()
|
||||||
.flatMap(
|
.flatMap(
|
||||||
id -> {
|
id -> {
|
||||||
List<Tuple2<String, String>> tmp = new ArrayList<>();
|
List<Tuple2<String, String>> tmp = new ArrayList<>();
|
||||||
tmp.add(new Tuple2<>(cc.getCcId(), id));
|
tmp.add(new Tuple2<>(cc._1(), id));
|
||||||
return tmp.stream();
|
return tmp.stream();
|
||||||
})
|
})
|
||||||
.iterator();
|
.iterator();
|
||||||
|
@ -144,13 +143,12 @@ public class Deduper implements Serializable {
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
.map(Relation::toEdgeRdd);
|
.map(Relation::toEdgeRdd);
|
||||||
|
|
||||||
JavaRDD<ConnectedComponent> ccs = JavaGraphProcessor
|
JavaPairRDD<String, List<String>> ccs = JavaGraphProcessor
|
||||||
.findCCs(vertexes, edgeRdd, maxIterations)
|
.findCCs(vertexes, edgeRdd, dedupConf.getWf().getMaxIterations());
|
||||||
.toJavaRDD();
|
|
||||||
|
|
||||||
JavaRDD<Relation> mergeRel = ccs
|
JavaRDD<Relation> mergeRel = ccs
|
||||||
.filter(k -> k.getDocs().size() > 1)
|
.filter(cc -> cc._2().size() > 1)
|
||||||
.flatMap(cc -> ccToMergeRel(cc, dedupConf))
|
.flatMap(cc -> Deduper.ccToMergeRel(cc, dedupConf))
|
||||||
.map(it -> new Relation(it._1(), it._2(), "mergeRel"));
|
.map(it -> new Relation(it._1(), it._2(), "mergeRel"));
|
||||||
|
|
||||||
final Dataset<Relation> mergeRels = spark
|
final Dataset<Relation> mergeRels = spark
|
||||||
|
@ -161,7 +159,7 @@ public class Deduper implements Serializable {
|
||||||
mergeRels.write().mode(SaveMode.Overwrite).parquet(mergeRelsPath);
|
mergeRels.write().mode(SaveMode.Overwrite).parquet(mergeRelsPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void createDedupEntity(DedupConfig dedupConf, String mergeRelsPath, String entitiesPath, SparkSession spark, String dedupEntityPath){
|
public static void createDedupEntity(DedupConfig dedupConf, String simRelsPath, String mergeRelsPath, String entitiesPath, SparkSession spark, String dedupEntityPath){
|
||||||
|
|
||||||
JavaPairRDD<String, String> entities = spark
|
JavaPairRDD<String, String> entities = spark
|
||||||
.read()
|
.read()
|
||||||
|
@ -172,7 +170,15 @@ public class Deduper implements Serializable {
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.mapToPair(t -> t);
|
.mapToPair(t -> t);
|
||||||
|
|
||||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
// <source_raw_id, relation(source, target)>
|
||||||
|
JavaPairRDD<String, Relation> simRels = spark
|
||||||
|
.read()
|
||||||
|
.load(simRelsPath)
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.mapToPair(r-> new Tuple2<>(r.getSource(), r));
|
||||||
|
|
||||||
|
// <raw_id, relation(dedup_id, raw_id)>
|
||||||
JavaPairRDD<String, Relation> mergeRels = spark
|
JavaPairRDD<String, Relation> mergeRels = spark
|
||||||
.read()
|
.read()
|
||||||
.load(mergeRelsPath)
|
.load(mergeRelsPath)
|
||||||
|
@ -185,7 +191,22 @@ public class Deduper implements Serializable {
|
||||||
.groupByKey()
|
.groupByKey()
|
||||||
.map(t-> entityMerger(t._1(), t._2().iterator()));
|
.map(t-> entityMerger(t._1(), t._2().iterator()));
|
||||||
|
|
||||||
dedupEntities.saveAsTextFile(dedupEntityPath);
|
JavaPairRDD<String, Iterable<Relation>> simRelsWithDedupId = simRels
|
||||||
|
.join(mergeRels)
|
||||||
|
.mapToPair(x -> new Tuple2<>(x._2()._2().getSource(), x._2()._1()))
|
||||||
|
.groupByKey();
|
||||||
|
|
||||||
|
JavaRDD<ConnectedComponent> groupEntity = mergeRels.join(entities)
|
||||||
|
.mapToPair(t -> new Tuple2<>(t._2()._1().getSource(), t._2()._2()))
|
||||||
|
.groupByKey()
|
||||||
|
.join(simRelsWithDedupId)
|
||||||
|
.map(x -> new ConnectedComponent(
|
||||||
|
x._1(),
|
||||||
|
x._2()._1(),
|
||||||
|
x._2()._2())
|
||||||
|
);
|
||||||
|
|
||||||
|
groupEntity.saveAsTextFile(dedupEntityPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -1,6 +1,8 @@
|
||||||
package eu.dnetlib.graph;
|
package eu.dnetlib.graph;
|
||||||
|
|
||||||
|
import com.clearspring.analytics.util.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import eu.dnetlib.pace.utils.Utility;
|
||||||
import eu.dnetlib.support.ConnectedComponent;
|
import eu.dnetlib.support.ConnectedComponent;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
@ -11,9 +13,12 @@ import scala.Tuple2;
|
||||||
import scala.reflect.ClassTag;
|
import scala.reflect.ClassTag;
|
||||||
import scala.reflect.ClassTag$;
|
import scala.reflect.ClassTag$;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
public class JavaGraphProcessor {
|
public class JavaGraphProcessor {
|
||||||
|
|
||||||
public static RDD<ConnectedComponent> findCCs(JavaPairRDD<Object, String> vertexes, JavaRDD<Edge<String>> edges, int maxIterations) {
|
//<ccId, list(json)>
|
||||||
|
public static JavaPairRDD<String, List<String>> findCCs(JavaPairRDD<Object, String> vertexes, JavaRDD<Edge<String>> edges, int maxIterations) {
|
||||||
|
|
||||||
ClassTag<String> stringTag = ClassTag$.MODULE$.apply(String.class);
|
ClassTag<String> stringTag = ClassTag$.MODULE$.apply(String.class);
|
||||||
Graph<String, String> graph =
|
Graph<String, String> graph =
|
||||||
|
@ -40,7 +45,11 @@ public class JavaGraphProcessor {
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
return joinResult.groupByKey().map(x -> new ConnectedComponent(Sets.newHashSet(x._2()))).rdd();
|
return joinResult
|
||||||
|
.groupByKey()
|
||||||
|
.map(x -> Lists.newArrayList(x._2()))
|
||||||
|
.zipWithUniqueId()
|
||||||
|
.mapToPair(x -> new Tuple2<>("dedup______::" + x._2().toString(), x._1()));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,20 +1,36 @@
|
||||||
package eu.dnetlib.jobs;
|
package eu.dnetlib.jobs;
|
||||||
|
|
||||||
|
import eu.dnetlib.Deduper;
|
||||||
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
import eu.dnetlib.pace.config.Type;
|
||||||
|
import eu.dnetlib.pace.model.FieldValueImpl;
|
||||||
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
|
import eu.dnetlib.pace.utils.Utility;
|
||||||
import eu.dnetlib.support.ArgumentApplicationParser;
|
import eu.dnetlib.support.ArgumentApplicationParser;
|
||||||
|
import eu.dnetlib.support.Block;
|
||||||
|
import eu.dnetlib.support.ConnectedComponent;
|
||||||
import eu.dnetlib.support.Relation;
|
import eu.dnetlib.support.Relation;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
import org.codehaus.jackson.map.ObjectMapper;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class SparkComputeStatistics extends AbstractSparkJob {
|
public class SparkComputeStatistics extends AbstractSparkJob {
|
||||||
|
|
||||||
|
@ -42,18 +58,42 @@ public class SparkComputeStatistics extends AbstractSparkJob {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() throws IOException {
|
public void run() throws IOException {
|
||||||
|
//https://towardsdatascience.com/7-evaluation-metrics-for-clustering-algorithms-bdc537ff54d2#:~:text=There%20are%20two%20types%20of,to%20all%20unsupervised%20learning%20results)
|
||||||
// read oozie parameters
|
// read oozie parameters
|
||||||
final String entitiesPath = parser.get("entitiesPath");
|
final String entitiesPath = parser.get("entitiesPath");
|
||||||
final String workingPath = parser.get("workingPath");
|
final String workingPath = parser.get("workingPath");
|
||||||
|
final String dedupConfPath = parser.get("dedupConfPath");
|
||||||
|
final String groundTruthFieldJPath = parser.get("groundTruthFieldJPath");
|
||||||
final int numPartitions = Optional
|
final int numPartitions = Optional
|
||||||
.ofNullable(parser.get("numPartitions"))
|
.ofNullable(parser.get("numPartitions"))
|
||||||
.map(Integer::valueOf)
|
.map(Integer::valueOf)
|
||||||
.orElse(NUM_PARTITIONS);
|
.orElse(NUM_PARTITIONS);
|
||||||
|
|
||||||
log.info("entitiesPath: '{}'", entitiesPath);
|
log.info("entitiesPath: '{}'", entitiesPath);
|
||||||
log.info("workingPath: '{}'", workingPath);
|
log.info("workingPath: '{}'", workingPath);
|
||||||
log.info("numPartitions: '{}'", numPartitions);
|
log.info("numPartitions: '{}'", numPartitions);
|
||||||
|
log.info("dedupConfPath: '{}'", dedupConfPath);
|
||||||
|
log.info("groundTruthFieldJPath: '{}'", groundTruthFieldJPath);
|
||||||
|
|
||||||
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
DedupConfig dedupConfig = loadDedupConfig(dedupConfPath);
|
||||||
|
|
||||||
|
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
||||||
|
.textFile(entitiesPath)
|
||||||
|
.repartition(numPartitions)
|
||||||
|
.mapToPair(
|
||||||
|
(PairFunction<String, String, MapDocument>) s -> {
|
||||||
|
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConfig, s);
|
||||||
|
//put in the map the groundTruthField used to compute statistics
|
||||||
|
d.getFieldMap().put("groundTruth", new FieldValueImpl(Type.String, "groundTruth", MapDocumentUtil.getJPathString(groundTruthFieldJPath, s)));
|
||||||
|
return new Tuple2<>(d.getIdentifier(), d);
|
||||||
|
});
|
||||||
|
|
||||||
|
JavaRDD<String> entities = mapDocuments.map(d -> d._2().getFieldMap().get("groundTruth").stringValue());
|
||||||
|
|
||||||
|
// create blocks
|
||||||
|
JavaRDD<List<String>> blocks = Deduper.createSortedBlocks(mapDocuments, dedupConfig)
|
||||||
|
.map(b -> b._2().getDocuments().stream().map(d -> d.getFieldMap().get("groundTruth").stringValue()).collect(Collectors.toList()));
|
||||||
|
|
||||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
||||||
JavaRDD<Relation> mergerels = spark
|
JavaRDD<Relation> mergerels = spark
|
||||||
|
@ -68,15 +108,38 @@ public class SparkComputeStatistics extends AbstractSparkJob {
|
||||||
.as(Encoders.bean(Relation.class))
|
.as(Encoders.bean(Relation.class))
|
||||||
.toJavaRDD();
|
.toJavaRDD();
|
||||||
|
|
||||||
|
JavaRDD<List<String>> groups = sc.textFile(workingPath + "/groupentities")
|
||||||
|
.map(e -> new ObjectMapper().readValue(e, ConnectedComponent.class))
|
||||||
|
.map(e -> e.getDocs().stream().map(d -> MapDocumentUtil.getJPathString(groundTruthFieldJPath, d)).collect(Collectors.toList()));
|
||||||
|
|
||||||
|
long entities_number = entities.count();
|
||||||
|
long blocks_number = blocks.count();
|
||||||
|
double blocks_randIndex = randIndex(blocks);
|
||||||
long simrels_number = simrels.count();
|
long simrels_number = simrels.count();
|
||||||
long mergerels_number = mergerels.count();
|
long mergerels_number = mergerels.count();
|
||||||
long connected_components = mergerels.groupBy(Relation::getSource).count();
|
double groups_randIndex = randIndex(groups);
|
||||||
|
long groups_number = groups.count();
|
||||||
|
long groundtruth_number = entities.filter(e -> !e.isEmpty()).count();
|
||||||
|
long correct_groups = groups.filter(x -> x.stream().distinct().count()==1).count();
|
||||||
|
long wrong_groups = groups_number - correct_groups;
|
||||||
|
|
||||||
writeStatsFileToHDFS(simrels_number, mergerels_number, connected_components, workingPath + "/stats_file");
|
String print =
|
||||||
|
"Entities : " + entities_number + "\n" +
|
||||||
|
"Ground Truth : " + groundtruth_number + "\n" +
|
||||||
|
"Blocks : " + blocks_number + "\n" +
|
||||||
|
"Blocks RI : " + blocks_randIndex + "\n" +
|
||||||
|
"SimRels : " + simrels_number + "\n" +
|
||||||
|
"MergeRels : " + mergerels_number + "\n" +
|
||||||
|
"Groups : " + groups_number + " (correct: " + correct_groups + ", wrong: " + wrong_groups + ")\n" +
|
||||||
|
"Groups RI : " + groups_randIndex;
|
||||||
|
|
||||||
|
System.out.println(print);
|
||||||
|
|
||||||
|
writeStatsFileToHDFS(groundtruth_number, entities_number, blocks_randIndex, groups_randIndex, blocks_number, simrels_number, mergerels_number, groups_number, workingPath + "/stats_file.txt");
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void writeStatsFileToHDFS(long simrels_number, long mergerels_number, long connected_components, String filePath) throws IOException {
|
public static void writeStatsFileToHDFS(long groundtruth_number, long entities_number, double blocks_randIndex, double groups_randIndex, long blocks_number, long simrels_number, long mergerels_number, long groups_number, String filePath) throws IOException {
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
|
|
||||||
FileSystem fs = FileSystem.get(conf);
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
@ -93,9 +156,14 @@ public class SparkComputeStatistics extends AbstractSparkJob {
|
||||||
}
|
}
|
||||||
|
|
||||||
String print =
|
String print =
|
||||||
"Similarity Relations : " + simrels_number + "\n" +
|
"Entities : " + entities_number + "\n" +
|
||||||
"Merge Relations : " + mergerels_number + "\n" +
|
"Ground Truth : " + groundtruth_number + "\n" +
|
||||||
"Connected Components : " + connected_components;
|
"Blocks : " + blocks_number + "\n" +
|
||||||
|
"Blocks RI : " + blocks_randIndex + "\n" +
|
||||||
|
"SimRels : " + simrels_number + "\n" +
|
||||||
|
"MergeRels : " + mergerels_number + "\n" +
|
||||||
|
"Groups : " + groups_number + "\n" +
|
||||||
|
"Groups RI : " + groups_randIndex;
|
||||||
|
|
||||||
// Create file to write
|
// Create file to write
|
||||||
FSDataOutputStream out = fs.create(outFile);
|
FSDataOutputStream out = fs.create(outFile);
|
||||||
|
@ -109,5 +177,31 @@ public class SparkComputeStatistics extends AbstractSparkJob {
|
||||||
e.printStackTrace();
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
//TODO find another maesure that takes into account all the elements outside of the group too
|
||||||
|
//RandIndex = number of pairwise correct predictions/total number of possible pairs (in the same cluster) -> bounded between 0 and 1
|
||||||
|
public double randIndex(JavaRDD<List<String>> clusters) {
|
||||||
|
|
||||||
|
Tuple2<Integer, Integer> reduce = clusters.map(c -> {
|
||||||
|
int num = 0;
|
||||||
|
for (String id : c.stream().distinct().filter(s -> !s.isEmpty()).collect(Collectors.toList())) {
|
||||||
|
int n = (int) c.stream().filter(i -> i.equals(id)).count();
|
||||||
|
num += binomialCoefficient(n);
|
||||||
|
}
|
||||||
|
int den = binomialCoefficient(c.size());
|
||||||
|
return new Tuple2<>(num, den);
|
||||||
|
})
|
||||||
|
.reduce((a, b) -> new Tuple2<>(a._1() + b._1(), a._2() + b._2()));
|
||||||
|
|
||||||
|
return (double)reduce._1()/ reduce._2();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int binomialCoefficient(int n)
|
||||||
|
{
|
||||||
|
return n*(n-1)/2;
|
||||||
|
}
|
||||||
|
|
||||||
|
//V-measure = harmonic mean of homogeneity and completeness, homogeneity = each cluster contains only members of a single class, completeness = all members of a given class are assigned to the same cluster
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -7,6 +7,7 @@ import eu.dnetlib.pace.utils.Utility;
|
||||||
import eu.dnetlib.support.ArgumentApplicationParser;
|
import eu.dnetlib.support.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.support.ConnectedComponent;
|
import eu.dnetlib.support.ConnectedComponent;
|
||||||
import eu.dnetlib.support.Relation;
|
import eu.dnetlib.support.Relation;
|
||||||
|
import org.apache.hadoop.io.compress.GzipCodec;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
@ -16,29 +17,32 @@ import org.apache.spark.sql.SparkSession;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
import scala.Tuple3;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Optional;
|
import java.util.*;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.StreamSupport;
|
||||||
|
|
||||||
public class SparkCreateDedupEntity extends AbstractSparkJob {
|
public class SparkCreateGroupEntity extends AbstractSparkJob {
|
||||||
|
|
||||||
private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.jobs.SparkCreateDedupEntity.class);
|
private static final Logger log = LoggerFactory.getLogger(SparkCreateGroupEntity.class);
|
||||||
|
|
||||||
public SparkCreateDedupEntity(ArgumentApplicationParser parser, SparkSession spark) {
|
public SparkCreateGroupEntity(ArgumentApplicationParser parser, SparkSession spark) {
|
||||||
super(parser, spark);
|
super(parser, spark);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||||
Utility.readResource("/jobs/parameters/createDedupEntity_parameters.json", SparkCreateDedupEntity.class)
|
Utility.readResource("/jobs/parameters/createGroupEntity_parameters.json", SparkCreateGroupEntity.class)
|
||||||
);
|
);
|
||||||
|
|
||||||
parser.parseArgument(args);
|
parser.parseArgument(args);
|
||||||
|
|
||||||
SparkConf conf = new SparkConf();
|
SparkConf conf = new SparkConf();
|
||||||
|
|
||||||
new SparkCreateDedupEntity(
|
new SparkCreateGroupEntity(
|
||||||
parser,
|
parser,
|
||||||
getSparkSession(conf)
|
getSparkSession(conf)
|
||||||
).run();
|
).run();
|
||||||
|
@ -63,6 +67,7 @@ public class SparkCreateDedupEntity extends AbstractSparkJob {
|
||||||
|
|
||||||
DedupConfig dedupConf = DedupConfig.load(readFileFromHDFS(dedupConfPath));
|
DedupConfig dedupConf = DedupConfig.load(readFileFromHDFS(dedupConfPath));
|
||||||
|
|
||||||
|
// <raw_id, json>
|
||||||
JavaPairRDD<String, String> entities = spark
|
JavaPairRDD<String, String> entities = spark
|
||||||
.read()
|
.read()
|
||||||
.textFile(entitiesPath)
|
.textFile(entitiesPath)
|
||||||
|
@ -72,7 +77,15 @@ public class SparkCreateDedupEntity extends AbstractSparkJob {
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.mapToPair(t -> t);
|
.mapToPair(t -> t);
|
||||||
|
|
||||||
// <source, target>: source is the dedup_id, target is the id of the mergedIn
|
// <source_raw_id, relation(source, target)>
|
||||||
|
JavaPairRDD<String, Relation> simRels = spark
|
||||||
|
.read()
|
||||||
|
.load(workingPath + "/simrels")
|
||||||
|
.as(Encoders.bean(Relation.class))
|
||||||
|
.toJavaRDD()
|
||||||
|
.mapToPair(r-> new Tuple2<>(r.getSource(), r));
|
||||||
|
|
||||||
|
// <raw_id, relation(dedup_id, raw_id)>
|
||||||
JavaPairRDD<String, Relation> mergeRels = spark
|
JavaPairRDD<String, Relation> mergeRels = spark
|
||||||
.read()
|
.read()
|
||||||
.load(workingPath + "/mergerels")
|
.load(workingPath + "/mergerels")
|
||||||
|
@ -80,12 +93,23 @@ public class SparkCreateDedupEntity extends AbstractSparkJob {
|
||||||
.toJavaRDD()
|
.toJavaRDD()
|
||||||
.mapToPair(r -> new Tuple2<>(r.getTarget(), r));
|
.mapToPair(r -> new Tuple2<>(r.getTarget(), r));
|
||||||
|
|
||||||
JavaRDD<ConnectedComponent> dedupEntities = mergeRels.join(entities)
|
// <dedup_id, simrel>
|
||||||
|
JavaPairRDD<String, Iterable<Relation>> simRelsWithDedupId = simRels
|
||||||
|
.join(mergeRels)
|
||||||
|
.mapToPair(x -> new Tuple2<>(x._2()._2().getSource(), x._2()._1()))
|
||||||
|
.groupByKey();
|
||||||
|
|
||||||
|
JavaRDD<ConnectedComponent> groupEntity = mergeRels.join(entities)
|
||||||
.mapToPair(t -> new Tuple2<>(t._2()._1().getSource(), t._2()._2()))
|
.mapToPair(t -> new Tuple2<>(t._2()._1().getSource(), t._2()._2()))
|
||||||
.groupByKey()
|
.groupByKey()
|
||||||
.map(t-> Deduper.entityMerger(t._1(), t._2().iterator()));
|
.join(simRelsWithDedupId)
|
||||||
|
.map(x -> new ConnectedComponent(
|
||||||
|
x._1(),
|
||||||
|
x._2()._1(),
|
||||||
|
x._2()._2())
|
||||||
|
);
|
||||||
|
|
||||||
dedupEntities.saveAsTextFile(workingPath + "dedupentity");
|
groupEntity.saveAsTextFile(workingPath + "/groupentities", GzipCodec.class);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
import static eu.dnetlib.Deduper.hash;
|
import static eu.dnetlib.Deduper.hash;
|
||||||
|
@ -85,12 +86,11 @@ public class SparkCreateMergeRels extends AbstractSparkJob {
|
||||||
.javaRDD()
|
.javaRDD()
|
||||||
.map(Relation::toEdgeRdd);
|
.map(Relation::toEdgeRdd);
|
||||||
|
|
||||||
JavaRDD<ConnectedComponent> ccs = JavaGraphProcessor
|
JavaPairRDD<String, List<String>> ccs = JavaGraphProcessor
|
||||||
.findCCs(vertexes, edgeRdd, dedupConf.getWf().getMaxIterations())
|
.findCCs(vertexes, edgeRdd, dedupConf.getWf().getMaxIterations());
|
||||||
.toJavaRDD();
|
|
||||||
|
|
||||||
JavaRDD<Relation> mergeRel = ccs
|
JavaRDD<Relation> mergeRel = ccs
|
||||||
.filter(k -> k.getDocs().size() > 1)
|
.filter(cc -> cc._2().size() > 1)
|
||||||
.flatMap(cc -> Deduper.ccToMergeRel(cc, dedupConf))
|
.flatMap(cc -> Deduper.ccToMergeRel(cc, dedupConf))
|
||||||
.map(it -> new Relation(it._1(), it._2(), "mergeRel"));
|
.map(it -> new Relation(it._1(), it._2(), "mergeRel"));
|
||||||
|
|
||||||
|
|
|
@ -14,6 +14,7 @@ import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
|
import org.apache.spark.sql.Encoder;
|
||||||
import org.apache.spark.sql.Encoders;
|
import org.apache.spark.sql.Encoders;
|
||||||
import org.apache.spark.sql.SaveMode;
|
import org.apache.spark.sql.SaveMode;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
|
|
|
@ -1,10 +1,7 @@
|
||||||
package eu.dnetlib.support;
|
package eu.dnetlib.support;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.*;
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
import java.util.stream.StreamSupport;
|
import java.util.stream.StreamSupport;
|
||||||
|
@ -12,6 +9,7 @@ import java.util.stream.StreamSupport;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
|
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||||
|
|
||||||
public class Block implements Serializable {
|
public class Block implements Serializable {
|
||||||
|
|
||||||
|
@ -23,6 +21,11 @@ public class Block implements Serializable {
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Block(String key, List<MapDocument> documents) {
|
||||||
|
this.key = key;
|
||||||
|
this.documents = documents;
|
||||||
|
}
|
||||||
|
|
||||||
public Block(String key, Iterable<MapDocument> documents) {
|
public Block(String key, Iterable<MapDocument> documents) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.documents = Lists.newArrayList(documents);
|
this.documents = Lists.newArrayList(documents);
|
||||||
|
|
|
@ -5,54 +5,35 @@ import java.io.Serializable;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import eu.dnetlib.pace.utils.Utility;
|
import com.google.common.collect.Sets;
|
||||||
import org.apache.commons.lang.StringUtils;
|
|
||||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
|
|
||||||
import eu.dnetlib.pace.util.PaceException;
|
import eu.dnetlib.pace.util.PaceException;
|
||||||
|
import org.codehaus.jackson.map.ObjectMapper;
|
||||||
|
|
||||||
public class ConnectedComponent implements Serializable {
|
public class ConnectedComponent implements Serializable {
|
||||||
|
|
||||||
private HashSet<String> docs;
|
private HashSet<String> docs;
|
||||||
private String ccId;
|
private String ccId;
|
||||||
|
private HashSet<Relation> simrels;
|
||||||
|
|
||||||
public ConnectedComponent() {
|
public ConnectedComponent() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public ConnectedComponent(String ccId, Set<String> docs, Set<Relation> simrels) {
|
||||||
|
this.docs = new HashSet<>(docs);
|
||||||
|
this.ccId = ccId;
|
||||||
|
this.simrels = new HashSet<>(simrels);
|
||||||
|
}
|
||||||
|
|
||||||
public ConnectedComponent(Set<String> docs) {
|
public ConnectedComponent(Set<String> docs) {
|
||||||
this.docs = new HashSet<>(docs);
|
this.docs = new HashSet<>(docs);
|
||||||
createID();
|
//initialization of id and relations missing
|
||||||
}
|
}
|
||||||
|
|
||||||
public String createID() {
|
public ConnectedComponent(String ccId, Iterable<String> docs, Iterable<Relation> simrels) {
|
||||||
if (docs.size() > 1) {
|
this.ccId = ccId;
|
||||||
final String s = getMin();
|
this.docs = Sets.newHashSet(docs);
|
||||||
ccId = "dedup::" + Utility.md5(s);
|
this.simrels = Sets.newHashSet(simrels);
|
||||||
return ccId;
|
|
||||||
} else {
|
|
||||||
return docs.iterator().next();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonIgnore
|
|
||||||
public String getMin() {
|
|
||||||
|
|
||||||
final StringBuilder min = new StringBuilder();
|
|
||||||
docs
|
|
||||||
.forEach(
|
|
||||||
i -> {
|
|
||||||
if (StringUtils.isBlank(min.toString())) {
|
|
||||||
min.append(i);
|
|
||||||
} else {
|
|
||||||
if (min.toString().compareTo(i) > 0) {
|
|
||||||
min.setLength(0);
|
|
||||||
min.append(i);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
});
|
|
||||||
return min.toString();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -80,4 +61,12 @@ public class ConnectedComponent implements Serializable {
|
||||||
public void setCcId(String ccId) {
|
public void setCcId(String ccId) {
|
||||||
this.ccId = ccId;
|
this.ccId = ccId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setSimrels(HashSet<Relation> simrels) {
|
||||||
|
this.simrels = simrels;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HashSet<Relation> getSimrels() {
|
||||||
|
return simrels;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,6 +16,10 @@
|
||||||
<name>dedupConfPath</name>
|
<name>dedupConfPath</name>
|
||||||
<description>path for the dedup configuration file</description>
|
<description>path for the dedup configuration file</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>groundTruthFieldJPath</name>
|
||||||
|
<description>jpath of the field to be used as ground truth</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
|
@ -138,6 +142,33 @@
|
||||||
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
||||||
<arg>--dedupConfPath</arg><arg>${dedupConfPath}</arg>
|
<arg>--dedupConfPath</arg><arg>${dedupConfPath}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
|
<ok to="CreateGroupEntities"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
|
||||||
|
<action name="CreateGroupEntities">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>Create Group Entities</name>
|
||||||
|
<class>eu.dnetlib.jobs.SparkCreateGroupEntity</class>
|
||||||
|
<jar>dnet-dedup-test-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--entitiesPath</arg><arg>${entitiesPath}</arg>
|
||||||
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
|
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
||||||
|
<arg>--dedupConfPath</arg><arg>${dedupConfPath}</arg>
|
||||||
|
</spark>
|
||||||
<ok to="ComputeStatistics"/>
|
<ok to="ComputeStatistics"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
@ -162,36 +193,12 @@
|
||||||
<arg>--entitiesPath</arg><arg>${entitiesPath}</arg>
|
<arg>--entitiesPath</arg><arg>${entitiesPath}</arg>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
<arg>--numPartitions</arg><arg>${numPartitions}</arg>
|
||||||
|
<arg>--dedupConfPath</arg><arg>${dedupConfPath}</arg>
|
||||||
|
<arg>--groundTruthFieldJPath</arg><arg>${groundTruthFieldJPath}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<!--<action name="CreateDedupEntities">-->
|
|
||||||
<!--<spark xmlns="uri:oozie:spark-action:0.2">-->
|
|
||||||
<!--<master>yarn</master>-->
|
|
||||||
<!--<mode>cluster</mode>-->
|
|
||||||
<!--<name>Create Dedup Entities</name>-->
|
|
||||||
<!--<class>eu.dnetlib.jobs.SparkCreateDedupEntity</class>-->
|
|
||||||
<!--<jar>dnet-dedup-test-${projectVersion}.jar</jar>-->
|
|
||||||
<!--<spark-opts>-->
|
|
||||||
<!----executor-memory=${sparkExecutorMemory}-->
|
|
||||||
<!----executor-cores=${sparkExecutorCores}-->
|
|
||||||
<!----driver-memory=${sparkDriverMemory}-->
|
|
||||||
<!----conf spark.extraListeners=${spark2ExtraListeners}-->
|
|
||||||
<!----conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}-->
|
|
||||||
<!----conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}-->
|
|
||||||
<!----conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}-->
|
|
||||||
<!----conf spark.sql.shuffle.partitions=3840-->
|
|
||||||
<!--</spark-opts>-->
|
|
||||||
<!--<arg>--entitiesPath</arg><arg>${entitiesPath}</arg>-->
|
|
||||||
<!--<arg>--workingPath</arg><arg>${workingPath}</arg>-->
|
|
||||||
<!--<arg>--numPartitions</arg><arg>${numPartitions}</arg>-->
|
|
||||||
<!--<arg>--dedupConfPath</arg><arg>${dedupConfPath}</arg>-->
|
|
||||||
<!--</spark>-->
|
|
||||||
<!--<ok to="End"/>-->
|
|
||||||
<!--<error to="Kill"/>-->
|
|
||||||
<!--</action>-->
|
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -16,5 +16,17 @@
|
||||||
"paramLongName": "numPartitions",
|
"paramLongName": "numPartitions",
|
||||||
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
"paramRequired": false
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dc",
|
||||||
|
"paramLongName": "dedupConfPath",
|
||||||
|
"paramDescription": "dedup configuration to be used",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "gt",
|
||||||
|
"paramLongName": "groundTruthFieldJPath",
|
||||||
|
"paramDescription": "field to be used as groundtruth",
|
||||||
|
"paramRequired": true
|
||||||
}
|
}
|
||||||
]
|
]
|
|
@ -1,7 +1,8 @@
|
||||||
package eu.dnetlib.pace;
|
package eu.dnetlib.pace;
|
||||||
|
|
||||||
import eu.dnetlib.Deduper;
|
import eu.dnetlib.Deduper;
|
||||||
import eu.dnetlib.jobs.SparkCreateDedupEntity;
|
import eu.dnetlib.jobs.SparkComputeStatistics;
|
||||||
|
import eu.dnetlib.jobs.SparkCreateGroupEntity;
|
||||||
import eu.dnetlib.jobs.SparkCreateMergeRels;
|
import eu.dnetlib.jobs.SparkCreateMergeRels;
|
||||||
import eu.dnetlib.jobs.SparkCreateSimRels;
|
import eu.dnetlib.jobs.SparkCreateSimRels;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
@ -50,7 +51,7 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
static JavaSparkContext context;
|
static JavaSparkContext context;
|
||||||
|
|
||||||
final String entitiesPath = Paths
|
final String entitiesPath = Paths
|
||||||
.get(DedupLocalTest.class.getResource("/eu/dnetlib/pace/examples/publications.dump.1000.json").toURI())
|
.get(DedupLocalTest.class.getResource("/eu/dnetlib/pace/examples/authors.dump.json").toURI())
|
||||||
.toFile()
|
.toFile()
|
||||||
.getAbsolutePath();
|
.getAbsolutePath();
|
||||||
|
|
||||||
|
@ -58,13 +59,14 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
final static String numPartitions = "20";
|
final static String numPartitions = "20";
|
||||||
|
|
||||||
final String dedupConfPath = Paths
|
final String dedupConfPath = Paths
|
||||||
.get(DedupLocalTest.class.getResource("/eu/dnetlib/pace/config/pubs.fdup.exp.json").toURI())
|
.get(DedupLocalTest.class.getResource("/eu/dnetlib/pace/config/authors.fdup.conf.json").toURI())
|
||||||
.toFile()
|
.toFile()
|
||||||
.getAbsolutePath();
|
.getAbsolutePath();
|
||||||
|
|
||||||
final static String simRelsPath = workingPath + "/simrels";
|
final static String simRelsPath = workingPath + "/simrels";
|
||||||
final static String mergeRelsPath = workingPath + "/mergerels";
|
final static String mergeRelsPath = workingPath + "/mergerels";
|
||||||
final static String dedupEntityPath = workingPath + "/dedupentities";
|
final static String groupEntityPath = workingPath + "/groupentities";
|
||||||
|
|
||||||
|
final static String groundTruthFieldJPath = "$.orcid";
|
||||||
|
|
||||||
public DedupLocalTest() throws URISyntaxException {
|
public DedupLocalTest() throws URISyntaxException {
|
||||||
}
|
}
|
||||||
|
@ -73,7 +75,7 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
//remove directories to clean workspace
|
//remove directories to clean workspace
|
||||||
FileUtils.deleteDirectory(new File(simRelsPath));
|
FileUtils.deleteDirectory(new File(simRelsPath));
|
||||||
FileUtils.deleteDirectory(new File(mergeRelsPath));
|
FileUtils.deleteDirectory(new File(mergeRelsPath));
|
||||||
FileUtils.deleteDirectory(new File(dedupEntityPath));
|
FileUtils.deleteDirectory(new File(groupEntityPath));
|
||||||
}
|
}
|
||||||
|
|
||||||
@BeforeAll
|
@BeforeAll
|
||||||
|
@ -155,9 +157,9 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@Order(3)
|
@Order(3)
|
||||||
public void createDedupEntityTest() throws Exception {
|
public void createGroupEntityTest() throws Exception {
|
||||||
|
|
||||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(Utility.readResource("/eu/dnetlib/pace/parameters/createDedupEntity_parameters.json", SparkCreateDedupEntity.class));
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(Utility.readResource("/eu/dnetlib/pace/parameters/createGroupEntity_parameters.json", SparkCreateGroupEntity.class));
|
||||||
|
|
||||||
parser.parseArgument(
|
parser.parseArgument(
|
||||||
new String[] {
|
new String[] {
|
||||||
|
@ -167,7 +169,27 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
"-dc", dedupConfPath
|
"-dc", dedupConfPath
|
||||||
});
|
});
|
||||||
|
|
||||||
new SparkCreateDedupEntity(
|
new SparkCreateGroupEntity(
|
||||||
|
parser,
|
||||||
|
spark
|
||||||
|
).run();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
@Order(4)
|
||||||
|
public void computeStatisticsTest() throws Exception {
|
||||||
|
ArgumentApplicationParser parser = new ArgumentApplicationParser(Utility.readResource("/eu/dnetlib/pace/parameters/computeStatistics_parameters.json", SparkComputeStatistics.class));
|
||||||
|
|
||||||
|
parser.parseArgument(
|
||||||
|
new String[] {
|
||||||
|
"-e", entitiesPath,
|
||||||
|
"-w", workingPath,
|
||||||
|
"-np", numPartitions,
|
||||||
|
"-dc", dedupConfPath,
|
||||||
|
"-gt", groundTruthFieldJPath
|
||||||
|
});
|
||||||
|
|
||||||
|
new SparkComputeStatistics(
|
||||||
parser,
|
parser,
|
||||||
spark
|
spark
|
||||||
).run();
|
).run();
|
||||||
|
@ -216,6 +238,7 @@ public class DedupLocalTest extends DedupTestUtils {
|
||||||
long before_dedupentity = System.currentTimeMillis();
|
long before_dedupentity = System.currentTimeMillis();
|
||||||
Deduper.createDedupEntity(
|
Deduper.createDedupEntity(
|
||||||
dedupConfig,
|
dedupConfig,
|
||||||
|
simRelsPath,
|
||||||
mergeRelsPath,
|
mergeRelsPath,
|
||||||
inputPath,
|
inputPath,
|
||||||
spark,
|
spark,
|
||||||
|
|
|
@ -5,7 +5,7 @@
|
||||||
"entityType": "author",
|
"entityType": "author",
|
||||||
"subEntityType": "author",
|
"subEntityType": "author",
|
||||||
"subEntityValue": "author",
|
"subEntityValue": "author",
|
||||||
"orderField": "name",
|
"orderField": "fullname",
|
||||||
"queueMaxSize": "200",
|
"queueMaxSize": "200",
|
||||||
"groupMaxSize": "100",
|
"groupMaxSize": "100",
|
||||||
"maxChildren": "100",
|
"maxChildren": "100",
|
||||||
|
@ -29,148 +29,103 @@
|
||||||
},
|
},
|
||||||
"pace": {
|
"pace": {
|
||||||
"clustering" : [
|
"clustering" : [
|
||||||
{ "name" : "lnfi", "fields" : [ "name" ], "params" : {} }
|
{ "name" : "lnfi", "fields" : [ "fullname" ], "params" : {} }
|
||||||
],
|
],
|
||||||
"decisionTree": {
|
"decisionTree": {
|
||||||
"start": {
|
"start": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "pub_id",
|
"field": "orcid",
|
||||||
"comparator": "exactMatch",
|
"comparator": "exactMatch",
|
||||||
"weight": 1,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {}
|
"params": {}
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold":1,
|
"threshold": 1.0,
|
||||||
"aggregation": "AVG",
|
|
||||||
"positive": "NO_MATCH",
|
|
||||||
"negative": "yearCheck",
|
|
||||||
"undefined": "yearCheck"
|
|
||||||
},
|
|
||||||
"yearCheck": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "year",
|
|
||||||
"comparator": "numbersComparator",
|
|
||||||
"weight": 1,
|
|
||||||
"countIfUndefined": "false",
|
|
||||||
"params": {}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 50,
|
|
||||||
"aggregation": "MAX",
|
"aggregation": "MAX",
|
||||||
"positive": "NO_MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "surnames",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "surnames",
|
"undefined": "orcids",
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "true"
|
||||||
},
|
},
|
||||||
"surnames": {
|
"orcids": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "orcids",
|
||||||
|
"comparator": "stringListMatch",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {"type": "count"}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 3.0,
|
||||||
|
"aggregation": "MAX",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "coauthors",
|
||||||
|
"undefined": "coauthors",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"coauthors": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "coauthors",
|
"field": "coauthors",
|
||||||
"comparator": "authorsMatch",
|
"comparator": "authorsMatch",
|
||||||
"weight": 1.0,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {
|
"params": {"type": "count"}
|
||||||
"surname_th": 0.75,
|
}
|
||||||
"fullname_th": 0.75,
|
],
|
||||||
"size_th": 20,
|
"threshold": 1.0,
|
||||||
"mode": "surname"
|
"aggregation": "MAX",
|
||||||
}
|
"positive": "topicsMatch",
|
||||||
|
"negative": "NO_MATCH",
|
||||||
|
"undefined": "topicsMatch",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"topicsMatch": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "topics",
|
||||||
|
"comparator": "cosineSimilarity",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {}
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold": 0.5,
|
"threshold": 0.5,
|
||||||
"aggregation": "MAX",
|
"aggregation": "MAX",
|
||||||
"positive": "MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "cityCheck",
|
|
||||||
"undefined": "cityCheck",
|
|
||||||
"ignoreUndefined": "true"
|
|
||||||
},
|
|
||||||
"cityCheck": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "org",
|
|
||||||
"comparator": "cityMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "true",
|
|
||||||
"params": {
|
|
||||||
"windowSize": "4"
|
|
||||||
}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 0.1,
|
|
||||||
"aggregation": "AVG",
|
|
||||||
"positive": "keywordCheck",
|
|
||||||
"negative": "NO_MATCH",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "keywordCheck",
|
"undefined": "NO_MATCH",
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "false"
|
||||||
},
|
|
||||||
"keywordCheck": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "org",
|
|
||||||
"comparator": "keywordMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "true",
|
|
||||||
"params": {
|
|
||||||
"windowSize": "4"
|
|
||||||
}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 0.5,
|
|
||||||
"aggregation": "AVG",
|
|
||||||
"positive": "orgCheck",
|
|
||||||
"negative": "NO_MATCH",
|
|
||||||
"undefined": "orgCheck",
|
|
||||||
"ignoreUndefined": "true"
|
|
||||||
},
|
|
||||||
"orgCheck": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "org",
|
|
||||||
"comparator": "jaroWinklerNormalizedName",
|
|
||||||
"weight": 1,
|
|
||||||
"countIfUndefined": "true",
|
|
||||||
"params": {
|
|
||||||
"windowSize": "4"
|
|
||||||
}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 0.7,
|
|
||||||
"aggregation": "AVG",
|
|
||||||
"positive": "MATCH",
|
|
||||||
"negative": "NO_MATCH",
|
|
||||||
"undefined": "MATCH",
|
|
||||||
"ignoreUndefined": "true"
|
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
"model": [
|
"model": [
|
||||||
{
|
{
|
||||||
"name": "name",
|
"name": "topics",
|
||||||
|
"type": "DoubleArray",
|
||||||
|
"path": "$.topics"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "fullname",
|
||||||
"type": "String",
|
"type": "String",
|
||||||
"path": "$.name"
|
"path": "$.fullname"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "orcid",
|
||||||
|
"type": "String",
|
||||||
|
"path": "$.orcid"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "coauthors",
|
"name": "coauthors",
|
||||||
"type": "List",
|
"type": "List",
|
||||||
"path": "$.coauthors[*].name",
|
"path": "$.coAuthors[*].fullname"
|
||||||
"size": 200
|
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "year",
|
"name": "orcids",
|
||||||
"type": "String",
|
"type": "List",
|
||||||
"path": "$.year"
|
"path": "$.coAuthors[*].orcid"
|
||||||
},
|
|
||||||
{
|
|
||||||
"name": "pub_id",
|
|
||||||
"type": "String",
|
|
||||||
"path": "$.pub_id"
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"name": "org",
|
|
||||||
"type": "String",
|
|
||||||
"path": "$.org"
|
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"blacklists": {},
|
"blacklists": {},
|
||||||
|
|
|
@ -5,7 +5,7 @@
|
||||||
"entityType": "author",
|
"entityType": "author",
|
||||||
"subEntityType": "author",
|
"subEntityType": "author",
|
||||||
"subEntityValue": "author",
|
"subEntityValue": "author",
|
||||||
"orderField": "name",
|
"orderField": "fullname",
|
||||||
"queueMaxSize": "200",
|
"queueMaxSize": "200",
|
||||||
"groupMaxSize": "100",
|
"groupMaxSize": "100",
|
||||||
"maxChildren": "100",
|
"maxChildren": "100",
|
||||||
|
@ -29,74 +29,103 @@
|
||||||
},
|
},
|
||||||
"pace": {
|
"pace": {
|
||||||
"clustering" : [
|
"clustering" : [
|
||||||
{ "name" : "lnfi", "fields" : [ "name" ], "params" : {} }
|
{ "name" : "lnfi", "fields" : [ "fullname" ], "params" : {} }
|
||||||
],
|
],
|
||||||
"decisionTree": {
|
"decisionTree": {
|
||||||
"start": {
|
"start": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "pub_id",
|
"field": "orcid",
|
||||||
"comparator": "exactMatch",
|
"comparator": "exactMatch",
|
||||||
"weight": 1,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {}
|
"params": {}
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold":1,
|
"threshold": 1.0,
|
||||||
"aggregation": "AVG",
|
"aggregation": "MAX",
|
||||||
"positive": "NO_MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "yearCheck",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "yearCheck"
|
"undefined": "orcids",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
},
|
},
|
||||||
"surnames": {
|
"orcids": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "orcids",
|
||||||
|
"comparator": "stringListMatch",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {"type": "count"}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 3.0,
|
||||||
|
"aggregation": "MAX",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "coauthors",
|
||||||
|
"undefined": "coauthors",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"coauthors": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "coauthors",
|
"field": "coauthors",
|
||||||
"comparator": "authorsMatch",
|
"comparator": "authorsMatch",
|
||||||
"weight": 1.0,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {
|
"params": {"type": "count"}
|
||||||
"surname_th": 0.75,
|
|
||||||
"fullname_th": 0.75,
|
|
||||||
"size_th": 20,
|
|
||||||
"mode": "surname"
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold": 0.5,
|
"threshold": 1.0,
|
||||||
|
"aggregation": "MAX",
|
||||||
|
"positive": "topicsMatch",
|
||||||
|
"negative": "NO_MATCH",
|
||||||
|
"undefined": "topicsMatch",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"topicsMatch": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "topics",
|
||||||
|
"comparator": "cosineSimilarity",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 1.0,
|
||||||
"aggregation": "MAX",
|
"aggregation": "MAX",
|
||||||
"positive": "MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "NO_MATCH",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "MATCH",
|
"undefined": "NO_MATCH",
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "false"
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
"model": [
|
"model": [
|
||||||
{
|
{
|
||||||
"name": "name",
|
"name": "topics",
|
||||||
|
"type": "DoubleArray",
|
||||||
|
"path": "$.topics"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "fullname",
|
||||||
"type": "String",
|
"type": "String",
|
||||||
"path": "$.name"
|
"path": "$.fullname"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "orcid",
|
||||||
|
"type": "String",
|
||||||
|
"path": "$.orcid"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "coauthors",
|
"name": "coauthors",
|
||||||
"type": "List",
|
"type": "List",
|
||||||
"path": "$.coauthors[*].name",
|
"path": "$.coAuthors[*].fullname"
|
||||||
"size": 200
|
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "year",
|
"name": "orcids",
|
||||||
"type": "String",
|
"type": "List",
|
||||||
"path": "$.year"
|
"path": "$.coAuthors[*].orcid"
|
||||||
},
|
|
||||||
{
|
|
||||||
"name": "pub_id",
|
|
||||||
"type": "String",
|
|
||||||
"path": "$.pub_id"
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"name": "org",
|
|
||||||
"type": "String",
|
|
||||||
"path": "$.org"
|
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"blacklists": {},
|
"blacklists": {},
|
||||||
|
|
|
@ -51,37 +51,6 @@
|
||||||
],
|
],
|
||||||
"decisionTree": {
|
"decisionTree": {
|
||||||
"start": {
|
"start": {
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "pid",
|
|
||||||
"comparator": "jsonListMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "false",
|
|
||||||
"params": {
|
|
||||||
"jpath_value": "$.value",
|
|
||||||
"jpath_classid": "$.qualifier.classid"
|
|
||||||
}
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"field": "pid",
|
|
||||||
"comparator": "jsonListMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "false",
|
|
||||||
"params": {
|
|
||||||
"jpath_value": "$.value",
|
|
||||||
"jpath_classid": "$.qualifier.classid",
|
|
||||||
"crossCompare": "alternateid"
|
|
||||||
}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 0.5,
|
|
||||||
"aggregation": "MAX",
|
|
||||||
"positive": "layer1",
|
|
||||||
"negative": "layer2",
|
|
||||||
"undefined": "layer2",
|
|
||||||
"ignoreUndefined": "true"
|
|
||||||
},
|
|
||||||
"layer1": {
|
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "title",
|
"field": "title",
|
||||||
|
@ -94,49 +63,8 @@
|
||||||
"threshold": 0.9,
|
"threshold": 0.9,
|
||||||
"aggregation": "AVG",
|
"aggregation": "AVG",
|
||||||
"positive": "MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "NO_MATCH",
|
"negative": "MATCH",
|
||||||
"undefined": "NO_MATCH",
|
"undefined": "MATCH",
|
||||||
"ignoreUndefined": "true"
|
|
||||||
},
|
|
||||||
"layer2": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "title",
|
|
||||||
"comparator": "titleVersionMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "false",
|
|
||||||
"params": {}
|
|
||||||
},
|
|
||||||
{
|
|
||||||
"field": "authors",
|
|
||||||
"comparator": "sizeMatch",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "false",
|
|
||||||
"params": {}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 1.0,
|
|
||||||
"aggregation": "AND",
|
|
||||||
"positive": "layer3",
|
|
||||||
"negative": "NO_MATCH",
|
|
||||||
"undefined": "layer3",
|
|
||||||
"ignoreUndefined": "false"
|
|
||||||
},
|
|
||||||
"layer3": {
|
|
||||||
"fields": [
|
|
||||||
{
|
|
||||||
"field": "title",
|
|
||||||
"comparator": "levensteinTitle",
|
|
||||||
"weight": 1.0,
|
|
||||||
"countIfUndefined": "true",
|
|
||||||
"params": {}
|
|
||||||
}
|
|
||||||
],
|
|
||||||
"threshold": 0.99,
|
|
||||||
"aggregation": "AVG",
|
|
||||||
"positive": "MATCH",
|
|
||||||
"negative": "NO_MATCH",
|
|
||||||
"undefined": "NO_MATCH",
|
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "true"
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,32 @@
|
||||||
|
[
|
||||||
|
{
|
||||||
|
"paramName": "e",
|
||||||
|
"paramLongName": "entitiesPath",
|
||||||
|
"paramDescription": "the input entities",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "w",
|
||||||
|
"paramLongName": "workingPath",
|
||||||
|
"paramDescription": "path of the working directory",
|
||||||
|
"paramRequired": true
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "np",
|
||||||
|
"paramLongName": "numPartitions",
|
||||||
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "dc",
|
||||||
|
"paramLongName": "dedupConfPath",
|
||||||
|
"paramDescription": "dedup configuration to be used",
|
||||||
|
"paramRequired": false
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"paramName": "gt",
|
||||||
|
"paramLongName": "groundTruthFieldJPath",
|
||||||
|
"paramDescription": "field to be used as groundtruth",
|
||||||
|
"paramRequired": false
|
||||||
|
}
|
||||||
|
]
|
|
@ -1,5 +1,5 @@
|
||||||
package eu.dnetlib.pace.config;
|
package eu.dnetlib.pace.config;
|
||||||
|
|
||||||
public enum Type {
|
public enum Type {
|
||||||
String, Int, List, JSON, URL, StringConcat
|
String, Int, List, JSON, URL, StringConcat, DoubleArray
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,4 +20,6 @@ public interface FieldValue extends Field {
|
||||||
*/
|
*/
|
||||||
public void setValue(final Object value);
|
public void setValue(final Object value);
|
||||||
|
|
||||||
|
public double[] doubleArrayValue();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -58,8 +58,10 @@ public class FieldValueImpl extends AbstractField implements FieldValue {
|
||||||
throw new RuntimeException(value.toString());
|
throw new RuntimeException(value.toString());
|
||||||
}
|
}
|
||||||
case URL:
|
case URL:
|
||||||
String str = value.toString();
|
String str = value.toString();
|
||||||
return StringUtils.isBlank(str) || !isValidURL(str);
|
return StringUtils.isBlank(str) || !isValidURL(str);
|
||||||
|
case DoubleArray:
|
||||||
|
return doubleArrayValue().length==0;
|
||||||
default:
|
default:
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -116,6 +118,10 @@ public class FieldValueImpl extends AbstractField implements FieldValue {
|
||||||
// }
|
// }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public double[] doubleArrayValue() {
|
||||||
|
return (double[])getValue();
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* (non-Javadoc)
|
* (non-Javadoc)
|
||||||
*
|
*
|
||||||
|
|
|
@ -52,7 +52,7 @@ public class AuthorsMatch extends AbstractComparator {
|
||||||
if (a.isEmpty() || b.isEmpty())
|
if (a.isEmpty() || b.isEmpty())
|
||||||
return -1;
|
return -1;
|
||||||
|
|
||||||
if (((FieldList) a).size() > SIZE_THRESHOLD || ((FieldList) a).size() > SIZE_THRESHOLD)
|
if (((FieldList) a).size() > SIZE_THRESHOLD || ((FieldList) b).size() > SIZE_THRESHOLD)
|
||||||
return 1.0;
|
return 1.0;
|
||||||
|
|
||||||
List<Person> aList = ((FieldList) a).stringList().stream().map(author -> new Person(author, false)).collect(Collectors.toList());
|
List<Person> aList = ((FieldList) a).stringList().stream().map(author -> new Person(author, false)).collect(Collectors.toList());
|
||||||
|
|
|
@ -0,0 +1,53 @@
|
||||||
|
package eu.dnetlib.pace.tree;
|
||||||
|
|
||||||
|
import eu.dnetlib.pace.config.Config;
|
||||||
|
import eu.dnetlib.pace.model.Field;
|
||||||
|
import eu.dnetlib.pace.model.FieldList;
|
||||||
|
import eu.dnetlib.pace.model.FieldValueImpl;
|
||||||
|
import eu.dnetlib.pace.model.Person;
|
||||||
|
import eu.dnetlib.pace.tree.support.AbstractComparator;
|
||||||
|
import eu.dnetlib.pace.tree.support.ComparatorClass;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ComparatorClass("cosineSimilarity")
|
||||||
|
public class CosineSimilarity extends AbstractComparator {
|
||||||
|
|
||||||
|
Map<String, String> params;
|
||||||
|
|
||||||
|
public CosineSimilarity(Map<String,String> params) {
|
||||||
|
super(params);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double compare(final Field a, final Field b, final Config conf) {
|
||||||
|
|
||||||
|
if (a.isEmpty() || b.isEmpty())
|
||||||
|
return -1;
|
||||||
|
|
||||||
|
double[] aVector = ((FieldValueImpl) a).doubleArrayValue();
|
||||||
|
double[] bVector = ((FieldValueImpl) b).doubleArrayValue();
|
||||||
|
|
||||||
|
return cosineSimilarity(aVector, bVector);
|
||||||
|
}
|
||||||
|
|
||||||
|
double cosineSimilarity(double[] a, double[] b) {
|
||||||
|
double dotProduct = 0;
|
||||||
|
double normASum = 0;
|
||||||
|
double normBSum = 0;
|
||||||
|
|
||||||
|
for(int i = 0; i < a.length; i ++) {
|
||||||
|
dotProduct += a[i] * b[i];
|
||||||
|
normASum += a[i] * a[i];
|
||||||
|
normBSum += b[i] * b[i];
|
||||||
|
}
|
||||||
|
|
||||||
|
double eucledianDist = Math.sqrt(normASum) * Math.sqrt(normBSum);
|
||||||
|
return dotProduct / eucledianDist;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
|
@ -19,9 +19,13 @@ public class StringListMatch extends AbstractComparator {
|
||||||
private static final Log log = LogFactory.getLog(StringListMatch.class);
|
private static final Log log = LogFactory.getLog(StringListMatch.class);
|
||||||
private Map<String, String> params;
|
private Map<String, String> params;
|
||||||
|
|
||||||
|
final private String TYPE; //percentage or count
|
||||||
|
|
||||||
public StringListMatch(final Map<String, String> params) {
|
public StringListMatch(final Map<String, String> params) {
|
||||||
super(params);
|
super(params);
|
||||||
this.params = params;
|
this.params = params;
|
||||||
|
|
||||||
|
TYPE = params.getOrDefault("type", "percentage");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -31,7 +35,7 @@ public class StringListMatch extends AbstractComparator {
|
||||||
final Set<String> pb = new HashSet<>(((FieldList) b).stringList());
|
final Set<String> pb = new HashSet<>(((FieldList) b).stringList());
|
||||||
|
|
||||||
if (pa.isEmpty() || pb.isEmpty()) {
|
if (pa.isEmpty() || pb.isEmpty()) {
|
||||||
return -1; //return undefined if one of the two lists of pids is empty
|
return -1; //return undefined if one of the two lists is empty
|
||||||
}
|
}
|
||||||
|
|
||||||
int incommon = Sets.intersection(pa, pb).size();
|
int incommon = Sets.intersection(pa, pb).size();
|
||||||
|
@ -41,7 +45,10 @@ public class StringListMatch extends AbstractComparator {
|
||||||
return 0.0;
|
return 0.0;
|
||||||
}
|
}
|
||||||
|
|
||||||
return (double)incommon / (incommon + simDiff);
|
if(TYPE.equals("percentage"))
|
||||||
|
return (double)incommon / (incommon + simDiff);
|
||||||
|
else
|
||||||
|
return incommon;
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -244,6 +244,5 @@ public class BlockProcessorForTesting {
|
||||||
final String type = dedupConf.getWf().getEntityType();
|
final String type = dedupConf.getWf().getEntityType();
|
||||||
|
|
||||||
context.emit(type, from, to);
|
context.emit(type, from, to);
|
||||||
context.emit(type, to, from);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,12 +7,10 @@ import com.jayway.jsonpath.JsonPath;
|
||||||
import com.jayway.jsonpath.Option;
|
import com.jayway.jsonpath.Option;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import eu.dnetlib.pace.config.Type;
|
import eu.dnetlib.pace.config.Type;
|
||||||
import eu.dnetlib.pace.model.Field;
|
import eu.dnetlib.pace.model.*;
|
||||||
import eu.dnetlib.pace.model.FieldListImpl;
|
|
||||||
import eu.dnetlib.pace.model.FieldValueImpl;
|
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
|
||||||
import net.minidev.json.JSONArray;
|
import net.minidev.json.JSONArray;
|
||||||
|
|
||||||
|
import java.math.BigDecimal;
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -46,6 +44,14 @@ public class MapDocumentUtil {
|
||||||
.forEach(fi::add);
|
.forEach(fi::add);
|
||||||
stringField.put(fdef.getName(), fi);
|
stringField.put(fdef.getName(), fi);
|
||||||
break;
|
break;
|
||||||
|
case DoubleArray:
|
||||||
|
stringField.put(
|
||||||
|
fdef.getName(),
|
||||||
|
new FieldValueImpl(Type.DoubleArray,
|
||||||
|
fdef.getName(),
|
||||||
|
getJPathArray(fdef.getPath(), json))
|
||||||
|
);
|
||||||
|
break;
|
||||||
case StringConcat:
|
case StringConcat:
|
||||||
String[] jpaths = fdef.getPath().split("\\|\\|\\|");
|
String[] jpaths = fdef.getPath().split("\\|\\|\\|");
|
||||||
stringField.put(
|
stringField.put(
|
||||||
|
@ -115,6 +121,30 @@ public class MapDocumentUtil {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static double[] getJPathArray(final String jsonPath, final String json) {
|
||||||
|
try {
|
||||||
|
Object o = JsonPath.read(json, jsonPath);
|
||||||
|
if (o instanceof double[])
|
||||||
|
return (double[]) o;
|
||||||
|
if (o instanceof JSONArray) {
|
||||||
|
Object[] objects = ((JSONArray) o).toArray();
|
||||||
|
double[] array = new double[objects.length];
|
||||||
|
for (int i = 0; i < objects.length; i++) {
|
||||||
|
if (objects[i] instanceof BigDecimal)
|
||||||
|
array[i] = ((BigDecimal)objects[i]).doubleValue();
|
||||||
|
else
|
||||||
|
array[i] = (double) objects[i];
|
||||||
|
}
|
||||||
|
return array;
|
||||||
|
}
|
||||||
|
return new double[0];
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
return new double[0];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public static String truncateValue(String value, int length) {
|
public static String truncateValue(String value, int length) {
|
||||||
if (value == null)
|
if (value == null)
|
||||||
|
|
|
@ -36,6 +36,10 @@ public abstract class AbstractPaceTest extends AbstractPaceFunctions {
|
||||||
return new FieldValueImpl(Type.URL, "url", s);
|
return new FieldValueImpl(Type.URL, "url", s);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected Field array(final double[] a) {
|
||||||
|
return new FieldValueImpl(Type.DoubleArray, "array", a);
|
||||||
|
}
|
||||||
|
|
||||||
protected Field createFieldList(List<String> strings, String fieldName){
|
protected Field createFieldList(List<String> strings, String fieldName){
|
||||||
|
|
||||||
List<FieldValueImpl> fieldValueStream = strings.stream().map(s -> new FieldValueImpl(Type.String, fieldName, s)).collect(Collectors.toList());
|
List<FieldValueImpl> fieldValueStream = strings.stream().map(s -> new FieldValueImpl(Type.String, fieldName, s)).collect(Collectors.toList());
|
||||||
|
|
|
@ -2,7 +2,9 @@ package eu.dnetlib.pace.comparators;
|
||||||
|
|
||||||
import eu.dnetlib.pace.AbstractPaceTest;
|
import eu.dnetlib.pace.AbstractPaceTest;
|
||||||
import eu.dnetlib.pace.clustering.NGramUtils;
|
import eu.dnetlib.pace.clustering.NGramUtils;
|
||||||
|
import eu.dnetlib.pace.config.Type;
|
||||||
import eu.dnetlib.pace.model.Field;
|
import eu.dnetlib.pace.model.Field;
|
||||||
|
import eu.dnetlib.pace.model.FieldValueImpl;
|
||||||
import eu.dnetlib.pace.tree.*;
|
import eu.dnetlib.pace.tree.*;
|
||||||
import eu.dnetlib.pace.config.DedupConfig;
|
import eu.dnetlib.pace.config.DedupConfig;
|
||||||
|
|
||||||
|
@ -284,5 +286,18 @@ public class ComparatorTest extends AbstractPaceTest {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void cosineSimilarity() {
|
||||||
|
|
||||||
|
CosineSimilarity cosineSimilarity = new CosineSimilarity(params);
|
||||||
|
|
||||||
|
Field a = new FieldValueImpl(Type.DoubleArray, "array", new double[]{1,2,3});
|
||||||
|
Field b = new FieldValueImpl(Type.DoubleArray, "array", new double[]{1,2,3});
|
||||||
|
|
||||||
|
double compare = cosineSimilarity.compare(a, b, conf);
|
||||||
|
|
||||||
|
System.out.println("compare = " + compare);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,6 +7,7 @@ import eu.dnetlib.pace.clustering.ClusteringClass;
|
||||||
import eu.dnetlib.pace.clustering.ClusteringCombiner;
|
import eu.dnetlib.pace.clustering.ClusteringCombiner;
|
||||||
import eu.dnetlib.pace.model.Field;
|
import eu.dnetlib.pace.model.Field;
|
||||||
import eu.dnetlib.pace.model.FieldList;
|
import eu.dnetlib.pace.model.FieldList;
|
||||||
|
import eu.dnetlib.pace.model.FieldValue;
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
import eu.dnetlib.pace.tree.JsonListMatch;
|
import eu.dnetlib.pace.tree.JsonListMatch;
|
||||||
import eu.dnetlib.pace.tree.support.AggType;
|
import eu.dnetlib.pace.tree.support.AggType;
|
||||||
|
@ -20,10 +21,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.*;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
|
||||||
|
@ -104,15 +102,15 @@ public class ConfigTest extends AbstractPaceTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void asMapDocumentTest2() {
|
public void authorAsMapDocument() {
|
||||||
|
|
||||||
DedupConfig dedupConf = DedupConfig.load(readFromClasspath("author.test.conf.json"));
|
DedupConfig dedupConf = DedupConfig.load(readFromClasspath("author.fdup.conf.json"));
|
||||||
|
|
||||||
final String json = readFromClasspath("author.json");
|
final String json = readFromClasspath("author.json");
|
||||||
|
|
||||||
final MapDocument mapDocument = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, json);
|
final MapDocument mapDocument = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, json);
|
||||||
|
|
||||||
System.out.println("mapDocument = " + mapDocument.getFieldMap().get("coauthors").stringValue());
|
System.out.println("mapDocument = " + Arrays.toString(((FieldValue) mapDocument.getFieldMap().get("topics")).doubleArrayValue()));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -29,71 +29,103 @@
|
||||||
},
|
},
|
||||||
"pace": {
|
"pace": {
|
||||||
"clustering" : [
|
"clustering" : [
|
||||||
{ "name" : "personClustering", "fields" : [ "fullname" ], "params" : {} },
|
{ "name" : "lnfi", "fields" : [ "fullname" ], "params" : {} }
|
||||||
{ "name" : "personHash", "fields" : [ "fullname" ], "params" : {} }
|
|
||||||
],
|
],
|
||||||
"decisionTree": {
|
"decisionTree": {
|
||||||
"start": {
|
"start": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "year",
|
"field": "orcid",
|
||||||
"comparator": "numbersComparator",
|
"comparator": "exactMatch",
|
||||||
"weight": 1,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {}
|
"params": {}
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold": 50,
|
"threshold": 1.0,
|
||||||
"aggregation": "MAX",
|
"aggregation": "MAX",
|
||||||
"positive": "NO_MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "surnames",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "surnames",
|
"undefined": "orcids",
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "true"
|
||||||
},
|
},
|
||||||
"surnames": {
|
"orcids": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "orcids",
|
||||||
|
"comparator": "stringListMatch",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {"type": "count"}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 3.0,
|
||||||
|
"aggregation": "MAX",
|
||||||
|
"positive": "MATCH",
|
||||||
|
"negative": "coauthors",
|
||||||
|
"undefined": "coauthors",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"coauthors": {
|
||||||
"fields": [
|
"fields": [
|
||||||
{
|
{
|
||||||
"field": "coauthors",
|
"field": "coauthors",
|
||||||
"comparator": "authorsMatch",
|
"comparator": "authorsMatch",
|
||||||
"weight": 1.0,
|
"weight": 1.0,
|
||||||
"countIfUndefined": "false",
|
"countIfUndefined": "true",
|
||||||
"params": {
|
"params": {"type": "count"}
|
||||||
"surname_th": 0.75,
|
|
||||||
"fullname_th": 0.75,
|
|
||||||
"size_th": 20,
|
|
||||||
"mode": "surname"
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"threshold": 0.6,
|
"threshold": 1.0,
|
||||||
|
"aggregation": "MAX",
|
||||||
|
"positive": "topicsMatch",
|
||||||
|
"negative": "NO_MATCH",
|
||||||
|
"undefined": "topicsMatch",
|
||||||
|
"ignoreUndefined": "true"
|
||||||
|
},
|
||||||
|
"topicsMatch": {
|
||||||
|
"fields": [
|
||||||
|
{
|
||||||
|
"field": "topics",
|
||||||
|
"comparator": "cosineSimilarity",
|
||||||
|
"weight": 1.0,
|
||||||
|
"countIfUndefined": "true",
|
||||||
|
"params": {}
|
||||||
|
}
|
||||||
|
],
|
||||||
|
"threshold": 1.0,
|
||||||
"aggregation": "MAX",
|
"aggregation": "MAX",
|
||||||
"positive": "MATCH",
|
"positive": "MATCH",
|
||||||
"negative": "NO_MATCH",
|
"negative": "NO_MATCH",
|
||||||
"undefined": "MATCH",
|
"undefined": "NO_MATCH",
|
||||||
"ignoreUndefined": "true"
|
"ignoreUndefined": "false"
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
"model": [
|
"model": [
|
||||||
|
{
|
||||||
|
"name": "topics",
|
||||||
|
"type": "DoubleArray",
|
||||||
|
"path": "$.topics"
|
||||||
|
},
|
||||||
{
|
{
|
||||||
"name": "fullname",
|
"name": "fullname",
|
||||||
"type": "String",
|
"type": "String",
|
||||||
"path": "$.name"
|
"path": "$.fullname"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name": "orcid",
|
||||||
|
"type": "String",
|
||||||
|
"path": "$.orcid"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "coauthors",
|
"name": "coauthors",
|
||||||
"type": "List",
|
"type": "List",
|
||||||
"path": "$.coauthors[*].name",
|
"path": "$.coAuthors[*].fullname"
|
||||||
"size": 200
|
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"name": "year",
|
"name": "orcids",
|
||||||
"type": "String",
|
"type": "List",
|
||||||
"path": "$.publication.year"
|
"path": "$.coAuthors[*].orcid"
|
||||||
},
|
|
||||||
{
|
|
||||||
"name": "title",
|
|
||||||
"type": "String",
|
|
||||||
"path": "$.publication.title"
|
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
"blacklists": {},
|
"blacklists": {},
|
|
@ -1 +1 @@
|
||||||
{"id": "f3389e7c8af1d806c06e2ab51f28a4b4", "name": "Aczél, János", "shortname": "Aczél, J.", "pid": "aczel.janos", "coauthors": [], "publication": {"year": "1955", "title": "L\\\"osung der Vektor-Funktionalgleichung der homogenen und inhomogenen $n$-dimensionalen einparametrigen ``Translation'' der erzeugenden Funktion von Kettenreaktionen und des station\\\"aren und nichtstation\\\"aren Bewegungsintegrals", "venue": "Acta Math. Acad. Sci. Hung. 6, 131-140 (1955)."}}
|
{"fullname":"Zaragoza, Maria Cleofé","firstname":"Maria Cleofé","lastname":"Zaragoza","coAuthors":[{"fullname":"Cambras, Trinitat","lastname":"Cambras","firstname":"Trinitat","orcid":"0000-0002-9009-4690"},{"fullname":"Castro-Marrero, Jesús","lastname":"Castro-Marrero","firstname":"Jesús","orcid":""},{"fullname":"Díez-Noguera, Antoni","lastname":"Díez-Noguera","firstname":"Antoni","orcid":""},{"fullname":"Alegre, José","lastname":"Alegre","firstname":"José","orcid":"0000-0002-7582-7585"}],"topics":[0.9522090839562252,0.04779091604377485],"orcid":"0000-0002-9797-0219","id":"author::1a10826c83c7f9f0dcebe7df05e37a2a","pubId":"50|pmid________::db7fd19db5a620eafad40cfb97f9690d"}
|
Loading…
Reference in New Issue