From cc63706347ff65fc1c57c63a673175473680770f Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Fri, 6 Dec 2019 13:38:00 +0100 Subject: [PATCH 01/13] Implemented deduplication on spark --- .../transformation/TransformationJobTest.java | 4 +- dhp-workflows/dhp-dedup/pom.xml | 61 + .../java/eu/dnetlib/dedup/DedupUtility.java | 94 + .../main/java/eu/dnetlib/dedup/Deduper.java | 159 ++ .../dedup/SparkCreateConnectedComponent.java | 80 + .../dnetlib/dedup/SparkCreateDedupRecord.java | 101 + .../eu/dnetlib/dedup/SparkCreateSimRels.java | 94 + .../java/eu/dnetlib/dedup/SparkReporter.java | 47 + .../dedup/graph/ConnectedComponent.java | 80 + .../dnetlib/dedup/graph/GraphProcessor.scala | 39 + .../dnetlib/dhp/dedup/conf/org.curr.conf.json | 1745 ++++++++++++++++ .../dhp/dedup/conf/org.curr.conf2.json | 1753 +++++++++++++++++ .../dnetlib/dhp/dedup/conf/pub.curr.conf.json | 280 +++ .../dnetlib/dhp/dedup/dedup_parameters.json | 7 + .../dhp/dedup/oozie_app/config-default.xml | 31 + .../dnetlib/dhp/dedup/oozie_app/workflow.xml | 101 + .../dnetlib/dedup/SparkCreateDedupTest.java | 62 + .../eu/dnetlib/dedup/jpath/JsonPathTest.java | 31 + .../eu/dnetlib/dedup/conf/org.curr.conf.json | 1753 +++++++++++++++++ .../eu/dnetlib/dedup/conf/sample.json | 1 + dhp-workflows/pom.xml | 2 + pom.xml | 79 +- 22 files changed, 6582 insertions(+), 22 deletions(-) create mode 100644 dhp-workflows/dhp-dedup/pom.xml create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkReporter.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java create mode 100644 dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/jpath/JsonPathTest.java create mode 100644 dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json create mode 100644 dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/sample.json diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java index d517cca00..5e5e42f1e 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/TransformationJobTest.java @@ -89,6 +89,8 @@ public class TransformationJobTest { "-rh", "", "-ro", "", "-rr", ""}); + + } @Test @@ -96,7 +98,7 @@ public class TransformationJobTest { final String path = this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); System.out.println("path = " + path); - Path tempDirWithPrefix = Files.createTempDirectory("mdsotre_output"); + Path tempDirWithPrefix = Files.createTempDirectory("mdstore_output"); System.out.println(tempDirWithPrefix.toFile().getAbsolutePath()); diff --git a/dhp-workflows/dhp-dedup/pom.xml b/dhp-workflows/dhp-dedup/pom.xml new file mode 100644 index 000000000..28ef6a453 --- /dev/null +++ b/dhp-workflows/dhp-dedup/pom.xml @@ -0,0 +1,61 @@ + + + + dhp-workflows + eu.dnetlib.dhp + 1.0.5-SNAPSHOT + + 4.0.0 + + dhp-dedup + + + + + org.apache.spark + spark-core_2.11 + + + org.apache.spark + spark-sql_2.11 + + + + eu.dnetlib.dhp + dhp-common + ${project.version} + + + eu.dnetlib.dhp + dhp-schemas + ${project.version} + + + com.arakelian + java-jq + + + + eu.dnetlib + dnet-pace-core + + + org.apache.spark + spark-graphx_2.11 + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java new file mode 100644 index 000000000..b65e866f1 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java @@ -0,0 +1,94 @@ +package eu.dnetlib.dedup; + +import com.google.common.collect.Sets; +import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.util.LongAccumulator; + +import java.io.IOException; +import java.io.StringWriter; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class DedupUtility { + + public static Map constructAccumulator(final DedupConfig dedupConf, final SparkContext context) { + + Map accumulators = new HashMap<>(); + + String acc1 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "records per hash key = 1"); + accumulators.put(acc1, context.longAccumulator(acc1)); + String acc2 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField()); + accumulators.put(acc2, context.longAccumulator(acc2)); + String acc3 = String.format("%s::%s",dedupConf.getWf().getEntityType(), String.format("Skipped records for count(%s) >= %s", dedupConf.getWf().getOrderField(), dedupConf.getWf().getGroupMaxSize())); + accumulators.put(acc3, context.longAccumulator(acc3)); + String acc4 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "skip list"); + accumulators.put(acc4, context.longAccumulator(acc4)); + String acc5 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)"); + accumulators.put(acc5, context.longAccumulator(acc5)); + String acc6 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold()); + accumulators.put(acc6, context.longAccumulator(acc6)); + + return accumulators; + } + + public static JavaRDD loadDataFromHDFS(String path, JavaSparkContext context) { + return context.textFile(path); + } + + public static void deleteIfExists(String path) throws IOException { + Configuration conf = new Configuration(); + FileSystem fileSystem = FileSystem.get(conf); + if (fileSystem.exists(new Path(path))){ + fileSystem.delete(new Path(path), true); + } + } + + public static DedupConfig loadConfigFromHDFS(String path) throws IOException { + + Configuration conf = new Configuration(); + FileSystem fileSystem = FileSystem.get(conf); + FSDataInputStream inputStream = new FSDataInputStream(fileSystem.open(new Path(path))); + + return DedupConfig.load(IOUtils.toString(inputStream, StandardCharsets.UTF_8.name())); + + } + + static String readFromClasspath(final String filename, final Class clazz) { + final StringWriter sw = new StringWriter(); + try { + IOUtils.copy(clazz.getResourceAsStream(filename), sw); + return sw.toString(); + } catch (final IOException e) { + throw new RuntimeException("cannot load resource from classpath: " + filename); + } + } + + static Set getGroupingKeys(DedupConfig conf, MapDocument doc) { + return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf)); + } + + public static String md5(final String s) { + try { + final MessageDigest md = MessageDigest.getInstance("MD5"); + md.update(s.getBytes("UTF-8")); + return new String(Hex.encodeHex(md.digest())); + } catch (final Exception e) { + System.err.println("Error creating id"); + return null; + } + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java new file mode 100644 index 000000000..51b991da5 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java @@ -0,0 +1,159 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.Field; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.BlockProcessor; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.util.LongAccumulator; +import scala.Serializable; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +public class Deduper implements Serializable { + + private static final Log log = LogFactory.getLog(Deduper.class); + + /** + * @return the list of relations generated by the deduplication + * @param: the spark context + * @param: list of JSON entities to be deduped + * @param: the dedup configuration + */ + public static JavaPairRDD dedup(JavaSparkContext context, JavaRDD entities, DedupConfig config) { + + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); + + //create vertexes of the graph: + JavaPairRDD mapDocs = mapToVertexes(context, entities, config); + + + //create blocks for deduplication + JavaPairRDD> blocks = createBlocks(context, mapDocs, config); + + //create relations by comparing only elements in the same group + return computeRelations(context, blocks, config); + +// final RDD> edgeRdd = relationRDD.map(it -> new Edge<>(it._1().hashCode(), it._2().hashCode(), "equalTo")).rdd(); +// +// RDD> vertexes = mapDocs.mapToPair((PairFunction, Object, MapDocument>) t -> new Tuple2((long) t._1().hashCode(), t._2())).rdd(); +// accumulators.forEach((name, acc) -> log.info(name + " -> " + acc.value())); +// +// return GraphProcessor.findCCs(vertexes, edgeRdd, 20).toJavaRDD(); + } + + /** + * @return the list of relations generated by the deduplication + * @param: the spark context + * @param: list of blocks + * @param: the dedup configuration + */ + public static JavaPairRDD computeRelations(JavaSparkContext context, JavaPairRDD> blocks, DedupConfig config) { + + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); + + return blocks.flatMapToPair((PairFlatMapFunction>, String, String>) it -> { + final SparkReporter reporter = new SparkReporter(accumulators); + new BlockProcessor(config).process(it._1(), it._2(), reporter); + return reporter.getRelations().iterator(); + + }).mapToPair( + (PairFunction, String, Tuple2>) item -> + new Tuple2>(item._1() + item._2(), item)) + .reduceByKey((a, b) -> a) + .mapToPair((PairFunction>, String, String>) Tuple2::_2); + } + + + /** + * @return the list of blocks based on clustering of dedup configuration + * @param: the spark context + * @param: list of entities: + * @param: the dedup configuration + */ + public static JavaPairRDD> createBlocks(JavaSparkContext context, JavaPairRDD mapDocs, DedupConfig config) { + return mapDocs + //the reduce is just to be sure that we haven't document with same id + .reduceByKey((a, b) -> a) + .map(Tuple2::_2) + //Clustering: from to List + .flatMapToPair((PairFlatMapFunction) a -> + DedupUtility.getGroupingKeys(config, a) + .stream() + .map(it -> new Tuple2<>(it, a)) + .collect(Collectors.toList()) + .iterator()) + .groupByKey(); + } + + + public static JavaPairRDD> createsortedBlocks(JavaSparkContext context, JavaPairRDD mapDocs, DedupConfig config) { + final String of = config.getWf().getOrderField(); + final int maxQueueSize = config.getWf().getGroupMaxSize(); + return mapDocs + //the reduce is just to be sure that we haven't document with same id + .reduceByKey((a, b) -> a) + .map(Tuple2::_2) + //Clustering: from to List + .flatMapToPair((PairFlatMapFunction>) a -> + DedupUtility.getGroupingKeys(config, a) + .stream() + .map(it -> { + List tmp = new ArrayList<>(); + tmp.add(a); + return new Tuple2<>(it, tmp); + } + ) + .collect(Collectors.toList()) + .iterator()) + .reduceByKey((Function2, List, List>) (v1, v2) -> { + v1.addAll(v2); + v1.sort(Comparator.comparing(a -> a.getFieldMap().get(of).stringValue())); + if (v1.size()> maxQueueSize) + return new ArrayList<>(v1.subList(0, maxQueueSize)); + return v1; + }); + } + + /** + * @return the list of vertexes: + * @param: the spark context + * @param: list of JSON entities + * @param: the dedup configuration + */ + public static JavaPairRDD mapToVertexes(JavaSparkContext context, JavaRDD entities, DedupConfig config) { + + return entities.mapToPair((PairFunction) s -> { + + MapDocument mapDocument = MapDocumentUtil.asMapDocumentWithJPath(config, s); + return new Tuple2(mapDocument.getIdentifier(), mapDocument); + + + }); + } + + public static JavaPairRDD computeRelations2(JavaSparkContext context, JavaPairRDD> blocks, DedupConfig config) { + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); + + return blocks.flatMapToPair((PairFlatMapFunction>, String, String>) it -> { + final SparkReporter reporter = new SparkReporter(accumulators); + new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter); + return reporter.getRelations().iterator(); + + }).mapToPair( + (PairFunction, String, Tuple2>) item -> + new Tuple2>(item._1() + item._2(), item)) + .reduceByKey((a, b) -> a) + .mapToPair((PairFunction>, String, String>) Tuple2::_2); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java new file mode 100644 index 000000000..182bb374a --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -0,0 +1,80 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.dedup.graph.ConnectedComponent; +import eu.dnetlib.dedup.graph.GraphProcessor; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.io.IOUtils; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.graphx.Edge; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.List; + +public class SparkCreateConnectedComponent { + + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateConnectedComponent.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCreateConnectedComponent.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String entity = parser.get("entity"); + final String targetPath = parser.get("targetPath"); + final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateConnectedComponent.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + + + final JavaPairRDD vertexes = sc.textFile(inputPath + "/" + entity) + .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) + .mapToPair((PairFunction) + s -> new Tuple2((long) s.hashCode(), s) + ); + + final Dataset similarityRelations = spark.read().load(targetPath + "/" + entity+"_simrel").as(Encoders.bean(Relation.class)); + + + final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(it.getSource().hashCode(), it.getTarget().hashCode(), it.getRelClass())).rdd(); + + + final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, 20).toJavaRDD(); + + + final Dataset mergeRelation = spark.createDataset(cc.filter(k->k.getDocIds().size()>1).flatMap((FlatMapFunction) c -> + c.getDocIds() + .stream() + .flatMap(id -> { + List tmp = new ArrayList<>(); + Relation r = new Relation(); + r.setSource(c.getCcId()); + r.setTarget(id); + r.setRelClass("merges"); + tmp.add(r); + r = new Relation(); + r.setTarget(c.getCcId()); + r.setSource(id); + r.setRelClass("isMergedIn"); + tmp.add(r); + return tmp.stream(); + }).iterator()).rdd(), Encoders.bean(Relation.class)); + + mergeRelation.write().mode("overwrite").save(targetPath+"/"+entity+"_mergeRels"); + + + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java new file mode 100644 index 000000000..6d6165b18 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java @@ -0,0 +1,101 @@ +package eu.dnetlib.dedup; + +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Lists; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.io.IOUtils; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +public class SparkCreateDedupRecord { + + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCreateDedupRecord.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String entity = parser.get("entity"); + final String targetPath = parser.get("targetPath"); + final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + + final JavaPairRDD inputJsonEntities = sc.textFile(inputPath + "/" + entity) + .mapToPair((PairFunction)it-> + new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) + ); + + +// JavaPairRDD mergeRels = spark +// .read().load(targetPath + "/" + entity+"_mergeRels").as(Encoders.bean(Relation.class)) +// .where("relClass=='merges'") +// .javaRDD() +// .mapToPair( +// (PairFunction)r-> +// new Tuple2(r.getTarget(), r.getSource()) +// ); +// +// +// final JavaPairRDD p = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); +// +// Comparator c = new Comparator() { +// @Override +// public int compare(String s, String t1) { +// return 0; +// } +// }; +// final JavaPairRDD stringStringJavaPairRDD = p.repartitionAndSortWithinPartitions(p.partitioner().get(), c); + + +// List inputValues = Arrays.asList( +// new Foo("k",5), +// new Foo("a",1), +// new Foo("a",30), +// new Foo("a",18), +// new Foo("a",22), +// new Foo("b",22), +// new Foo("c",5), +// new Foo("a",5), +// new Foo("s",1), +// new Foo("h",4) +// ); +// +// +// final JavaPairRDD fooFighters = sc.parallelize(inputValues).mapToPair((PairFunction) i -> new Tuple2(i, i)); +// +// +// FooComparator c = new FooComparator(); +// final List>> result = +// fooFighters.repartitionAndSortWithinPartitions(new FooPartitioner(fooFighters.getNumPartitions()), c) +// .mapToPair((PairFunction, String, Foo>) t-> new Tuple2(t._1().getValue(), t._2()) ) +// .groupByKey() +// .mapValues((Function, List>) Lists::newArrayList) +// .collect(); +// +// +// System.out.println(result); + + } + +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java new file mode 100644 index 000000000..a7aa18f89 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java @@ -0,0 +1,94 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.dedup.graph.ConnectedComponent; +import eu.dnetlib.dedup.graph.GraphProcessor; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.io.IOUtils; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.graphx.Edge; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.List; + + +/** + * This Spark class creates similarity relations between entities, saving result + * + * param request: + * sourcePath + * entityType + * target Path + */ +public class SparkCreateSimRels { + + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCreateSimRels.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String entity = parser.get("entity"); + final String targetPath = parser.get("targetPath"); + final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + + + final long total = sc.textFile(inputPath + "/" + entity).count(); + + JavaPairRDD vertexes = sc.textFile(inputPath + "/" + entity) + .map(s->{ + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf,s); + return new Tuple2<>(d.getIdentifier(), d);}) + .mapToPair((PairFunction, Object, MapDocument>) t -> new Tuple2((long) t._1().hashCode(), t._2())); + + + + + JavaPairRDD mapDocument = vertexes.mapToPair((PairFunction, String, MapDocument>) item -> new Tuple2(item._2().getIdentifier(), item._2())); + + //create blocks for deduplication + JavaPairRDD> blocks = Deduper.createsortedBlocks(sc,mapDocument, dedupConf); + + + //create relations by comparing only elements in the same group + final JavaPairRDD dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf); + + + final JavaRDD isSimilarToRDD = dedupRels.map(simRel -> { + final Relation r = new Relation(); + r.setSource(simRel._1()); + r.setTarget(simRel._2()); + r.setRelClass("isSimilarTo"); + return r; + }); + + spark.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)).write().mode("overwrite").save(targetPath+"/"+entity+"_simrel"); + + + + + + + + + + + } + + + +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkReporter.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkReporter.java new file mode 100644 index 000000000..165a10b25 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkReporter.java @@ -0,0 +1,47 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.pace.util.Reporter; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.spark.util.LongAccumulator; +import scala.Serializable; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class SparkReporter implements Serializable, Reporter { + + final List> relations = new ArrayList<>(); + private static final Log log = LogFactory.getLog(SparkReporter.class); + Map accumulators; + + public SparkReporter(Map accumulators){ + this.accumulators = accumulators; + } + + public void incrementCounter(String counterGroup, String counterName, long delta, Map accumulators) { + + final String accumulatorName = String.format("%s::%s", counterGroup, counterName); + if (accumulators.containsKey(accumulatorName)){ + accumulators.get(accumulatorName).add(delta); + } + + } + + @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) { + relations.add(new Tuple2<>(from, to)); + } + + public List> getRelations() { + return relations; + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java new file mode 100644 index 000000000..27a61c02d --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java @@ -0,0 +1,80 @@ +package eu.dnetlib.dedup.graph; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dedup.DedupUtility; +import eu.dnetlib.pace.util.PaceException; +import org.apache.commons.lang.StringUtils; +import org.codehaus.jackson.annotate.JsonIgnore; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Set; + +public class ConnectedComponent implements Serializable { + + private Set docIds; + private String ccId; + + + public ConnectedComponent() { + } + + public ConnectedComponent(Set docIds) { + this.docIds = docIds; + createID(); + } + + public String createID() { + if (docIds.size() > 1) { + final String s = getMin(); + String prefix = s.split("\\|")[0]; + ccId =prefix + "|dedup_______::" + DedupUtility.md5(s); + return ccId; + } else { + return docIds.iterator().next(); + } + } + + @JsonIgnore + public String getMin(){ + + final StringBuilder min = new StringBuilder(); + docIds.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 + public String toString(){ + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.writeValueAsString(this); + } catch (IOException e) { + throw new PaceException("Failed to create Json: ", e); + } + } + + public Set getDocIds() { + return docIds; + } + + public void setDocIds(Set docIds) { + this.docIds = docIds; + } + + public String getCcId() { + return ccId; + } + + public void setCcId(String ccId) { + this.ccId = ccId; + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala new file mode 100644 index 000000000..ab2be9170 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala @@ -0,0 +1,39 @@ +package eu.dnetlib.dedup.graph + + +import eu.dnetlib.pace.model.MapDocument +import org.apache.spark.graphx._ +import org.apache.spark.rdd.RDD + +import scala.collection.JavaConversions; + +object GraphProcessor { + + def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int): RDD[ConnectedComponent] = { + val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby + val cc = graph.connectedComponents(maxIterations).vertices + + val joinResult = vertexes.leftOuterJoin(cc).map { + case (id, (openaireId, cc)) => { + if (cc.isEmpty) { + (id, openaireId) + } + else { + (cc.get, openaireId) + } + } + } + val connectedComponents = joinResult.groupByKey() + .map[ConnectedComponent](cc => asConnectedComponent(cc)) + (connectedComponents) + } + + + + def asConnectedComponent(group: (VertexId, Iterable[String])): ConnectedComponent = { + val docs = group._2.toSet[String] + val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs)); + connectedComponent + } + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json new file mode 100644 index 000000000..a878d2419 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json @@ -0,0 +1,1745 @@ +{ + "wf": { + "threshold": "0.9", + "dedupRun": "001", + "entityType": "organization", + "orderField": "legalname", + "queueMaxSize": "2000", + "groupMaxSize": "50", + "slidingWindowSize": "200", + "idPath": ".id", + "rootBuilder": [ + "organization", + "projectOrganization_participation_isParticipant", + "datasourceOrganization_provision_isProvidedBy" + ], + "includeChildren": "true" + }, + "pace": { + "clustering": [ + { + "name": "sortedngrampairs", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "ngramLen": "3" + } + }, + { + "name": "suffixprefix", + "fields": [ + "legalname" + ], + "params": { + "max": 1, + "len": "3" + } + }, + { + "name": "urlclustering", + "fields": [ + "websiteurl" + ], + "params": {} + }, + { + "name": "keywordsclustering", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "windowSize": 4 + } + } + ], + "strictConditions": [ + { + "name": "exactMatch", + "fields": [ + "gridid" + ] + } + ], + "conditions": [ + { + "name": "DomainExactMatch", + "fields": [ + "websiteurl" + ] + }, + { + "name": "exactMatch", + "fields": [ + "country" + ] + } + ], + "model": [ + { + "name": "country", + "algo": "Null", + "type": "String", + "weight": "0", + "ignoreMissing": "false", + "path": ".country.classid" + }, + { + "name": "legalshortname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.1", + "ignoreMissing": "true", + "path": ".legalshortname.value" + }, + { + "name": "legalname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.9", + "ignoreMissing": "false", + "path": ".legalname.value", + "params": { + "windowSize": 4, + "threshold": 0.7 + } + }, + { + "name": "websiteurl", + "algo": "Null", + "type": "URL", + "weight": "0", + "ignoreMissing": "true", + "path": ".websiteurl.value", + "params": { + "host": 0.5, + "path": 0.5 + } + } + ], + "blacklists": { + "legalname": [] + }, + "synonyms": { + "key::1": [ + "university", + "università", + "università studi", + "universitario", + "universitaria", + "université", + "universitaire", + "universitaires", + "universidad", + "universitade", + "Universität", + "universitaet", + "Uniwersytet", + "университет", + "universiteit", + "πανεπιστήμιο", + "universitesi", + "universiteti" + ], + "key::2": [ + "studies", + "studi", + "études", + "estudios", + "estudos", + "Studien", + "studia", + "исследования", + "studies", + "σπουδές" + ], + "key::3": [ + "advanced", + "superiore", + "supérieur", + "supérieure", + "supérieurs", + "supérieures", + "avancado", + "avancados", + "fortgeschrittene", + "fortgeschritten", + "zaawansowany", + "передовой", + "gevorderd", + "gevorderde", + "προχωρημένος", + "προχωρημένη", + "προχωρημένο", + "προχωρημένες", + "προχωρημένα", + "wyzsza" + ], + "key::4": [ + "institute", + "istituto", + "institut", + "instituto", + "instituto", + "Institut", + "instytut", + "институт", + "instituut", + "ινστιτούτο" + ], + "key::5": [ + "hospital", + "ospedale", + "hôpital", + "hospital", + "hospital", + "Krankenhaus", + "szpital", + "больница", + "ziekenhuis", + "νοσοκομείο" + ], + "key::6": [ + "research", + "ricerca", + "recherche", + "investigacion", + "pesquisa", + "Forschung", + "badania", + "исследования", + "onderzoek", + "έρευνα", + "erevna", + "erevnas" + ], + "key::7": [ + "college", + "collegio", + "université", + "colegio", + "faculdade", + "Hochschule", + "Szkoła Wyższa", + "Высшая школа", + "universiteit", + "κολλέγιο" + ], + "key::8": [ + "foundation", + "fondazione", + "fondation", + "fundación", + "fundação", + "Stiftung", + "Fundacja", + "фонд", + "stichting", + "ίδρυμα", + "idryma" + ], + "key::9": [ + "center", + "centro", + "centre", + "centro", + "centro", + "zentrum", + "centrum", + "центр", + "centrum", + "κέντρο" + ], + "key::10": [ + "national", + "nazionale", + "national", + "nationale", + "nationaux", + "nationales", + "nacional", + "nacional", + "national", + "krajowy", + "национальный", + "nationaal", + "nationale", + "εθνικό" + ], + "key::11": [ + "association", + "associazione", + "association", + "asociación", + "associação", + "Verein", + "verband", + "stowarzyszenie", + "ассоциация", + "associatie" + ], + "key::12": [ + "society", + "societa", + "société", + "sociedad", + "sociedade", + "gesellschaft", + "społeczeństwo", + "общество", + "maatschappij", + "κοινωνία" + ], + "key::13": [ + "international", + "internazionale", + "international", + "internacional", + "internacional", + "international", + "międzynarodowy", + "Международный", + "internationaal", + "internationale", + "διεθνής", + "διεθνή", + "διεθνές" + ], + "key::14": [ + "community", + "comunita", + "communauté", + "comunidad", + "comunidade", + "Gemeinschaft", + "społeczność", + "сообщество", + "gemeenschap", + "κοινότητα" + ], + "key::15": [ + "school", + "scuola", + "école", + "escuela", + "escola", + "schule", + "Szkoła", + "школа", + "school", + "σχολείο" + ], + "key::16": [ + "education", + "educazione", + "éducation", + "educacion", + "Educação", + "Bildung", + "Edukacja", + "образование", + "opleiding", + "εκπαίδευση" + ], + "key::17": [ + "academy", + "accademia", + "académie", + "academia", + "academia", + "Akademie", + "akademie", + "академия", + "academie", + "ακαδημία" + ], + "key::18": [ + "public", + "pubblico", + "public", + "publique", + "publics", + "publiques", + "publico", + "publico", + "Öffentlichkeit", + "publiczny", + "публичный", + "publiek", + "publieke", + "δημόσιος", + "δημόσια", + "δημόσιο" + ], + "key::19": [ + "museum", + "museo", + "musée", + "mueso", + "museu", + "museum", + "muzeum", + "музей", + "museum", + "μουσείο" + ], + "key::20": [ + "group", + "gruppo", + "groupe", + "grupo", + "grupo", + "gruppe", + "grupa", + "группа", + "groep", + "ομάδα", + "όμιλος" + ], + "key::21": [ + "department", + "dipartimento", + "département", + "departamento", + "departamento", + "abteilung", + "departament", + "отдел", + "afdeling", + "τμήμα" + ], + "key::22": [ + "council", + "consiglio", + "conseil", + "Consejo", + "conselho", + "gesellschaft", + "rada", + "совет", + "raad", + "συμβούλιο" + ], + "key::23": [ + "library", + "biblioteca", + "bibliothèque", + "biblioteca", + "biblioteca", + "Bibliothek", + "biblioteka", + "библиотека", + "bibliotheek", + "βιβλιοθήκη" + ], + "key::24": [ + "ministry", + "ministero", + "ministère", + "ministerio", + "ministério", + "Ministerium", + "ministerstwo", + "министерство", + "ministerie", + "υπουργείο" + ], + "key::25": [ + "services", + "servizi", + "services", + "servicios", + "Serviços", + "Dienstleistungen", + "usługi", + "услуги", + "diensten", + "υπηρεσίες" + ], + "key::26": [ + "central", + "centrale", + "central", + "centrale", + "centrales", + "central", + "central", + "zentral", + "centralny", + "цетральный", + "centraal", + "κεντρικός", + "κεντρική", + "κεντρικό", + "κεντρικά" + ], + "key::27": [ + "general", + "generale", + "général", + "générale", + "généraux", + "générales", + "general", + "geral", + "general", + "Allgemeines", + "general", + "общий", + "algemeen", + "algemene", + "γενικός", + "γενική", + "γενικό", + "γενικά" + ], + "key::28": [ + "applied", + "applicati", + "appliqué", + "appliquée", + "appliqués", + "appliquées", + "aplicado", + "aplicada", + "angewendet", + "stosowany", + "прикладной", + "toegepast", + "toegepaste", + "εφαρμοσμένος", + "εφαρμοσμένη", + "εφαρμοσμένο", + "εφαρμοσμένα" + ], + "key::29": [ + "european", + "europee", + "europea", + "européen", + "européenne", + "européens", + "européennes", + "europeo", + "europeu", + "europäisch", + "europejski", + "европейский", + "Europees", + "Europese", + "ευρωπαϊκός", + "ευρωπαϊκή", + "ευρωπαϊκό", + "ευρωπαϊκά" + ], + "key::30": [ + "agency", + "agenzia", + "agence", + "agencia", + "agencia", + "agentur", + "agencja", + "агенция", + "agentschap", + "πρακτορείο" + ], + "key::31": [ + "laboratory", + "laboratorio", + "laboratoire", + "laboratorio", + "laboratorio", + "labor", + "laboratorium", + "лаборатория", + "laboratorium", + "εργαστήριο" + ], + "key::32": [ + "industry", + "industria", + "industrie", + "индустрия", + "industrie", + "βιομηχανία" + ], + "key::33": [ + "industrial", + "industriale", + "industriel", + "industrielle", + "industriels", + "industrielles", + "индустриальный", + "industrieel", + "βιομηχανικός", + "βιομηχανική", + "βιομηχανικό", + "βιομηχανικά", + "βιομηχανικές" + ], + "key::34": [ + "consortium", + "consorzio", + "consortium", + "консорциум", + "consortium", + "κοινοπραξία" + ], + "key::35": [ + "organization", + "organizzazione", + "organisation", + "organización", + "organização", + "organizacja", + "организация", + "organisatie", + "οργανισμός" + ], + "key::36": [ + "authority", + "autorità", + "autorité", + "авторитет", + "autoriteit" + ], + "key::37": [ + "federation", + "federazione", + "fédération", + "федерация", + "federatie", + "ομοσπονδία" + ], + "key::38": [ + "observatory", + "osservatorio", + "observatoire", + "обсерватория", + "observatorium", + "αστεροσκοπείο" + ], + "key::39": [ + "bureau", + "ufficio", + "bureau", + "офис", + "bureau", + "γραφείο" + ], + "key::40": [ + "company", + "impresa", + "compagnie", + "société", + "компания", + "bedrijf", + "εταιρία" + ], + "key::41": [ + "polytechnic", + "politecnico", + "polytechnique", + "политехника", + "polytechnisch", + "πολυτεχνείο", + "universita politecnica", + "polytechnic university", + "universidad politecnica", + "universitat politecnica", + "politechnika", + "politechniki", + "university technology", + "university science technology" + ], + "key::42": [ + "coalition", + "coalizione", + "coalition", + "коалиция", + "coalitie", + "συνασπισμός" + ], + "key::43": [ + "initiative", + "iniziativa", + "initiative", + "инициатива", + "initiatief", + "πρωτοβουλία" + ], + "key::44": [ + "academic", + "accademico", + "académique", + "universitaire", + "акадеческий academisch", + "ακαδημαϊκός", + "ακαδημαϊκή", + "ακαδημαϊκό", + "ακαδημαϊκές", + "ακαδημαϊκοί" + ], + "key::45": [ + "institution", + "istituzione", + "institution", + "институциональный", + "instelling", + "ινστιτούτο" + ], + "key::46": [ + "division", + "divisione", + "division", + "отделение", + "divisie", + "τμήμα" + ], + "key::47": [ + "committee", + "comitato", + "comité", + "комитет", + "commissie", + "επιτροπή" + ], + "key::48": [ + "promotion", + "promozione", + "продвижение", + "proothisis", + "forderung" + ], + "key::49": [ + "medical", + "medicine", + "clinical", + "medicina", + "clinici", + "médico", + "medicina", + "clínica", + "médico", + "medicina", + "clínica", + "medizinisch", + "Medizin", + "klinisch", + "medisch", + "geneeskunde", + "klinisch", + "ιατρικός", + "ιατρική", + "ιατρικό", + "ιατρικά", + "κλινικός", + "κλινική", + "κλινικό", + "κλινικά", + "tıbbi", + "tıp", + "klinik", + "orvosi", + "orvostudomány", + "klinikai", + "zdravniški", + "medicinski", + "klinični", + "meditsiini", + "kliinik", + "kliiniline" + ], + "key::50": [ + "technology", + "technological", + "tecnologia", + "tecnologie", + "tecnología", + "tecnológico", + "tecnologia", + "tecnológico", + "Technologie", + "technologisch", + "technologie", + "technologisch", + "τεχνολογία", + "τεχνολογικός", + "τεχνολογική", + "τεχνολογικό", + "teknoloji", + "teknolojik", + "technológia", + "technológiai", + "tehnologija", + "tehnološki", + "tehnoloogia", + "tehnoloogiline", + "technologii", + "technical", + "texniki", + "teknik" + ], + "key::51": [ + "science", + "scientific", + "scienza", + "scientifiche", + "scienze", + "ciencia", + "científico", + "ciência", + "científico", + "Wissenschaft", + "wissenschaftlich", + "wetenschap", + "wetenschappelijk", + "επιστήμη", + "επιστημονικός", + "επιστημονική", + "επιστημονικό", + "επιστημονικά", + "bilim", + "bilimsel", + "tudomány", + "tudományos", + "znanost", + "znanstveni", + "teadus", + "teaduslik", + "" + ], + "key::52": [ + "engineering", + "ingegneria", + "ingeniería", + "engenharia", + "Ingenieurwissenschaft", + "ingenieurswetenschappen", + "bouwkunde", + "μηχανικός", + "μηχανική", + "μηχανικό", + "mühendislik", + "mérnöki", + "Inženirstvo", + "inseneeria", + "inseneri", + "" + ], + "key::53": [ + "management", + "gestione", + "gestionale", + "gestionali", + "gestión", + "administración", + "gestão", + "administração", + "Verwaltung", + "management", + "διαχείριση", + "yönetim", + "menedzsment", + "vodstvo", + "upravljanje", + "management", + "juhtkond", + "juhtimine", + "haldus", + "" + ], + "key::54": [ + "energy", + "energia", + "energía", + "energia", + "Energie", + "energie", + "ενέργεια", + "enerji", + "energia", + "energija", + "energia", + "" + ], + "key::55": [ + "agricultural", + "agriculture", + "agricoltura", + "agricole", + "agrícola", + "agricultura", + "agrícola", + "agricultura", + "landwirtschaftlich", + "Landwirtschaft", + "landbouwkundig", + "landbouw", + "αγροτικός", + "αγροτική", + "αγροτικό", + "γεωργικός", + "γεωργική", + "γεωργικό", + "γεωργία", + "tarımsal", + "tarım", + "mezőgazdasági", + "mezőgazdaság", + "poljedelski", + "poljedelstvo", + "põllumajandus", + "põllumajanduslik", + "" + ], + "key::56": [ + "information", + "informazione", + "información", + "informação", + "Information", + "informatie", + "πληροφορία", + "bilgi", + "információ", + "informacija", + "informatsioon", + "informatycznych", + "" + ], + "key::57": [ + "social", + "sociali", + "social", + "social", + "Sozial", + "sociaal", + "maatschappelijk", + "κοινωνικός", + "κοινωνική", + "κοινωνικό", + "κοινωνικά", + "sosyal", + "szociális", + "družbeni", + "sotsiaal", + "sotsiaalne", + "" + ], + "key::58": [ + "environmental", + "ambiente", + "medioambiental", + "ambiente", + "medioambiente", + "meioambiente", + "Umwelt", + "milieu", + "milieuwetenschap", + "milieukunde", + "περιβαλλοντικός", + "περιβαλλοντική", + "περιβαλλοντικό", + "περιβαλλοντικά", + "çevre", + "környezeti", + "okoliški", + "keskonna", + "" + ], + "key::59": [ + "business", + "economia", + "economiche", + "economica", + "negocio", + "empresa", + "negócio", + "Unternehmen", + "bedrijf", + "bedrijfskunde", + "επιχείρηση", + "iş", + "üzleti", + "posel", + "ettevõte/äri", + "" + ], + "key::60": [ + "pharmaceuticals", + "pharmacy", + "farmacia", + "farmaceutica", + "farmacéutica", + "farmacia", + "farmacêutica", + "farmácia", + "Pharmazeutika", + "Arzneimittelkunde", + "farmaceutica", + "geneesmiddelen", + "apotheek", + "φαρμακευτικός", + "φαρμακευτική", + "φαρμακευτικό", + "φαρμακευτικά", + "φαρμακείο", + "ilaç", + "eczane", + "gyógyszerészeti", + "gyógyszertár", + "farmacevtika", + "lekarništvo", + "farmaatsia", + "farmatseutiline", + "" + ], + "key::61": [ + "healthcare", + "health services", + "salute", + "atenciónmédica", + "cuidadodelasalud", + "cuidadoscomasaúde", + "Gesundheitswesen", + "gezondheidszorg", + "ιατροφαρμακευτικήπερίθαλψη", + "sağlıkhizmeti", + "egészségügy", + "zdravstvo", + "tervishoid", + "tervishoiu", + "" + ], + "key::62": [ + "history", + "storia", + "historia", + "história", + "Geschichte", + "geschiedenis", + "geschiedkunde", + "ιστορία", + "tarih", + "történelem", + "zgodovina", + "ajalugu", + "" + ], + "key::63": [ + "materials", + "materiali", + "materia", + "materiales", + "materiais", + "materialen", + "υλικά", + "τεκμήρια", + "malzemeler", + "anyagok", + "materiali", + "materjalid", + "vahendid", + "" + ], + "key::64": [ + "economics", + "economia", + "economiche", + "economica", + "economía", + "economia", + "Wirtschaft", + "economie", + "οικονομικά", + "οικονομικέςεπιστήμες", + "ekonomi", + "közgazdaságtan", + "gospodarstvo", + "ekonomija", + "majanduslik", + "majandus", + "" + ], + "key::65": [ + "therapeutics", + "terapeutica", + "terapéutica", + "terapêutica", + "therapie", + "θεραπευτική", + "tedavibilimi", + "gyógykezelés", + "terapevtika", + "terapeutiline", + "ravi", + "" + ], + "key::66": [ + "oncology", + "oncologia", + "oncologico", + "oncología", + "oncologia", + "Onkologie", + "oncologie", + "ογκολογία", + "onkoloji", + "onkológia", + "onkologija", + "onkoloogia", + "" + ], + "key::67": [ + "natural", + "naturali", + "naturale", + "natural", + "natural", + "natürlich", + "natuurlijk", + "φυσικός", + "φυσική", + "φυσικό", + "φυσικά", + "doğal", + "természetes", + "naraven", + "loodus", + "" + ], + "key::68": [ + "educational", + "educazione", + "pedagogia", + "educacional", + "educativo", + "educacional", + "pädagogisch", + "educatief", + "εκπαιδευτικός", + "εκπαιδευτική", + "εκπαιδευτικό", + "εκπαιδευτικά", + "eğitimsel", + "oktatási", + "izobraževalen", + "haridus", + "hariduslik", + "" + ], + "key::69": [ + "biomedical", + "biomedica", + "biomédico", + "biomédico", + "biomedizinisch", + "biomedisch", + "βιοιατρικός", + "βιοιατρική", + "βιοιατρικό", + "βιοιατρικά", + "biyomedikal", + "orvosbiológiai", + "biomedicinski", + "biomeditsiiniline", + "" + ], + "key::70": [ + "veterinary", + "veterinaria", + "veterinarie", + "veterinaria", + "veterinária", + "tierärtzlich", + "veterinair", + "veeartsenijlkunde", + "κτηνιατρικός", + "κτηνιατρική", + "κτηνιατρικό", + "κτηνιατρικά", + "veteriner", + "állatorvosi", + "veterinar", + "veterinarski", + "veterinaaria", + "" + ], + "key::71": [ + "chemistry", + "chimica", + "química", + "química", + "Chemie", + "chemie", + "scheikunde", + "χημεία", + "kimya", + "kémia", + "kemija", + "keemia", + "" + ], + "key::72": [ + "security", + "sicurezza", + "seguridad", + "segurança", + "Sicherheit", + "veiligheid", + "ασφάλεια", + "güvenlik", + "biztonsági", + "varnost", + "turvalisus", + "julgeolek", + "" + ], + "key::73": [ + "biotechnology", + "biotecnologia", + "biotecnologie", + "biotecnología", + "biotecnologia", + "Biotechnologie", + "biotechnologie", + "βιοτεχνολογία", + "biyoteknoloji", + "biotechnológia", + "biotehnologija", + "biotehnoloogia", + "" + ], + "key::74": [ + "military", + "militare", + "militari", + "militar", + "militar", + "Militär", + "militair", + "leger", + "στρατιωτικός", + "στρατιωτική", + "στρατιωτικό", + "στρατιωτικά", + "askeri", + "katonai", + "vojaški", + "vojni", + "militaar", + "wojskowa", + "" + ], + "key::75": [ + "theological", + "teologia", + "teologico", + "teológico", + "tecnológica", + "theologisch", + "theologisch", + "θεολογικός", + "θεολογική", + "θεολογικό", + "θεολογικά", + "teolojik", + "technológiai", + "teološki", + "teoloogia", + "usuteadus", + "teoloogiline", + "" + ], + "key::76": [ + "electronics", + "elettronica", + "electrónica", + "eletrônicos", + "Elektronik", + "elektronica", + "ηλεκτρονική", + "elektronik", + "elektronika", + "elektronika", + "elektroonika", + "" + ], + "key::77": [ + "forestry", + "forestale", + "forestali", + "silvicultura", + "forestal", + "floresta", + "Forstwirtschaft", + "bosbouw", + "δασοκομία", + "δασολογία", + "ormancılık", + "erdészet", + "gozdarstvo", + "metsandus", + "" + ], + "key::78": [ + "maritime", + "marittima", + "marittime", + "marittimo", + "marítimo", + "marítimo", + "maritiem", + "ναυτικός", + "ναυτική", + "ναυτικό", + "ναυτικά", + "ναυτιλιακός", + "ναυτιλιακή", + "ναυτιλιακό", + "ναυτιλιακά", + "θαλάσσιος", + "θαλάσσια", + "θαλάσσιο", + "denizcilik", + "tengeri", + "morski", + "mere", + "merendus", + "" + ], + "key::79": [ + "sports", + "sport", + "deportes", + "esportes", + "Sport", + "sport", + "sportwetenschappen", + "άθληση", + "γυμναστικήδραστηριότητα", + "spor", + "sport", + "šport", + "sport", + "spordi", + "" + ], + "key::80": [ + "surgery", + "chirurgia", + "chirurgiche", + "cirugía", + "cirurgia", + "Chirurgie", + "chirurgie", + "heelkunde", + "εγχείρηση", + "επέμβαση", + "χειρουργικήεπέμβαση", + "cerrahi", + "sebészet", + "kirurgija", + "kirurgia", + "" + ], + "key::81": [ + "cultural", + "culturale", + "culturali", + "cultura", + "cultural", + "cultural", + "kulturell", + "cultureel", + "πολιτιστικός", + "πολιτιστική", + "πολιτιστικό", + "πολιτισμικός", + "πολιτισμική", + "πολιτισμικό", + "kültürel", + "kultúrális", + "kulturni", + "kultuuri", + "kultuuriline", + "" + ], + "key::82": [ + "computerscience", + "informatica", + "ordenador", + "computadora", + "informática", + "computación", + "cienciasdelacomputación", + "ciênciadacomputação", + "Computer", + "computer", + "υπολογιστής", + "ηλεκτρονικόςυπολογιστής", + "bilgisayar", + "számítógép", + "računalnik", + "arvuti", + "" + ], + "key::83": [ + "finance", + "financial", + "finanza", + "finanziarie", + "finanza", + "financiero", + "finanças", + "financeiro", + "Finanzen", + "finanziell", + "financiën", + "financieel", + "χρηματοοικονομικά", + "χρηματοδότηση", + "finanse", + "finansal", + "pénzügy", + "pénzügyi", + "finance", + "finančni", + "finants", + "finantsiline", + "" + ], + "key::84": [ + "communication", + "comunicazione", + "comuniciación", + "comunicação", + "Kommunikation", + "communication", + "επικοινωνία", + "iletişim", + "kommunikáció", + "komuniciranje", + "kommunikatsioon", + "" + ], + "key::85": [ + "justice", + "giustizia", + "justicia", + "justiça", + "Recht", + "Justiz", + "justitie", + "gerechtigheid", + "δικαιοσύνη", + "υπουργείοδικαιοσύνης", + "δίκαιο", + "adalet", + "igazságügy", + "pravo", + "õigus", + "" + ], + "key::86": [ + "aerospace", + "aerospaziale", + "aerospaziali", + "aeroespacio", + "aeroespaço", + "Luftfahrt", + "luchtvaart", + "ruimtevaart", + "αεροπορικός", + "αεροπορική", + "αεροπορικό", + "αεροναυπηγικός", + "αεροναυπηγική", + "αεροναυπηγικό", + "αεροναυπηγικά", + "havacılıkveuzay", + "légtér", + "zrakoplovstvo", + "atmosfäär", + "kosmos", + "" + ], + "key::87": [ + "dermatology", + "dermatologia", + "dermatología", + "dermatologia", + "Dermatologie", + "dermatologie", + "δρματολογία", + "dermatoloji", + "bőrgyógyászat", + "dermatológia", + "dermatologija", + "dermatoloogia", + "" + ], + "key::88": [ + "architecture", + "architettura", + "arquitectura", + "arquitetura", + "Architektur", + "architectuur", + "αρχιτεκτονική", + "mimarlık", + "építészet", + "arhitektura", + "arhitektuur", + "" + ], + "key::89": [ + "mathematics", + "matematica", + "matematiche", + "matemáticas", + "matemáticas", + "Mathematik", + "wiskunde", + "mathematica", + "μαθηματικά", + "matematik", + "matematika", + "matematika", + "matemaatika", + "" + ], + "key::90": [ + "language", + "lingue", + "linguistica", + "linguistiche", + "lenguaje", + "idioma", + "língua", + "idioma", + "Sprache", + "taal", + "taalkunde", + "γλώσσα", + "dil", + "nyelv", + "jezik", + "keel", + "" + ], + "key::91": [ + "neuroscience", + "neuroscienza", + "neurociencia", + "neurociência", + "Neurowissenschaft", + "neurowetenschappen", + "νευροεπιστήμη", + "nörobilim", + "idegtudomány", + "nevroznanost", + "neuroteadused", + "" + ], + "key::92": [ + "automation", + "automazione", + "automatización", + "automação", + "Automatisierung", + "automatisering", + "αυτοματοποίηση", + "otomasyon", + "automatizálás", + "avtomatizacija", + "automatiseeritud", + "" + ], + "key::93": [ + "pediatric", + "pediatria", + "pediatriche", + "pediatrico", + "pediátrico", + "pediatría", + "pediátrico", + "pediatria", + "pädiatrisch", + "pediatrische", + "παιδιατρική", + "pediatrik", + "gyermekgyógyászat", + "pediatrija", + "pediaatria", + "" + ], + "key::94": [ + "photonics", + "fotonica", + "fotoniche", + "fotónica", + "fotônica", + "Photonik", + "fotonica", + "φωτονική", + "fotonik", + "fotonika", + "fotonika", + "fotoonika", + "" + ], + "key::95": [ + "mechanics", + "meccanica", + "meccaniche", + "mecánica", + "mecânica", + "Mechanik", + "Maschinenbau", + "mechanica", + "werktuigkunde", + "μηχανικής", + "mekanik", + "gépészet", + "mehanika", + "mehaanika", + "" + ], + "key::96": [ + "psychiatrics", + "psichiatria", + "psichiatrica", + "psichiatriche", + "psiquiatría", + "psiquiatria", + "Psychiatrie", + "psychiatrie", + "ψυχιατρική", + "psikiyatrik", + "pszihiátria", + "psihiatrija", + "psühhaatria", + "" + ], + "key::97": [ + "psychology", + "fisiologia", + "psicología", + "psicologia", + "Psychologie", + "psychologie", + "ψυχολογία", + "psikoloji", + "pszihológia", + "psihologija", + "psühholoogia", + "" + ], + "key::98": [ + "automotive", + "industriaautomobilistica", + "industriadelautomóvil", + "automotriz", + "industriaautomotriz", + "automotivo", + "Automobilindustrie", + "autoindustrie", + "αυτοκίνητος", + "αυτοκίνητη", + "αυτοκίνητο", + "αυτοκινούμενος", + "αυτοκινούμενη", + "αυτοκινούμενο", + "αυτοκινητιστικός", + "αυτοκινητιστική", + "αυτοκινητιστικό", + "otomotiv", + "autóipari", + "samogiben", + "avtomobilskaindustrija", + "auto-", + "" + ], + "key::99": [ + "neurology", + "neurologia", + "neurologiche", + "neurología", + "neurologia", + "Neurologie", + "neurologie", + "zenuwleer", + "νευρολογία", + "nöroloji", + "neurológia", + "ideggyógyászat", + "nevrologija", + "neuroloogia", + "" + ], + "key::100": [ + "geology", + "geologia", + "geologiche", + "geología", + "geologia", + "Geologie", + "geologie", + "aardkunde", + "γεωλογία", + "jeoloji", + "geológia", + "földtudomány", + "geologija", + "geoloogia", + "" + ], + "key::101": [ + "microbiology", + "microbiologia", + "micro-biologia", + "microbiologiche", + "microbiología", + "microbiologia", + "Mikrobiologie", + "microbiologie", + "μικροβιολογία", + "mikrobiyoloji", + "mikrobiológia", + "mikrobiologija", + "mikrobioloogia", + "" + ], + "key::102": [ + "informatics", + "informatica", + "informática", + "informática", + "informatica", + "" + ], + "key::103": [ + "forschungsgemeinschaft", + "comunita ricerca", + "research community", + "research foundation", + "research association" + ], + "key::104": [ + "commerce", + "ticaret", + "ticarət", + "commercio", + "trade", + "handel", + "comercio" + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json new file mode 100644 index 000000000..2aeb4c582 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json @@ -0,0 +1,1753 @@ +{ + "wf": { + "threshold": "0.9", + "dedupRun": "001", + "entityType": "organization", + "orderField": "legalname", + "queueMaxSize": "2000", + "groupMaxSize": "50", + "slidingWindowSize": "200", + "idPath": "$.id", + "rootBuilder": [ + "organization", + "projectOrganization_participation_isParticipant", + "datasourceOrganization_provision_isProvidedBy" + ], + "includeChildren": "true" + }, + "pace": { + "clustering": [ + { + "name": "sortedngrampairs", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "ngramLen": "3" + } + }, + { + "name": "suffixprefix", + "fields": [ + "legalname" + ], + "params": { + "max": 1, + "len": "3" + } + }, + { + "name": "urlclustering", + "fields": [ + "websiteurl" + ], + "params": {} + }, + { + "name": "keywordsclustering", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "windowSize": 4 + } + } + ], + "strictConditions": [ + { + "name": "exactMatch", + "fields": [ + "gridid" + ] + } + ], + "conditions": [ + { + "name": "DomainExactMatch", + "fields": [ + "websiteurl" + ] + }, + { + "name": "exactMatch", + "fields": [ + "country" + ] + } + ], + "model": [ + { + "name": "country", + "algo": "Null", + "type": "String", + "weight": "0", + "ignoreMissing": "false", + "path": "$.country.classid" + }, + { + "name": "legalshortname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.1", + "ignoreMissing": "true", + "path": "$.legalshortname.value" + }, + { + "name": "legalname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.9", + "ignoreMissing": "false", + "path": "$.legalname.value", + "params": { + "windowSize": 4, + "threshold": 0.7 + } + }, + { + "name": "websiteurl", + "algo": "Null", + "type": "URL", + "weight": "0", + "ignoreMissing": "true", + "path": "$.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" + } + ], + "blacklists": { + "legalname": [] + }, + "synonyms": { + "key::1": [ + "university", + "università", + "università studi", + "universitario", + "universitaria", + "université", + "universitaire", + "universitaires", + "universidad", + "universitade", + "Universität", + "universitaet", + "Uniwersytet", + "университет", + "universiteit", + "πανεπιστήμιο", + "universitesi", + "universiteti" + ], + "key::2": [ + "studies", + "studi", + "études", + "estudios", + "estudos", + "Studien", + "studia", + "исследования", + "studies", + "σπουδές" + ], + "key::3": [ + "advanced", + "superiore", + "supérieur", + "supérieure", + "supérieurs", + "supérieures", + "avancado", + "avancados", + "fortgeschrittene", + "fortgeschritten", + "zaawansowany", + "передовой", + "gevorderd", + "gevorderde", + "προχωρημένος", + "προχωρημένη", + "προχωρημένο", + "προχωρημένες", + "προχωρημένα", + "wyzsza" + ], + "key::4": [ + "institute", + "istituto", + "institut", + "instituto", + "instituto", + "Institut", + "instytut", + "институт", + "instituut", + "ινστιτούτο" + ], + "key::5": [ + "hospital", + "ospedale", + "hôpital", + "hospital", + "hospital", + "Krankenhaus", + "szpital", + "больница", + "ziekenhuis", + "νοσοκομείο" + ], + "key::6": [ + "research", + "ricerca", + "recherche", + "investigacion", + "pesquisa", + "Forschung", + "badania", + "исследования", + "onderzoek", + "έρευνα", + "erevna", + "erevnas" + ], + "key::7": [ + "college", + "collegio", + "université", + "colegio", + "faculdade", + "Hochschule", + "Szkoła Wyższa", + "Высшая школа", + "universiteit", + "κολλέγιο" + ], + "key::8": [ + "foundation", + "fondazione", + "fondation", + "fundación", + "fundação", + "Stiftung", + "Fundacja", + "фонд", + "stichting", + "ίδρυμα", + "idryma" + ], + "key::9": [ + "center", + "centro", + "centre", + "centro", + "centro", + "zentrum", + "centrum", + "центр", + "centrum", + "κέντρο" + ], + "key::10": [ + "national", + "nazionale", + "national", + "nationale", + "nationaux", + "nationales", + "nacional", + "nacional", + "national", + "krajowy", + "национальный", + "nationaal", + "nationale", + "εθνικό" + ], + "key::11": [ + "association", + "associazione", + "association", + "asociación", + "associação", + "Verein", + "verband", + "stowarzyszenie", + "ассоциация", + "associatie" + ], + "key::12": [ + "society", + "societa", + "société", + "sociedad", + "sociedade", + "gesellschaft", + "społeczeństwo", + "общество", + "maatschappij", + "κοινωνία" + ], + "key::13": [ + "international", + "internazionale", + "international", + "internacional", + "internacional", + "international", + "międzynarodowy", + "Международный", + "internationaal", + "internationale", + "διεθνής", + "διεθνή", + "διεθνές" + ], + "key::14": [ + "community", + "comunita", + "communauté", + "comunidad", + "comunidade", + "Gemeinschaft", + "społeczność", + "сообщество", + "gemeenschap", + "κοινότητα" + ], + "key::15": [ + "school", + "scuola", + "école", + "escuela", + "escola", + "schule", + "Szkoła", + "школа", + "school", + "σχολείο" + ], + "key::16": [ + "education", + "educazione", + "éducation", + "educacion", + "Educação", + "Bildung", + "Edukacja", + "образование", + "opleiding", + "εκπαίδευση" + ], + "key::17": [ + "academy", + "accademia", + "académie", + "academia", + "academia", + "Akademie", + "akademie", + "академия", + "academie", + "ακαδημία" + ], + "key::18": [ + "public", + "pubblico", + "public", + "publique", + "publics", + "publiques", + "publico", + "publico", + "Öffentlichkeit", + "publiczny", + "публичный", + "publiek", + "publieke", + "δημόσιος", + "δημόσια", + "δημόσιο" + ], + "key::19": [ + "museum", + "museo", + "musée", + "mueso", + "museu", + "museum", + "muzeum", + "музей", + "museum", + "μουσείο" + ], + "key::20": [ + "group", + "gruppo", + "groupe", + "grupo", + "grupo", + "gruppe", + "grupa", + "группа", + "groep", + "ομάδα", + "όμιλος" + ], + "key::21": [ + "department", + "dipartimento", + "département", + "departamento", + "departamento", + "abteilung", + "departament", + "отдел", + "afdeling", + "τμήμα" + ], + "key::22": [ + "council", + "consiglio", + "conseil", + "Consejo", + "conselho", + "gesellschaft", + "rada", + "совет", + "raad", + "συμβούλιο" + ], + "key::23": [ + "library", + "biblioteca", + "bibliothèque", + "biblioteca", + "biblioteca", + "Bibliothek", + "biblioteka", + "библиотека", + "bibliotheek", + "βιβλιοθήκη" + ], + "key::24": [ + "ministry", + "ministero", + "ministère", + "ministerio", + "ministério", + "Ministerium", + "ministerstwo", + "министерство", + "ministerie", + "υπουργείο" + ], + "key::25": [ + "services", + "servizi", + "services", + "servicios", + "Serviços", + "Dienstleistungen", + "usługi", + "услуги", + "diensten", + "υπηρεσίες" + ], + "key::26": [ + "central", + "centrale", + "central", + "centrale", + "centrales", + "central", + "central", + "zentral", + "centralny", + "цетральный", + "centraal", + "κεντρικός", + "κεντρική", + "κεντρικό", + "κεντρικά" + ], + "key::27": [ + "general", + "generale", + "général", + "générale", + "généraux", + "générales", + "general", + "geral", + "general", + "Allgemeines", + "general", + "общий", + "algemeen", + "algemene", + "γενικός", + "γενική", + "γενικό", + "γενικά" + ], + "key::28": [ + "applied", + "applicati", + "appliqué", + "appliquée", + "appliqués", + "appliquées", + "aplicado", + "aplicada", + "angewendet", + "stosowany", + "прикладной", + "toegepast", + "toegepaste", + "εφαρμοσμένος", + "εφαρμοσμένη", + "εφαρμοσμένο", + "εφαρμοσμένα" + ], + "key::29": [ + "european", + "europee", + "europea", + "européen", + "européenne", + "européens", + "européennes", + "europeo", + "europeu", + "europäisch", + "europejski", + "европейский", + "Europees", + "Europese", + "ευρωπαϊκός", + "ευρωπαϊκή", + "ευρωπαϊκό", + "ευρωπαϊκά" + ], + "key::30": [ + "agency", + "agenzia", + "agence", + "agencia", + "agencia", + "agentur", + "agencja", + "агенция", + "agentschap", + "πρακτορείο" + ], + "key::31": [ + "laboratory", + "laboratorio", + "laboratoire", + "laboratorio", + "laboratorio", + "labor", + "laboratorium", + "лаборатория", + "laboratorium", + "εργαστήριο" + ], + "key::32": [ + "industry", + "industria", + "industrie", + "индустрия", + "industrie", + "βιομηχανία" + ], + "key::33": [ + "industrial", + "industriale", + "industriel", + "industrielle", + "industriels", + "industrielles", + "индустриальный", + "industrieel", + "βιομηχανικός", + "βιομηχανική", + "βιομηχανικό", + "βιομηχανικά", + "βιομηχανικές" + ], + "key::34": [ + "consortium", + "consorzio", + "consortium", + "консорциум", + "consortium", + "κοινοπραξία" + ], + "key::35": [ + "organization", + "organizzazione", + "organisation", + "organización", + "organização", + "organizacja", + "организация", + "organisatie", + "οργανισμός" + ], + "key::36": [ + "authority", + "autorità", + "autorité", + "авторитет", + "autoriteit" + ], + "key::37": [ + "federation", + "federazione", + "fédération", + "федерация", + "federatie", + "ομοσπονδία" + ], + "key::38": [ + "observatory", + "osservatorio", + "observatoire", + "обсерватория", + "observatorium", + "αστεροσκοπείο" + ], + "key::39": [ + "bureau", + "ufficio", + "bureau", + "офис", + "bureau", + "γραφείο" + ], + "key::40": [ + "company", + "impresa", + "compagnie", + "société", + "компания", + "bedrijf", + "εταιρία" + ], + "key::41": [ + "polytechnic", + "politecnico", + "polytechnique", + "политехника", + "polytechnisch", + "πολυτεχνείο", + "universita politecnica", + "polytechnic university", + "universidad politecnica", + "universitat politecnica", + "politechnika", + "politechniki", + "university technology", + "university science technology" + ], + "key::42": [ + "coalition", + "coalizione", + "coalition", + "коалиция", + "coalitie", + "συνασπισμός" + ], + "key::43": [ + "initiative", + "iniziativa", + "initiative", + "инициатива", + "initiatief", + "πρωτοβουλία" + ], + "key::44": [ + "academic", + "accademico", + "académique", + "universitaire", + "акадеческий academisch", + "ακαδημαϊκός", + "ακαδημαϊκή", + "ακαδημαϊκό", + "ακαδημαϊκές", + "ακαδημαϊκοί" + ], + "key::45": [ + "institution", + "istituzione", + "institution", + "институциональный", + "instelling", + "ινστιτούτο" + ], + "key::46": [ + "division", + "divisione", + "division", + "отделение", + "divisie", + "τμήμα" + ], + "key::47": [ + "committee", + "comitato", + "comité", + "комитет", + "commissie", + "επιτροπή" + ], + "key::48": [ + "promotion", + "promozione", + "продвижение", + "proothisis", + "forderung" + ], + "key::49": [ + "medical", + "medicine", + "clinical", + "medicina", + "clinici", + "médico", + "medicina", + "clínica", + "médico", + "medicina", + "clínica", + "medizinisch", + "Medizin", + "klinisch", + "medisch", + "geneeskunde", + "klinisch", + "ιατρικός", + "ιατρική", + "ιατρικό", + "ιατρικά", + "κλινικός", + "κλινική", + "κλινικό", + "κλινικά", + "tıbbi", + "tıp", + "klinik", + "orvosi", + "orvostudomány", + "klinikai", + "zdravniški", + "medicinski", + "klinični", + "meditsiini", + "kliinik", + "kliiniline" + ], + "key::50": [ + "technology", + "technological", + "tecnologia", + "tecnologie", + "tecnología", + "tecnológico", + "tecnologia", + "tecnológico", + "Technologie", + "technologisch", + "technologie", + "technologisch", + "τεχνολογία", + "τεχνολογικός", + "τεχνολογική", + "τεχνολογικό", + "teknoloji", + "teknolojik", + "technológia", + "technológiai", + "tehnologija", + "tehnološki", + "tehnoloogia", + "tehnoloogiline", + "technologii", + "technical", + "texniki", + "teknik" + ], + "key::51": [ + "science", + "scientific", + "scienza", + "scientifiche", + "scienze", + "ciencia", + "científico", + "ciência", + "científico", + "Wissenschaft", + "wissenschaftlich", + "wetenschap", + "wetenschappelijk", + "επιστήμη", + "επιστημονικός", + "επιστημονική", + "επιστημονικό", + "επιστημονικά", + "bilim", + "bilimsel", + "tudomány", + "tudományos", + "znanost", + "znanstveni", + "teadus", + "teaduslik", + "" + ], + "key::52": [ + "engineering", + "ingegneria", + "ingeniería", + "engenharia", + "Ingenieurwissenschaft", + "ingenieurswetenschappen", + "bouwkunde", + "μηχανικός", + "μηχανική", + "μηχανικό", + "mühendislik", + "mérnöki", + "Inženirstvo", + "inseneeria", + "inseneri", + "" + ], + "key::53": [ + "management", + "gestione", + "gestionale", + "gestionali", + "gestión", + "administración", + "gestão", + "administração", + "Verwaltung", + "management", + "διαχείριση", + "yönetim", + "menedzsment", + "vodstvo", + "upravljanje", + "management", + "juhtkond", + "juhtimine", + "haldus", + "" + ], + "key::54": [ + "energy", + "energia", + "energía", + "energia", + "Energie", + "energie", + "ενέργεια", + "enerji", + "energia", + "energija", + "energia", + "" + ], + "key::55": [ + "agricultural", + "agriculture", + "agricoltura", + "agricole", + "agrícola", + "agricultura", + "agrícola", + "agricultura", + "landwirtschaftlich", + "Landwirtschaft", + "landbouwkundig", + "landbouw", + "αγροτικός", + "αγροτική", + "αγροτικό", + "γεωργικός", + "γεωργική", + "γεωργικό", + "γεωργία", + "tarımsal", + "tarım", + "mezőgazdasági", + "mezőgazdaság", + "poljedelski", + "poljedelstvo", + "põllumajandus", + "põllumajanduslik", + "" + ], + "key::56": [ + "information", + "informazione", + "información", + "informação", + "Information", + "informatie", + "πληροφορία", + "bilgi", + "információ", + "informacija", + "informatsioon", + "informatycznych", + "" + ], + "key::57": [ + "social", + "sociali", + "social", + "social", + "Sozial", + "sociaal", + "maatschappelijk", + "κοινωνικός", + "κοινωνική", + "κοινωνικό", + "κοινωνικά", + "sosyal", + "szociális", + "družbeni", + "sotsiaal", + "sotsiaalne", + "" + ], + "key::58": [ + "environmental", + "ambiente", + "medioambiental", + "ambiente", + "medioambiente", + "meioambiente", + "Umwelt", + "milieu", + "milieuwetenschap", + "milieukunde", + "περιβαλλοντικός", + "περιβαλλοντική", + "περιβαλλοντικό", + "περιβαλλοντικά", + "çevre", + "környezeti", + "okoliški", + "keskonna", + "" + ], + "key::59": [ + "business", + "economia", + "economiche", + "economica", + "negocio", + "empresa", + "negócio", + "Unternehmen", + "bedrijf", + "bedrijfskunde", + "επιχείρηση", + "iş", + "üzleti", + "posel", + "ettevõte/äri", + "" + ], + "key::60": [ + "pharmaceuticals", + "pharmacy", + "farmacia", + "farmaceutica", + "farmacéutica", + "farmacia", + "farmacêutica", + "farmácia", + "Pharmazeutika", + "Arzneimittelkunde", + "farmaceutica", + "geneesmiddelen", + "apotheek", + "φαρμακευτικός", + "φαρμακευτική", + "φαρμακευτικό", + "φαρμακευτικά", + "φαρμακείο", + "ilaç", + "eczane", + "gyógyszerészeti", + "gyógyszertár", + "farmacevtika", + "lekarništvo", + "farmaatsia", + "farmatseutiline", + "" + ], + "key::61": [ + "healthcare", + "health services", + "salute", + "atenciónmédica", + "cuidadodelasalud", + "cuidadoscomasaúde", + "Gesundheitswesen", + "gezondheidszorg", + "ιατροφαρμακευτικήπερίθαλψη", + "sağlıkhizmeti", + "egészségügy", + "zdravstvo", + "tervishoid", + "tervishoiu", + "" + ], + "key::62": [ + "history", + "storia", + "historia", + "história", + "Geschichte", + "geschiedenis", + "geschiedkunde", + "ιστορία", + "tarih", + "történelem", + "zgodovina", + "ajalugu", + "" + ], + "key::63": [ + "materials", + "materiali", + "materia", + "materiales", + "materiais", + "materialen", + "υλικά", + "τεκμήρια", + "malzemeler", + "anyagok", + "materiali", + "materjalid", + "vahendid", + "" + ], + "key::64": [ + "economics", + "economia", + "economiche", + "economica", + "economía", + "economia", + "Wirtschaft", + "economie", + "οικονομικά", + "οικονομικέςεπιστήμες", + "ekonomi", + "közgazdaságtan", + "gospodarstvo", + "ekonomija", + "majanduslik", + "majandus", + "" + ], + "key::65": [ + "therapeutics", + "terapeutica", + "terapéutica", + "terapêutica", + "therapie", + "θεραπευτική", + "tedavibilimi", + "gyógykezelés", + "terapevtika", + "terapeutiline", + "ravi", + "" + ], + "key::66": [ + "oncology", + "oncologia", + "oncologico", + "oncología", + "oncologia", + "Onkologie", + "oncologie", + "ογκολογία", + "onkoloji", + "onkológia", + "onkologija", + "onkoloogia", + "" + ], + "key::67": [ + "natural", + "naturali", + "naturale", + "natural", + "natural", + "natürlich", + "natuurlijk", + "φυσικός", + "φυσική", + "φυσικό", + "φυσικά", + "doğal", + "természetes", + "naraven", + "loodus", + "" + ], + "key::68": [ + "educational", + "educazione", + "pedagogia", + "educacional", + "educativo", + "educacional", + "pädagogisch", + "educatief", + "εκπαιδευτικός", + "εκπαιδευτική", + "εκπαιδευτικό", + "εκπαιδευτικά", + "eğitimsel", + "oktatási", + "izobraževalen", + "haridus", + "hariduslik", + "" + ], + "key::69": [ + "biomedical", + "biomedica", + "biomédico", + "biomédico", + "biomedizinisch", + "biomedisch", + "βιοιατρικός", + "βιοιατρική", + "βιοιατρικό", + "βιοιατρικά", + "biyomedikal", + "orvosbiológiai", + "biomedicinski", + "biomeditsiiniline", + "" + ], + "key::70": [ + "veterinary", + "veterinaria", + "veterinarie", + "veterinaria", + "veterinária", + "tierärtzlich", + "veterinair", + "veeartsenijlkunde", + "κτηνιατρικός", + "κτηνιατρική", + "κτηνιατρικό", + "κτηνιατρικά", + "veteriner", + "állatorvosi", + "veterinar", + "veterinarski", + "veterinaaria", + "" + ], + "key::71": [ + "chemistry", + "chimica", + "química", + "química", + "Chemie", + "chemie", + "scheikunde", + "χημεία", + "kimya", + "kémia", + "kemija", + "keemia", + "" + ], + "key::72": [ + "security", + "sicurezza", + "seguridad", + "segurança", + "Sicherheit", + "veiligheid", + "ασφάλεια", + "güvenlik", + "biztonsági", + "varnost", + "turvalisus", + "julgeolek", + "" + ], + "key::73": [ + "biotechnology", + "biotecnologia", + "biotecnologie", + "biotecnología", + "biotecnologia", + "Biotechnologie", + "biotechnologie", + "βιοτεχνολογία", + "biyoteknoloji", + "biotechnológia", + "biotehnologija", + "biotehnoloogia", + "" + ], + "key::74": [ + "military", + "militare", + "militari", + "militar", + "militar", + "Militär", + "militair", + "leger", + "στρατιωτικός", + "στρατιωτική", + "στρατιωτικό", + "στρατιωτικά", + "askeri", + "katonai", + "vojaški", + "vojni", + "militaar", + "wojskowa", + "" + ], + "key::75": [ + "theological", + "teologia", + "teologico", + "teológico", + "tecnológica", + "theologisch", + "theologisch", + "θεολογικός", + "θεολογική", + "θεολογικό", + "θεολογικά", + "teolojik", + "technológiai", + "teološki", + "teoloogia", + "usuteadus", + "teoloogiline", + "" + ], + "key::76": [ + "electronics", + "elettronica", + "electrónica", + "eletrônicos", + "Elektronik", + "elektronica", + "ηλεκτρονική", + "elektronik", + "elektronika", + "elektronika", + "elektroonika", + "" + ], + "key::77": [ + "forestry", + "forestale", + "forestali", + "silvicultura", + "forestal", + "floresta", + "Forstwirtschaft", + "bosbouw", + "δασοκομία", + "δασολογία", + "ormancılık", + "erdészet", + "gozdarstvo", + "metsandus", + "" + ], + "key::78": [ + "maritime", + "marittima", + "marittime", + "marittimo", + "marítimo", + "marítimo", + "maritiem", + "ναυτικός", + "ναυτική", + "ναυτικό", + "ναυτικά", + "ναυτιλιακός", + "ναυτιλιακή", + "ναυτιλιακό", + "ναυτιλιακά", + "θαλάσσιος", + "θαλάσσια", + "θαλάσσιο", + "denizcilik", + "tengeri", + "morski", + "mere", + "merendus", + "" + ], + "key::79": [ + "sports", + "sport", + "deportes", + "esportes", + "Sport", + "sport", + "sportwetenschappen", + "άθληση", + "γυμναστικήδραστηριότητα", + "spor", + "sport", + "šport", + "sport", + "spordi", + "" + ], + "key::80": [ + "surgery", + "chirurgia", + "chirurgiche", + "cirugía", + "cirurgia", + "Chirurgie", + "chirurgie", + "heelkunde", + "εγχείρηση", + "επέμβαση", + "χειρουργικήεπέμβαση", + "cerrahi", + "sebészet", + "kirurgija", + "kirurgia", + "" + ], + "key::81": [ + "cultural", + "culturale", + "culturali", + "cultura", + "cultural", + "cultural", + "kulturell", + "cultureel", + "πολιτιστικός", + "πολιτιστική", + "πολιτιστικό", + "πολιτισμικός", + "πολιτισμική", + "πολιτισμικό", + "kültürel", + "kultúrális", + "kulturni", + "kultuuri", + "kultuuriline", + "" + ], + "key::82": [ + "computerscience", + "informatica", + "ordenador", + "computadora", + "informática", + "computación", + "cienciasdelacomputación", + "ciênciadacomputação", + "Computer", + "computer", + "υπολογιστής", + "ηλεκτρονικόςυπολογιστής", + "bilgisayar", + "számítógép", + "računalnik", + "arvuti", + "" + ], + "key::83": [ + "finance", + "financial", + "finanza", + "finanziarie", + "finanza", + "financiero", + "finanças", + "financeiro", + "Finanzen", + "finanziell", + "financiën", + "financieel", + "χρηματοοικονομικά", + "χρηματοδότηση", + "finanse", + "finansal", + "pénzügy", + "pénzügyi", + "finance", + "finančni", + "finants", + "finantsiline", + "" + ], + "key::84": [ + "communication", + "comunicazione", + "comuniciación", + "comunicação", + "Kommunikation", + "communication", + "επικοινωνία", + "iletişim", + "kommunikáció", + "komuniciranje", + "kommunikatsioon", + "" + ], + "key::85": [ + "justice", + "giustizia", + "justicia", + "justiça", + "Recht", + "Justiz", + "justitie", + "gerechtigheid", + "δικαιοσύνη", + "υπουργείοδικαιοσύνης", + "δίκαιο", + "adalet", + "igazságügy", + "pravo", + "õigus", + "" + ], + "key::86": [ + "aerospace", + "aerospaziale", + "aerospaziali", + "aeroespacio", + "aeroespaço", + "Luftfahrt", + "luchtvaart", + "ruimtevaart", + "αεροπορικός", + "αεροπορική", + "αεροπορικό", + "αεροναυπηγικός", + "αεροναυπηγική", + "αεροναυπηγικό", + "αεροναυπηγικά", + "havacılıkveuzay", + "légtér", + "zrakoplovstvo", + "atmosfäär", + "kosmos", + "" + ], + "key::87": [ + "dermatology", + "dermatologia", + "dermatología", + "dermatologia", + "Dermatologie", + "dermatologie", + "δρματολογία", + "dermatoloji", + "bőrgyógyászat", + "dermatológia", + "dermatologija", + "dermatoloogia", + "" + ], + "key::88": [ + "architecture", + "architettura", + "arquitectura", + "arquitetura", + "Architektur", + "architectuur", + "αρχιτεκτονική", + "mimarlık", + "építészet", + "arhitektura", + "arhitektuur", + "" + ], + "key::89": [ + "mathematics", + "matematica", + "matematiche", + "matemáticas", + "matemáticas", + "Mathematik", + "wiskunde", + "mathematica", + "μαθηματικά", + "matematik", + "matematika", + "matematika", + "matemaatika", + "" + ], + "key::90": [ + "language", + "lingue", + "linguistica", + "linguistiche", + "lenguaje", + "idioma", + "língua", + "idioma", + "Sprache", + "taal", + "taalkunde", + "γλώσσα", + "dil", + "nyelv", + "jezik", + "keel", + "" + ], + "key::91": [ + "neuroscience", + "neuroscienza", + "neurociencia", + "neurociência", + "Neurowissenschaft", + "neurowetenschappen", + "νευροεπιστήμη", + "nörobilim", + "idegtudomány", + "nevroznanost", + "neuroteadused", + "" + ], + "key::92": [ + "automation", + "automazione", + "automatización", + "automação", + "Automatisierung", + "automatisering", + "αυτοματοποίηση", + "otomasyon", + "automatizálás", + "avtomatizacija", + "automatiseeritud", + "" + ], + "key::93": [ + "pediatric", + "pediatria", + "pediatriche", + "pediatrico", + "pediátrico", + "pediatría", + "pediátrico", + "pediatria", + "pädiatrisch", + "pediatrische", + "παιδιατρική", + "pediatrik", + "gyermekgyógyászat", + "pediatrija", + "pediaatria", + "" + ], + "key::94": [ + "photonics", + "fotonica", + "fotoniche", + "fotónica", + "fotônica", + "Photonik", + "fotonica", + "φωτονική", + "fotonik", + "fotonika", + "fotonika", + "fotoonika", + "" + ], + "key::95": [ + "mechanics", + "meccanica", + "meccaniche", + "mecánica", + "mecânica", + "Mechanik", + "Maschinenbau", + "mechanica", + "werktuigkunde", + "μηχανικής", + "mekanik", + "gépészet", + "mehanika", + "mehaanika", + "" + ], + "key::96": [ + "psychiatrics", + "psichiatria", + "psichiatrica", + "psichiatriche", + "psiquiatría", + "psiquiatria", + "Psychiatrie", + "psychiatrie", + "ψυχιατρική", + "psikiyatrik", + "pszihiátria", + "psihiatrija", + "psühhaatria", + "" + ], + "key::97": [ + "psychology", + "fisiologia", + "psicología", + "psicologia", + "Psychologie", + "psychologie", + "ψυχολογία", + "psikoloji", + "pszihológia", + "psihologija", + "psühholoogia", + "" + ], + "key::98": [ + "automotive", + "industriaautomobilistica", + "industriadelautomóvil", + "automotriz", + "industriaautomotriz", + "automotivo", + "Automobilindustrie", + "autoindustrie", + "αυτοκίνητος", + "αυτοκίνητη", + "αυτοκίνητο", + "αυτοκινούμενος", + "αυτοκινούμενη", + "αυτοκινούμενο", + "αυτοκινητιστικός", + "αυτοκινητιστική", + "αυτοκινητιστικό", + "otomotiv", + "autóipari", + "samogiben", + "avtomobilskaindustrija", + "auto-", + "" + ], + "key::99": [ + "neurology", + "neurologia", + "neurologiche", + "neurología", + "neurologia", + "Neurologie", + "neurologie", + "zenuwleer", + "νευρολογία", + "nöroloji", + "neurológia", + "ideggyógyászat", + "nevrologija", + "neuroloogia", + "" + ], + "key::100": [ + "geology", + "geologia", + "geologiche", + "geología", + "geologia", + "Geologie", + "geologie", + "aardkunde", + "γεωλογία", + "jeoloji", + "geológia", + "földtudomány", + "geologija", + "geoloogia", + "" + ], + "key::101": [ + "microbiology", + "microbiologia", + "micro-biologia", + "microbiologiche", + "microbiología", + "microbiologia", + "Mikrobiologie", + "microbiologie", + "μικροβιολογία", + "mikrobiyoloji", + "mikrobiológia", + "mikrobiologija", + "mikrobioloogia", + "" + ], + "key::102": [ + "informatics", + "informatica", + "informática", + "informática", + "informatica", + "" + ], + "key::103": [ + "forschungsgemeinschaft", + "comunita ricerca", + "research community", + "research foundation", + "research association" + ], + "key::104": [ + "commerce", + "ticaret", + "ticarət", + "commercio", + "trade", + "handel", + "comercio" + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json new file mode 100644 index 000000000..3e861fb71 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json @@ -0,0 +1,280 @@ +{ + "wf" : { + "threshold" : "0.99", + "dedupRun" : "001", + "entityType" : "result", + "subEntityType" : "resulttype", + "subEntityValue" : "publication", + "orderField" : "title", + "queueMaxSize" : "2000", + "groupMaxSize" : "100", + "maxChildren" : "100", + "idPath": "$.id", + "slidingWindowSize" : "200", + "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_isAffiliatedWith", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], + "includeChildren" : "true" + }, + "pace" : { + "clustering" : [ + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } + ], + "strictConditions" : [ + { "name" : "pidMatch", "fields" : [ "pid" ] } + ], + "conditions" : [ + { "name" : "titleVersionMatch", "fields" : [ "title" ] }, + { "name" : "sizeMatch", "fields" : [ "authors" ] } + ], + "model" : [ + { "name" : "doi", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "true", "path" : "$.pid[?(@.qualifier.classid ==\"doi\")].value" }, + { "name" : "pid", "algo" : "Null", "type" : "JSON", "weight" : "0.0", "ignoreMissing" : "true", "path" : "$.pid", "overrideMatch" : "true" }, + { "name" : "title", "algo" : "LevensteinTitle", "type" : "String", "weight" : "1.0", "ignoreMissing" : "false", "path" : "$.title[?(@.qualifier.classid ==\"main title\")].value", "length" : 250, "size" : 5 }, + { "name" : "authors", "algo" : "Null", "type" : "List", "weight" : "0.0", "ignoreMissing" : "true", "path" : "$.author[*].fullname", "size" : 200 }, + { "name" : "resulttype", "algo" : "Null", "type" : "String", "weight" : "0.0", "ignoreMissing" : "false", "path" : "$.resulttype.classid" } + ], + "synonyms": {}, + "blacklists" : { + "title" : [ + "^Inside Front Cover$", + "(?i)^Poster presentations$", + "^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$", + "^Problems with perinatal pathology\\.?$", + "(?i)^Cases? of Puerperal Convulsions$", + "(?i)^Operative Gyna?ecology$", + "(?i)^Mind the gap\\!?\\:?$", + "^Chronic fatigue syndrome\\.?$", + "^Cartas? ao editor Letters? to the Editor$", + "^Note from the Editor$", + "^Anesthesia Abstract$", + + "^Annual report$", + "(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$", + "(?i)^Graph and Table of Infectious Diseases?$", + "^Presentation$", + "(?i)^Reviews and Information on Publications$", + "(?i)^PUBLIC HEALTH SERVICES?$", + "(?i)^COMBINED TEXT-?BOOK OF OBSTETRICS AND GYN(Æ|ae)COLOGY$", + "(?i)^Adrese autora$", + "(?i)^Systematic Part .*\\. Catalogus Fossilium Austriae, Band 2: Echinoidea neogenica$", + "(?i)^Acknowledgement to Referees$", + "(?i)^Behçet's disease\\.?$", + "(?i)^Isolation and identification of restriction endonuclease.*$", + "(?i)^CEREBROVASCULAR DISEASES?.?$", + "(?i)^Screening for abdominal aortic aneurysms?\\.?$", + "^Event management$", + "(?i)^Breakfast and Crohn's disease.*\\.?$", + "^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$", + "(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$", + "^Gushi hakubutsugaku$", + + "^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$", + "^Intestinal spirocha?etosis$", + "^Treatment of Rodent Ulcer$", + "(?i)^\\W*Cloud Computing\\W*$", + "^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$", + "^Free Communications, Poster Presentations: Session [A-F]$", + + "^“The Historical Aspects? of Quackery\\.?”$", + "^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$", + "^P(er|re)-Mile Premiums for Auto Insurance\\.?$", + "(?i)^Case Report$", + "^Boletín Informativo$", + "(?i)^Glioblastoma Multiforme$", + "(?i)^Nuevos táxones animales descritos en la península Ibérica y Macaronesia desde 1994 \\(.*\\)$", + "^Zaměstnanecké výhody$", + "(?i)^The Economics of Terrorism and Counter-Terrorism: A Survey \\(Part .*\\)$", + "(?i)^Carotid body tumours?\\.?$", + "(?i)^\\[Españoles en Francia : La condición Emigrante.*\\]$", + "^Avant-propos$", + "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Head(s)? and Capital(s)?$", + "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Bases?$", + "(?i)^PUBLIC HEALTH VERSUS THE STATE$", + "^Viñetas de Cortázar$", + "(?i)^Search for heavy neutrinos and W(\\[|_|\\(|_\\{|-)?R(\\]|\\)|\\})? bosons with right-handed couplings in a left-right symmetric model in pp collisions at.*TeV(\\.)?$", + "(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$", + "(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$", + "(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$", + + "(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$", + "^Aus der AGMB$", + + "^Znanstveno-stručni prilozi$", + "(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$", + "(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$", + "(?i)^Hodnocení finanční situace podniku a návrhy na její zlepšení$", + "^Finanční analýza podniku$", + "^Financial analysis( of business)?$", + "(?i)^Textbook of Gyn(a)?(Æ)?(e)?cology$", + "^Jikken nihon shūshinsho$", + "(?i)^CORONER('|s)(s|') INQUESTS$", + "(?i)^(Μελέτη παραγόντων )?risk management( για ανάπτυξη και εφαρμογή ενός πληροφοριακού συστήματος| και ανάπτυξη συστήματος)?$", + "(?i)^Consultants' contract(s)?$", + "(?i)^Upute autorima$", + "(?i)^Bijdrage tot de Kennis van den Godsdienst der Dajaks van Lan(d|f)ak en Tajan$", + "^Joshi shin kokubun$", + "^Kōtō shōgaku dokuhon nōson'yō$", + "^Jinjō shōgaku shōka$", + "^Shōgaku shūjichō$", + "^Nihon joshi dokuhon$", + "^Joshi shin dokuhon$", + "^Chūtō kanbun dokuhon$", + "^Wabun dokuhon$", + "(?i)^(Analysis of economy selected village or town|Rozbor hospodaření vybrané obce či města)$", + "(?i)^cardiac rehabilitation$", + "(?i)^Analytical summary$", + "^Thesaurus resolutionum Sacrae Congregationis Concilii$", + "(?i)^Sumario analítico(\\s{1})?(Analitic summary)?$", + "^Prikazi i osvrti$", + "^Rodinný dům s provozovnou$", + "^Family house with an establishment$", + "^Shinsei chūtō shin kokugun$", + "^Pulmonary alveolar proteinosis(\\.?)$", + "^Shinshū kanbun$", + "^Viñeta(s?) de Rodríguez$", + "(?i)^RUBRIKA UREDNIKA$", + "^A Matching Model of the Academic Publication Market$", + "^Yōgaku kōyō$", + + "^Internetový marketing$", + "^Internet marketing$", + "^Chūtō kokugo dokuhon$", + "^Kokugo dokuhon$", + "^Antibiotic Cover for Dental Extraction(s?)$", + "^Strategie podniku$", + "^Strategy of an Enterprise$", + "(?i)^respiratory disease(s?)(\\.?)$", + "^Award(s?) for Gallantry in Civil Defence$", + "^Podniková kultura$", + "^Corporate Culture$", + "^Severe hyponatraemia in hospital inpatient(s?)(\\.?)$", + "^Pracovní motivace$", + "^Work Motivation$", + "^Kaitei kōtō jogaku dokuhon$", + "^Konsolidovaná účetní závěrka$", + "^Consolidated Financial Statements$", + "(?i)^intracranial tumour(s?)$", + "^Climate Change Mitigation Options and Directed Technical Change: A Decentralized Equilibrium Analysis$", + "^\\[CERVECERIAS MAHOU(\\.|\\:) INTERIOR\\] \\[Material gráfico\\]$", + "^Housing Market Dynamics(\\:|\\.) On the Contribution of Income Shocks and Credit Constraint(s?)$", + "^\\[Funciones auxiliares de la música en Radio París,.*\\]$", + "^Úroveň motivačního procesu jako způsobu vedení lidí$", + "^The level of motivation process as a leadership$", + "^Pay-beds in N(\\.?)H(\\.?)S(\\.?) Hospitals$", + "(?i)^news and events$", + "(?i)^NOVOSTI I DOGAĐAJI$", + "^Sansū no gakushū$", + "^Posouzení informačního systému firmy a návrh změn$", + "^Information System Assessment and Proposal for ICT Modification$", + "^Stresové zatížení pracovníků ve vybrané profesi$", + "^Stress load in a specific job$", + + "^Sunday: Poster Sessions, Pt.*$", + "^Monday: Poster Sessions, Pt.*$", + "^Wednesday: Poster Sessions, Pt.*", + "^Tuesday: Poster Sessions, Pt.*$", + + "^Analýza reklamy$", + "^Analysis of advertising$", + + "^Shōgaku shūshinsho$", + "^Shōgaku sansū$", + "^Shintei joshi kokubun$", + "^Taishō joshi kokubun dokuhon$", + "^Joshi kokubun$", + + "^Účetní uzávěrka a účetní závěrka v ČR$", + "(?i)^The \"?Causes\"? of Cancer$", + "^Normas para la publicación de artículos$", + "^Editor('|s)(s|') [Rr]eply$", + "^Editor(’|s)(s|’) letter$", + "^Redaktoriaus žodis$", + "^DISCUSSION ON THE PRECEDING PAPER$", + "^Kōtō shōgaku shūshinsho jidōyō$", + "^Shōgaku nihon rekishi$", + "^(Theory of the flow of action currents in isolated myelinated nerve fibers).*$", + "^Préface$", + "^Occupational [Hh]ealth [Ss]ervices.$", + "^In Memoriam Professor Toshiyuki TAKESHIMA$", + "^Účetní závěrka ve vybraném podniku.*$", + "^Financial statements in selected company$", + "^Abdominal [Aa]ortic [Aa]neurysms.*$", + "^Pseudomyxoma peritonei$", + "^Kazalo autora$", + + "(?i)^uvodna riječ$", + "^Motivace jako způsob vedení lidí$", + "^Motivation as a leadership$", + "^Polyfunkční dům$", + "^Multi\\-funkcional building$", + "^Podnikatelský plán$", + "(?i)^Podnikatelský záměr$", + "(?i)^Business Plan$", + "^Oceňování nemovitostí$", + "^Marketingová komunikace$", + "^Marketing communication$", + "^Sumario Analítico$", + "^Riječ uredništva$", + "^Savjetovanja i priredbe$", + "^Índice$", + "^(Starobosanski nadpisi).*$", + "^Vzdělávání pracovníků v organizaci$", + "^Staff training in organization$", + "^(Life Histories of North American Geometridae).*$", + "^Strategická analýza podniku$", + "^Strategic Analysis of an Enterprise$", + "^Sadržaj$", + "^Upute suradnicima$", + "^Rodinný dům$", + "(?i)^Fami(l)?ly house$", + "^Upute autorima$", + "^Strategic Analysis$", + "^Finanční analýza vybraného podniku$", + "^Finanční analýza$", + "^Riječ urednika$", + "(?i)^Content(s?)$", + "(?i)^Inhalt$", + "^Jinjō shōgaku shūshinsho jidōyō$", + "(?i)^Index$", + "^Chūgaku kokubun kyōkasho$", + "^Retrato de una mujer$", + "^Retrato de un hombre$", + "^Kōtō shōgaku dokuhon$", + "^Shotōka kokugo$", + "^Shōgaku dokuhon$", + "^Jinjō shōgaku kokugo dokuhon$", + "^Shinsei kokugo dokuhon$", + "^Teikoku dokuhon$", + "^Instructions to Authors$", + "^KİTAP TAHLİLİ$", + "^PRZEGLĄD PIŚMIENNICTWA$", + "(?i)^Presentación$", + "^İçindekiler$", + "(?i)^Tabl?e of contents$", + "^(CODICE DEL BEATO DE LOS REYES FERNANDO I Y SANCHA).*$", + "^(\\[MADRID\\. BIBL\\. NAC\\. N.*KING FERDINAND I.*FROM SAN ISIDORO DE LEON\\. FACUNDUS SCRIPSIT DATED.*\\]).*", + "^Editorial( Board)?$", + "(?i)^Editorial \\(English\\)$", + "^Editörden$", + "^(Corpus Oral Dialectal \\(COD\\)\\.).*$", + "^(Kiri Karl Morgensternile).*$", + "^(\\[Eksliibris Aleksandr).*\\]$", + "^(\\[Eksliibris Aleksandr).*$", + "^(Eksliibris Aleksandr).*$", + "^(Kiri A\\. de Vignolles).*$", + "^(2 kirja Karl Morgensternile).*$", + "^(Pirita kloostri idaosa arheoloogilised).*$", + "^(Kiri tundmatule).*$", + "^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$", + "^(Eksliibris Nikolai Birukovile).*$", + "^(Eksliibris Nikolai Issakovile).*$", + "^(WHP Cruise Summary Information of section).*$", + "^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$", + "^(Measurement of the spin\\-dependent structure function).*", + "(?i)^.*authors['’′]? reply\\.?$", + "(?i)^.*authors['’′]? response\\.?$" + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json new file mode 100644 index 000000000..d9a0dc8b9 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json @@ -0,0 +1,7 @@ +[ + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequential file to read", "paramRequired": true}, + {"paramName":"e", "paramLongName":"entity", "paramDescription": "the type of entity to be deduped", "paramRequired": true}, + {"paramName":"c", "paramLongName":"dedupConf", "paramDescription": "dedup configuration to be used", "paramRequired": true}, + {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "target path to save dedup result", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml new file mode 100644 index 000000000..ddea85590 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml @@ -0,0 +1,31 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hive_db_name + openaire + + + dedupConf + {"wf":{"threshold":"0.9","dedupRun":"001","entityType":"organization","orderField":"legalname","queueMaxSize":"2000","groupMaxSize":"50","slidingWindowSize":"200","idPath":".id","rootBuilder":["organization","projectOrganization_participation_isParticipant","datasourceOrganization_provision_isProvidedBy"],"includeChildren":"true"},"pace":{"clustering":[{"name":"sortedngrampairs","fields":["legalname"],"params":{"max":2,"ngramLen":"3"}},{"name":"suffixprefix","fields":["legalname"],"params":{"max":1,"len":"3"}},{"name":"urlclustering","fields":["websiteurl"],"params":{}},{"name":"keywordsclustering","fields":["legalname"],"params":{"max":2,"windowSize":4}}],"strictConditions":[{"name":"exactMatch","fields":["gridid"]}],"conditions":[{"name":"DomainExactMatch","fields":["websiteurl"]},{"name":"exactMatch","fields":["country"]}],"model":[{"name":"country","algo":"Null","type":"String","weight":"0","ignoreMissing":"false","path":".country.classid"},{"name":"legalshortname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.1","ignoreMissing":"true","path":".legalshortname.value"},{"name":"legalname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.9","ignoreMissing":"false","path":".legalname.value","params":{"windowSize":4,"threshold":0.7}},{"name":"websiteurl","algo":"Null","type":"URL","weight":"0","ignoreMissing":"true","path":".websiteurl.value","params":{"host":0.5,"path":0.5}},{"name":"gridid","algo":"Null","type":"String","weight":"0.0","ignoreMissing":"true","path":".pid[] | select(.qualifier.classid==\"grid\") | .value"}],"blacklists":{"legalname":[]},"synonyms":{"key::1":["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti"],"key::2":["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"],"key::3":["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"],"key::4":["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"],"key::5":["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"],"key::6":["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"],"key::7":["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"],"key::8":["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"],"key::9":["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"],"key::10":["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"],"key::11":["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"],"key::12":["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"],"key::13":["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"],"key::14":["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"],"key::15":["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"],"key::16":["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"],"key::17":["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"],"key::18":["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"],"key::19":["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"],"key::20":["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"],"key::21":["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"],"key::22":["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"],"key::23":["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"],"key::24":["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"],"key::25":["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"],"key::26":["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"],"key::27":["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"],"key::28":["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"],"key::29":["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"],"key::30":["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"],"key::31":["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"],"key::32":["industry","industria","industrie","индустрия","industrie","βιομηχανία"],"key::33":["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"],"key::34":["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"],"key::35":["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"],"key::36":["authority","autorità","autorité","авторитет","autoriteit"],"key::37":["federation","federazione","fédération","федерация","federatie","ομοσπονδία"],"key::38":["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"],"key::39":["bureau","ufficio","bureau","офис","bureau","γραφείο"],"key::40":["company","impresa","compagnie","société","компания","bedrijf","εταιρία"],"key::41":["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"],"key::42":["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"],"key::43":["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"],"key::44":["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"],"key::45":["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"],"key::46":["division","divisione","division","отделение","divisie","τμήμα"],"key::47":["committee","comitato","comité","комитет","commissie","επιτροπή"],"key::48":["promotion","promozione","продвижение","proothisis","forderung"],"key::49":["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"],"key::50":["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"],"key::51":["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""],"key::52":["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""],"key::53":["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""],"key::54":["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""],"key::55":["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""],"key::56":["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""],"key::57":["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""],"key::58":["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""],"key::59":["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""],"key::60":["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""],"key::61":["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""],"key::62":["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""],"key::63":["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""],"key::64":["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""],"key::65":["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""],"key::66":["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""],"key::67":["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""],"key::68":["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""],"key::69":["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""],"key::70":["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""],"key::71":["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""],"key::72":["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""],"key::73":["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""],"key::74":["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""],"key::75":["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""],"key::76":["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""],"key::77":["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""],"key::78":["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""],"key::79":["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""],"key::80":["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""],"key::81":["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""],"key::82":["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""],"key::83":["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""],"key::84":["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""],"key::85":["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""],"key::86":["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""],"key::87":["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""],"key::88":["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""],"key::89":["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""],"key::90":["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""],"key::91":["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""],"key::92":["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""],"key::93":["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""],"key::94":["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""],"key::95":["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""],"key::96":["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""],"key::97":["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""],"key::98":["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""],"key::99":["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""],"key::100":["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""],"key::101":["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""],"key::102":["informatics","informatica","informática","informática","informatica",""],"key::103":["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"],"key::104":["commerce","ticaret","ticarət","commercio","trade","handel","comercio"]}}} + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml new file mode 100644 index 000000000..7f9de8af5 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml @@ -0,0 +1,101 @@ + + + + sourcePath + the source path + + + entity + the entity that should be processed + + + dedupConf + the dedup Configuration + + + targetPath + the target path + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + Create Similarity Relations + eu.dnetlib.dedup.SparkCreateSimRels + dhp-dedup-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} --conf + spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf + spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf + spark.sql.warehouse.dir="/user/hive/warehouse" + + -mtyarn-cluster + --sourcePath${sourcePath} + --targetPath${targetPath} + --entity${entity} + --dedupConf${dedupConf} + + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + Create Connected Components + eu.dnetlib.dedup.SparkCreateConnectedComponent + dhp-dedup-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} --conf + spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf + spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf + spark.sql.warehouse.dir="/user/hive/warehouse" + + -mtyarn-cluster + --sourcePath${sourcePath} + --targetPath${targetPath} + --entity${entity} + --dedupConf${dedupConf} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java new file mode 100644 index 000000000..b16b45ef1 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -0,0 +1,62 @@ +package eu.dnetlib.dedup; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; + +public class SparkCreateDedupTest { + + + + @Before + public void setUp() throws IOException { + FileUtils.deleteDirectory(new File("/tmp/pub_dedup_vertex")); + FileUtils.deleteDirectory(new File("/tmp/pub_dedup_rels")); + } + + + + @Test + @Ignore + public void dedupTest() throws Exception { + final String configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json")); + + + SparkCreateSimRels.main(new String[] { + "-mt", "local[*]", + "-s", "/home/sandro/betadump", + "-e", "publication", + "-c", configuration, + "-t", "/tmp/dedup", + }); + + SparkCreateConnectedComponent.main(new String[] { + "-mt", "local[*]", + "-s", "/home/sandro/betadump", + "-e", "publication", + "-c", configuration, + "-t", "/tmp/dedup", + }); + } + + @Test + @Ignore + public void dedupRecordTest() throws Exception { + SparkCreateDedupRecord.main(new String[] { + "-mt", "local[*]", + "-s", "/home/sandro/betadump", + "-e", "publication", + "-c", "configuration", + "-t", "/tmp/dedup", + }); + } + + + + +} diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/jpath/JsonPathTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/jpath/JsonPathTest.java new file mode 100644 index 000000000..7a63cfe24 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/jpath/JsonPathTest.java @@ -0,0 +1,31 @@ +package eu.dnetlib.dedup.jpath; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.jayway.jsonpath.JsonPath; +import org.apache.commons.io.IOUtils; +import org.junit.Test; +import java.util.List; +import java.util.Map; + +public class JsonPathTest { + + @Test + public void testJPath () throws Exception { + final String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/sample.json")); + List> pid = JsonPath.read(json, "$.pid[*]"); +// System.out.println(json); + + pid.forEach(it -> { + try { + System.out.println(new ObjectMapper().writeValueAsString(it)); + } catch (JsonProcessingException e) { + e.printStackTrace(); + } + }); + + + + + } +} diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json new file mode 100644 index 000000000..55c1d2066 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json @@ -0,0 +1,1753 @@ +{ + "wf": { + "threshold": "0.9", + "dedupRun": "001", + "entityType": "organization", + "orderField": "legalname", + "queueMaxSize": "2000", + "groupMaxSize": "50", + "slidingWindowSize": "200", + "idPath": "$.id", + "rootBuilder": [ + "organization", + "projectOrganization_participation_isParticipant", + "datasourceOrganization_provision_isProvidedBy" + ], + "includeChildren": "true" + }, + "pace": { + "clustering": [ + { + "name": "sortedngrampairs", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "ngramLen": "3" + } + }, + { + "name": "suffixprefix", + "fields": [ + "legalname" + ], + "params": { + "max": 1, + "len": "3" + } + }, + { + "name": "urlclustering", + "fields": [ + "websiteurl" + ], + "params": {} + }, + { + "name": "keywordsclustering", + "fields": [ + "legalname" + ], + "params": { + "max": 2, + "windowSize": 4 + } + } + ], + "strictConditions": [ + { + "name": "exactMatch", + "fields": [ + "gridid" + ] + } + ], + "conditions": [ + { + "name": "DomainExactMatch", + "fields": [ + "websiteurl" + ] + }, + { + "name": "exactMatch", + "fields": [ + "country" + ] + } + ], + "model": [ + { + "name": "country", + "algo": "Null", + "type": "String", + "weight": "0", + "ignoreMissing": "false", + "path": ".country.classid" + }, + { + "name": "legalshortname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.1", + "ignoreMissing": "true", + "path": ".legalshortname.value" + }, + { + "name": "legalname", + "algo": "JaroWinklerNormalizedName", + "type": "String", + "weight": "0.9", + "ignoreMissing": "false", + "path": ".legalname.value", + "params": { + "windowSize": 4, + "threshold": 0.7 + } + }, + { + "name": "websiteurl", + "algo": "Null", + "type": "URL", + "weight": "0", + "ignoreMissing": "true", + "path": ".websiteurl.value", + "params": { + "host": 0.5, + "path": 0.5 + } + }, + { + "name": "gridid", + "algo": "Null", + "type": "String", + "weight": "0.0", + "ignoreMissing": "true", + "path": ".pid[] | select(.qualifier.classid==\"grid\") | .value" + } + ], + "blacklists": { + "legalname": [] + }, + "synonyms": { + "key::1": [ + "university", + "università", + "università studi", + "universitario", + "universitaria", + "université", + "universitaire", + "universitaires", + "universidad", + "universitade", + "Universität", + "universitaet", + "Uniwersytet", + "университет", + "universiteit", + "πανεπιστήμιο", + "universitesi", + "universiteti" + ], + "key::2": [ + "studies", + "studi", + "études", + "estudios", + "estudos", + "Studien", + "studia", + "исследования", + "studies", + "σπουδές" + ], + "key::3": [ + "advanced", + "superiore", + "supérieur", + "supérieure", + "supérieurs", + "supérieures", + "avancado", + "avancados", + "fortgeschrittene", + "fortgeschritten", + "zaawansowany", + "передовой", + "gevorderd", + "gevorderde", + "προχωρημένος", + "προχωρημένη", + "προχωρημένο", + "προχωρημένες", + "προχωρημένα", + "wyzsza" + ], + "key::4": [ + "institute", + "istituto", + "institut", + "instituto", + "instituto", + "Institut", + "instytut", + "институт", + "instituut", + "ινστιτούτο" + ], + "key::5": [ + "hospital", + "ospedale", + "hôpital", + "hospital", + "hospital", + "Krankenhaus", + "szpital", + "больница", + "ziekenhuis", + "νοσοκομείο" + ], + "key::6": [ + "research", + "ricerca", + "recherche", + "investigacion", + "pesquisa", + "Forschung", + "badania", + "исследования", + "onderzoek", + "έρευνα", + "erevna", + "erevnas" + ], + "key::7": [ + "college", + "collegio", + "université", + "colegio", + "faculdade", + "Hochschule", + "Szkoła Wyższa", + "Высшая школа", + "universiteit", + "κολλέγιο" + ], + "key::8": [ + "foundation", + "fondazione", + "fondation", + "fundación", + "fundação", + "Stiftung", + "Fundacja", + "фонд", + "stichting", + "ίδρυμα", + "idryma" + ], + "key::9": [ + "center", + "centro", + "centre", + "centro", + "centro", + "zentrum", + "centrum", + "центр", + "centrum", + "κέντρο" + ], + "key::10": [ + "national", + "nazionale", + "national", + "nationale", + "nationaux", + "nationales", + "nacional", + "nacional", + "national", + "krajowy", + "национальный", + "nationaal", + "nationale", + "εθνικό" + ], + "key::11": [ + "association", + "associazione", + "association", + "asociación", + "associação", + "Verein", + "verband", + "stowarzyszenie", + "ассоциация", + "associatie" + ], + "key::12": [ + "society", + "societa", + "société", + "sociedad", + "sociedade", + "gesellschaft", + "społeczeństwo", + "общество", + "maatschappij", + "κοινωνία" + ], + "key::13": [ + "international", + "internazionale", + "international", + "internacional", + "internacional", + "international", + "międzynarodowy", + "Международный", + "internationaal", + "internationale", + "διεθνής", + "διεθνή", + "διεθνές" + ], + "key::14": [ + "community", + "comunita", + "communauté", + "comunidad", + "comunidade", + "Gemeinschaft", + "społeczność", + "сообщество", + "gemeenschap", + "κοινότητα" + ], + "key::15": [ + "school", + "scuola", + "école", + "escuela", + "escola", + "schule", + "Szkoła", + "школа", + "school", + "σχολείο" + ], + "key::16": [ + "education", + "educazione", + "éducation", + "educacion", + "Educação", + "Bildung", + "Edukacja", + "образование", + "opleiding", + "εκπαίδευση" + ], + "key::17": [ + "academy", + "accademia", + "académie", + "academia", + "academia", + "Akademie", + "akademie", + "академия", + "academie", + "ακαδημία" + ], + "key::18": [ + "public", + "pubblico", + "public", + "publique", + "publics", + "publiques", + "publico", + "publico", + "Öffentlichkeit", + "publiczny", + "публичный", + "publiek", + "publieke", + "δημόσιος", + "δημόσια", + "δημόσιο" + ], + "key::19": [ + "museum", + "museo", + "musée", + "mueso", + "museu", + "museum", + "muzeum", + "музей", + "museum", + "μουσείο" + ], + "key::20": [ + "group", + "gruppo", + "groupe", + "grupo", + "grupo", + "gruppe", + "grupa", + "группа", + "groep", + "ομάδα", + "όμιλος" + ], + "key::21": [ + "department", + "dipartimento", + "département", + "departamento", + "departamento", + "abteilung", + "departament", + "отдел", + "afdeling", + "τμήμα" + ], + "key::22": [ + "council", + "consiglio", + "conseil", + "Consejo", + "conselho", + "gesellschaft", + "rada", + "совет", + "raad", + "συμβούλιο" + ], + "key::23": [ + "library", + "biblioteca", + "bibliothèque", + "biblioteca", + "biblioteca", + "Bibliothek", + "biblioteka", + "библиотека", + "bibliotheek", + "βιβλιοθήκη" + ], + "key::24": [ + "ministry", + "ministero", + "ministère", + "ministerio", + "ministério", + "Ministerium", + "ministerstwo", + "министерство", + "ministerie", + "υπουργείο" + ], + "key::25": [ + "services", + "servizi", + "services", + "servicios", + "Serviços", + "Dienstleistungen", + "usługi", + "услуги", + "diensten", + "υπηρεσίες" + ], + "key::26": [ + "central", + "centrale", + "central", + "centrale", + "centrales", + "central", + "central", + "zentral", + "centralny", + "цетральный", + "centraal", + "κεντρικός", + "κεντρική", + "κεντρικό", + "κεντρικά" + ], + "key::27": [ + "general", + "generale", + "général", + "générale", + "généraux", + "générales", + "general", + "geral", + "general", + "Allgemeines", + "general", + "общий", + "algemeen", + "algemene", + "γενικός", + "γενική", + "γενικό", + "γενικά" + ], + "key::28": [ + "applied", + "applicati", + "appliqué", + "appliquée", + "appliqués", + "appliquées", + "aplicado", + "aplicada", + "angewendet", + "stosowany", + "прикладной", + "toegepast", + "toegepaste", + "εφαρμοσμένος", + "εφαρμοσμένη", + "εφαρμοσμένο", + "εφαρμοσμένα" + ], + "key::29": [ + "european", + "europee", + "europea", + "européen", + "européenne", + "européens", + "européennes", + "europeo", + "europeu", + "europäisch", + "europejski", + "европейский", + "Europees", + "Europese", + "ευρωπαϊκός", + "ευρωπαϊκή", + "ευρωπαϊκό", + "ευρωπαϊκά" + ], + "key::30": [ + "agency", + "agenzia", + "agence", + "agencia", + "agencia", + "agentur", + "agencja", + "агенция", + "agentschap", + "πρακτορείο" + ], + "key::31": [ + "laboratory", + "laboratorio", + "laboratoire", + "laboratorio", + "laboratorio", + "labor", + "laboratorium", + "лаборатория", + "laboratorium", + "εργαστήριο" + ], + "key::32": [ + "industry", + "industria", + "industrie", + "индустрия", + "industrie", + "βιομηχανία" + ], + "key::33": [ + "industrial", + "industriale", + "industriel", + "industrielle", + "industriels", + "industrielles", + "индустриальный", + "industrieel", + "βιομηχανικός", + "βιομηχανική", + "βιομηχανικό", + "βιομηχανικά", + "βιομηχανικές" + ], + "key::34": [ + "consortium", + "consorzio", + "consortium", + "консорциум", + "consortium", + "κοινοπραξία" + ], + "key::35": [ + "organization", + "organizzazione", + "organisation", + "organización", + "organização", + "organizacja", + "организация", + "organisatie", + "οργανισμός" + ], + "key::36": [ + "authority", + "autorità", + "autorité", + "авторитет", + "autoriteit" + ], + "key::37": [ + "federation", + "federazione", + "fédération", + "федерация", + "federatie", + "ομοσπονδία" + ], + "key::38": [ + "observatory", + "osservatorio", + "observatoire", + "обсерватория", + "observatorium", + "αστεροσκοπείο" + ], + "key::39": [ + "bureau", + "ufficio", + "bureau", + "офис", + "bureau", + "γραφείο" + ], + "key::40": [ + "company", + "impresa", + "compagnie", + "société", + "компания", + "bedrijf", + "εταιρία" + ], + "key::41": [ + "polytechnic", + "politecnico", + "polytechnique", + "политехника", + "polytechnisch", + "πολυτεχνείο", + "universita politecnica", + "polytechnic university", + "universidad politecnica", + "universitat politecnica", + "politechnika", + "politechniki", + "university technology", + "university science technology" + ], + "key::42": [ + "coalition", + "coalizione", + "coalition", + "коалиция", + "coalitie", + "συνασπισμός" + ], + "key::43": [ + "initiative", + "iniziativa", + "initiative", + "инициатива", + "initiatief", + "πρωτοβουλία" + ], + "key::44": [ + "academic", + "accademico", + "académique", + "universitaire", + "акадеческий academisch", + "ακαδημαϊκός", + "ακαδημαϊκή", + "ακαδημαϊκό", + "ακαδημαϊκές", + "ακαδημαϊκοί" + ], + "key::45": [ + "institution", + "istituzione", + "institution", + "институциональный", + "instelling", + "ινστιτούτο" + ], + "key::46": [ + "division", + "divisione", + "division", + "отделение", + "divisie", + "τμήμα" + ], + "key::47": [ + "committee", + "comitato", + "comité", + "комитет", + "commissie", + "επιτροπή" + ], + "key::48": [ + "promotion", + "promozione", + "продвижение", + "proothisis", + "forderung" + ], + "key::49": [ + "medical", + "medicine", + "clinical", + "medicina", + "clinici", + "médico", + "medicina", + "clínica", + "médico", + "medicina", + "clínica", + "medizinisch", + "Medizin", + "klinisch", + "medisch", + "geneeskunde", + "klinisch", + "ιατρικός", + "ιατρική", + "ιατρικό", + "ιατρικά", + "κλινικός", + "κλινική", + "κλινικό", + "κλινικά", + "tıbbi", + "tıp", + "klinik", + "orvosi", + "orvostudomány", + "klinikai", + "zdravniški", + "medicinski", + "klinični", + "meditsiini", + "kliinik", + "kliiniline" + ], + "key::50": [ + "technology", + "technological", + "tecnologia", + "tecnologie", + "tecnología", + "tecnológico", + "tecnologia", + "tecnológico", + "Technologie", + "technologisch", + "technologie", + "technologisch", + "τεχνολογία", + "τεχνολογικός", + "τεχνολογική", + "τεχνολογικό", + "teknoloji", + "teknolojik", + "technológia", + "technológiai", + "tehnologija", + "tehnološki", + "tehnoloogia", + "tehnoloogiline", + "technologii", + "technical", + "texniki", + "teknik" + ], + "key::51": [ + "science", + "scientific", + "scienza", + "scientifiche", + "scienze", + "ciencia", + "científico", + "ciência", + "científico", + "Wissenschaft", + "wissenschaftlich", + "wetenschap", + "wetenschappelijk", + "επιστήμη", + "επιστημονικός", + "επιστημονική", + "επιστημονικό", + "επιστημονικά", + "bilim", + "bilimsel", + "tudomány", + "tudományos", + "znanost", + "znanstveni", + "teadus", + "teaduslik", + "" + ], + "key::52": [ + "engineering", + "ingegneria", + "ingeniería", + "engenharia", + "Ingenieurwissenschaft", + "ingenieurswetenschappen", + "bouwkunde", + "μηχανικός", + "μηχανική", + "μηχανικό", + "mühendislik", + "mérnöki", + "Inženirstvo", + "inseneeria", + "inseneri", + "" + ], + "key::53": [ + "management", + "gestione", + "gestionale", + "gestionali", + "gestión", + "administración", + "gestão", + "administração", + "Verwaltung", + "management", + "διαχείριση", + "yönetim", + "menedzsment", + "vodstvo", + "upravljanje", + "management", + "juhtkond", + "juhtimine", + "haldus", + "" + ], + "key::54": [ + "energy", + "energia", + "energía", + "energia", + "Energie", + "energie", + "ενέργεια", + "enerji", + "energia", + "energija", + "energia", + "" + ], + "key::55": [ + "agricultural", + "agriculture", + "agricoltura", + "agricole", + "agrícola", + "agricultura", + "agrícola", + "agricultura", + "landwirtschaftlich", + "Landwirtschaft", + "landbouwkundig", + "landbouw", + "αγροτικός", + "αγροτική", + "αγροτικό", + "γεωργικός", + "γεωργική", + "γεωργικό", + "γεωργία", + "tarımsal", + "tarım", + "mezőgazdasági", + "mezőgazdaság", + "poljedelski", + "poljedelstvo", + "põllumajandus", + "põllumajanduslik", + "" + ], + "key::56": [ + "information", + "informazione", + "información", + "informação", + "Information", + "informatie", + "πληροφορία", + "bilgi", + "információ", + "informacija", + "informatsioon", + "informatycznych", + "" + ], + "key::57": [ + "social", + "sociali", + "social", + "social", + "Sozial", + "sociaal", + "maatschappelijk", + "κοινωνικός", + "κοινωνική", + "κοινωνικό", + "κοινωνικά", + "sosyal", + "szociális", + "družbeni", + "sotsiaal", + "sotsiaalne", + "" + ], + "key::58": [ + "environmental", + "ambiente", + "medioambiental", + "ambiente", + "medioambiente", + "meioambiente", + "Umwelt", + "milieu", + "milieuwetenschap", + "milieukunde", + "περιβαλλοντικός", + "περιβαλλοντική", + "περιβαλλοντικό", + "περιβαλλοντικά", + "çevre", + "környezeti", + "okoliški", + "keskonna", + "" + ], + "key::59": [ + "business", + "economia", + "economiche", + "economica", + "negocio", + "empresa", + "negócio", + "Unternehmen", + "bedrijf", + "bedrijfskunde", + "επιχείρηση", + "iş", + "üzleti", + "posel", + "ettevõte/äri", + "" + ], + "key::60": [ + "pharmaceuticals", + "pharmacy", + "farmacia", + "farmaceutica", + "farmacéutica", + "farmacia", + "farmacêutica", + "farmácia", + "Pharmazeutika", + "Arzneimittelkunde", + "farmaceutica", + "geneesmiddelen", + "apotheek", + "φαρμακευτικός", + "φαρμακευτική", + "φαρμακευτικό", + "φαρμακευτικά", + "φαρμακείο", + "ilaç", + "eczane", + "gyógyszerészeti", + "gyógyszertár", + "farmacevtika", + "lekarništvo", + "farmaatsia", + "farmatseutiline", + "" + ], + "key::61": [ + "healthcare", + "health services", + "salute", + "atenciónmédica", + "cuidadodelasalud", + "cuidadoscomasaúde", + "Gesundheitswesen", + "gezondheidszorg", + "ιατροφαρμακευτικήπερίθαλψη", + "sağlıkhizmeti", + "egészségügy", + "zdravstvo", + "tervishoid", + "tervishoiu", + "" + ], + "key::62": [ + "history", + "storia", + "historia", + "história", + "Geschichte", + "geschiedenis", + "geschiedkunde", + "ιστορία", + "tarih", + "történelem", + "zgodovina", + "ajalugu", + "" + ], + "key::63": [ + "materials", + "materiali", + "materia", + "materiales", + "materiais", + "materialen", + "υλικά", + "τεκμήρια", + "malzemeler", + "anyagok", + "materiali", + "materjalid", + "vahendid", + "" + ], + "key::64": [ + "economics", + "economia", + "economiche", + "economica", + "economía", + "economia", + "Wirtschaft", + "economie", + "οικονομικά", + "οικονομικέςεπιστήμες", + "ekonomi", + "közgazdaságtan", + "gospodarstvo", + "ekonomija", + "majanduslik", + "majandus", + "" + ], + "key::65": [ + "therapeutics", + "terapeutica", + "terapéutica", + "terapêutica", + "therapie", + "θεραπευτική", + "tedavibilimi", + "gyógykezelés", + "terapevtika", + "terapeutiline", + "ravi", + "" + ], + "key::66": [ + "oncology", + "oncologia", + "oncologico", + "oncología", + "oncologia", + "Onkologie", + "oncologie", + "ογκολογία", + "onkoloji", + "onkológia", + "onkologija", + "onkoloogia", + "" + ], + "key::67": [ + "natural", + "naturali", + "naturale", + "natural", + "natural", + "natürlich", + "natuurlijk", + "φυσικός", + "φυσική", + "φυσικό", + "φυσικά", + "doğal", + "természetes", + "naraven", + "loodus", + "" + ], + "key::68": [ + "educational", + "educazione", + "pedagogia", + "educacional", + "educativo", + "educacional", + "pädagogisch", + "educatief", + "εκπαιδευτικός", + "εκπαιδευτική", + "εκπαιδευτικό", + "εκπαιδευτικά", + "eğitimsel", + "oktatási", + "izobraževalen", + "haridus", + "hariduslik", + "" + ], + "key::69": [ + "biomedical", + "biomedica", + "biomédico", + "biomédico", + "biomedizinisch", + "biomedisch", + "βιοιατρικός", + "βιοιατρική", + "βιοιατρικό", + "βιοιατρικά", + "biyomedikal", + "orvosbiológiai", + "biomedicinski", + "biomeditsiiniline", + "" + ], + "key::70": [ + "veterinary", + "veterinaria", + "veterinarie", + "veterinaria", + "veterinária", + "tierärtzlich", + "veterinair", + "veeartsenijlkunde", + "κτηνιατρικός", + "κτηνιατρική", + "κτηνιατρικό", + "κτηνιατρικά", + "veteriner", + "állatorvosi", + "veterinar", + "veterinarski", + "veterinaaria", + "" + ], + "key::71": [ + "chemistry", + "chimica", + "química", + "química", + "Chemie", + "chemie", + "scheikunde", + "χημεία", + "kimya", + "kémia", + "kemija", + "keemia", + "" + ], + "key::72": [ + "security", + "sicurezza", + "seguridad", + "segurança", + "Sicherheit", + "veiligheid", + "ασφάλεια", + "güvenlik", + "biztonsági", + "varnost", + "turvalisus", + "julgeolek", + "" + ], + "key::73": [ + "biotechnology", + "biotecnologia", + "biotecnologie", + "biotecnología", + "biotecnologia", + "Biotechnologie", + "biotechnologie", + "βιοτεχνολογία", + "biyoteknoloji", + "biotechnológia", + "biotehnologija", + "biotehnoloogia", + "" + ], + "key::74": [ + "military", + "militare", + "militari", + "militar", + "militar", + "Militär", + "militair", + "leger", + "στρατιωτικός", + "στρατιωτική", + "στρατιωτικό", + "στρατιωτικά", + "askeri", + "katonai", + "vojaški", + "vojni", + "militaar", + "wojskowa", + "" + ], + "key::75": [ + "theological", + "teologia", + "teologico", + "teológico", + "tecnológica", + "theologisch", + "theologisch", + "θεολογικός", + "θεολογική", + "θεολογικό", + "θεολογικά", + "teolojik", + "technológiai", + "teološki", + "teoloogia", + "usuteadus", + "teoloogiline", + "" + ], + "key::76": [ + "electronics", + "elettronica", + "electrónica", + "eletrônicos", + "Elektronik", + "elektronica", + "ηλεκτρονική", + "elektronik", + "elektronika", + "elektronika", + "elektroonika", + "" + ], + "key::77": [ + "forestry", + "forestale", + "forestali", + "silvicultura", + "forestal", + "floresta", + "Forstwirtschaft", + "bosbouw", + "δασοκομία", + "δασολογία", + "ormancılık", + "erdészet", + "gozdarstvo", + "metsandus", + "" + ], + "key::78": [ + "maritime", + "marittima", + "marittime", + "marittimo", + "marítimo", + "marítimo", + "maritiem", + "ναυτικός", + "ναυτική", + "ναυτικό", + "ναυτικά", + "ναυτιλιακός", + "ναυτιλιακή", + "ναυτιλιακό", + "ναυτιλιακά", + "θαλάσσιος", + "θαλάσσια", + "θαλάσσιο", + "denizcilik", + "tengeri", + "morski", + "mere", + "merendus", + "" + ], + "key::79": [ + "sports", + "sport", + "deportes", + "esportes", + "Sport", + "sport", + "sportwetenschappen", + "άθληση", + "γυμναστικήδραστηριότητα", + "spor", + "sport", + "šport", + "sport", + "spordi", + "" + ], + "key::80": [ + "surgery", + "chirurgia", + "chirurgiche", + "cirugía", + "cirurgia", + "Chirurgie", + "chirurgie", + "heelkunde", + "εγχείρηση", + "επέμβαση", + "χειρουργικήεπέμβαση", + "cerrahi", + "sebészet", + "kirurgija", + "kirurgia", + "" + ], + "key::81": [ + "cultural", + "culturale", + "culturali", + "cultura", + "cultural", + "cultural", + "kulturell", + "cultureel", + "πολιτιστικός", + "πολιτιστική", + "πολιτιστικό", + "πολιτισμικός", + "πολιτισμική", + "πολιτισμικό", + "kültürel", + "kultúrális", + "kulturni", + "kultuuri", + "kultuuriline", + "" + ], + "key::82": [ + "computerscience", + "informatica", + "ordenador", + "computadora", + "informática", + "computación", + "cienciasdelacomputación", + "ciênciadacomputação", + "Computer", + "computer", + "υπολογιστής", + "ηλεκτρονικόςυπολογιστής", + "bilgisayar", + "számítógép", + "računalnik", + "arvuti", + "" + ], + "key::83": [ + "finance", + "financial", + "finanza", + "finanziarie", + "finanza", + "financiero", + "finanças", + "financeiro", + "Finanzen", + "finanziell", + "financiën", + "financieel", + "χρηματοοικονομικά", + "χρηματοδότηση", + "finanse", + "finansal", + "pénzügy", + "pénzügyi", + "finance", + "finančni", + "finants", + "finantsiline", + "" + ], + "key::84": [ + "communication", + "comunicazione", + "comuniciación", + "comunicação", + "Kommunikation", + "communication", + "επικοινωνία", + "iletişim", + "kommunikáció", + "komuniciranje", + "kommunikatsioon", + "" + ], + "key::85": [ + "justice", + "giustizia", + "justicia", + "justiça", + "Recht", + "Justiz", + "justitie", + "gerechtigheid", + "δικαιοσύνη", + "υπουργείοδικαιοσύνης", + "δίκαιο", + "adalet", + "igazságügy", + "pravo", + "õigus", + "" + ], + "key::86": [ + "aerospace", + "aerospaziale", + "aerospaziali", + "aeroespacio", + "aeroespaço", + "Luftfahrt", + "luchtvaart", + "ruimtevaart", + "αεροπορικός", + "αεροπορική", + "αεροπορικό", + "αεροναυπηγικός", + "αεροναυπηγική", + "αεροναυπηγικό", + "αεροναυπηγικά", + "havacılıkveuzay", + "légtér", + "zrakoplovstvo", + "atmosfäär", + "kosmos", + "" + ], + "key::87": [ + "dermatology", + "dermatologia", + "dermatología", + "dermatologia", + "Dermatologie", + "dermatologie", + "δρματολογία", + "dermatoloji", + "bőrgyógyászat", + "dermatológia", + "dermatologija", + "dermatoloogia", + "" + ], + "key::88": [ + "architecture", + "architettura", + "arquitectura", + "arquitetura", + "Architektur", + "architectuur", + "αρχιτεκτονική", + "mimarlık", + "építészet", + "arhitektura", + "arhitektuur", + "" + ], + "key::89": [ + "mathematics", + "matematica", + "matematiche", + "matemáticas", + "matemáticas", + "Mathematik", + "wiskunde", + "mathematica", + "μαθηματικά", + "matematik", + "matematika", + "matematika", + "matemaatika", + "" + ], + "key::90": [ + "language", + "lingue", + "linguistica", + "linguistiche", + "lenguaje", + "idioma", + "língua", + "idioma", + "Sprache", + "taal", + "taalkunde", + "γλώσσα", + "dil", + "nyelv", + "jezik", + "keel", + "" + ], + "key::91": [ + "neuroscience", + "neuroscienza", + "neurociencia", + "neurociência", + "Neurowissenschaft", + "neurowetenschappen", + "νευροεπιστήμη", + "nörobilim", + "idegtudomány", + "nevroznanost", + "neuroteadused", + "" + ], + "key::92": [ + "automation", + "automazione", + "automatización", + "automação", + "Automatisierung", + "automatisering", + "αυτοματοποίηση", + "otomasyon", + "automatizálás", + "avtomatizacija", + "automatiseeritud", + "" + ], + "key::93": [ + "pediatric", + "pediatria", + "pediatriche", + "pediatrico", + "pediátrico", + "pediatría", + "pediátrico", + "pediatria", + "pädiatrisch", + "pediatrische", + "παιδιατρική", + "pediatrik", + "gyermekgyógyászat", + "pediatrija", + "pediaatria", + "" + ], + "key::94": [ + "photonics", + "fotonica", + "fotoniche", + "fotónica", + "fotônica", + "Photonik", + "fotonica", + "φωτονική", + "fotonik", + "fotonika", + "fotonika", + "fotoonika", + "" + ], + "key::95": [ + "mechanics", + "meccanica", + "meccaniche", + "mecánica", + "mecânica", + "Mechanik", + "Maschinenbau", + "mechanica", + "werktuigkunde", + "μηχανικής", + "mekanik", + "gépészet", + "mehanika", + "mehaanika", + "" + ], + "key::96": [ + "psychiatrics", + "psichiatria", + "psichiatrica", + "psichiatriche", + "psiquiatría", + "psiquiatria", + "Psychiatrie", + "psychiatrie", + "ψυχιατρική", + "psikiyatrik", + "pszihiátria", + "psihiatrija", + "psühhaatria", + "" + ], + "key::97": [ + "psychology", + "fisiologia", + "psicología", + "psicologia", + "Psychologie", + "psychologie", + "ψυχολογία", + "psikoloji", + "pszihológia", + "psihologija", + "psühholoogia", + "" + ], + "key::98": [ + "automotive", + "industriaautomobilistica", + "industriadelautomóvil", + "automotriz", + "industriaautomotriz", + "automotivo", + "Automobilindustrie", + "autoindustrie", + "αυτοκίνητος", + "αυτοκίνητη", + "αυτοκίνητο", + "αυτοκινούμενος", + "αυτοκινούμενη", + "αυτοκινούμενο", + "αυτοκινητιστικός", + "αυτοκινητιστική", + "αυτοκινητιστικό", + "otomotiv", + "autóipari", + "samogiben", + "avtomobilskaindustrija", + "auto-", + "" + ], + "key::99": [ + "neurology", + "neurologia", + "neurologiche", + "neurología", + "neurologia", + "Neurologie", + "neurologie", + "zenuwleer", + "νευρολογία", + "nöroloji", + "neurológia", + "ideggyógyászat", + "nevrologija", + "neuroloogia", + "" + ], + "key::100": [ + "geology", + "geologia", + "geologiche", + "geología", + "geologia", + "Geologie", + "geologie", + "aardkunde", + "γεωλογία", + "jeoloji", + "geológia", + "földtudomány", + "geologija", + "geoloogia", + "" + ], + "key::101": [ + "microbiology", + "microbiologia", + "micro-biologia", + "microbiologiche", + "microbiología", + "microbiologia", + "Mikrobiologie", + "microbiologie", + "μικροβιολογία", + "mikrobiyoloji", + "mikrobiológia", + "mikrobiologija", + "mikrobioloogia", + "" + ], + "key::102": [ + "informatics", + "informatica", + "informática", + "informática", + "informatica", + "" + ], + "key::103": [ + "forschungsgemeinschaft", + "comunita ricerca", + "research community", + "research foundation", + "research association" + ], + "key::104": [ + "commerce", + "ticaret", + "ticarət", + "commercio", + "trade", + "handel", + "comercio" + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/sample.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/sample.json new file mode 100644 index 000000000..1e31429bb --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/sample.json @@ -0,0 +1 @@ +{"eclegalbody": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "ecresearchorganization": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "logourl": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "pid": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "qualifier": {"classid": "urn", "classname": "urn", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "http://en.wikipedia.org/wiki/Trudeau_Institute"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "qualifier": {"classid": "grid", "classname": "grid", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "grid.250945.f"}, {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "qualifier": {"classid": "mag_id", "classname": "Microsoft Academic Graph Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "https://academic.microsoft.com/#/detail/125275712"}], "websiteurl": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": "http://www.trudeauinstitute.org/"}, "ecnutscode": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "legalname": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": "Trudeau Institute"}, "collectedfrom": [{"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": "Microsoft Academic Graph", "key": "10|openaire____::5f532a3fc4f1ea403f37070f59a7a53a"}], "dataInfo": {"invisible": false, "provenanceaction": {"classid": "sysimport:actionset", "classname": "sysimport:actionset", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "deletedbyinference": true, "inferred": true, "inferenceprovenance": "dedup-similarity-organization-simple"}, "alternativeNames": [], "echighereducation": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "id": "20|microsoft___::0f02f8d8b343f10fd7ecb83ad9682e21", "eclegalperson": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "lastupdatetimestamp": 0, "ecinternationalorganizationeurinterests": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "ecnonprofit": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "ecenterprise": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "ecinternationalorganization": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "legalshortname": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}, "country": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "extraInfo": [], "originalId": ["https://academic.microsoft.com/#/detail/125275712"], "ecsmevalidated": {"dataInfo": {"invisible": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "deletedbyinference": false, "inferred": false, "inferenceprovenance": ""}, "value": ""}} diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index 7aa581523..bd2ca9704 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -17,6 +17,7 @@ dhp-aggregation dhp-distcp dhp-graph-mapper + dhp-dedup @@ -310,6 +311,7 @@ + diff --git a/pom.xml b/pom.xml index e385ff6eb..5d3dce34a 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,12 @@ ${dhp.spark.version} provided + + org.apache.spark + spark-graphx_2.11 + ${dhp.spark.version} + provided + org.apache.commons @@ -177,6 +183,17 @@ ${dhp.jackson.version} provided + + + com.fasterxml.jackson.core + jackson-core + ${dhp.jackson.version} + provided + + + + + com.fasterxml.jackson.core jackson-annotations @@ -190,6 +207,12 @@ provided + + eu.dnetlib + dnet-pace-core + 4.0.0-SNAPSHOT + + javax.persistence @@ -203,6 +226,16 @@ amqp-client 5.6.0 + + com.jayway.jsonpath + json-path + 2.4.0 + + + com.arakelian + java-jq + 0.10.1 + org.apache.oozie @@ -259,27 +292,6 @@ - - eu.dnetlib - protoc-jar-maven-plugin - 1.1.0 - - - generate-sources - - run - - - ${google.protobuf.version} - - src/main/resources - - src/gen/java - - - - - org.apache.maven.plugins maven-surefire-plugin @@ -342,6 +354,31 @@ + + net.alchim31.maven + scala-maven-plugin + 4.0.1 + + + scala-compile-first + initialize + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.version} + + From 4b66b471a4a77ed34f7fb4fa097586cc2a7be921 Mon Sep 17 00:00:00 2001 From: miconis Date: Tue, 10 Dec 2019 14:57:16 +0100 Subject: [PATCH 02/13] implementation of the sorting by trust mechanism and the merge of oaf entities --- .../eu/dnetlib/dhp/schema/oaf/Context.java | 19 + .../java/eu/dnetlib/dhp/schema/oaf/Field.java | 21 + .../eu/dnetlib/dhp/schema/oaf/KeyValue.java | 23 + .../eu/dnetlib/dhp/schema/oaf/OafEntity.java | 31 +- .../dnetlib/dhp/schema/oaf/Publication.java | 12 + .../eu/dnetlib/dhp/schema/oaf/Qualifier.java | 28 + .../eu/dnetlib/dhp/schema/oaf/Result.java | 171 +- .../dhp/schema/oaf/StructuredProperty.java | 24 + .../eu/dnetlib/dhp/schema/oaf/MergeTest.java | 89 + .../eu/dnetlib/dhp/transform/ext_simple.xsl | 2 +- .../resources/eu/dnetlib/dhp/transform/tr.xml | 12 +- .../eu/dnetlib/dedup/DedupRecordFactory.java | 169 ++ .../java/eu/dnetlib/dedup/OafComparator.java | 15 + .../java/eu/dnetlib/dedup/OafEntityType.java | 13 + .../main/java/eu/dnetlib/dedup/OafKey.java | 31 + .../java/eu/dnetlib/dedup/OafPartitioner.java | 59 + .../dedup/SparkCreateConnectedComponent.java | 4 +- .../dnetlib/dedup/SparkCreateDedupRecord.java | 82 +- .../eu/dnetlib/dedup/SparkCreateSimRels.java | 3 +- .../dnetlib/dedup/graph/GraphProcessor.scala | 4 +- .../dnetlib/dhp/dedup/conf/org.curr.conf.json | 1745 ----------------- .../dhp/dedup/oozie_app/config-default.xml | 14 +- .../dnetlib/dedup/SparkCreateDedupTest.java | 33 +- .../eu/dnetlib/dedup/conf/org.curr.conf.json | 10 +- .../dnetlib}/dedup/conf/org.curr.conf2.json | 0 .../eu/dnetlib}/dedup/conf/pub.curr.conf.json | 0 .../dhp/distcp/oozie_app/config-default.xml | 8 +- .../dnetlib/dhp/distcp/oozie_app/workflow.xml | 4 +- .../dhp/graph/oozie_app/config-default.xml | 12 +- dhp-workflows/docs/oozie-installer.markdown | 4 +- dhp-workflows/pom.xml | 4 +- 31 files changed, 769 insertions(+), 1877 deletions(-) create mode 100644 dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java delete mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json rename dhp-workflows/dhp-dedup/src/{main/resources/eu/dnetlib/dhp => test/resources/eu/dnetlib}/dedup/conf/org.curr.conf2.json (100%) rename dhp-workflows/dhp-dedup/src/{main/resources/eu/dnetlib/dhp => test/resources/eu/dnetlib}/dedup/conf/pub.curr.conf.json (100%) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java index 64e23088e..8f46a0e23 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java @@ -23,4 +23,23 @@ public class Context implements Serializable { public void setDataInfo(List dataInfo) { this.dataInfo = dataInfo; } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + Context other = (Context) obj; + + return id.equals(other.getId()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java index 8834900c9..b645d275f 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java @@ -23,4 +23,25 @@ public class Field implements Serializable { public void setDataInfo(DataInfo dataInfo) { this.dataInfo = dataInfo; } + + @Override + public int hashCode(){ + return getValue().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + Field other = (Field) obj; + + return getValue().equals(other.getValue()); + } + + } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java index 1c4c7e6ef..fd394e188 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java @@ -33,4 +33,27 @@ public class KeyValue implements Serializable { public void setDataInfo(DataInfo dataInfo) { this.dataInfo = dataInfo; } + + public String toComparableString() { + return String.format("%s::%s", key != null ? key.toLowerCase() : "", value != null ? value.toLowerCase() : ""); + } + + @Override + public int hashCode() { + return toComparableString().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + KeyValue other = (KeyValue) obj; + + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java index 791667b46..955ddfd01 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java @@ -1,7 +1,8 @@ package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; -import java.util.List; +import java.util.*; +import java.util.stream.Collectors; public abstract class OafEntity extends Oaf implements Serializable { @@ -84,4 +85,32 @@ public abstract class OafEntity extends Oaf implements Serializable { public void setOaiprovenance(OAIProvenance oaiprovenance) { this.oaiprovenance = oaiprovenance; } + + public void mergeFrom(OafEntity e) { + + if (e == null) + return; + + originalId = mergeLists(originalId, e.getOriginalId()); + + collectedfrom = mergeLists(collectedfrom, e.getCollectedfrom()); + + pid = mergeLists(pid, e.getPid()); + + dateofcollection = e.getDateofcollection(); + + dateoftransformation = e.getDateoftransformation(); + + extraInfo = mergeLists(extraInfo, e.getExtraInfo()); + + oaiprovenance = e.getOaiprovenance(); + + } + + protected List mergeLists(final List... lists) { + + return Arrays.stream(lists).filter(Objects::nonNull).flatMap(List::stream).distinct().collect(Collectors.toList()); + } + + } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java index 9ca9cd3d6..181062f32 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java @@ -14,4 +14,16 @@ public class Publication extends Result implements Serializable { public void setJournal(Journal journal) { this.journal = journal; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + Publication p = (Publication) e; + + if (p.getJournal() != null) + journal = p.getJournal(); + } + + } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java index 60889535d..9c52d7310 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java @@ -40,4 +40,32 @@ public class Qualifier implements Serializable { public void setSchemename(String schemename) { this.schemename = schemename; } + + public String toComparableString() { + return String.format("%s::%s::%s::%s", + classid != null ? classid : "", + classname != null ? classname : "", + schemeid != null ? schemeid : "", + schemename != null ? schemename : ""); + } + + @Override + public int hashCode() { + return toComparableString().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + Qualifier other = (Qualifier) obj; + + return toComparableString() + .equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 7b08e71c2..0e34d8ba6 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -1,7 +1,11 @@ package eu.dnetlib.dhp.schema.oaf; +import org.apache.commons.lang3.StringUtils; + import java.io.Serializable; import java.util.List; +import java.util.Map; +import java.util.Objects; public abstract class Result extends OafEntity implements Serializable { @@ -12,35 +16,35 @@ public abstract class Result extends OafEntity implements Serializable { // common fields private Qualifier language; - + private List country; private List subject; - + private List title; - + private List relevantdate; private List> description; - + private Field dateofacceptance; - + private Field publisher; - + private Field embargoenddate; - + private List> source; - + private List> fulltext; // remove candidate - + private List> format; - + private List> contributor; - + private Qualifier resourcetype; - + private List> coverage; - + private Field refereed; //peer-review status private List context; @@ -240,4 +244,145 @@ public abstract class Result extends OafEntity implements Serializable { this.processingchargecurrency = processingchargecurrency; return this; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + Result r = (Result) e; + + mergeAuthors(r.getAuthor()); + + //TODO mergeFrom is used only for create Dedup Records since the creation of these two fields requires more complex functions (maybe they will be filled in an external function) +// if (author == null) +// author = r.getAuthor(); //authors will be replaced because they could be too much +// dateofacceptance = r.getDateofacceptance(); +// instance = mergeLists(instance, r.getInstance()); + + if (r.getResulttype() != null) + resulttype = r.getResulttype(); + + if (r.getLanguage() != null) + language = r.getLanguage(); + + country = mergeLists(country, r.getCountry()); + + subject = mergeLists(subject, r.getSubject()); + + title = mergeLists(title, r.getTitle()); + + relevantdate = mergeLists(relevantdate, r.getRelevantdate()); + + description = mergeLists(description, r.getDescription()); + + if (r.getPublisher() != null) + publisher = r.getPublisher(); + + if (r.getEmbargoenddate() != null) + embargoenddate = r.getEmbargoenddate(); + + source = mergeLists(source, r.getSource()); + + fulltext = mergeLists(fulltext, r.getFulltext()); + + format = mergeLists(format, r.getFormat()); + + contributor = mergeLists(contributor, r.getContributor()); + + if (r.getResourcetype() != null) + resourcetype = r.getResourcetype(); + + coverage = mergeLists(coverage, r.getCoverage()); + + if (r.getRefereed() != null) + refereed = r.getRefereed(); + + context = mergeLists(context, r.getContext()); + + if (r.getProcessingchargeamount() != null) + processingchargeamount = r.getProcessingchargeamount(); + + if (r.getProcessingchargecurrency() != null) + processingchargecurrency = r.getProcessingchargecurrency(); + + externalReference = mergeLists(externalReference, r.getExternalReference()); + + } + + public void mergeAuthors(List authors){ + int c1 = countAuthorsPids(author); + int c2 = countAuthorsPids(authors); + int s1 = authorsSize(author); + int s2 = authorsSize(authors); + + + //if both have no authors with pids and authors is bigger than author + if (c1 == 0 && c2 == 0 && author.size() authors){ + if (authors == null) + return -1; + + return (int) authors.stream().map(this::extractAuthorPid).filter(Objects::nonNull).filter(StringUtils::isNotBlank).count(); + } + + public int authorsSize(List authors){ + if (authors == null) + return 0; + return authors.size(); + } + + public String extractAuthorPid(Author a){ + + if(a == null || a.getPid() == null || a.getPid().size() == 0) + return null; + + StringBuilder mainPid = new StringBuilder(); + + a.getPid().forEach(pid ->{ + if (pid.getQualifier().getClassid().equalsIgnoreCase("orcid")) { + mainPid.setLength(0); + mainPid.append(pid.getValue()); + } + else { + if(mainPid.length() == 0) + mainPid.append(pid.getValue()); + } + }); + + return mainPid.toString(); + + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java index 79ebdd7f9..ea2370c7a 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java @@ -33,4 +33,28 @@ public class StructuredProperty implements Serializable { public void setDataInfo(DataInfo dataInfo) { this.dataInfo = dataInfo; } + + public String toComparableString(){ + return String.format("%s::%s", value != null ? value.toLowerCase() : "", qualifier != null ? qualifier.toComparableString().toLowerCase() : ""); + } + + @Override + public int hashCode() { + return toComparableString().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + StructuredProperty other = (StructuredProperty) obj; + + return toComparableString() + .equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java new file mode 100644 index 000000000..e487ddcba --- /dev/null +++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java @@ -0,0 +1,89 @@ +package eu.dnetlib.dhp.schema.oaf; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class MergeTest { + + OafEntity oaf; + + @Before + public void setUp() { + oaf = new Publication(); + } + + @Test + public void mergeListsTest() { + + //string list merge test + List a = Arrays.asList("a", "b", "c", "e"); + List b = Arrays.asList("a", "b", "c", "d"); + List c = null; + + System.out.println("merge result 1 = " + oaf.mergeLists(a, b)); + + System.out.println("merge result 2 = " + oaf.mergeLists(a, c)); + + System.out.println("merge result 3 = " + oaf.mergeLists(c, c)); + } + + @Test + public void mergePublicationCollectedFromTest() { + + Publication a = new Publication(); + Publication b = new Publication(); + + a.setCollectedfrom(Arrays.asList(setKV("a", "open"), setKV("b", "closed"))); + b.setCollectedfrom(Arrays.asList(setKV("A", "open"), setKV("b", "Open"))); + + a.mergeFrom(b); + + Assert.assertNotNull(a.getCollectedfrom()); + Assert.assertEquals(3, a.getCollectedfrom().size()); + + } + + @Test + public void mergePublicationSubjectTest() { + + Publication a = new Publication(); + Publication b = new Publication(); + + a.setSubject(Arrays.asList(setSP("a", "open", "classe"), setSP("b", "open", "classe"))); + b.setSubject(Arrays.asList(setSP("A", "open", "classe"), setSP("c", "open", "classe"))); + + a.mergeFrom(b); + + Assert.assertNotNull(a.getSubject()); + Assert.assertEquals(3, a.getSubject().size()); + + } + + private KeyValue setKV(final String key, final String value) { + + KeyValue k = new KeyValue(); + + k.setKey(key); + k.setValue(value); + + return k; + } + + private StructuredProperty setSP(final String value, final String schema, final String classname) { + StructuredProperty s = new StructuredProperty(); + s.setValue(value); + Qualifier q = new Qualifier(); + q.setClassname(classname); + q.setClassid(classname); + q.setSchemename(schema); + q.setSchemeid(schema); + s.setQualifier(q); + return s; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl index cef50aa95..5f5ed5a3b 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl @@ -9,7 +9,7 @@ - + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml index a9eae8576..ef6d9f7ac 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml @@ -1,11 +1,11 @@
- - - - - + + + + +
@@ -24,7 +24,7 @@ - + diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java new file mode 100644 index 000000000..371e80349 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -0,0 +1,169 @@ +package eu.dnetlib.dedup; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.lang.NotImplementedException; +import org.apache.commons.lang.StringUtils; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.codehaus.jackson.map.ObjectMapper; +import scala.Tuple2; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Random; + +import static java.util.stream.Collectors.toMap; + +public class DedupRecordFactory { + + public JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf){ + + // + final JavaPairRDD inputJsonEntities = sc.textFile(entitiesInputPath) + .mapToPair((PairFunction) it-> + new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) + ); + + //: source is the dedup_id, target is the id of the mergedIn + JavaPairRDD mergeRels = spark + .read().load(mergeRelsInputPath).as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .mapToPair( + (PairFunction)r-> + new Tuple2(r.getTarget(), r.getSource()) + ); + + // + final JavaPairRDD joinResult = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); + + JavaPairRDD keyJson = joinResult.mapToPair((PairFunction, OafKey, String>) json -> { + + String idValue = json._1(); + + String trust = MapDocumentUtil.getJPathString("$.dataInfo.trust", json._2()); + + //TODO remember to replace this with the actual trust retrieving + if (StringUtils.isBlank(trust)) { + Random generator = new Random(); + int number = generator.nextInt(20); + double result = (number / 100.0) + 0.80; + trust = "" + result; + } + + return new Tuple2(new OafKey(idValue, trust), json._2()); + }); + + OafComparator c = new OafComparator(); + // + JavaPairRDD> sortedJoinResult = keyJson.repartitionAndSortWithinPartitions(new OafPartitioner(keyJson.getNumPartitions()), c) + .mapToPair((PairFunction, String, String>) t -> new Tuple2(t._1().getDedupId(), t._2())) + .groupByKey(); + + + switch(entityType){ + case Publication: + return sortedJoinResult.map(this::publicationMerger); + case Dataset: + return sortedJoinResult.map(this::datasetMerger); + case Project: + return sortedJoinResult.map(this::projectMerger); + case Software: + return sortedJoinResult.map(this::softwareMerger); + case Datasource: + return sortedJoinResult.map(this::datasourceMerger); + case Organization: + return sortedJoinResult.map(this::organizationMerger); + case OtherResearchProduct: + return sortedJoinResult.map(this::otherresearchproductMerger); + default: + return null; + } + + } + + private Publication publicationMerger(Tuple2> e){ + + Publication p = new Publication(); //the result of the merge, to be returned at the end + + p.setId(e._1()); + + final ObjectMapper mapper = new ObjectMapper(); + + final Collection dateofacceptance = Lists.newArrayList(); + final Collection> authors = Lists.newArrayList(); + final Collection> instances = Lists.newArrayList(); + + StringBuilder trust = new StringBuilder("0.0"); + + e._2().forEach(pub -> { + try { + Publication publication = mapper.readValue(pub, Publication.class); + + final String currentTrust = publication.getDataInfo().getTrust(); + if (!currentTrust.equals("1.0")) { + trust.setLength(0); + trust.append(currentTrust); + } + + p.mergeFrom(publication); + + //add to the list if they are not null + if (publication.getDateofacceptance() != null) + dateofacceptance.add(publication.getDateofacceptance().getValue()); + if (publication.getAuthor() != null) + authors.add(publication.getAuthor()); + if (publication.getInstance() != null) + instances.add(publication.getInstance()); + + } catch (Exception exc){} + + }); + + p.setAuthor(null); //TODO create a single list of authors to put in the final publication + + + return p; + } + + private Dataset datasetMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + + private Project projectMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + + private Software softwareMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + + private Datasource datasourceMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + + private Organization organizationMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + + private OtherResearchProduct otherresearchproductMerger(Tuple2> e){ + + throw new NotImplementedException(); + } + +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java new file mode 100644 index 000000000..2ab78db7c --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java @@ -0,0 +1,15 @@ +package eu.dnetlib.dedup; +import com.google.common.collect.ComparisonChain; +import java.io.Serializable; +import java.util.Comparator; + +public class OafComparator implements Comparator, Serializable { + + @Override + public int compare(OafKey a, OafKey b) { + return ComparisonChain.start() + .compare(a.getDedupId(), b.getDedupId()) + .compare(a.getTrust(), b.getTrust()) + .result(); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java new file mode 100644 index 000000000..4ff2fa873 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java @@ -0,0 +1,13 @@ +package eu.dnetlib.dedup; + +public enum OafEntityType { + + Datasource, + Organization, + Project, + Dataset, + OtherResearchProduct, + Software, + Publication + +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java new file mode 100644 index 000000000..f66b0457e --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java @@ -0,0 +1,31 @@ +package eu.dnetlib.dedup; + +import java.io.Serializable; +public class OafKey implements Serializable { + + private String dedupId; + private String trust; + + public OafKey(String dedupId, String trust) { + this.dedupId = dedupId; + this.trust = trust; + } + public OafKey() { + } + public String getDedupId() { + return dedupId; + } + public void setDedupId(String dedupId) { + this.dedupId = dedupId; + } + public String getTrust() { + return trust; + } + public void setTrust(String trust) { + this.trust = trust; + } + @Override + public String toString(){ + return String.format("%s->%d", dedupId,trust); + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java new file mode 100644 index 000000000..20885fd0b --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java @@ -0,0 +1,59 @@ +package eu.dnetlib.dedup; + +import org.apache.spark.Partitioner; + +import java.io.Serializable; + +public class OafPartitioner extends Partitioner implements Serializable { + + private final int numPartitions; + + public OafPartitioner(int partitions) { + assert (partitions > 0); + this.numPartitions = partitions; + } + + @Override + public int numPartitions() { + return numPartitions; + } + + @Override + public int getPartition(Object key) { + if (key instanceof OafKey) { + @SuppressWarnings("unchecked") + OafKey item = (OafKey) key; + return Math.abs(item.getDedupId().hashCode() % numPartitions); + } else { + throw new IllegalArgumentException("Unexpected Key"); + } + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + numPartitions; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof OafPartitioner)) { + return false; + } + // + OafPartitioner other = (OafPartitioner) obj; + if (numPartitions != other.numPartitions) { + return false; + } + // + return true; + } +} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index 182bb374a..eacf3d479 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -37,8 +37,8 @@ public class SparkCreateConnectedComponent { final String inputPath = parser.get("sourcePath"); final String entity = parser.get("entity"); final String targetPath = parser.get("targetPath"); - final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateConnectedComponent.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); - +// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateConnectedComponent.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); final JavaPairRDD vertexes = sc.textFile(inputPath + "/" + entity) .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java index 6d6165b18..56bdc20f1 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java @@ -3,6 +3,7 @@ package eu.dnetlib.dedup; import com.google.common.collect.ComparisonChain; import com.google.common.collect.Lists; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Publication; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.util.MapDocumentUtil; @@ -17,6 +18,7 @@ import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; import scala.Tuple2; import java.util.ArrayList; @@ -27,26 +29,28 @@ import java.util.List; public class SparkCreateDedupRecord { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = SparkSession - .builder() - .appName(SparkCreateDedupRecord.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String entity = parser.get("entity"); - final String targetPath = parser.get("targetPath"); - final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); - - final JavaPairRDD inputJsonEntities = sc.textFile(inputPath + "/" + entity) - .mapToPair((PairFunction)it-> - new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) - ); - +// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); +// parser.parseArgument(args); +// final SparkSession spark = SparkSession +// .builder() +// .appName(SparkCreateDedupRecord.class.getSimpleName()) +// .master(parser.get("master")) +// .getOrCreate(); +// +// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); +// final String inputPath = parser.get("sourcePath"); +// final String entity = parser.get("entity"); +// final String targetPath = parser.get("targetPath"); +//// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); +// final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); +// +// // +// final JavaPairRDD inputJsonEntities = sc.textFile(inputPath + "/" + entity) +// .mapToPair((PairFunction)it-> +// new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) +// ); +// //: source is the dedup_id, target is the id of the mergedIn // JavaPairRDD mergeRels = spark // .read().load(targetPath + "/" + entity+"_mergeRels").as(Encoders.bean(Relation.class)) // .where("relClass=='merges'") @@ -56,46 +60,12 @@ public class SparkCreateDedupRecord { // new Tuple2(r.getTarget(), r.getSource()) // ); // -// +// // // final JavaPairRDD p = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); -// -// Comparator c = new Comparator() { -// @Override -// public int compare(String s, String t1) { -// return 0; -// } -// }; -// final JavaPairRDD stringStringJavaPairRDD = p.repartitionAndSortWithinPartitions(p.partitioner().get(), c); + StructType schema = Encoders.bean(Publication.class).schema(); -// List inputValues = Arrays.asList( -// new Foo("k",5), -// new Foo("a",1), -// new Foo("a",30), -// new Foo("a",18), -// new Foo("a",22), -// new Foo("b",22), -// new Foo("c",5), -// new Foo("a",5), -// new Foo("s",1), -// new Foo("h",4) -// ); -// -// -// final JavaPairRDD fooFighters = sc.parallelize(inputValues).mapToPair((PairFunction) i -> new Tuple2(i, i)); -// -// -// FooComparator c = new FooComparator(); -// final List>> result = -// fooFighters.repartitionAndSortWithinPartitions(new FooPartitioner(fooFighters.getNumPartitions()), c) -// .mapToPair((PairFunction, String, Foo>) t-> new Tuple2(t._1().getValue(), t._2()) ) -// .groupByKey() -// .mapValues((Function, List>) Lists::newArrayList) -// .collect(); -// -// -// System.out.println(result); - + System.out.println(schema); } } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java index a7aa18f89..2ae85baf3 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java @@ -44,7 +44,8 @@ public class SparkCreateSimRels { final String inputPath = parser.get("sourcePath"); final String entity = parser.get("entity"); final String targetPath = parser.get("targetPath"); - final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); +// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); final long total = sc.textFile(inputPath + "/" + entity).count(); diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala index ab2be9170..38c695152 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/graph/GraphProcessor.scala @@ -1,7 +1,5 @@ package eu.dnetlib.dedup.graph - -import eu.dnetlib.pace.model.MapDocument import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD @@ -25,7 +23,7 @@ object GraphProcessor { } val connectedComponents = joinResult.groupByKey() .map[ConnectedComponent](cc => asConnectedComponent(cc)) - (connectedComponents) + connectedComponents } diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json deleted file mode 100644 index a878d2419..000000000 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json +++ /dev/null @@ -1,1745 +0,0 @@ -{ - "wf": { - "threshold": "0.9", - "dedupRun": "001", - "entityType": "organization", - "orderField": "legalname", - "queueMaxSize": "2000", - "groupMaxSize": "50", - "slidingWindowSize": "200", - "idPath": ".id", - "rootBuilder": [ - "organization", - "projectOrganization_participation_isParticipant", - "datasourceOrganization_provision_isProvidedBy" - ], - "includeChildren": "true" - }, - "pace": { - "clustering": [ - { - "name": "sortedngrampairs", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "ngramLen": "3" - } - }, - { - "name": "suffixprefix", - "fields": [ - "legalname" - ], - "params": { - "max": 1, - "len": "3" - } - }, - { - "name": "urlclustering", - "fields": [ - "websiteurl" - ], - "params": {} - }, - { - "name": "keywordsclustering", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "windowSize": 4 - } - } - ], - "strictConditions": [ - { - "name": "exactMatch", - "fields": [ - "gridid" - ] - } - ], - "conditions": [ - { - "name": "DomainExactMatch", - "fields": [ - "websiteurl" - ] - }, - { - "name": "exactMatch", - "fields": [ - "country" - ] - } - ], - "model": [ - { - "name": "country", - "algo": "Null", - "type": "String", - "weight": "0", - "ignoreMissing": "false", - "path": ".country.classid" - }, - { - "name": "legalshortname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.1", - "ignoreMissing": "true", - "path": ".legalshortname.value" - }, - { - "name": "legalname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.9", - "ignoreMissing": "false", - "path": ".legalname.value", - "params": { - "windowSize": 4, - "threshold": 0.7 - } - }, - { - "name": "websiteurl", - "algo": "Null", - "type": "URL", - "weight": "0", - "ignoreMissing": "true", - "path": ".websiteurl.value", - "params": { - "host": 0.5, - "path": 0.5 - } - } - ], - "blacklists": { - "legalname": [] - }, - "synonyms": { - "key::1": [ - "university", - "università", - "università studi", - "universitario", - "universitaria", - "université", - "universitaire", - "universitaires", - "universidad", - "universitade", - "Universität", - "universitaet", - "Uniwersytet", - "университет", - "universiteit", - "πανεπιστήμιο", - "universitesi", - "universiteti" - ], - "key::2": [ - "studies", - "studi", - "études", - "estudios", - "estudos", - "Studien", - "studia", - "исследования", - "studies", - "σπουδές" - ], - "key::3": [ - "advanced", - "superiore", - "supérieur", - "supérieure", - "supérieurs", - "supérieures", - "avancado", - "avancados", - "fortgeschrittene", - "fortgeschritten", - "zaawansowany", - "передовой", - "gevorderd", - "gevorderde", - "προχωρημένος", - "προχωρημένη", - "προχωρημένο", - "προχωρημένες", - "προχωρημένα", - "wyzsza" - ], - "key::4": [ - "institute", - "istituto", - "institut", - "instituto", - "instituto", - "Institut", - "instytut", - "институт", - "instituut", - "ινστιτούτο" - ], - "key::5": [ - "hospital", - "ospedale", - "hôpital", - "hospital", - "hospital", - "Krankenhaus", - "szpital", - "больница", - "ziekenhuis", - "νοσοκομείο" - ], - "key::6": [ - "research", - "ricerca", - "recherche", - "investigacion", - "pesquisa", - "Forschung", - "badania", - "исследования", - "onderzoek", - "έρευνα", - "erevna", - "erevnas" - ], - "key::7": [ - "college", - "collegio", - "université", - "colegio", - "faculdade", - "Hochschule", - "Szkoła Wyższa", - "Высшая школа", - "universiteit", - "κολλέγιο" - ], - "key::8": [ - "foundation", - "fondazione", - "fondation", - "fundación", - "fundação", - "Stiftung", - "Fundacja", - "фонд", - "stichting", - "ίδρυμα", - "idryma" - ], - "key::9": [ - "center", - "centro", - "centre", - "centro", - "centro", - "zentrum", - "centrum", - "центр", - "centrum", - "κέντρο" - ], - "key::10": [ - "national", - "nazionale", - "national", - "nationale", - "nationaux", - "nationales", - "nacional", - "nacional", - "national", - "krajowy", - "национальный", - "nationaal", - "nationale", - "εθνικό" - ], - "key::11": [ - "association", - "associazione", - "association", - "asociación", - "associação", - "Verein", - "verband", - "stowarzyszenie", - "ассоциация", - "associatie" - ], - "key::12": [ - "society", - "societa", - "société", - "sociedad", - "sociedade", - "gesellschaft", - "społeczeństwo", - "общество", - "maatschappij", - "κοινωνία" - ], - "key::13": [ - "international", - "internazionale", - "international", - "internacional", - "internacional", - "international", - "międzynarodowy", - "Международный", - "internationaal", - "internationale", - "διεθνής", - "διεθνή", - "διεθνές" - ], - "key::14": [ - "community", - "comunita", - "communauté", - "comunidad", - "comunidade", - "Gemeinschaft", - "społeczność", - "сообщество", - "gemeenschap", - "κοινότητα" - ], - "key::15": [ - "school", - "scuola", - "école", - "escuela", - "escola", - "schule", - "Szkoła", - "школа", - "school", - "σχολείο" - ], - "key::16": [ - "education", - "educazione", - "éducation", - "educacion", - "Educação", - "Bildung", - "Edukacja", - "образование", - "opleiding", - "εκπαίδευση" - ], - "key::17": [ - "academy", - "accademia", - "académie", - "academia", - "academia", - "Akademie", - "akademie", - "академия", - "academie", - "ακαδημία" - ], - "key::18": [ - "public", - "pubblico", - "public", - "publique", - "publics", - "publiques", - "publico", - "publico", - "Öffentlichkeit", - "publiczny", - "публичный", - "publiek", - "publieke", - "δημόσιος", - "δημόσια", - "δημόσιο" - ], - "key::19": [ - "museum", - "museo", - "musée", - "mueso", - "museu", - "museum", - "muzeum", - "музей", - "museum", - "μουσείο" - ], - "key::20": [ - "group", - "gruppo", - "groupe", - "grupo", - "grupo", - "gruppe", - "grupa", - "группа", - "groep", - "ομάδα", - "όμιλος" - ], - "key::21": [ - "department", - "dipartimento", - "département", - "departamento", - "departamento", - "abteilung", - "departament", - "отдел", - "afdeling", - "τμήμα" - ], - "key::22": [ - "council", - "consiglio", - "conseil", - "Consejo", - "conselho", - "gesellschaft", - "rada", - "совет", - "raad", - "συμβούλιο" - ], - "key::23": [ - "library", - "biblioteca", - "bibliothèque", - "biblioteca", - "biblioteca", - "Bibliothek", - "biblioteka", - "библиотека", - "bibliotheek", - "βιβλιοθήκη" - ], - "key::24": [ - "ministry", - "ministero", - "ministère", - "ministerio", - "ministério", - "Ministerium", - "ministerstwo", - "министерство", - "ministerie", - "υπουργείο" - ], - "key::25": [ - "services", - "servizi", - "services", - "servicios", - "Serviços", - "Dienstleistungen", - "usługi", - "услуги", - "diensten", - "υπηρεσίες" - ], - "key::26": [ - "central", - "centrale", - "central", - "centrale", - "centrales", - "central", - "central", - "zentral", - "centralny", - "цетральный", - "centraal", - "κεντρικός", - "κεντρική", - "κεντρικό", - "κεντρικά" - ], - "key::27": [ - "general", - "generale", - "général", - "générale", - "généraux", - "générales", - "general", - "geral", - "general", - "Allgemeines", - "general", - "общий", - "algemeen", - "algemene", - "γενικός", - "γενική", - "γενικό", - "γενικά" - ], - "key::28": [ - "applied", - "applicati", - "appliqué", - "appliquée", - "appliqués", - "appliquées", - "aplicado", - "aplicada", - "angewendet", - "stosowany", - "прикладной", - "toegepast", - "toegepaste", - "εφαρμοσμένος", - "εφαρμοσμένη", - "εφαρμοσμένο", - "εφαρμοσμένα" - ], - "key::29": [ - "european", - "europee", - "europea", - "européen", - "européenne", - "européens", - "européennes", - "europeo", - "europeu", - "europäisch", - "europejski", - "европейский", - "Europees", - "Europese", - "ευρωπαϊκός", - "ευρωπαϊκή", - "ευρωπαϊκό", - "ευρωπαϊκά" - ], - "key::30": [ - "agency", - "agenzia", - "agence", - "agencia", - "agencia", - "agentur", - "agencja", - "агенция", - "agentschap", - "πρακτορείο" - ], - "key::31": [ - "laboratory", - "laboratorio", - "laboratoire", - "laboratorio", - "laboratorio", - "labor", - "laboratorium", - "лаборатория", - "laboratorium", - "εργαστήριο" - ], - "key::32": [ - "industry", - "industria", - "industrie", - "индустрия", - "industrie", - "βιομηχανία" - ], - "key::33": [ - "industrial", - "industriale", - "industriel", - "industrielle", - "industriels", - "industrielles", - "индустриальный", - "industrieel", - "βιομηχανικός", - "βιομηχανική", - "βιομηχανικό", - "βιομηχανικά", - "βιομηχανικές" - ], - "key::34": [ - "consortium", - "consorzio", - "consortium", - "консорциум", - "consortium", - "κοινοπραξία" - ], - "key::35": [ - "organization", - "organizzazione", - "organisation", - "organización", - "organização", - "organizacja", - "организация", - "organisatie", - "οργανισμός" - ], - "key::36": [ - "authority", - "autorità", - "autorité", - "авторитет", - "autoriteit" - ], - "key::37": [ - "federation", - "federazione", - "fédération", - "федерация", - "federatie", - "ομοσπονδία" - ], - "key::38": [ - "observatory", - "osservatorio", - "observatoire", - "обсерватория", - "observatorium", - "αστεροσκοπείο" - ], - "key::39": [ - "bureau", - "ufficio", - "bureau", - "офис", - "bureau", - "γραφείο" - ], - "key::40": [ - "company", - "impresa", - "compagnie", - "société", - "компания", - "bedrijf", - "εταιρία" - ], - "key::41": [ - "polytechnic", - "politecnico", - "polytechnique", - "политехника", - "polytechnisch", - "πολυτεχνείο", - "universita politecnica", - "polytechnic university", - "universidad politecnica", - "universitat politecnica", - "politechnika", - "politechniki", - "university technology", - "university science technology" - ], - "key::42": [ - "coalition", - "coalizione", - "coalition", - "коалиция", - "coalitie", - "συνασπισμός" - ], - "key::43": [ - "initiative", - "iniziativa", - "initiative", - "инициатива", - "initiatief", - "πρωτοβουλία" - ], - "key::44": [ - "academic", - "accademico", - "académique", - "universitaire", - "акадеческий academisch", - "ακαδημαϊκός", - "ακαδημαϊκή", - "ακαδημαϊκό", - "ακαδημαϊκές", - "ακαδημαϊκοί" - ], - "key::45": [ - "institution", - "istituzione", - "institution", - "институциональный", - "instelling", - "ινστιτούτο" - ], - "key::46": [ - "division", - "divisione", - "division", - "отделение", - "divisie", - "τμήμα" - ], - "key::47": [ - "committee", - "comitato", - "comité", - "комитет", - "commissie", - "επιτροπή" - ], - "key::48": [ - "promotion", - "promozione", - "продвижение", - "proothisis", - "forderung" - ], - "key::49": [ - "medical", - "medicine", - "clinical", - "medicina", - "clinici", - "médico", - "medicina", - "clínica", - "médico", - "medicina", - "clínica", - "medizinisch", - "Medizin", - "klinisch", - "medisch", - "geneeskunde", - "klinisch", - "ιατρικός", - "ιατρική", - "ιατρικό", - "ιατρικά", - "κλινικός", - "κλινική", - "κλινικό", - "κλινικά", - "tıbbi", - "tıp", - "klinik", - "orvosi", - "orvostudomány", - "klinikai", - "zdravniški", - "medicinski", - "klinični", - "meditsiini", - "kliinik", - "kliiniline" - ], - "key::50": [ - "technology", - "technological", - "tecnologia", - "tecnologie", - "tecnología", - "tecnológico", - "tecnologia", - "tecnológico", - "Technologie", - "technologisch", - "technologie", - "technologisch", - "τεχνολογία", - "τεχνολογικός", - "τεχνολογική", - "τεχνολογικό", - "teknoloji", - "teknolojik", - "technológia", - "technológiai", - "tehnologija", - "tehnološki", - "tehnoloogia", - "tehnoloogiline", - "technologii", - "technical", - "texniki", - "teknik" - ], - "key::51": [ - "science", - "scientific", - "scienza", - "scientifiche", - "scienze", - "ciencia", - "científico", - "ciência", - "científico", - "Wissenschaft", - "wissenschaftlich", - "wetenschap", - "wetenschappelijk", - "επιστήμη", - "επιστημονικός", - "επιστημονική", - "επιστημονικό", - "επιστημονικά", - "bilim", - "bilimsel", - "tudomány", - "tudományos", - "znanost", - "znanstveni", - "teadus", - "teaduslik", - "" - ], - "key::52": [ - "engineering", - "ingegneria", - "ingeniería", - "engenharia", - "Ingenieurwissenschaft", - "ingenieurswetenschappen", - "bouwkunde", - "μηχανικός", - "μηχανική", - "μηχανικό", - "mühendislik", - "mérnöki", - "Inženirstvo", - "inseneeria", - "inseneri", - "" - ], - "key::53": [ - "management", - "gestione", - "gestionale", - "gestionali", - "gestión", - "administración", - "gestão", - "administração", - "Verwaltung", - "management", - "διαχείριση", - "yönetim", - "menedzsment", - "vodstvo", - "upravljanje", - "management", - "juhtkond", - "juhtimine", - "haldus", - "" - ], - "key::54": [ - "energy", - "energia", - "energía", - "energia", - "Energie", - "energie", - "ενέργεια", - "enerji", - "energia", - "energija", - "energia", - "" - ], - "key::55": [ - "agricultural", - "agriculture", - "agricoltura", - "agricole", - "agrícola", - "agricultura", - "agrícola", - "agricultura", - "landwirtschaftlich", - "Landwirtschaft", - "landbouwkundig", - "landbouw", - "αγροτικός", - "αγροτική", - "αγροτικό", - "γεωργικός", - "γεωργική", - "γεωργικό", - "γεωργία", - "tarımsal", - "tarım", - "mezőgazdasági", - "mezőgazdaság", - "poljedelski", - "poljedelstvo", - "põllumajandus", - "põllumajanduslik", - "" - ], - "key::56": [ - "information", - "informazione", - "información", - "informação", - "Information", - "informatie", - "πληροφορία", - "bilgi", - "információ", - "informacija", - "informatsioon", - "informatycznych", - "" - ], - "key::57": [ - "social", - "sociali", - "social", - "social", - "Sozial", - "sociaal", - "maatschappelijk", - "κοινωνικός", - "κοινωνική", - "κοινωνικό", - "κοινωνικά", - "sosyal", - "szociális", - "družbeni", - "sotsiaal", - "sotsiaalne", - "" - ], - "key::58": [ - "environmental", - "ambiente", - "medioambiental", - "ambiente", - "medioambiente", - "meioambiente", - "Umwelt", - "milieu", - "milieuwetenschap", - "milieukunde", - "περιβαλλοντικός", - "περιβαλλοντική", - "περιβαλλοντικό", - "περιβαλλοντικά", - "çevre", - "környezeti", - "okoliški", - "keskonna", - "" - ], - "key::59": [ - "business", - "economia", - "economiche", - "economica", - "negocio", - "empresa", - "negócio", - "Unternehmen", - "bedrijf", - "bedrijfskunde", - "επιχείρηση", - "iş", - "üzleti", - "posel", - "ettevõte/äri", - "" - ], - "key::60": [ - "pharmaceuticals", - "pharmacy", - "farmacia", - "farmaceutica", - "farmacéutica", - "farmacia", - "farmacêutica", - "farmácia", - "Pharmazeutika", - "Arzneimittelkunde", - "farmaceutica", - "geneesmiddelen", - "apotheek", - "φαρμακευτικός", - "φαρμακευτική", - "φαρμακευτικό", - "φαρμακευτικά", - "φαρμακείο", - "ilaç", - "eczane", - "gyógyszerészeti", - "gyógyszertár", - "farmacevtika", - "lekarništvo", - "farmaatsia", - "farmatseutiline", - "" - ], - "key::61": [ - "healthcare", - "health services", - "salute", - "atenciónmédica", - "cuidadodelasalud", - "cuidadoscomasaúde", - "Gesundheitswesen", - "gezondheidszorg", - "ιατροφαρμακευτικήπερίθαλψη", - "sağlıkhizmeti", - "egészségügy", - "zdravstvo", - "tervishoid", - "tervishoiu", - "" - ], - "key::62": [ - "history", - "storia", - "historia", - "história", - "Geschichte", - "geschiedenis", - "geschiedkunde", - "ιστορία", - "tarih", - "történelem", - "zgodovina", - "ajalugu", - "" - ], - "key::63": [ - "materials", - "materiali", - "materia", - "materiales", - "materiais", - "materialen", - "υλικά", - "τεκμήρια", - "malzemeler", - "anyagok", - "materiali", - "materjalid", - "vahendid", - "" - ], - "key::64": [ - "economics", - "economia", - "economiche", - "economica", - "economía", - "economia", - "Wirtschaft", - "economie", - "οικονομικά", - "οικονομικέςεπιστήμες", - "ekonomi", - "közgazdaságtan", - "gospodarstvo", - "ekonomija", - "majanduslik", - "majandus", - "" - ], - "key::65": [ - "therapeutics", - "terapeutica", - "terapéutica", - "terapêutica", - "therapie", - "θεραπευτική", - "tedavibilimi", - "gyógykezelés", - "terapevtika", - "terapeutiline", - "ravi", - "" - ], - "key::66": [ - "oncology", - "oncologia", - "oncologico", - "oncología", - "oncologia", - "Onkologie", - "oncologie", - "ογκολογία", - "onkoloji", - "onkológia", - "onkologija", - "onkoloogia", - "" - ], - "key::67": [ - "natural", - "naturali", - "naturale", - "natural", - "natural", - "natürlich", - "natuurlijk", - "φυσικός", - "φυσική", - "φυσικό", - "φυσικά", - "doğal", - "természetes", - "naraven", - "loodus", - "" - ], - "key::68": [ - "educational", - "educazione", - "pedagogia", - "educacional", - "educativo", - "educacional", - "pädagogisch", - "educatief", - "εκπαιδευτικός", - "εκπαιδευτική", - "εκπαιδευτικό", - "εκπαιδευτικά", - "eğitimsel", - "oktatási", - "izobraževalen", - "haridus", - "hariduslik", - "" - ], - "key::69": [ - "biomedical", - "biomedica", - "biomédico", - "biomédico", - "biomedizinisch", - "biomedisch", - "βιοιατρικός", - "βιοιατρική", - "βιοιατρικό", - "βιοιατρικά", - "biyomedikal", - "orvosbiológiai", - "biomedicinski", - "biomeditsiiniline", - "" - ], - "key::70": [ - "veterinary", - "veterinaria", - "veterinarie", - "veterinaria", - "veterinária", - "tierärtzlich", - "veterinair", - "veeartsenijlkunde", - "κτηνιατρικός", - "κτηνιατρική", - "κτηνιατρικό", - "κτηνιατρικά", - "veteriner", - "állatorvosi", - "veterinar", - "veterinarski", - "veterinaaria", - "" - ], - "key::71": [ - "chemistry", - "chimica", - "química", - "química", - "Chemie", - "chemie", - "scheikunde", - "χημεία", - "kimya", - "kémia", - "kemija", - "keemia", - "" - ], - "key::72": [ - "security", - "sicurezza", - "seguridad", - "segurança", - "Sicherheit", - "veiligheid", - "ασφάλεια", - "güvenlik", - "biztonsági", - "varnost", - "turvalisus", - "julgeolek", - "" - ], - "key::73": [ - "biotechnology", - "biotecnologia", - "biotecnologie", - "biotecnología", - "biotecnologia", - "Biotechnologie", - "biotechnologie", - "βιοτεχνολογία", - "biyoteknoloji", - "biotechnológia", - "biotehnologija", - "biotehnoloogia", - "" - ], - "key::74": [ - "military", - "militare", - "militari", - "militar", - "militar", - "Militär", - "militair", - "leger", - "στρατιωτικός", - "στρατιωτική", - "στρατιωτικό", - "στρατιωτικά", - "askeri", - "katonai", - "vojaški", - "vojni", - "militaar", - "wojskowa", - "" - ], - "key::75": [ - "theological", - "teologia", - "teologico", - "teológico", - "tecnológica", - "theologisch", - "theologisch", - "θεολογικός", - "θεολογική", - "θεολογικό", - "θεολογικά", - "teolojik", - "technológiai", - "teološki", - "teoloogia", - "usuteadus", - "teoloogiline", - "" - ], - "key::76": [ - "electronics", - "elettronica", - "electrónica", - "eletrônicos", - "Elektronik", - "elektronica", - "ηλεκτρονική", - "elektronik", - "elektronika", - "elektronika", - "elektroonika", - "" - ], - "key::77": [ - "forestry", - "forestale", - "forestali", - "silvicultura", - "forestal", - "floresta", - "Forstwirtschaft", - "bosbouw", - "δασοκομία", - "δασολογία", - "ormancılık", - "erdészet", - "gozdarstvo", - "metsandus", - "" - ], - "key::78": [ - "maritime", - "marittima", - "marittime", - "marittimo", - "marítimo", - "marítimo", - "maritiem", - "ναυτικός", - "ναυτική", - "ναυτικό", - "ναυτικά", - "ναυτιλιακός", - "ναυτιλιακή", - "ναυτιλιακό", - "ναυτιλιακά", - "θαλάσσιος", - "θαλάσσια", - "θαλάσσιο", - "denizcilik", - "tengeri", - "morski", - "mere", - "merendus", - "" - ], - "key::79": [ - "sports", - "sport", - "deportes", - "esportes", - "Sport", - "sport", - "sportwetenschappen", - "άθληση", - "γυμναστικήδραστηριότητα", - "spor", - "sport", - "šport", - "sport", - "spordi", - "" - ], - "key::80": [ - "surgery", - "chirurgia", - "chirurgiche", - "cirugía", - "cirurgia", - "Chirurgie", - "chirurgie", - "heelkunde", - "εγχείρηση", - "επέμβαση", - "χειρουργικήεπέμβαση", - "cerrahi", - "sebészet", - "kirurgija", - "kirurgia", - "" - ], - "key::81": [ - "cultural", - "culturale", - "culturali", - "cultura", - "cultural", - "cultural", - "kulturell", - "cultureel", - "πολιτιστικός", - "πολιτιστική", - "πολιτιστικό", - "πολιτισμικός", - "πολιτισμική", - "πολιτισμικό", - "kültürel", - "kultúrális", - "kulturni", - "kultuuri", - "kultuuriline", - "" - ], - "key::82": [ - "computerscience", - "informatica", - "ordenador", - "computadora", - "informática", - "computación", - "cienciasdelacomputación", - "ciênciadacomputação", - "Computer", - "computer", - "υπολογιστής", - "ηλεκτρονικόςυπολογιστής", - "bilgisayar", - "számítógép", - "računalnik", - "arvuti", - "" - ], - "key::83": [ - "finance", - "financial", - "finanza", - "finanziarie", - "finanza", - "financiero", - "finanças", - "financeiro", - "Finanzen", - "finanziell", - "financiën", - "financieel", - "χρηματοοικονομικά", - "χρηματοδότηση", - "finanse", - "finansal", - "pénzügy", - "pénzügyi", - "finance", - "finančni", - "finants", - "finantsiline", - "" - ], - "key::84": [ - "communication", - "comunicazione", - "comuniciación", - "comunicação", - "Kommunikation", - "communication", - "επικοινωνία", - "iletişim", - "kommunikáció", - "komuniciranje", - "kommunikatsioon", - "" - ], - "key::85": [ - "justice", - "giustizia", - "justicia", - "justiça", - "Recht", - "Justiz", - "justitie", - "gerechtigheid", - "δικαιοσύνη", - "υπουργείοδικαιοσύνης", - "δίκαιο", - "adalet", - "igazságügy", - "pravo", - "õigus", - "" - ], - "key::86": [ - "aerospace", - "aerospaziale", - "aerospaziali", - "aeroespacio", - "aeroespaço", - "Luftfahrt", - "luchtvaart", - "ruimtevaart", - "αεροπορικός", - "αεροπορική", - "αεροπορικό", - "αεροναυπηγικός", - "αεροναυπηγική", - "αεροναυπηγικό", - "αεροναυπηγικά", - "havacılıkveuzay", - "légtér", - "zrakoplovstvo", - "atmosfäär", - "kosmos", - "" - ], - "key::87": [ - "dermatology", - "dermatologia", - "dermatología", - "dermatologia", - "Dermatologie", - "dermatologie", - "δρματολογία", - "dermatoloji", - "bőrgyógyászat", - "dermatológia", - "dermatologija", - "dermatoloogia", - "" - ], - "key::88": [ - "architecture", - "architettura", - "arquitectura", - "arquitetura", - "Architektur", - "architectuur", - "αρχιτεκτονική", - "mimarlık", - "építészet", - "arhitektura", - "arhitektuur", - "" - ], - "key::89": [ - "mathematics", - "matematica", - "matematiche", - "matemáticas", - "matemáticas", - "Mathematik", - "wiskunde", - "mathematica", - "μαθηματικά", - "matematik", - "matematika", - "matematika", - "matemaatika", - "" - ], - "key::90": [ - "language", - "lingue", - "linguistica", - "linguistiche", - "lenguaje", - "idioma", - "língua", - "idioma", - "Sprache", - "taal", - "taalkunde", - "γλώσσα", - "dil", - "nyelv", - "jezik", - "keel", - "" - ], - "key::91": [ - "neuroscience", - "neuroscienza", - "neurociencia", - "neurociência", - "Neurowissenschaft", - "neurowetenschappen", - "νευροεπιστήμη", - "nörobilim", - "idegtudomány", - "nevroznanost", - "neuroteadused", - "" - ], - "key::92": [ - "automation", - "automazione", - "automatización", - "automação", - "Automatisierung", - "automatisering", - "αυτοματοποίηση", - "otomasyon", - "automatizálás", - "avtomatizacija", - "automatiseeritud", - "" - ], - "key::93": [ - "pediatric", - "pediatria", - "pediatriche", - "pediatrico", - "pediátrico", - "pediatría", - "pediátrico", - "pediatria", - "pädiatrisch", - "pediatrische", - "παιδιατρική", - "pediatrik", - "gyermekgyógyászat", - "pediatrija", - "pediaatria", - "" - ], - "key::94": [ - "photonics", - "fotonica", - "fotoniche", - "fotónica", - "fotônica", - "Photonik", - "fotonica", - "φωτονική", - "fotonik", - "fotonika", - "fotonika", - "fotoonika", - "" - ], - "key::95": [ - "mechanics", - "meccanica", - "meccaniche", - "mecánica", - "mecânica", - "Mechanik", - "Maschinenbau", - "mechanica", - "werktuigkunde", - "μηχανικής", - "mekanik", - "gépészet", - "mehanika", - "mehaanika", - "" - ], - "key::96": [ - "psychiatrics", - "psichiatria", - "psichiatrica", - "psichiatriche", - "psiquiatría", - "psiquiatria", - "Psychiatrie", - "psychiatrie", - "ψυχιατρική", - "psikiyatrik", - "pszihiátria", - "psihiatrija", - "psühhaatria", - "" - ], - "key::97": [ - "psychology", - "fisiologia", - "psicología", - "psicologia", - "Psychologie", - "psychologie", - "ψυχολογία", - "psikoloji", - "pszihológia", - "psihologija", - "psühholoogia", - "" - ], - "key::98": [ - "automotive", - "industriaautomobilistica", - "industriadelautomóvil", - "automotriz", - "industriaautomotriz", - "automotivo", - "Automobilindustrie", - "autoindustrie", - "αυτοκίνητος", - "αυτοκίνητη", - "αυτοκίνητο", - "αυτοκινούμενος", - "αυτοκινούμενη", - "αυτοκινούμενο", - "αυτοκινητιστικός", - "αυτοκινητιστική", - "αυτοκινητιστικό", - "otomotiv", - "autóipari", - "samogiben", - "avtomobilskaindustrija", - "auto-", - "" - ], - "key::99": [ - "neurology", - "neurologia", - "neurologiche", - "neurología", - "neurologia", - "Neurologie", - "neurologie", - "zenuwleer", - "νευρολογία", - "nöroloji", - "neurológia", - "ideggyógyászat", - "nevrologija", - "neuroloogia", - "" - ], - "key::100": [ - "geology", - "geologia", - "geologiche", - "geología", - "geologia", - "Geologie", - "geologie", - "aardkunde", - "γεωλογία", - "jeoloji", - "geológia", - "földtudomány", - "geologija", - "geoloogia", - "" - ], - "key::101": [ - "microbiology", - "microbiologia", - "micro-biologia", - "microbiologiche", - "microbiología", - "microbiologia", - "Mikrobiologie", - "microbiologie", - "μικροβιολογία", - "mikrobiyoloji", - "mikrobiológia", - "mikrobiologija", - "mikrobioloogia", - "" - ], - "key::102": [ - "informatics", - "informatica", - "informática", - "informática", - "informatica", - "" - ], - "key::103": [ - "forschungsgemeinschaft", - "comunita ricerca", - "research community", - "research foundation", - "research association" - ], - "key::104": [ - "commerce", - "ticaret", - "ticarət", - "commercio", - "trade", - "handel", - "comercio" - ] - } - } -} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml index ddea85590..e654bbbb6 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml @@ -1,31 +1,31 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 oozie.use.system.libpath - true + true oozie.action.sharelib.for.spark - spark2 + spark2 hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 hive_db_name - openaire + openaire dedupConf - {"wf":{"threshold":"0.9","dedupRun":"001","entityType":"organization","orderField":"legalname","queueMaxSize":"2000","groupMaxSize":"50","slidingWindowSize":"200","idPath":".id","rootBuilder":["organization","projectOrganization_participation_isParticipant","datasourceOrganization_provision_isProvidedBy"],"includeChildren":"true"},"pace":{"clustering":[{"name":"sortedngrampairs","fields":["legalname"],"params":{"max":2,"ngramLen":"3"}},{"name":"suffixprefix","fields":["legalname"],"params":{"max":1,"len":"3"}},{"name":"urlclustering","fields":["websiteurl"],"params":{}},{"name":"keywordsclustering","fields":["legalname"],"params":{"max":2,"windowSize":4}}],"strictConditions":[{"name":"exactMatch","fields":["gridid"]}],"conditions":[{"name":"DomainExactMatch","fields":["websiteurl"]},{"name":"exactMatch","fields":["country"]}],"model":[{"name":"country","algo":"Null","type":"String","weight":"0","ignoreMissing":"false","path":".country.classid"},{"name":"legalshortname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.1","ignoreMissing":"true","path":".legalshortname.value"},{"name":"legalname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.9","ignoreMissing":"false","path":".legalname.value","params":{"windowSize":4,"threshold":0.7}},{"name":"websiteurl","algo":"Null","type":"URL","weight":"0","ignoreMissing":"true","path":".websiteurl.value","params":{"host":0.5,"path":0.5}},{"name":"gridid","algo":"Null","type":"String","weight":"0.0","ignoreMissing":"true","path":".pid[] | select(.qualifier.classid==\"grid\") | .value"}],"blacklists":{"legalname":[]},"synonyms":{"key::1":["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti"],"key::2":["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"],"key::3":["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"],"key::4":["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"],"key::5":["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"],"key::6":["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"],"key::7":["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"],"key::8":["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"],"key::9":["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"],"key::10":["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"],"key::11":["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"],"key::12":["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"],"key::13":["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"],"key::14":["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"],"key::15":["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"],"key::16":["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"],"key::17":["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"],"key::18":["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"],"key::19":["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"],"key::20":["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"],"key::21":["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"],"key::22":["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"],"key::23":["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"],"key::24":["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"],"key::25":["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"],"key::26":["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"],"key::27":["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"],"key::28":["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"],"key::29":["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"],"key::30":["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"],"key::31":["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"],"key::32":["industry","industria","industrie","индустрия","industrie","βιομηχανία"],"key::33":["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"],"key::34":["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"],"key::35":["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"],"key::36":["authority","autorità","autorité","авторитет","autoriteit"],"key::37":["federation","federazione","fédération","федерация","federatie","ομοσπονδία"],"key::38":["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"],"key::39":["bureau","ufficio","bureau","офис","bureau","γραφείο"],"key::40":["company","impresa","compagnie","société","компания","bedrijf","εταιρία"],"key::41":["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"],"key::42":["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"],"key::43":["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"],"key::44":["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"],"key::45":["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"],"key::46":["division","divisione","division","отделение","divisie","τμήμα"],"key::47":["committee","comitato","comité","комитет","commissie","επιτροπή"],"key::48":["promotion","promozione","продвижение","proothisis","forderung"],"key::49":["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"],"key::50":["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"],"key::51":["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""],"key::52":["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""],"key::53":["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""],"key::54":["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""],"key::55":["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""],"key::56":["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""],"key::57":["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""],"key::58":["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""],"key::59":["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""],"key::60":["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""],"key::61":["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""],"key::62":["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""],"key::63":["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""],"key::64":["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""],"key::65":["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""],"key::66":["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""],"key::67":["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""],"key::68":["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""],"key::69":["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""],"key::70":["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""],"key::71":["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""],"key::72":["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""],"key::73":["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""],"key::74":["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""],"key::75":["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""],"key::76":["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""],"key::77":["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""],"key::78":["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""],"key::79":["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""],"key::80":["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""],"key::81":["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""],"key::82":["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""],"key::83":["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""],"key::84":["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""],"key::85":["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""],"key::86":["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""],"key::87":["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""],"key::88":["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""],"key::89":["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""],"key::90":["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""],"key::91":["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""],"key::92":["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""],"key::93":["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""],"key::94":["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""],"key::95":["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""],"key::96":["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""],"key::97":["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""],"key::98":["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""],"key::99":["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""],"key::100":["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""],"key::101":["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""],"key::102":["informatics","informatica","informática","informática","informatica",""],"key::103":["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"],"key::104":["commerce","ticaret","ticarət","commercio","trade","handel","comercio"]}}} + {"wf":{"threshold":"0.9","dedupRun":"001","entityType":"organization","orderField":"legalname","queueMaxSize":"2000","groupMaxSize":"50","slidingWindowSize":"200","idPath":".id","rootBuilder":["organization","projectOrganization_participation_isParticipant","datasourceOrganization_provision_isProvidedBy"],"includeChildren":"true"},"pace":{"clustering":[{"name":"sortedngrampairs","fields":["legalname"],"params":{"max":2,"ngramLen":"3"}},{"name":"suffixprefix","fields":["legalname"],"params":{"max":1,"len":"3"}},{"name":"urlclustering","fields":["websiteurl"],"params":{}},{"name":"keywordsclustering","fields":["legalname"],"params":{"max":2,"windowSize":4}}],"strictConditions":[{"name":"exactMatch","fields":["gridid"]}],"conditions":[{"name":"DomainExactMatch","fields":["websiteurl"]},{"name":"exactMatch","fields":["country"]}],"model":[{"name":"country","algo":"Null","type":"String","weight":"0","ignoreMissing":"false","path":".country.classid"},{"name":"legalshortname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.1","ignoreMissing":"true","path":".legalshortname.dedupId"},{"name":"legalname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.9","ignoreMissing":"false","path":".legalname.dedupId","params":{"windowSize":4,"threshold":0.7}},{"name":"websiteurl","algo":"Null","type":"URL","weight":"0","ignoreMissing":"true","path":".websiteurl.dedupId","params":{"host":0.5,"path":0.5}},{"name":"gridid","algo":"Null","type":"String","weight":"0.0","ignoreMissing":"true","path":".pid[] | select(.qualifier.classid==\"grid\") | .dedupId"}],"blacklists":{"legalname":[]},"synonyms":{"key::1":["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti"],"key::2":["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"],"key::3":["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"],"key::4":["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"],"key::5":["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"],"key::6":["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"],"key::7":["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"],"key::8":["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"],"key::9":["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"],"key::10":["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"],"key::11":["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"],"key::12":["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"],"key::13":["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"],"key::14":["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"],"key::15":["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"],"key::16":["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"],"key::17":["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"],"key::18":["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"],"key::19":["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"],"key::20":["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"],"key::21":["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"],"key::22":["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"],"key::23":["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"],"key::24":["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"],"key::25":["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"],"key::26":["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"],"key::27":["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"],"key::28":["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"],"key::29":["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"],"key::30":["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"],"key::31":["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"],"key::32":["industry","industria","industrie","индустрия","industrie","βιομηχανία"],"key::33":["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"],"key::34":["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"],"key::35":["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"],"key::36":["authority","autorità","autorité","авторитет","autoriteit"],"key::37":["federation","federazione","fédération","федерация","federatie","ομοσπονδία"],"key::38":["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"],"key::39":["bureau","ufficio","bureau","офис","bureau","γραφείο"],"key::40":["company","impresa","compagnie","société","компания","bedrijf","εταιρία"],"key::41":["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"],"key::42":["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"],"key::43":["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"],"key::44":["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"],"key::45":["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"],"key::46":["division","divisione","division","отделение","divisie","τμήμα"],"key::47":["committee","comitato","comité","комитет","commissie","επιτροπή"],"key::48":["promotion","promozione","продвижение","proothisis","forderung"],"key::49":["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"],"key::50":["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"],"key::51":["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""],"key::52":["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""],"key::53":["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""],"key::54":["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""],"key::55":["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""],"key::56":["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""],"key::57":["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""],"key::58":["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""],"key::59":["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""],"key::60":["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""],"key::61":["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""],"key::62":["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""],"key::63":["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""],"key::64":["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""],"key::65":["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""],"key::66":["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""],"key::67":["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""],"key::68":["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""],"key::69":["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""],"key::70":["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""],"key::71":["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""],"key::72":["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""],"key::73":["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""],"key::74":["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""],"key::75":["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""],"key::76":["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""],"key::77":["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""],"key::78":["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""],"key::79":["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""],"key::80":["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""],"key::81":["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""],"key::82":["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""],"key::83":["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""],"key::84":["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""],"key::85":["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""],"key::86":["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""],"key::87":["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""],"key::88":["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""],"key::89":["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""],"key::90":["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""],"key::91":["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""],"key::92":["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""],"key::93":["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""],"key::94":["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""],"key::95":["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""],"key::96":["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""],"key::97":["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""],"key::98":["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""],"key::99":["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""],"key::100":["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""],"key::101":["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""],"key::102":["informatics","informatica","informática","informática","informatica",""],"key::103":["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"],"key::104":["commerce","ticaret","ticarət","commercio","trade","handel","comercio"]}}} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index b16b45ef1..f35baa9f8 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -1,5 +1,7 @@ package eu.dnetlib.dedup; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.schema.oaf.Publication; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.junit.Before; @@ -8,36 +10,37 @@ import org.junit.Test; import java.io.File; import java.io.IOException; +import java.util.List; public class SparkCreateDedupTest { - + String configuration; @Before public void setUp() throws IOException { - FileUtils.deleteDirectory(new File("/tmp/pub_dedup_vertex")); - FileUtils.deleteDirectory(new File("/tmp/pub_dedup_rels")); + configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/pub.curr.conf.json")); + } - - @Test @Ignore - public void dedupTest() throws Exception { - final String configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json")); - - + public void createSimRelsTest() throws Exception { SparkCreateSimRels.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", "publication", "-c", configuration, "-t", "/tmp/dedup", }); + } + + @Test + @Ignore + public void createCCTest() throws Exception { SparkCreateConnectedComponent.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", "publication", "-c", configuration, "-t", "/tmp/dedup", @@ -49,14 +52,10 @@ public class SparkCreateDedupTest { public void dedupRecordTest() throws Exception { SparkCreateDedupRecord.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", "publication", - "-c", "configuration", + "-c", configuration, "-t", "/tmp/dedup", }); } - - - - } diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json index 55c1d2066..a878d2419 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json @@ -7,7 +7,7 @@ "queueMaxSize": "2000", "groupMaxSize": "50", "slidingWindowSize": "200", - "idPath": "$.id", + "idPath": ".id", "rootBuilder": [ "organization", "projectOrganization_participation_isParticipant", @@ -117,14 +117,6 @@ "host": 0.5, "path": 0.5 } - }, - { - "name": "gridid", - "algo": "Null", - "type": "String", - "weight": "0.0", - "ignoreMissing": "true", - "path": ".pid[] | select(.qualifier.classid==\"grid\") | .value" } ], "blacklists": { diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json similarity index 100% rename from dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json rename to dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub.curr.conf.json similarity index 100% rename from dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json rename to dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub.curr.conf.json diff --git a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml index 905fb9984..292ec14c0 100644 --- a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml @@ -1,18 +1,18 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 sourceNN - webhdfs://namenode2.hadoop.dm.openaire.eu:50071 + webhdfs://namenode2.hadoop.dm.openaire.eu:50071 oozie.use.system.libpath - true + true \ No newline at end of file diff --git a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml index 91b97332b..5fe802118 100644 --- a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml @@ -14,12 +14,12 @@ hbase_dump_distcp_memory_mb - 6144 + 6144 memory for distcp action copying InfoSpace dump from remote cluster hbase_dump_distcp_num_maps - 1 + 1 maximum number of simultaneous copies of InfoSpace dump from remote location diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml index fcab9dd00..abac9bba2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml @@ -1,26 +1,26 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 oozie.use.system.libpath - true + true oozie.action.sharelib.for.spark - spark2 + spark2 hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 hive_db_name - openaire + openaire \ No newline at end of file diff --git a/dhp-workflows/docs/oozie-installer.markdown b/dhp-workflows/docs/oozie-installer.markdown index b9486ad5e..90360ec35 100644 --- a/dhp-workflows/docs/oozie-installer.markdown +++ b/dhp-workflows/docs/oozie-installer.markdown @@ -54,7 +54,7 @@ Properties overriding order is the following: 2. `~/.dhp/application.properties` defined properties 3. `${workflow.source.dir}/job.properties` 4. `job-override.properties` (located in the project root dir) -5. `maven -Dparam=value` +5. `maven -Dparam=dedupId` where the maven `-Dparam` property is overriding all the other ones. @@ -73,7 +73,7 @@ Workflow definition requirements This property can be set using maven `-D` switch. -`[oozie_app]` is the default directory name however it can be set to any value as soon as `oozieAppDir` property is provided with directory name as value. +`[oozie_app]` is the default directory name however it can be set to any dedupId as soon as `oozieAppDir` property is provided with directory name as dedupId. Subworkflows are supported as well and subworkflow directories should be nested within `[oozie_app]` directory. diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index bd2ca9704..65227a782 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -73,7 +73,7 @@ attach-test-resources - + provided @@ -326,7 +326,7 @@ - + From abd9034da06d6eb9df16fe93a6688fb00045e2af Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 11 Dec 2019 15:43:24 +0100 Subject: [PATCH 03/13] implemented DedupRecord factory with the merge of publications --- .gitignore | 3 +- dhp-common/pom.xml | 10 +- .../ArgumentApplicationParser.java | 41 +++- .../dhp/application/OptionsParameter.java | 9 + .../ArgumentApplicationParserTest.java | 10 + .../eu/dnetlib/application/parameters.json | 21 +- .../eu/dnetlib/dhp/schema/oaf/Instance.java | 27 +++ .../eu/dnetlib/dhp/schema/oaf/Result.java | 84 +------- .../eu/dnetlib/dhp/transform/ext_simple.xsl | 2 +- .../resources/eu/dnetlib/dhp/transform/tr.xml | 12 +- .../java/eu/dnetlib/dedup/DatePicker.java | 119 +++++++++++ .../eu/dnetlib/dedup/DedupRecordFactory.java | 83 ++++---- .../java/eu/dnetlib/dedup/DedupUtility.java | 196 +++++++++++++++++- .../java/eu/dnetlib/dedup/OafEntityType.java | 16 +- .../dedup/SparkCreateConnectedComponent.java | 4 +- .../dnetlib/dedup/SparkCreateDedupRecord.java | 74 ++----- .../eu/dnetlib/dedup/SparkCreateSimRels.java | 2 +- .../dhp/dedup/dedupRecord_parameters.json | 33 +++ .../dnetlib/dhp/dedup/dedup_parameters.json | 36 +++- .../dhp/dedup/oozie_app/config-default.xml | 17 +- .../dnetlib/dhp/dedup/oozie_app/workflow.xml | 25 +++ .../eu/dnetlib/dedup/MergeAuthorTest.java | 61 ++++++ .../dnetlib/dedup/SparkCreateDedupTest.java | 23 +- .../eu/dnetlib/dedup/json/authors_merge.json | 3 + .../dhp/distcp/oozie_app/config-default.xml | 8 +- .../dnetlib/dhp/distcp/oozie_app/workflow.xml | 4 +- .../dhp/graph/oozie_app/config-default.xml | 12 +- dhp-workflows/docs/oozie-installer.markdown | 4 +- dhp-workflows/pom.xml | 4 +- pom.xml | 5 + 30 files changed, 686 insertions(+), 262 deletions(-) create mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DatePicker.java create mode 100644 dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json create mode 100644 dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/MergeAuthorTest.java create mode 100644 dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/json/authors_merge.json diff --git a/.gitignore b/.gitignore index 486eacee9..3f00d9729 100644 --- a/.gitignore +++ b/.gitignore @@ -18,4 +18,5 @@ /*/build /build spark-warehouse -/dhp-workflows/dhp-graph-mapper/job-override.properties +/*/*/job-override.properties + diff --git a/dhp-common/pom.xml b/dhp-common/pom.xml index 6fac06b68..43c2a3834 100644 --- a/dhp-common/pom.xml +++ b/dhp-common/pom.xml @@ -17,6 +17,10 @@ commons-cli commons-cli + + commons-io + commons-io + org.apache.commons commons-lang3 @@ -29,21 +33,15 @@ javax.persistence javax.persistence-api - com.fasterxml.jackson.core jackson-databind - com.rabbitmq amqp-client - - commons-io - commons-io - diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java b/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java index a4970a928..cbfc5caf1 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java @@ -2,17 +2,25 @@ package eu.dnetlib.dhp.application; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.cli.*; +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.io.IOUtils; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.Serializable; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; +import java.io.StringWriter; +import java.util.*; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; +import java.util.zip.Inflater; public class ArgumentApplicationParser implements Serializable { private final Options options = new Options(); private final Map objectMap = new HashMap<>(); + private final List compressedValues = new ArrayList<>(); + public ArgumentApplicationParser(final String json_configuration) throws Exception { final ObjectMapper mapper = new ObjectMapper(); final OptionsParameter[] configuration = mapper.readValue(json_configuration, OptionsParameter[].class); @@ -29,6 +37,9 @@ public class ArgumentApplicationParser implements Serializable { final Option o = new Option(conf.getParamName(), true, conf.getParamDescription()); o.setLongOpt(conf.getParamLongName()); o.setRequired(conf.isParamRequired()); + if (conf.isCompressed()) { + compressedValues.add(conf.getParamLongName()); + } return o; }).forEach(options::addOption); @@ -38,10 +49,32 @@ public class ArgumentApplicationParser implements Serializable { } + + public static String decompressValue(final String abstractCompressed) { + try { + byte[] byteArray = Base64.decodeBase64(abstractCompressed.getBytes()); + GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(byteArray)); + final StringWriter stringWriter = new StringWriter(); + IOUtils.copy(gis, stringWriter); + return stringWriter.toString(); + } catch (Throwable e) { + System.out.println("Wrong value to decompress:" + abstractCompressed); + throw new RuntimeException(e); + } + } + + public static String compressArgument(final String value) throws Exception{ + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GZIPOutputStream gzip = new GZIPOutputStream(out); + gzip.write(value.getBytes()); + gzip.close(); + return java.util.Base64.getEncoder().encodeToString(out.toByteArray()); + } + public void parseArgument(final String[] args) throws Exception { CommandLineParser parser = new BasicParser(); CommandLine cmd = parser.parse(options, args); - Arrays.stream(cmd.getOptions()).forEach(it -> objectMap.put(it.getLongOpt(), it.getValue())); + Arrays.stream(cmd.getOptions()).forEach(it -> objectMap.put(it.getLongOpt(), compressedValues.contains(it.getLongOpt())? decompressValue(it.getValue()): it.getValue())); } public String get(final String key) { diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java b/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java index 92079fce7..4e7c2826b 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java @@ -7,6 +7,7 @@ public class OptionsParameter { private String paramLongName; private String paramDescription; private boolean paramRequired; + private boolean compressed; public OptionsParameter() { } @@ -26,4 +27,12 @@ public class OptionsParameter { public boolean isParamRequired() { return paramRequired; } + + public boolean isCompressed() { + return compressed; + } + + public void setCompressed(boolean compressed) { + this.compressed = compressed; + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java index 2033919b9..fdea3c2d4 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java @@ -3,6 +3,10 @@ package eu.dnetlib.dhp.application; import org.apache.commons.io.IOUtils; import org.junit.Test; +import java.io.ByteArrayOutputStream; +import java.util.Base64; +import java.util.zip.GZIPOutputStream; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -24,6 +28,7 @@ public class ArgumentApplicationParserTest { "-ro", "value7", "-rr", "value8", "-w", "value9", + "-cc", ArgumentApplicationParser.compressArgument(jsonConfiguration) }); assertNotNull(parser.get("hdfsPath")); assertNotNull(parser.get("apidescriptor")); @@ -45,7 +50,12 @@ public class ArgumentApplicationParserTest { assertEquals("value7", parser.get("rabbitOngoingQueue")); assertEquals("value8", parser.get("rabbitReportQueue")); assertEquals("value9", parser.get("workflowId")); + assertEquals(jsonConfiguration, parser.get("ccCoco")); } + + + + } diff --git a/dhp-common/src/test/resources/eu/dnetlib/application/parameters.json b/dhp-common/src/test/resources/eu/dnetlib/application/parameters.json index 60c2d391a..13c199166 100644 --- a/dhp-common/src/test/resources/eu/dnetlib/application/parameters.json +++ b/dhp-common/src/test/resources/eu/dnetlib/application/parameters.json @@ -1,12 +1,13 @@ [ - {"paramName":"p", "paramLongName":"hdfsPath", "paramDescription": "the path where storing the sequential file", "paramRequired": true}, - {"paramName":"a", "paramLongName":"apidescriptor", "paramDescription": "the JSON encoding of the API Descriptor", "paramRequired": true}, - {"paramName":"n", "paramLongName":"namenode", "paramDescription": "the Name Node URI", "paramRequired": true}, - {"paramName":"u", "paramLongName":"userHDFS", "paramDescription": "the user wich create the hdfs seq file", "paramRequired": true}, - {"paramName":"ru", "paramLongName":"rabbitUser", "paramDescription": "the user to connect with RabbitMq for messaging", "paramRequired": true}, - {"paramName":"rp", "paramLongName":"rabbitPassWord", "paramDescription": "the password to connect with RabbitMq for messaging", "paramRequired": true}, - {"paramName":"rh", "paramLongName":"rabbitHost", "paramDescription": "the host of the RabbitMq server", "paramRequired": true}, - {"paramName":"ro", "paramLongName":"rabbitOngoingQueue", "paramDescription": "the name of the ongoing queue", "paramRequired": true}, - {"paramName":"rr", "paramLongName":"rabbitReportQueue", "paramDescription": "the name of the report queue", "paramRequired": true}, - {"paramName":"w", "paramLongName":"workflowId", "paramDescription": "the identifier of the dnet Workflow", "paramRequired": true} + {"paramName":"p", "paramLongName":"hdfsPath", "paramDescription": "the path where storing the sequential file", "paramRequired": true}, + {"paramName":"a", "paramLongName":"apidescriptor", "paramDescription": "the JSON encoding of the API Descriptor", "paramRequired": true}, + {"paramName":"n", "paramLongName":"namenode", "paramDescription": "the Name Node URI", "paramRequired": true}, + {"paramName":"u", "paramLongName":"userHDFS", "paramDescription": "the user wich create the hdfs seq file", "paramRequired": true}, + {"paramName":"ru", "paramLongName":"rabbitUser", "paramDescription": "the user to connect with RabbitMq for messaging", "paramRequired": true}, + {"paramName":"rp", "paramLongName":"rabbitPassWord", "paramDescription": "the password to connect with RabbitMq for messaging", "paramRequired": true}, + {"paramName":"rh", "paramLongName":"rabbitHost", "paramDescription": "the host of the RabbitMq server", "paramRequired": true}, + {"paramName":"ro", "paramLongName":"rabbitOngoingQueue", "paramDescription": "the name of the ongoing queue", "paramRequired": true}, + {"paramName":"rr", "paramLongName":"rabbitReportQueue", "paramDescription": "the name of the report queue", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workflowId", "paramDescription": "the identifier of the dnet Workflow", "paramRequired": true}, + {"paramName":"cc", "paramLongName":"ccCoco", "paramDescription": "the identifier of the dnet Workflow", "compressed":true,"paramRequired": true} ] \ No newline at end of file diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java index f27704c5c..dc5ac61e8 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java @@ -84,4 +84,31 @@ public class Instance implements Serializable { public void setDateofacceptance(Field dateofacceptance) { this.dateofacceptance = dateofacceptance; } + public String toComparableString(){ + return String.format("%s::%s::%s::%s", + hostedby != null && hostedby.getKey()!= null ? hostedby.getKey().toLowerCase() : "", + accessright!= null && accessright.getClassid()!= null ? accessright.getClassid() : "", + instancetype!= null && instancetype.getClassid()!= null ? instancetype.getClassid() : "", + url != null ? url:""); + } + + @Override + public int hashCode() { + return toComparableString().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + Instance other = (Instance) obj; + + return toComparableString() + .equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 0e34d8ba6..1bb7f6a67 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -6,6 +6,8 @@ import java.io.Serializable; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; public abstract class Result extends OafEntity implements Serializable { @@ -251,13 +253,12 @@ public abstract class Result extends OafEntity implements Serializable { Result r = (Result) e; - mergeAuthors(r.getAuthor()); + //TODO mergeFrom is used only for create Dedup Records since the creation of these two fields requires more complex functions (maybe they will be filled in an external function) -// if (author == null) -// author = r.getAuthor(); //authors will be replaced because they could be too much // dateofacceptance = r.getDateofacceptance(); -// instance = mergeLists(instance, r.getInstance()); + + instance = mergeLists(instance, r.getInstance()); if (r.getResulttype() != null) resulttype = r.getResulttype(); @@ -309,80 +310,5 @@ public abstract class Result extends OafEntity implements Serializable { } - public void mergeAuthors(List authors){ - int c1 = countAuthorsPids(author); - int c2 = countAuthorsPids(authors); - int s1 = authorsSize(author); - int s2 = authorsSize(authors); - - //if both have no authors with pids and authors is bigger than author - if (c1 == 0 && c2 == 0 && author.size() authors){ - if (authors == null) - return -1; - - return (int) authors.stream().map(this::extractAuthorPid).filter(Objects::nonNull).filter(StringUtils::isNotBlank).count(); - } - - public int authorsSize(List authors){ - if (authors == null) - return 0; - return authors.size(); - } - - public String extractAuthorPid(Author a){ - - if(a == null || a.getPid() == null || a.getPid().size() == 0) - return null; - - StringBuilder mainPid = new StringBuilder(); - - a.getPid().forEach(pid ->{ - if (pid.getQualifier().getClassid().equalsIgnoreCase("orcid")) { - mainPid.setLength(0); - mainPid.append(pid.getValue()); - } - else { - if(mainPid.length() == 0) - mainPid.append(pid.getValue()); - } - }); - - return mainPid.toString(); - - } } diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl index 5f5ed5a3b..cef50aa95 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/ext_simple.xsl @@ -9,7 +9,7 @@ - + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml index ef6d9f7ac..a9eae8576 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/tr.xml @@ -1,11 +1,11 @@
- - - - - + + + + +
@@ -24,7 +24,7 @@ - + diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DatePicker.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DatePicker.java new file mode 100644 index 000000000..73f178edc --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DatePicker.java @@ -0,0 +1,119 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.dhp.schema.oaf.Field; +import org.apache.commons.lang.StringUtils; + +import java.time.Year; +import java.util.*; +import java.util.stream.Collectors; + +import static java.util.Collections.reverseOrder; +import static java.util.Map.Entry.comparingByValue; +import static java.util.stream.Collectors.toMap; +import static org.apache.commons.lang.StringUtils.endsWith; +import static org.apache.commons.lang.StringUtils.substringBefore; + +public class DatePicker { + + private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}"; + private static final String DATE_DEFAULT_SUFFIX = "01-01"; + private static final int YEAR_LB = 1300; + private static final int YEAR_UB = Year.now().getValue() + 5; + + public static Field pick(final Collection dateofacceptance) { + + final Map frequencies = dateofacceptance + .parallelStream() + .filter(StringUtils::isNotBlank) + .collect( + Collectors.toConcurrentMap( + w -> w, w -> 1, Integer::sum)); + + if (frequencies.isEmpty()) { + return new Field<>(); + } + + final Field date = new Field<>(); + date.setValue(frequencies.keySet().iterator().next()); + + // let's sort this map by values first, filtering out invalid dates + final Map sorted = frequencies + .entrySet() + .stream() + .filter(d -> StringUtils.isNotBlank(d.getKey())) + .filter(d -> d.getKey().matches(DATE_PATTERN)) + .filter(d -> inRange(d.getKey())) + .sorted(reverseOrder(comparingByValue())) + .collect( + toMap( + Map.Entry::getKey, + Map.Entry::getValue, (e1, e2) -> e2, + LinkedHashMap::new)); + + // shortcut + if (sorted.size() == 0) { + return date; + } + + // voting method (1/3 + 1) wins + if (sorted.size() >= 3) { + final int acceptThreshold = (sorted.size() / 3) + 1; + final List accepted = sorted.entrySet().stream() + .filter(e -> e.getValue() >= acceptThreshold) + .map(e -> e.getKey()) + .collect(Collectors.toList()); + + // cannot find strong majority + if (accepted.isEmpty()) { + final int max = sorted.values().iterator().next(); + Optional first = sorted.entrySet().stream() + .filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) + .map(Map.Entry::getKey) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } + + date.setValue(sorted.keySet().iterator().next()); + return date; + } + + if (accepted.size() == 1) { + date.setValue(accepted.get(0)); + return date; + } else { + final Optional first = accepted.stream() + .filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX)) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } + + return date; + } + + //1st non YYYY-01-01 is returned + } else { + if (sorted.size() == 2) { + for (Map.Entry e : sorted.entrySet()) { + if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) { + date.setValue(e.getKey()); + return date; + } + } + } + + // none of the dates seems good enough, return the 1st one + date.setValue(sorted.keySet().iterator().next()); + return date; + } + } + + private static boolean inRange(final String date) { + final int year = Integer.parseInt(substringBefore(date, "-")); + return year >= YEAR_LB && year <= YEAR_UB; + } + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index 371e80349..000be640a 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -1,6 +1,5 @@ package eu.dnetlib.dedup; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.pace.config.DedupConfig; @@ -10,23 +9,20 @@ import org.apache.commons.lang.StringUtils; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.codehaus.jackson.map.ObjectMapper; import scala.Tuple2; -import java.io.IOException; import java.util.Collection; -import java.util.List; import java.util.Random; import static java.util.stream.Collectors.toMap; public class DedupRecordFactory { - public JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf){ + public static JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf){ // final JavaPairRDD inputJsonEntities = sc.textFile(entitiesInputPath) @@ -51,7 +47,12 @@ public class DedupRecordFactory { String idValue = json._1(); - String trust = MapDocumentUtil.getJPathString("$.dataInfo.trust", json._2()); + String trust =""; + try { + trust = MapDocumentUtil.getJPathString("$.dataInfo.trust", json._2()); + } catch (Throwable e) { + + } //TODO remember to replace this with the actual trust retrieving if (StringUtils.isBlank(trust)) { @@ -71,28 +72,32 @@ public class DedupRecordFactory { .groupByKey(); + + + + switch(entityType){ - case Publication: - return sortedJoinResult.map(this::publicationMerger); - case Dataset: - return sortedJoinResult.map(this::datasetMerger); - case Project: - return sortedJoinResult.map(this::projectMerger); - case Software: - return sortedJoinResult.map(this::softwareMerger); - case Datasource: - return sortedJoinResult.map(this::datasourceMerger); - case Organization: - return sortedJoinResult.map(this::organizationMerger); - case OtherResearchProduct: - return sortedJoinResult.map(this::otherresearchproductMerger); + case publication: + return sortedJoinResult.map(DedupRecordFactory::publicationMerger); + case dataset: + return sortedJoinResult.map(DedupRecordFactory::datasetMerger); + case project: + return sortedJoinResult.map(DedupRecordFactory::projectMerger); + case software: + return sortedJoinResult.map(DedupRecordFactory::softwareMerger); + case datasource: + return sortedJoinResult.map(DedupRecordFactory::datasourceMerger); + case organization: + return sortedJoinResult.map(DedupRecordFactory::organizationMerger); + case otherresearchproduct: + return sortedJoinResult.map(DedupRecordFactory::otherresearchproductMerger); default: return null; } } - private Publication publicationMerger(Tuple2> e){ + private static Publication publicationMerger(Tuple2> e){ Publication p = new Publication(); //the result of the merge, to be returned at the end @@ -101,67 +106,59 @@ public class DedupRecordFactory { final ObjectMapper mapper = new ObjectMapper(); final Collection dateofacceptance = Lists.newArrayList(); - final Collection> authors = Lists.newArrayList(); - final Collection> instances = Lists.newArrayList(); + StringBuilder trust = new StringBuilder("0.0"); + if (e._2() != null) e._2().forEach(pub -> { try { Publication publication = mapper.readValue(pub, Publication.class); final String currentTrust = publication.getDataInfo().getTrust(); - if (!currentTrust.equals("1.0")) { + if (!"1.0".equals(currentTrust)) { trust.setLength(0); trust.append(currentTrust); } - p.mergeFrom(publication); - + p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); //add to the list if they are not null if (publication.getDateofacceptance() != null) dateofacceptance.add(publication.getDateofacceptance().getValue()); - if (publication.getAuthor() != null) - authors.add(publication.getAuthor()); - if (publication.getInstance() != null) - instances.add(publication.getInstance()); - - } catch (Exception exc){} - + } catch (Exception exc){ + throw new RuntimeException(exc); + } }); - - p.setAuthor(null); //TODO create a single list of authors to put in the final publication - - + p.setDateofacceptance(DatePicker.pick(dateofacceptance)); return p; } - private Dataset datasetMerger(Tuple2> e){ + private static Dataset datasetMerger(Tuple2> e){ throw new NotImplementedException(); } - private Project projectMerger(Tuple2> e){ + private static Project projectMerger(Tuple2> e){ throw new NotImplementedException(); } - private Software softwareMerger(Tuple2> e){ + private static Software softwareMerger(Tuple2> e){ throw new NotImplementedException(); } - private Datasource datasourceMerger(Tuple2> e){ + private static Datasource datasourceMerger(Tuple2> e){ throw new NotImplementedException(); } - private Organization organizationMerger(Tuple2> e){ + private static Organization organizationMerger(Tuple2> e){ throw new NotImplementedException(); } - private OtherResearchProduct otherresearchproductMerger(Tuple2> e){ + private static OtherResearchProduct otherresearchproductMerger(Tuple2> e){ throw new NotImplementedException(); } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java index b65e866f1..388ab9b69 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java @@ -1,11 +1,17 @@ package eu.dnetlib.dedup; import com.google.common.collect.Sets; +import com.wcohen.ss.JaroWinkler; +import eu.dnetlib.dhp.schema.oaf.Author; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; import eu.dnetlib.pace.config.DedupConfig; + import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.model.Person; import org.apache.commons.codec.binary.Hex; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -14,32 +20,35 @@ import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.util.LongAccumulator; +import scala.Tuple2; import java.io.IOException; import java.io.StringWriter; import java.nio.charset.StandardCharsets; import java.security.MessageDigest; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; +import java.text.Normalizer; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class DedupUtility { + private static final Double THRESHOLD = 0.95; public static Map constructAccumulator(final DedupConfig dedupConf, final SparkContext context) { Map accumulators = new HashMap<>(); - String acc1 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "records per hash key = 1"); + String acc1 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "records per hash key = 1"); accumulators.put(acc1, context.longAccumulator(acc1)); - String acc2 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField()); + String acc2 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "missing " + dedupConf.getWf().getOrderField()); accumulators.put(acc2, context.longAccumulator(acc2)); - String acc3 = String.format("%s::%s",dedupConf.getWf().getEntityType(), String.format("Skipped records for count(%s) >= %s", dedupConf.getWf().getOrderField(), dedupConf.getWf().getGroupMaxSize())); + String acc3 = String.format("%s::%s", dedupConf.getWf().getEntityType(), String.format("Skipped records for count(%s) >= %s", dedupConf.getWf().getOrderField(), dedupConf.getWf().getGroupMaxSize())); accumulators.put(acc3, context.longAccumulator(acc3)); - String acc4 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "skip list"); + String acc4 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "skip list"); accumulators.put(acc4, context.longAccumulator(acc4)); - String acc5 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)"); + String acc5 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "dedupSimilarity (x2)"); accumulators.put(acc5, context.longAccumulator(acc5)); - String acc6 = String.format("%s::%s",dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold()); + String acc6 = String.format("%s::%s", dedupConf.getWf().getEntityType(), "d < " + dedupConf.getWf().getThreshold()); accumulators.put(acc6, context.longAccumulator(acc6)); return accumulators; @@ -52,7 +61,7 @@ public class DedupUtility { public static void deleteIfExists(String path) throws IOException { Configuration conf = new Configuration(); FileSystem fileSystem = FileSystem.get(conf); - if (fileSystem.exists(new Path(path))){ + if (fileSystem.exists(new Path(path))) { fileSystem.delete(new Path(path), true); } } @@ -91,4 +100,171 @@ public class DedupUtility { return null; } } + + + public static List mergeAuthor(final List a, final List b) { + int pa = countAuthorsPids(a); + int pb = countAuthorsPids(b); + List base, enrich; + int sa = authorsSize(a); + int sb = authorsSize(b); + + if(pa == pb){ + base = sa>sb?a:b; + enrich = sa>sb?b:a; + } else { + base = pa>pb?a:b; + enrich = pa>pb?b:a; + } + enrichPidFromList(base, enrich); + return base; + + + +// //if both have no authors with pids +// if (pa < 1 && pb < 1) { +// //B is bigger than A +// if (sa < sb) +// return b; +// //A is bigger than B +// else +// return a; +// } +// //If A has author with pids +// if (pa > 0) { +// //B has no author with pid +// if (pb < 1) +// return a; +// //B has author with pid +// else { +// enrichPidFromList(a, b); +// return a; +// } +// } +// //If B has author with pids +// //A has no author with pid +// if (pa < 1) +// return b; +// //A has author with pid +// else { +// enrichPidFromList(b, a); +// return b; +// } + } + + private static void enrichPidFromList(List base, List enrich) { + if(base==null || enrich == null) + return; + final Map basePidAuthorMap = base.stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap(a -> a.getPid() + .stream() + .map(p -> new Tuple2<>(p.toComparableString(), a)) + ).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); + + final List> pidToEnrich = enrich + .stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap(a -> a.getPid().stream().filter(p -> !basePidAuthorMap.containsKey(p.toComparableString())).map(p -> new Tuple2<>(p, a))) + .collect(Collectors.toList()); + + + pidToEnrich.forEach(a -> { + Optional> simAuhtor = base.stream().map(ba -> new Tuple2<>(sim(ba, a._2()), ba)).max(Comparator.comparing(Tuple2::_1)); + if (simAuhtor.isPresent() && simAuhtor.get()._1()> THRESHOLD) { + Author r = simAuhtor.get()._2(); + r.getPid().add(a._1()); + } + }); + } + + public static String createEntityPath(final String basePath, final String entityType) { + return String.format("%s/%s", basePath,entityType); + } + + public static String createSimRelPath(final String basePath, final String entityType) { + return String.format("%s/%s_simRel", basePath,entityType); + } + + public static String createMergeRelPath(final String basePath, final String entityType) { + return String.format("%s/%s_mergeRel", basePath,entityType); + } + + private static Double sim(Author a, Author b) { + + final Person pa = parse(a); + final Person pb = parse(b); + + if (pa.isAccurate() & pb.isAccurate()) { + return new JaroWinkler().score( + normalize(pa.getSurnameString()), + normalize(pb.getSurnameString())); + } else { + return new JaroWinkler().score( + normalize(pa.getNormalisedFullname()), + normalize(pb.getNormalisedFullname())); + } + } + + private static String normalize(final String s) { + return nfd(s).toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError in case of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim(); + } + + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } + private static Person parse(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return new Person(author.getSurname() + ", " + author.getName(), false); + } else { + return new Person(author.getFullname(), false); + } + } + + + private static int countAuthorsPids(List authors) { + if (authors == null) + return 0; + + return (int) authors.stream().map(DedupUtility::extractAuthorPid).filter(Objects::nonNull).filter(StringUtils::isNotBlank).count(); + } + + private static int authorsSize(List authors) { + if (authors == null) + return 0; + return authors.size(); + } + + + private static boolean isAccurate(final Author a) { + return StringUtils.isNotBlank(a.getName()) && StringUtils.isNotBlank(a.getSurname()); + } + + private static String extractAuthorPid(Author a) { + + if (a == null || a.getPid() == null || a.getPid().size() == 0) + return null; + + StringBuilder mainPid = new StringBuilder(); + + a.getPid().forEach(pid -> { + if (pid.getQualifier().getClassid().equalsIgnoreCase("orcid")) { + mainPid.setLength(0); + mainPid.append(pid.getValue()); + } else { + if (mainPid.length() == 0) + mainPid.append(pid.getValue()); + } + }); + + return mainPid.toString(); + + } } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java index 4ff2fa873..fb347ed51 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafEntityType.java @@ -2,12 +2,14 @@ package eu.dnetlib.dedup; public enum OafEntityType { - Datasource, - Organization, - Project, - Dataset, - OtherResearchProduct, - Software, - Publication + datasource, + organization, + project, + dataset, + otherresearchproduct, + software, + publication + + } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index eacf3d479..9783e93d6 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -46,7 +46,7 @@ public class SparkCreateConnectedComponent { s -> new Tuple2((long) s.hashCode(), s) ); - final Dataset similarityRelations = spark.read().load(targetPath + "/" + entity+"_simrel").as(Encoders.bean(Relation.class)); + final Dataset similarityRelations = spark.read().load(DedupUtility.createSimRelPath(targetPath,entity)).as(Encoders.bean(Relation.class)); final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(it.getSource().hashCode(), it.getTarget().hashCode(), it.getRelClass())).rdd(); @@ -73,7 +73,7 @@ public class SparkCreateConnectedComponent { return tmp.stream(); }).iterator()).rdd(), Encoders.bean(Relation.class)); - mergeRelation.write().mode("overwrite").save(targetPath+"/"+entity+"_mergeRels"); + mergeRelation.write().mode("overwrite").save(DedupUtility.createMergeRelPath(targetPath,entity)); } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java index 56bdc20f1..db2306526 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java @@ -1,71 +1,39 @@ package eu.dnetlib.dedup; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Lists; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.util.MapDocumentUtil; import org.apache.commons.io.IOUtils; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.Optional; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.types.StructType; -import scala.Tuple2; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; public class SparkCreateDedupRecord { public static void main(String[] args) throws Exception { -// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedup_parameters.json"))); -// parser.parseArgument(args); -// final SparkSession spark = SparkSession -// .builder() -// .appName(SparkCreateDedupRecord.class.getSimpleName()) -// .master(parser.get("master")) -// .getOrCreate(); -// -// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); -// final String inputPath = parser.get("sourcePath"); -// final String entity = parser.get("entity"); -// final String targetPath = parser.get("targetPath"); -//// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); -// final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); -// -// // -// final JavaPairRDD inputJsonEntities = sc.textFile(inputPath + "/" + entity) -// .mapToPair((PairFunction)it-> -// new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) -// ); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCreateDedupRecord.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); -// //: source is the dedup_id, target is the id of the mergedIn -// JavaPairRDD mergeRels = spark -// .read().load(targetPath + "/" + entity+"_mergeRels").as(Encoders.bean(Relation.class)) -// .where("relClass=='merges'") -// .javaRDD() -// .mapToPair( -// (PairFunction)r-> -// new Tuple2(r.getTarget(), r.getSource()) -// ); -// -// // -// final JavaPairRDD p = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String sourcePath = parser.get("sourcePath"); + final String entity = parser.get("entity"); + final String dedupPath = parser.get("dedupPath"); +// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateDedupRecord.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); + final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); + + final JavaRDD dedupRecord = DedupRecordFactory.createDedupRecord(sc, spark, DedupUtility.createMergeRelPath(dedupPath,entity), DedupUtility.createEntityPath(sourcePath,entity), OafEntityType.valueOf(entity), dedupConf); + dedupRecord.map(r-> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(r); + }).saveAsTextFile(dedupPath+"/"+entity+"_dedup_record_json"); - StructType schema = Encoders.bean(Publication.class).schema(); - System.out.println(schema); } } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java index 2ae85baf3..48d442d04 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java @@ -77,7 +77,7 @@ public class SparkCreateSimRels { return r; }); - spark.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)).write().mode("overwrite").save(targetPath+"/"+entity+"_simrel"); + spark.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)).write().mode("overwrite").save( DedupUtility.createSimRelPath(targetPath,entity)); diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json new file mode 100644 index 000000000..de744dfb6 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedupRecord_parameters.json @@ -0,0 +1,33 @@ +[ + { + "paramName": "mt", + "paramLongName": "master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the path of the sequential file to read", + "paramRequired": true + }, + { + "paramName": "e", + "paramLongName": "entity", + "paramDescription": "the type of entity to be deduped", + "paramRequired": true + }, + { + "paramName": "c", + "paramLongName": "dedupConf", + "paramDescription": "dedup configuration to be used", + "compressed": true, + "paramRequired": true + }, + { + "paramName": "d", + "paramLongName": "dedupPath", + "paramDescription": "dedup path to load mergeRelation", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json index d9a0dc8b9..8ba8515d0 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/dedup_parameters.json @@ -1,7 +1,33 @@ [ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequential file to read", "paramRequired": true}, - {"paramName":"e", "paramLongName":"entity", "paramDescription": "the type of entity to be deduped", "paramRequired": true}, - {"paramName":"c", "paramLongName":"dedupConf", "paramDescription": "dedup configuration to be used", "paramRequired": true}, - {"paramName":"t", "paramLongName":"targetPath", "paramDescription": "target path to save dedup result", "paramRequired": true} + { + "paramName": "mt", + "paramLongName": "master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the path of the sequential file to read", + "paramRequired": true + }, + { + "paramName": "e", + "paramLongName": "entity", + "paramDescription": "the type of entity to be deduped", + "paramRequired": true + }, + { + "paramName": "c", + "paramLongName": "dedupConf", + "paramDescription": "dedup configuration to be used", + "compressed": true, + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "target path to save dedup result", + "paramRequired": true + } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml index e654bbbb6..fcab9dd00 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/config-default.xml @@ -1,31 +1,26 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 oozie.use.system.libpath - true + true oozie.action.sharelib.for.spark - spark2 + spark2 hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 hive_db_name - openaire + openaire - - dedupConf - {"wf":{"threshold":"0.9","dedupRun":"001","entityType":"organization","orderField":"legalname","queueMaxSize":"2000","groupMaxSize":"50","slidingWindowSize":"200","idPath":".id","rootBuilder":["organization","projectOrganization_participation_isParticipant","datasourceOrganization_provision_isProvidedBy"],"includeChildren":"true"},"pace":{"clustering":[{"name":"sortedngrampairs","fields":["legalname"],"params":{"max":2,"ngramLen":"3"}},{"name":"suffixprefix","fields":["legalname"],"params":{"max":1,"len":"3"}},{"name":"urlclustering","fields":["websiteurl"],"params":{}},{"name":"keywordsclustering","fields":["legalname"],"params":{"max":2,"windowSize":4}}],"strictConditions":[{"name":"exactMatch","fields":["gridid"]}],"conditions":[{"name":"DomainExactMatch","fields":["websiteurl"]},{"name":"exactMatch","fields":["country"]}],"model":[{"name":"country","algo":"Null","type":"String","weight":"0","ignoreMissing":"false","path":".country.classid"},{"name":"legalshortname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.1","ignoreMissing":"true","path":".legalshortname.dedupId"},{"name":"legalname","algo":"JaroWinklerNormalizedName","type":"String","weight":"0.9","ignoreMissing":"false","path":".legalname.dedupId","params":{"windowSize":4,"threshold":0.7}},{"name":"websiteurl","algo":"Null","type":"URL","weight":"0","ignoreMissing":"true","path":".websiteurl.dedupId","params":{"host":0.5,"path":0.5}},{"name":"gridid","algo":"Null","type":"String","weight":"0.0","ignoreMissing":"true","path":".pid[] | select(.qualifier.classid==\"grid\") | .dedupId"}],"blacklists":{"legalname":[]},"synonyms":{"key::1":["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti"],"key::2":["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"],"key::3":["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"],"key::4":["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"],"key::5":["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"],"key::6":["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"],"key::7":["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"],"key::8":["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"],"key::9":["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"],"key::10":["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"],"key::11":["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"],"key::12":["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"],"key::13":["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"],"key::14":["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"],"key::15":["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"],"key::16":["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"],"key::17":["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"],"key::18":["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"],"key::19":["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"],"key::20":["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"],"key::21":["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"],"key::22":["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"],"key::23":["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"],"key::24":["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"],"key::25":["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"],"key::26":["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"],"key::27":["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"],"key::28":["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"],"key::29":["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"],"key::30":["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"],"key::31":["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"],"key::32":["industry","industria","industrie","индустрия","industrie","βιομηχανία"],"key::33":["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"],"key::34":["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"],"key::35":["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"],"key::36":["authority","autorità","autorité","авторитет","autoriteit"],"key::37":["federation","federazione","fédération","федерация","federatie","ομοσπονδία"],"key::38":["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"],"key::39":["bureau","ufficio","bureau","офис","bureau","γραφείο"],"key::40":["company","impresa","compagnie","société","компания","bedrijf","εταιρία"],"key::41":["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"],"key::42":["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"],"key::43":["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"],"key::44":["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"],"key::45":["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"],"key::46":["division","divisione","division","отделение","divisie","τμήμα"],"key::47":["committee","comitato","comité","комитет","commissie","επιτροπή"],"key::48":["promotion","promozione","продвижение","proothisis","forderung"],"key::49":["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"],"key::50":["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"],"key::51":["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""],"key::52":["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""],"key::53":["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""],"key::54":["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""],"key::55":["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""],"key::56":["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""],"key::57":["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""],"key::58":["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""],"key::59":["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""],"key::60":["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""],"key::61":["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""],"key::62":["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""],"key::63":["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""],"key::64":["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""],"key::65":["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""],"key::66":["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""],"key::67":["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""],"key::68":["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""],"key::69":["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""],"key::70":["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""],"key::71":["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""],"key::72":["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""],"key::73":["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""],"key::74":["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""],"key::75":["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""],"key::76":["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""],"key::77":["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""],"key::78":["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""],"key::79":["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""],"key::80":["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""],"key::81":["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""],"key::82":["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""],"key::83":["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""],"key::84":["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""],"key::85":["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""],"key::86":["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""],"key::87":["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""],"key::88":["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""],"key::89":["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""],"key::90":["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""],"key::91":["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""],"key::92":["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""],"key::93":["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""],"key::94":["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""],"key::95":["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""],"key::96":["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""],"key::97":["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""],"key::98":["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""],"key::99":["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""],"key::100":["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""],"key::101":["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""],"key::102":["informatics","informatica","informática","informática","informatica",""],"key::103":["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"],"key::104":["commerce","ticaret","ticarət","commercio","trade","handel","comercio"]}}} - - \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml index 7f9de8af5..09dd3a315 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml @@ -93,6 +93,31 @@ --entity${entity} --dedupConf${dedupConf} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + Create Connected Components + eu.dnetlib.dedup.SparkCreateDedupRecord + dhp-dedup-${projectVersion}.jar + --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} --conf + spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" --conf + spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" --conf + spark.sql.warehouse.dir="/user/hive/warehouse" + + -mtyarn-cluster + --sourcePath${sourcePath} + --dedupPath${dedupPath} + --entity${entity} + --dedupConf${dedupConf} + diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/MergeAuthorTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/MergeAuthorTest.java new file mode 100644 index 000000000..817f2075c --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/MergeAuthorTest.java @@ -0,0 +1,61 @@ +package eu.dnetlib.dedup; + +import eu.dnetlib.dhp.schema.oaf.Publication; +import org.apache.commons.io.IOUtils; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +public class MergeAuthorTest { + + List publicationsToMerge; + final ObjectMapper mapper = new ObjectMapper(); + + @Before + public void setUp() throws Exception { + final String json = IOUtils.toString(this.getClass().getResourceAsStream("/eu/dnetlib/dedup/json/authors_merge.json")); + + + publicationsToMerge = Arrays.asList(json.split("\n")).stream().map(s-> { + try { + return mapper.readValue(s, Publication.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).collect(Collectors.toList()); + + + + } + + + @Test + public void test() throws Exception { + Publication dedup = new Publication(); + + + publicationsToMerge.forEach(p-> { + dedup.mergeFrom(p); + dedup.setAuthor(DedupUtility.mergeAuthor(dedup.getAuthor(),p.getAuthor())); + }); + + + + + + + + + System.out.println(mapper.writeValueAsString(dedup)); + + + } + + + +} diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index f35baa9f8..5d5576dd8 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -1,6 +1,7 @@ package eu.dnetlib.dedup; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Publication; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -27,9 +28,9 @@ public class SparkCreateDedupTest { public void createSimRelsTest() throws Exception { SparkCreateSimRels.main(new String[] { "-mt", "local[*]", - "-s", "/Users/miconis/dumps", + "-s", "/home/sandro/betadump", "-e", "publication", - "-c", configuration, + "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", }); } @@ -40,9 +41,9 @@ public class SparkCreateDedupTest { SparkCreateConnectedComponent.main(new String[] { "-mt", "local[*]", - "-s", "/Users/miconis/dumps", + "-s", "/home/sandro/betadump", "-e", "publication", - "-c", configuration, + "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", }); } @@ -52,10 +53,18 @@ public class SparkCreateDedupTest { public void dedupRecordTest() throws Exception { SparkCreateDedupRecord.main(new String[] { "-mt", "local[*]", - "-s", "/Users/miconis/dumps", + "-s", "/home/sandro/betadump", "-e", "publication", - "-c", configuration, - "-t", "/tmp/dedup", + "-c", ArgumentApplicationParser.compressArgument(configuration), + "-d", "/tmp/dedup", }); } + + @Test + public void printCC() throws Exception { + System.out.println(ArgumentApplicationParser.compressArgument(configuration)); + } + + + } diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/json/authors_merge.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/json/authors_merge.json new file mode 100644 index 000000000..4e8b66d1b --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/json/authors_merge.json @@ -0,0 +1,3 @@ +{"journal":{"name":"","issnPrinted":"","issnOnline":"","issnLinking":"","ep":"","iss":"","sp":"","vol":"","edition":"","conferenceplace":"","conferencedate":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"author":[{"fullname":"Nemkov, Pavel G.","name":"","surname":"","rank":1,"pid":[],"affiliation":[]},{"fullname":"Gayubo, Severiano F.","name":"","surname":"","rank":2,"pid":[{"value":"ORCID1","qualifier":{"classid":"orcid","classname":"orcid","schemeid":"dnet:pidType","schemename":"dnet:pidType"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"affiliation":[]},{"fullname":"Ciccio Pasticcio","name":"","surname":"","rank":2,"pid":[],"affiliation":[]}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"und","classname":"Undetermined","schemeid":"dent:languages","schemename":"dent:languages"},"country":[],"subject":[{"value":"Biodiversity","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Taxonomy","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Animalia","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Arthropoda","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Insecta","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Hymenoptera","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Crabronidae","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"title":[{"value":"A New Species Of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) From Turkmenistan","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"relevantdate":[{"value":"2003-12-31","qualifier":{"classid":"dnet:date","classname":"dnet:date","schemeid":"dnet:date","schemename":"dnet:date"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"description":[{"value":"Nemkov, Pavel G., Gayubo, Severiano F. (2003): A new species of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) from Turkmenistan. Zootaxa 144: 1-4, DOI: 10.5281/zenodo.156314","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"dateofacceptance":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"publisher":{"value":"Zenodo","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"embargoenddate":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":{"classid":"","classname":"","schemeid":"","schemename":""},"coverage":[],"refereed":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"context":[],"id":"50|scholexplore::ceb3a5d32107897a0df1178211e3e9ca","originalId":[],"collectedfrom":[{"key":"10|openaire____::e034d6a11054f5ade9221ebac484e864","value":"scholExplorer","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"pid":[{"value":"10.5281/zenodo.156314","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"extraInfo":[],"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":true,"inferenceprovenance":"dedup-similarity-result-levenstein","provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":0} +{"journal":{"name":"","issnPrinted":"","issnOnline":"","issnLinking":"","ep":"","iss":"","sp":"","vol":"","edition":"","conferenceplace":"","conferencedate":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"author":[{"fullname":"Nemkov, Pavel G.","name":"","surname":"","rank":1,"pid":[{"value":"ORCIDDIO","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"dnet:pidType","schemename":"dnet:pidType"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"affiliation":[]},{"fullname":"Gayubo, Severiano F.","name":"","surname":"","rank":2,"pid":[{"value":"MAGGLES","qualifier":{"classid":"mag","classname":"mag","schemeid":"dnet:pidType","schemename":"dnet:pidType"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"affiliation":[]}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"und","classname":"Undetermined","schemeid":"dent:languages","schemename":"dent:languages"},"country":[],"subject":[{"value":"Biodiversity","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Taxonomy","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Animalia","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Arthropoda","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Insecta","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Hymenoptera","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Crabronidae","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"title":[{"value":"A New Species Of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) From Turkmenistan","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"relevantdate":[{"value":"2003-12-31","qualifier":{"classid":"dnet:date","classname":"dnet:date","schemeid":"dnet:date","schemename":"dnet:date"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"description":[{"value":"Nemkov, Pavel G., Gayubo, Severiano F. (2003): A new species of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) from Turkmenistan. Zootaxa 144: 1-4, DOI: 10.5281/zenodo.156314","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"dateofacceptance":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"publisher":{"value":"Zenodo","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"embargoenddate":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":{"classid":"","classname":"","schemeid":"","schemename":""},"coverage":[],"refereed":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"context":[],"id":"50|scholexplore::ceb3a5d32107897a0df1178211e3e9ca","originalId":[],"collectedfrom":[{"key":"10|openaire____::e034d6a11054f5ade9221ebac484e864","value":"scholExplorer","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"pid":[{"value":"10.5281/zenodo.156314","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"extraInfo":[],"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":true,"inferenceprovenance":"dedup-similarity-result-levenstein","provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":0} +{"journal":{"name":"","issnPrinted":"","issnOnline":"","issnLinking":"","ep":"","iss":"","sp":"","vol":"","edition":"","conferenceplace":"","conferencedate":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"author":[],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"und","classname":"Undetermined","schemeid":"dent:languages","schemename":"dent:languages"},"country":[],"subject":[{"value":"Biodiversity","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Taxonomy","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Animalia","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Arthropoda","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Insecta","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Hymenoptera","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},{"value":"Crabronidae","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject","schemename":"dnet:subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"title":[{"value":"A New Species Of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) From Turkmenistan","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"relevantdate":[{"value":"2003-12-31","qualifier":{"classid":"dnet:date","classname":"dnet:date","schemeid":"dnet:date","schemename":"dnet:date"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"description":[{"value":"Nemkov, Pavel G., Gayubo, Severiano F. (2003): A new species of Nysson Latreille (Hymenoptera: Crabronidae: Bembicinae) from Turkmenistan. Zootaxa 144: 1-4, DOI: 10.5281/zenodo.156314","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"dateofacceptance":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"publisher":{"value":"Zenodo","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"embargoenddate":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":{"classid":"","classname":"","schemeid":"","schemename":""},"coverage":[],"refereed":{"value":"","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}},"context":[],"id":"50|scholexplore::ceb3a5d32107897a0df1178211e3e9ca","originalId":[],"collectedfrom":[{"key":"10|openaire____::e034d6a11054f5ade9221ebac484e864","value":"scholExplorer","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"pid":[{"value":"10.5281/zenodo.156314","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"inferenceprovenance":"","provenanceaction":{"classid":"","classname":"","schemeid":"","schemename":""}}}],"extraInfo":[],"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":true,"inferenceprovenance":"dedup-similarity-result-levenstein","provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":0} diff --git a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml index 292ec14c0..905fb9984 100644 --- a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/config-default.xml @@ -1,18 +1,18 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 sourceNN - webhdfs://namenode2.hadoop.dm.openaire.eu:50071 + webhdfs://namenode2.hadoop.dm.openaire.eu:50071 oozie.use.system.libpath - true + true \ No newline at end of file diff --git a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml index 5fe802118..91b97332b 100644 --- a/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-distcp/src/main/resources/eu/dnetlib/dhp/distcp/oozie_app/workflow.xml @@ -14,12 +14,12 @@ hbase_dump_distcp_memory_mb - 6144 + 6144 memory for distcp action copying InfoSpace dump from remote cluster hbase_dump_distcp_num_maps - 1 + 1 maximum number of simultaneous copies of InfoSpace dump from remote location diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml index abac9bba2..fcab9dd00 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml @@ -1,26 +1,26 @@ jobTracker - yarnRM + yarnRM nameNode - hdfs://nameservice1 + hdfs://nameservice1 oozie.use.system.libpath - true + true oozie.action.sharelib.for.spark - spark2 + spark2 hive_metastore_uris - thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 hive_db_name - openaire + openaire \ No newline at end of file diff --git a/dhp-workflows/docs/oozie-installer.markdown b/dhp-workflows/docs/oozie-installer.markdown index 90360ec35..b1953a54e 100644 --- a/dhp-workflows/docs/oozie-installer.markdown +++ b/dhp-workflows/docs/oozie-installer.markdown @@ -54,7 +54,7 @@ Properties overriding order is the following: 2. `~/.dhp/application.properties` defined properties 3. `${workflow.source.dir}/job.properties` 4. `job-override.properties` (located in the project root dir) -5. `maven -Dparam=dedupId` +5. `maven -Dparam=value` where the maven `-Dparam` property is overriding all the other ones. @@ -73,7 +73,7 @@ Workflow definition requirements This property can be set using maven `-D` switch. -`[oozie_app]` is the default directory name however it can be set to any dedupId as soon as `oozieAppDir` property is provided with directory name as dedupId. +`[oozie_app]` is the default directory name however it can be set to any value as soon as `oozieAppDir` property is provided with directory name as value. Subworkflows are supported as well and subworkflow directories should be nested within `[oozie_app]` directory. diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index 65227a782..bd2ca9704 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -73,7 +73,7 @@ attach-test-resources - + provided @@ -326,7 +326,7 @@ - + diff --git a/pom.xml b/pom.xml index 5d3dce34a..cc39e46f2 100644 --- a/pom.xml +++ b/pom.xml @@ -236,6 +236,11 @@ java-jq 0.10.1 + + edu.cmu + secondstring + 1.0.0 + org.apache.oozie From 6b45e37e22fc2ecd61e54982626e1fb99ca17a4a Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 11 Dec 2019 16:57:37 +0100 Subject: [PATCH 04/13] implemented DedupRecord factory with the merge of organizations --- .../dnetlib/dhp/schema/oaf/Organization.java | 23 + .../eu/dnetlib/dhp/schema/oaf/Result.java | 23 +- .../dhp/schema/oaf/StructuredProperty.java | 2 +- .../eu/dnetlib/dedup/DedupRecordFactory.java | 94 +- .../dnetlib/dedup/SparkCreateDedupTest.java | 9 +- .../eu/dnetlib/dedup/conf/org.curr.conf.json | 18 +- .../eu/dnetlib/dedup/conf/org.curr.conf2.json | 1753 ----------------- 7 files changed, 113 insertions(+), 1809 deletions(-) delete mode 100644 dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java index 6f89eca7e..c3e9a7007 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java @@ -164,4 +164,27 @@ public class Organization extends OafEntity implements Serializable { public void setCountry(Qualifier country) { this.country = country; } + + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + final Organization o = (Organization) e; + legalshortname = o.getLegalshortname() != null ? o.getLegalshortname() : legalshortname; + legalname = o.getLegalname() != null ? o.getLegalname() : legalname; + alternativeNames = mergeLists(o.getAlternativeNames(), alternativeNames); + websiteurl = o.getWebsiteurl() != null ? o.getWebsiteurl() : websiteurl; + logourl = o.getLogourl() != null ? o.getLogourl() : logourl; + eclegalbody = o.getEclegalbody() != null ? o.getEclegalbody() : eclegalbody; + eclegalperson = o.getEclegalperson() != null ? o.getEclegalperson() : eclegalperson; + ecnonprofit = o.getEcnonprofit() != null ? o.getEcnonprofit() : ecnonprofit; + ecresearchorganization = o.getEcresearchorganization() != null ? o.getEcresearchorganization() : ecresearchorganization; + echighereducation = o.getEchighereducation() != null ? o.getEchighereducation() : echighereducation; + ecinternationalorganizationeurinterests = o.getEcinternationalorganizationeurinterests() != null ? o.getEcinternationalorganizationeurinterests() : ecinternationalorganizationeurinterests; + ecinternationalorganization = o.getEcinternationalorganization() != null ? o.getEcinternationalorganization() : ecinternationalorganization; + ecenterprise = o.getEcenterprise() != null ? o.getEcenterprise() :ecenterprise; + ecsmevalidated = o.getEcsmevalidated() != null ? o.getEcsmevalidated() :ecsmevalidated; + ecnutscode = o.getEcnutscode() != null ? o.getEcnutscode() :ecnutscode; + country = o.getCountry() != null ? o.getCountry() :country; + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 1bb7f6a67..7fb7aef10 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -3,10 +3,7 @@ package eu.dnetlib.dhp.schema.oaf; import org.apache.commons.lang3.StringUtils; import java.io.Serializable; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; +import java.util.*; import java.util.stream.Collectors; public abstract class Result extends OafEntity implements Serializable { @@ -253,11 +250,6 @@ public abstract class Result extends OafEntity implements Serializable { Result r = (Result) e; - - - //TODO mergeFrom is used only for create Dedup Records since the creation of these two fields requires more complex functions (maybe they will be filled in an external function) -// dateofacceptance = r.getDateofacceptance(); - instance = mergeLists(instance, r.getInstance()); if (r.getResulttype() != null) @@ -274,7 +266,7 @@ public abstract class Result extends OafEntity implements Serializable { relevantdate = mergeLists(relevantdate, r.getRelevantdate()); - description = mergeLists(description, r.getDescription()); + description = longestLists(description, r.getDescription()); if (r.getPublisher() != null) publisher = r.getPublisher(); @@ -310,5 +302,16 @@ public abstract class Result extends OafEntity implements Serializable { } + private List> longestLists(List> a, List> b) { + if(a == null || b == null) + return a==null?b:a; + if (a.size()== b.size()) { + int msa = a.stream().filter(i -> i.getValue() != null).map(i -> i.getValue().length()).max(Comparator.naturalOrder()).orElse(0); + int msb = b.stream().filter(i -> i.getValue() != null).map(i -> i.getValue().length()).max(Comparator.naturalOrder()).orElse(0); + return msa>msb?a:b; + } + return a.size()> b.size()?a:b; + } + } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java index ea2370c7a..f6c6b7335 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java @@ -35,7 +35,7 @@ public class StructuredProperty implements Serializable { } public String toComparableString(){ - return String.format("%s::%s", value != null ? value.toLowerCase() : "", qualifier != null ? qualifier.toComparableString().toLowerCase() : ""); + return value != null ? value.toLowerCase() : ""; } @Override diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index 000be640a..c6e3efbc4 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -22,32 +22,32 @@ import static java.util.stream.Collectors.toMap; public class DedupRecordFactory { - public static JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf){ + public static JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf) { // final JavaPairRDD inputJsonEntities = sc.textFile(entitiesInputPath) - .mapToPair((PairFunction) it-> - new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it),it) + .mapToPair((PairFunction) it -> + new Tuple2(MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it) ); //: source is the dedup_id, target is the id of the mergedIn - JavaPairRDD mergeRels = spark + JavaPairRDD mergeRels = spark .read().load(mergeRelsInputPath).as(Encoders.bean(Relation.class)) .where("relClass=='merges'") .javaRDD() .mapToPair( - (PairFunction)r-> + (PairFunction) r -> new Tuple2(r.getTarget(), r.getSource()) ); // final JavaPairRDD joinResult = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); - JavaPairRDD keyJson = joinResult.mapToPair((PairFunction, OafKey, String>) json -> { + JavaPairRDD keyJson = joinResult.mapToPair((PairFunction, OafKey, String>) json -> { String idValue = json._1(); - String trust =""; + String trust = ""; try { trust = MapDocumentUtil.getJPathString("$.dataInfo.trust", json._2()); } catch (Throwable e) { @@ -72,11 +72,7 @@ public class DedupRecordFactory { .groupByKey(); - - - - - switch(entityType){ + switch (entityType) { case publication: return sortedJoinResult.map(DedupRecordFactory::publicationMerger); case dataset: @@ -97,7 +93,7 @@ public class DedupRecordFactory { } - private static Publication publicationMerger(Tuple2> e){ + private static Publication publicationMerger(Tuple2> e) { Publication p = new Publication(); //the result of the merge, to be returned at the end @@ -111,54 +107,80 @@ public class DedupRecordFactory { StringBuilder trust = new StringBuilder("0.0"); if (e._2() != null) - e._2().forEach(pub -> { - try { - Publication publication = mapper.readValue(pub, Publication.class); + e._2().forEach(pub -> { + try { + Publication publication = mapper.readValue(pub, Publication.class); - final String currentTrust = publication.getDataInfo().getTrust(); - if (!"1.0".equals(currentTrust)) { - trust.setLength(0); - trust.append(currentTrust); + final String currentTrust = publication.getDataInfo().getTrust(); + if (!"1.0".equals(currentTrust)) { + trust.setLength(0); + trust.append(currentTrust); + } + p.mergeFrom(publication); + p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); + //add to the list if they are not null + if (publication.getDateofacceptance() != null) + dateofacceptance.add(publication.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); } - p.mergeFrom(publication); - p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); - //add to the list if they are not null - if (publication.getDateofacceptance() != null) - dateofacceptance.add(publication.getDateofacceptance().getValue()); - } catch (Exception exc){ - throw new RuntimeException(exc); - } - }); + }); p.setDateofacceptance(DatePicker.pick(dateofacceptance)); return p; } - private static Dataset datasetMerger(Tuple2> e){ + private static Dataset datasetMerger(Tuple2> e) { throw new NotImplementedException(); } - private static Project projectMerger(Tuple2> e){ + private static Project projectMerger(Tuple2> e) { throw new NotImplementedException(); } - private static Software softwareMerger(Tuple2> e){ + private static Software softwareMerger(Tuple2> e) { throw new NotImplementedException(); } - private static Datasource datasourceMerger(Tuple2> e){ + private static Datasource datasourceMerger(Tuple2> e) { throw new NotImplementedException(); } - private static Organization organizationMerger(Tuple2> e){ + private static Organization organizationMerger(Tuple2> e) { - throw new NotImplementedException(); + Organization o = new Organization(); //the result of the merge, to be returned at the end + + o.setId(e._1()); + + final ObjectMapper mapper = new ObjectMapper(); + + + StringBuilder trust = new StringBuilder("0.0"); + + if (e._2() != null) + e._2().forEach(pub -> { + try { + Organization organization = mapper.readValue(pub, Organization.class); + + final String currentTrust = organization.getDataInfo().getTrust(); + if (!"1.0".equals(currentTrust)) { + trust.setLength(0); + trust.append(currentTrust); + } + o.mergeFrom(organization); + + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + + return o; } - private static OtherResearchProduct otherresearchproductMerger(Tuple2> e){ + private static OtherResearchProduct otherresearchproductMerger(Tuple2> e) { throw new NotImplementedException(); } diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index 5d5576dd8..7aa8a4302 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -16,10 +16,11 @@ import java.util.List; public class SparkCreateDedupTest { String configuration; + String entity = "organization"; @Before public void setUp() throws IOException { - configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/pub.curr.conf.json")); + configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json")); } @@ -29,7 +30,7 @@ public class SparkCreateDedupTest { SparkCreateSimRels.main(new String[] { "-mt", "local[*]", "-s", "/home/sandro/betadump", - "-e", "publication", + "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", }); @@ -42,7 +43,7 @@ public class SparkCreateDedupTest { SparkCreateConnectedComponent.main(new String[] { "-mt", "local[*]", "-s", "/home/sandro/betadump", - "-e", "publication", + "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", }); @@ -54,7 +55,7 @@ public class SparkCreateDedupTest { SparkCreateDedupRecord.main(new String[] { "-mt", "local[*]", "-s", "/home/sandro/betadump", - "-e", "publication", + "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-d", "/tmp/dedup", }); diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json index a878d2419..2aeb4c582 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json @@ -7,7 +7,7 @@ "queueMaxSize": "2000", "groupMaxSize": "50", "slidingWindowSize": "200", - "idPath": ".id", + "idPath": "$.id", "rootBuilder": [ "organization", "projectOrganization_participation_isParticipant", @@ -84,7 +84,7 @@ "type": "String", "weight": "0", "ignoreMissing": "false", - "path": ".country.classid" + "path": "$.country.classid" }, { "name": "legalshortname", @@ -92,7 +92,7 @@ "type": "String", "weight": "0.1", "ignoreMissing": "true", - "path": ".legalshortname.value" + "path": "$.legalshortname.value" }, { "name": "legalname", @@ -100,7 +100,7 @@ "type": "String", "weight": "0.9", "ignoreMissing": "false", - "path": ".legalname.value", + "path": "$.legalname.value", "params": { "windowSize": 4, "threshold": 0.7 @@ -112,11 +112,19 @@ "type": "URL", "weight": "0", "ignoreMissing": "true", - "path": ".websiteurl.value", + "path": "$.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" } ], "blacklists": { diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json deleted file mode 100644 index 2aeb4c582..000000000 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf2.json +++ /dev/null @@ -1,1753 +0,0 @@ -{ - "wf": { - "threshold": "0.9", - "dedupRun": "001", - "entityType": "organization", - "orderField": "legalname", - "queueMaxSize": "2000", - "groupMaxSize": "50", - "slidingWindowSize": "200", - "idPath": "$.id", - "rootBuilder": [ - "organization", - "projectOrganization_participation_isParticipant", - "datasourceOrganization_provision_isProvidedBy" - ], - "includeChildren": "true" - }, - "pace": { - "clustering": [ - { - "name": "sortedngrampairs", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "ngramLen": "3" - } - }, - { - "name": "suffixprefix", - "fields": [ - "legalname" - ], - "params": { - "max": 1, - "len": "3" - } - }, - { - "name": "urlclustering", - "fields": [ - "websiteurl" - ], - "params": {} - }, - { - "name": "keywordsclustering", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "windowSize": 4 - } - } - ], - "strictConditions": [ - { - "name": "exactMatch", - "fields": [ - "gridid" - ] - } - ], - "conditions": [ - { - "name": "DomainExactMatch", - "fields": [ - "websiteurl" - ] - }, - { - "name": "exactMatch", - "fields": [ - "country" - ] - } - ], - "model": [ - { - "name": "country", - "algo": "Null", - "type": "String", - "weight": "0", - "ignoreMissing": "false", - "path": "$.country.classid" - }, - { - "name": "legalshortname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.1", - "ignoreMissing": "true", - "path": "$.legalshortname.value" - }, - { - "name": "legalname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.9", - "ignoreMissing": "false", - "path": "$.legalname.value", - "params": { - "windowSize": 4, - "threshold": 0.7 - } - }, - { - "name": "websiteurl", - "algo": "Null", - "type": "URL", - "weight": "0", - "ignoreMissing": "true", - "path": "$.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" - } - ], - "blacklists": { - "legalname": [] - }, - "synonyms": { - "key::1": [ - "university", - "università", - "università studi", - "universitario", - "universitaria", - "université", - "universitaire", - "universitaires", - "universidad", - "universitade", - "Universität", - "universitaet", - "Uniwersytet", - "университет", - "universiteit", - "πανεπιστήμιο", - "universitesi", - "universiteti" - ], - "key::2": [ - "studies", - "studi", - "études", - "estudios", - "estudos", - "Studien", - "studia", - "исследования", - "studies", - "σπουδές" - ], - "key::3": [ - "advanced", - "superiore", - "supérieur", - "supérieure", - "supérieurs", - "supérieures", - "avancado", - "avancados", - "fortgeschrittene", - "fortgeschritten", - "zaawansowany", - "передовой", - "gevorderd", - "gevorderde", - "προχωρημένος", - "προχωρημένη", - "προχωρημένο", - "προχωρημένες", - "προχωρημένα", - "wyzsza" - ], - "key::4": [ - "institute", - "istituto", - "institut", - "instituto", - "instituto", - "Institut", - "instytut", - "институт", - "instituut", - "ινστιτούτο" - ], - "key::5": [ - "hospital", - "ospedale", - "hôpital", - "hospital", - "hospital", - "Krankenhaus", - "szpital", - "больница", - "ziekenhuis", - "νοσοκομείο" - ], - "key::6": [ - "research", - "ricerca", - "recherche", - "investigacion", - "pesquisa", - "Forschung", - "badania", - "исследования", - "onderzoek", - "έρευνα", - "erevna", - "erevnas" - ], - "key::7": [ - "college", - "collegio", - "université", - "colegio", - "faculdade", - "Hochschule", - "Szkoła Wyższa", - "Высшая школа", - "universiteit", - "κολλέγιο" - ], - "key::8": [ - "foundation", - "fondazione", - "fondation", - "fundación", - "fundação", - "Stiftung", - "Fundacja", - "фонд", - "stichting", - "ίδρυμα", - "idryma" - ], - "key::9": [ - "center", - "centro", - "centre", - "centro", - "centro", - "zentrum", - "centrum", - "центр", - "centrum", - "κέντρο" - ], - "key::10": [ - "national", - "nazionale", - "national", - "nationale", - "nationaux", - "nationales", - "nacional", - "nacional", - "national", - "krajowy", - "национальный", - "nationaal", - "nationale", - "εθνικό" - ], - "key::11": [ - "association", - "associazione", - "association", - "asociación", - "associação", - "Verein", - "verband", - "stowarzyszenie", - "ассоциация", - "associatie" - ], - "key::12": [ - "society", - "societa", - "société", - "sociedad", - "sociedade", - "gesellschaft", - "społeczeństwo", - "общество", - "maatschappij", - "κοινωνία" - ], - "key::13": [ - "international", - "internazionale", - "international", - "internacional", - "internacional", - "international", - "międzynarodowy", - "Международный", - "internationaal", - "internationale", - "διεθνής", - "διεθνή", - "διεθνές" - ], - "key::14": [ - "community", - "comunita", - "communauté", - "comunidad", - "comunidade", - "Gemeinschaft", - "społeczność", - "сообщество", - "gemeenschap", - "κοινότητα" - ], - "key::15": [ - "school", - "scuola", - "école", - "escuela", - "escola", - "schule", - "Szkoła", - "школа", - "school", - "σχολείο" - ], - "key::16": [ - "education", - "educazione", - "éducation", - "educacion", - "Educação", - "Bildung", - "Edukacja", - "образование", - "opleiding", - "εκπαίδευση" - ], - "key::17": [ - "academy", - "accademia", - "académie", - "academia", - "academia", - "Akademie", - "akademie", - "академия", - "academie", - "ακαδημία" - ], - "key::18": [ - "public", - "pubblico", - "public", - "publique", - "publics", - "publiques", - "publico", - "publico", - "Öffentlichkeit", - "publiczny", - "публичный", - "publiek", - "publieke", - "δημόσιος", - "δημόσια", - "δημόσιο" - ], - "key::19": [ - "museum", - "museo", - "musée", - "mueso", - "museu", - "museum", - "muzeum", - "музей", - "museum", - "μουσείο" - ], - "key::20": [ - "group", - "gruppo", - "groupe", - "grupo", - "grupo", - "gruppe", - "grupa", - "группа", - "groep", - "ομάδα", - "όμιλος" - ], - "key::21": [ - "department", - "dipartimento", - "département", - "departamento", - "departamento", - "abteilung", - "departament", - "отдел", - "afdeling", - "τμήμα" - ], - "key::22": [ - "council", - "consiglio", - "conseil", - "Consejo", - "conselho", - "gesellschaft", - "rada", - "совет", - "raad", - "συμβούλιο" - ], - "key::23": [ - "library", - "biblioteca", - "bibliothèque", - "biblioteca", - "biblioteca", - "Bibliothek", - "biblioteka", - "библиотека", - "bibliotheek", - "βιβλιοθήκη" - ], - "key::24": [ - "ministry", - "ministero", - "ministère", - "ministerio", - "ministério", - "Ministerium", - "ministerstwo", - "министерство", - "ministerie", - "υπουργείο" - ], - "key::25": [ - "services", - "servizi", - "services", - "servicios", - "Serviços", - "Dienstleistungen", - "usługi", - "услуги", - "diensten", - "υπηρεσίες" - ], - "key::26": [ - "central", - "centrale", - "central", - "centrale", - "centrales", - "central", - "central", - "zentral", - "centralny", - "цетральный", - "centraal", - "κεντρικός", - "κεντρική", - "κεντρικό", - "κεντρικά" - ], - "key::27": [ - "general", - "generale", - "général", - "générale", - "généraux", - "générales", - "general", - "geral", - "general", - "Allgemeines", - "general", - "общий", - "algemeen", - "algemene", - "γενικός", - "γενική", - "γενικό", - "γενικά" - ], - "key::28": [ - "applied", - "applicati", - "appliqué", - "appliquée", - "appliqués", - "appliquées", - "aplicado", - "aplicada", - "angewendet", - "stosowany", - "прикладной", - "toegepast", - "toegepaste", - "εφαρμοσμένος", - "εφαρμοσμένη", - "εφαρμοσμένο", - "εφαρμοσμένα" - ], - "key::29": [ - "european", - "europee", - "europea", - "européen", - "européenne", - "européens", - "européennes", - "europeo", - "europeu", - "europäisch", - "europejski", - "европейский", - "Europees", - "Europese", - "ευρωπαϊκός", - "ευρωπαϊκή", - "ευρωπαϊκό", - "ευρωπαϊκά" - ], - "key::30": [ - "agency", - "agenzia", - "agence", - "agencia", - "agencia", - "agentur", - "agencja", - "агенция", - "agentschap", - "πρακτορείο" - ], - "key::31": [ - "laboratory", - "laboratorio", - "laboratoire", - "laboratorio", - "laboratorio", - "labor", - "laboratorium", - "лаборатория", - "laboratorium", - "εργαστήριο" - ], - "key::32": [ - "industry", - "industria", - "industrie", - "индустрия", - "industrie", - "βιομηχανία" - ], - "key::33": [ - "industrial", - "industriale", - "industriel", - "industrielle", - "industriels", - "industrielles", - "индустриальный", - "industrieel", - "βιομηχανικός", - "βιομηχανική", - "βιομηχανικό", - "βιομηχανικά", - "βιομηχανικές" - ], - "key::34": [ - "consortium", - "consorzio", - "consortium", - "консорциум", - "consortium", - "κοινοπραξία" - ], - "key::35": [ - "organization", - "organizzazione", - "organisation", - "organización", - "organização", - "organizacja", - "организация", - "organisatie", - "οργανισμός" - ], - "key::36": [ - "authority", - "autorità", - "autorité", - "авторитет", - "autoriteit" - ], - "key::37": [ - "federation", - "federazione", - "fédération", - "федерация", - "federatie", - "ομοσπονδία" - ], - "key::38": [ - "observatory", - "osservatorio", - "observatoire", - "обсерватория", - "observatorium", - "αστεροσκοπείο" - ], - "key::39": [ - "bureau", - "ufficio", - "bureau", - "офис", - "bureau", - "γραφείο" - ], - "key::40": [ - "company", - "impresa", - "compagnie", - "société", - "компания", - "bedrijf", - "εταιρία" - ], - "key::41": [ - "polytechnic", - "politecnico", - "polytechnique", - "политехника", - "polytechnisch", - "πολυτεχνείο", - "universita politecnica", - "polytechnic university", - "universidad politecnica", - "universitat politecnica", - "politechnika", - "politechniki", - "university technology", - "university science technology" - ], - "key::42": [ - "coalition", - "coalizione", - "coalition", - "коалиция", - "coalitie", - "συνασπισμός" - ], - "key::43": [ - "initiative", - "iniziativa", - "initiative", - "инициатива", - "initiatief", - "πρωτοβουλία" - ], - "key::44": [ - "academic", - "accademico", - "académique", - "universitaire", - "акадеческий academisch", - "ακαδημαϊκός", - "ακαδημαϊκή", - "ακαδημαϊκό", - "ακαδημαϊκές", - "ακαδημαϊκοί" - ], - "key::45": [ - "institution", - "istituzione", - "institution", - "институциональный", - "instelling", - "ινστιτούτο" - ], - "key::46": [ - "division", - "divisione", - "division", - "отделение", - "divisie", - "τμήμα" - ], - "key::47": [ - "committee", - "comitato", - "comité", - "комитет", - "commissie", - "επιτροπή" - ], - "key::48": [ - "promotion", - "promozione", - "продвижение", - "proothisis", - "forderung" - ], - "key::49": [ - "medical", - "medicine", - "clinical", - "medicina", - "clinici", - "médico", - "medicina", - "clínica", - "médico", - "medicina", - "clínica", - "medizinisch", - "Medizin", - "klinisch", - "medisch", - "geneeskunde", - "klinisch", - "ιατρικός", - "ιατρική", - "ιατρικό", - "ιατρικά", - "κλινικός", - "κλινική", - "κλινικό", - "κλινικά", - "tıbbi", - "tıp", - "klinik", - "orvosi", - "orvostudomány", - "klinikai", - "zdravniški", - "medicinski", - "klinični", - "meditsiini", - "kliinik", - "kliiniline" - ], - "key::50": [ - "technology", - "technological", - "tecnologia", - "tecnologie", - "tecnología", - "tecnológico", - "tecnologia", - "tecnológico", - "Technologie", - "technologisch", - "technologie", - "technologisch", - "τεχνολογία", - "τεχνολογικός", - "τεχνολογική", - "τεχνολογικό", - "teknoloji", - "teknolojik", - "technológia", - "technológiai", - "tehnologija", - "tehnološki", - "tehnoloogia", - "tehnoloogiline", - "technologii", - "technical", - "texniki", - "teknik" - ], - "key::51": [ - "science", - "scientific", - "scienza", - "scientifiche", - "scienze", - "ciencia", - "científico", - "ciência", - "científico", - "Wissenschaft", - "wissenschaftlich", - "wetenschap", - "wetenschappelijk", - "επιστήμη", - "επιστημονικός", - "επιστημονική", - "επιστημονικό", - "επιστημονικά", - "bilim", - "bilimsel", - "tudomány", - "tudományos", - "znanost", - "znanstveni", - "teadus", - "teaduslik", - "" - ], - "key::52": [ - "engineering", - "ingegneria", - "ingeniería", - "engenharia", - "Ingenieurwissenschaft", - "ingenieurswetenschappen", - "bouwkunde", - "μηχανικός", - "μηχανική", - "μηχανικό", - "mühendislik", - "mérnöki", - "Inženirstvo", - "inseneeria", - "inseneri", - "" - ], - "key::53": [ - "management", - "gestione", - "gestionale", - "gestionali", - "gestión", - "administración", - "gestão", - "administração", - "Verwaltung", - "management", - "διαχείριση", - "yönetim", - "menedzsment", - "vodstvo", - "upravljanje", - "management", - "juhtkond", - "juhtimine", - "haldus", - "" - ], - "key::54": [ - "energy", - "energia", - "energía", - "energia", - "Energie", - "energie", - "ενέργεια", - "enerji", - "energia", - "energija", - "energia", - "" - ], - "key::55": [ - "agricultural", - "agriculture", - "agricoltura", - "agricole", - "agrícola", - "agricultura", - "agrícola", - "agricultura", - "landwirtschaftlich", - "Landwirtschaft", - "landbouwkundig", - "landbouw", - "αγροτικός", - "αγροτική", - "αγροτικό", - "γεωργικός", - "γεωργική", - "γεωργικό", - "γεωργία", - "tarımsal", - "tarım", - "mezőgazdasági", - "mezőgazdaság", - "poljedelski", - "poljedelstvo", - "põllumajandus", - "põllumajanduslik", - "" - ], - "key::56": [ - "information", - "informazione", - "información", - "informação", - "Information", - "informatie", - "πληροφορία", - "bilgi", - "információ", - "informacija", - "informatsioon", - "informatycznych", - "" - ], - "key::57": [ - "social", - "sociali", - "social", - "social", - "Sozial", - "sociaal", - "maatschappelijk", - "κοινωνικός", - "κοινωνική", - "κοινωνικό", - "κοινωνικά", - "sosyal", - "szociális", - "družbeni", - "sotsiaal", - "sotsiaalne", - "" - ], - "key::58": [ - "environmental", - "ambiente", - "medioambiental", - "ambiente", - "medioambiente", - "meioambiente", - "Umwelt", - "milieu", - "milieuwetenschap", - "milieukunde", - "περιβαλλοντικός", - "περιβαλλοντική", - "περιβαλλοντικό", - "περιβαλλοντικά", - "çevre", - "környezeti", - "okoliški", - "keskonna", - "" - ], - "key::59": [ - "business", - "economia", - "economiche", - "economica", - "negocio", - "empresa", - "negócio", - "Unternehmen", - "bedrijf", - "bedrijfskunde", - "επιχείρηση", - "iş", - "üzleti", - "posel", - "ettevõte/äri", - "" - ], - "key::60": [ - "pharmaceuticals", - "pharmacy", - "farmacia", - "farmaceutica", - "farmacéutica", - "farmacia", - "farmacêutica", - "farmácia", - "Pharmazeutika", - "Arzneimittelkunde", - "farmaceutica", - "geneesmiddelen", - "apotheek", - "φαρμακευτικός", - "φαρμακευτική", - "φαρμακευτικό", - "φαρμακευτικά", - "φαρμακείο", - "ilaç", - "eczane", - "gyógyszerészeti", - "gyógyszertár", - "farmacevtika", - "lekarništvo", - "farmaatsia", - "farmatseutiline", - "" - ], - "key::61": [ - "healthcare", - "health services", - "salute", - "atenciónmédica", - "cuidadodelasalud", - "cuidadoscomasaúde", - "Gesundheitswesen", - "gezondheidszorg", - "ιατροφαρμακευτικήπερίθαλψη", - "sağlıkhizmeti", - "egészségügy", - "zdravstvo", - "tervishoid", - "tervishoiu", - "" - ], - "key::62": [ - "history", - "storia", - "historia", - "história", - "Geschichte", - "geschiedenis", - "geschiedkunde", - "ιστορία", - "tarih", - "történelem", - "zgodovina", - "ajalugu", - "" - ], - "key::63": [ - "materials", - "materiali", - "materia", - "materiales", - "materiais", - "materialen", - "υλικά", - "τεκμήρια", - "malzemeler", - "anyagok", - "materiali", - "materjalid", - "vahendid", - "" - ], - "key::64": [ - "economics", - "economia", - "economiche", - "economica", - "economía", - "economia", - "Wirtschaft", - "economie", - "οικονομικά", - "οικονομικέςεπιστήμες", - "ekonomi", - "közgazdaságtan", - "gospodarstvo", - "ekonomija", - "majanduslik", - "majandus", - "" - ], - "key::65": [ - "therapeutics", - "terapeutica", - "terapéutica", - "terapêutica", - "therapie", - "θεραπευτική", - "tedavibilimi", - "gyógykezelés", - "terapevtika", - "terapeutiline", - "ravi", - "" - ], - "key::66": [ - "oncology", - "oncologia", - "oncologico", - "oncología", - "oncologia", - "Onkologie", - "oncologie", - "ογκολογία", - "onkoloji", - "onkológia", - "onkologija", - "onkoloogia", - "" - ], - "key::67": [ - "natural", - "naturali", - "naturale", - "natural", - "natural", - "natürlich", - "natuurlijk", - "φυσικός", - "φυσική", - "φυσικό", - "φυσικά", - "doğal", - "természetes", - "naraven", - "loodus", - "" - ], - "key::68": [ - "educational", - "educazione", - "pedagogia", - "educacional", - "educativo", - "educacional", - "pädagogisch", - "educatief", - "εκπαιδευτικός", - "εκπαιδευτική", - "εκπαιδευτικό", - "εκπαιδευτικά", - "eğitimsel", - "oktatási", - "izobraževalen", - "haridus", - "hariduslik", - "" - ], - "key::69": [ - "biomedical", - "biomedica", - "biomédico", - "biomédico", - "biomedizinisch", - "biomedisch", - "βιοιατρικός", - "βιοιατρική", - "βιοιατρικό", - "βιοιατρικά", - "biyomedikal", - "orvosbiológiai", - "biomedicinski", - "biomeditsiiniline", - "" - ], - "key::70": [ - "veterinary", - "veterinaria", - "veterinarie", - "veterinaria", - "veterinária", - "tierärtzlich", - "veterinair", - "veeartsenijlkunde", - "κτηνιατρικός", - "κτηνιατρική", - "κτηνιατρικό", - "κτηνιατρικά", - "veteriner", - "állatorvosi", - "veterinar", - "veterinarski", - "veterinaaria", - "" - ], - "key::71": [ - "chemistry", - "chimica", - "química", - "química", - "Chemie", - "chemie", - "scheikunde", - "χημεία", - "kimya", - "kémia", - "kemija", - "keemia", - "" - ], - "key::72": [ - "security", - "sicurezza", - "seguridad", - "segurança", - "Sicherheit", - "veiligheid", - "ασφάλεια", - "güvenlik", - "biztonsági", - "varnost", - "turvalisus", - "julgeolek", - "" - ], - "key::73": [ - "biotechnology", - "biotecnologia", - "biotecnologie", - "biotecnología", - "biotecnologia", - "Biotechnologie", - "biotechnologie", - "βιοτεχνολογία", - "biyoteknoloji", - "biotechnológia", - "biotehnologija", - "biotehnoloogia", - "" - ], - "key::74": [ - "military", - "militare", - "militari", - "militar", - "militar", - "Militär", - "militair", - "leger", - "στρατιωτικός", - "στρατιωτική", - "στρατιωτικό", - "στρατιωτικά", - "askeri", - "katonai", - "vojaški", - "vojni", - "militaar", - "wojskowa", - "" - ], - "key::75": [ - "theological", - "teologia", - "teologico", - "teológico", - "tecnológica", - "theologisch", - "theologisch", - "θεολογικός", - "θεολογική", - "θεολογικό", - "θεολογικά", - "teolojik", - "technológiai", - "teološki", - "teoloogia", - "usuteadus", - "teoloogiline", - "" - ], - "key::76": [ - "electronics", - "elettronica", - "electrónica", - "eletrônicos", - "Elektronik", - "elektronica", - "ηλεκτρονική", - "elektronik", - "elektronika", - "elektronika", - "elektroonika", - "" - ], - "key::77": [ - "forestry", - "forestale", - "forestali", - "silvicultura", - "forestal", - "floresta", - "Forstwirtschaft", - "bosbouw", - "δασοκομία", - "δασολογία", - "ormancılık", - "erdészet", - "gozdarstvo", - "metsandus", - "" - ], - "key::78": [ - "maritime", - "marittima", - "marittime", - "marittimo", - "marítimo", - "marítimo", - "maritiem", - "ναυτικός", - "ναυτική", - "ναυτικό", - "ναυτικά", - "ναυτιλιακός", - "ναυτιλιακή", - "ναυτιλιακό", - "ναυτιλιακά", - "θαλάσσιος", - "θαλάσσια", - "θαλάσσιο", - "denizcilik", - "tengeri", - "morski", - "mere", - "merendus", - "" - ], - "key::79": [ - "sports", - "sport", - "deportes", - "esportes", - "Sport", - "sport", - "sportwetenschappen", - "άθληση", - "γυμναστικήδραστηριότητα", - "spor", - "sport", - "šport", - "sport", - "spordi", - "" - ], - "key::80": [ - "surgery", - "chirurgia", - "chirurgiche", - "cirugía", - "cirurgia", - "Chirurgie", - "chirurgie", - "heelkunde", - "εγχείρηση", - "επέμβαση", - "χειρουργικήεπέμβαση", - "cerrahi", - "sebészet", - "kirurgija", - "kirurgia", - "" - ], - "key::81": [ - "cultural", - "culturale", - "culturali", - "cultura", - "cultural", - "cultural", - "kulturell", - "cultureel", - "πολιτιστικός", - "πολιτιστική", - "πολιτιστικό", - "πολιτισμικός", - "πολιτισμική", - "πολιτισμικό", - "kültürel", - "kultúrális", - "kulturni", - "kultuuri", - "kultuuriline", - "" - ], - "key::82": [ - "computerscience", - "informatica", - "ordenador", - "computadora", - "informática", - "computación", - "cienciasdelacomputación", - "ciênciadacomputação", - "Computer", - "computer", - "υπολογιστής", - "ηλεκτρονικόςυπολογιστής", - "bilgisayar", - "számítógép", - "računalnik", - "arvuti", - "" - ], - "key::83": [ - "finance", - "financial", - "finanza", - "finanziarie", - "finanza", - "financiero", - "finanças", - "financeiro", - "Finanzen", - "finanziell", - "financiën", - "financieel", - "χρηματοοικονομικά", - "χρηματοδότηση", - "finanse", - "finansal", - "pénzügy", - "pénzügyi", - "finance", - "finančni", - "finants", - "finantsiline", - "" - ], - "key::84": [ - "communication", - "comunicazione", - "comuniciación", - "comunicação", - "Kommunikation", - "communication", - "επικοινωνία", - "iletişim", - "kommunikáció", - "komuniciranje", - "kommunikatsioon", - "" - ], - "key::85": [ - "justice", - "giustizia", - "justicia", - "justiça", - "Recht", - "Justiz", - "justitie", - "gerechtigheid", - "δικαιοσύνη", - "υπουργείοδικαιοσύνης", - "δίκαιο", - "adalet", - "igazságügy", - "pravo", - "õigus", - "" - ], - "key::86": [ - "aerospace", - "aerospaziale", - "aerospaziali", - "aeroespacio", - "aeroespaço", - "Luftfahrt", - "luchtvaart", - "ruimtevaart", - "αεροπορικός", - "αεροπορική", - "αεροπορικό", - "αεροναυπηγικός", - "αεροναυπηγική", - "αεροναυπηγικό", - "αεροναυπηγικά", - "havacılıkveuzay", - "légtér", - "zrakoplovstvo", - "atmosfäär", - "kosmos", - "" - ], - "key::87": [ - "dermatology", - "dermatologia", - "dermatología", - "dermatologia", - "Dermatologie", - "dermatologie", - "δρματολογία", - "dermatoloji", - "bőrgyógyászat", - "dermatológia", - "dermatologija", - "dermatoloogia", - "" - ], - "key::88": [ - "architecture", - "architettura", - "arquitectura", - "arquitetura", - "Architektur", - "architectuur", - "αρχιτεκτονική", - "mimarlık", - "építészet", - "arhitektura", - "arhitektuur", - "" - ], - "key::89": [ - "mathematics", - "matematica", - "matematiche", - "matemáticas", - "matemáticas", - "Mathematik", - "wiskunde", - "mathematica", - "μαθηματικά", - "matematik", - "matematika", - "matematika", - "matemaatika", - "" - ], - "key::90": [ - "language", - "lingue", - "linguistica", - "linguistiche", - "lenguaje", - "idioma", - "língua", - "idioma", - "Sprache", - "taal", - "taalkunde", - "γλώσσα", - "dil", - "nyelv", - "jezik", - "keel", - "" - ], - "key::91": [ - "neuroscience", - "neuroscienza", - "neurociencia", - "neurociência", - "Neurowissenschaft", - "neurowetenschappen", - "νευροεπιστήμη", - "nörobilim", - "idegtudomány", - "nevroznanost", - "neuroteadused", - "" - ], - "key::92": [ - "automation", - "automazione", - "automatización", - "automação", - "Automatisierung", - "automatisering", - "αυτοματοποίηση", - "otomasyon", - "automatizálás", - "avtomatizacija", - "automatiseeritud", - "" - ], - "key::93": [ - "pediatric", - "pediatria", - "pediatriche", - "pediatrico", - "pediátrico", - "pediatría", - "pediátrico", - "pediatria", - "pädiatrisch", - "pediatrische", - "παιδιατρική", - "pediatrik", - "gyermekgyógyászat", - "pediatrija", - "pediaatria", - "" - ], - "key::94": [ - "photonics", - "fotonica", - "fotoniche", - "fotónica", - "fotônica", - "Photonik", - "fotonica", - "φωτονική", - "fotonik", - "fotonika", - "fotonika", - "fotoonika", - "" - ], - "key::95": [ - "mechanics", - "meccanica", - "meccaniche", - "mecánica", - "mecânica", - "Mechanik", - "Maschinenbau", - "mechanica", - "werktuigkunde", - "μηχανικής", - "mekanik", - "gépészet", - "mehanika", - "mehaanika", - "" - ], - "key::96": [ - "psychiatrics", - "psichiatria", - "psichiatrica", - "psichiatriche", - "psiquiatría", - "psiquiatria", - "Psychiatrie", - "psychiatrie", - "ψυχιατρική", - "psikiyatrik", - "pszihiátria", - "psihiatrija", - "psühhaatria", - "" - ], - "key::97": [ - "psychology", - "fisiologia", - "psicología", - "psicologia", - "Psychologie", - "psychologie", - "ψυχολογία", - "psikoloji", - "pszihológia", - "psihologija", - "psühholoogia", - "" - ], - "key::98": [ - "automotive", - "industriaautomobilistica", - "industriadelautomóvil", - "automotriz", - "industriaautomotriz", - "automotivo", - "Automobilindustrie", - "autoindustrie", - "αυτοκίνητος", - "αυτοκίνητη", - "αυτοκίνητο", - "αυτοκινούμενος", - "αυτοκινούμενη", - "αυτοκινούμενο", - "αυτοκινητιστικός", - "αυτοκινητιστική", - "αυτοκινητιστικό", - "otomotiv", - "autóipari", - "samogiben", - "avtomobilskaindustrija", - "auto-", - "" - ], - "key::99": [ - "neurology", - "neurologia", - "neurologiche", - "neurología", - "neurologia", - "Neurologie", - "neurologie", - "zenuwleer", - "νευρολογία", - "nöroloji", - "neurológia", - "ideggyógyászat", - "nevrologija", - "neuroloogia", - "" - ], - "key::100": [ - "geology", - "geologia", - "geologiche", - "geología", - "geologia", - "Geologie", - "geologie", - "aardkunde", - "γεωλογία", - "jeoloji", - "geológia", - "földtudomány", - "geologija", - "geoloogia", - "" - ], - "key::101": [ - "microbiology", - "microbiologia", - "micro-biologia", - "microbiologiche", - "microbiología", - "microbiologia", - "Mikrobiologie", - "microbiologie", - "μικροβιολογία", - "mikrobiyoloji", - "mikrobiológia", - "mikrobiologija", - "mikrobioloogia", - "" - ], - "key::102": [ - "informatics", - "informatica", - "informática", - "informática", - "informatica", - "" - ], - "key::103": [ - "forschungsgemeinschaft", - "comunita ricerca", - "research community", - "research foundation", - "research association" - ], - "key::104": [ - "commerce", - "ticaret", - "ticarət", - "commercio", - "trade", - "handel", - "comercio" - ] - } - } -} \ No newline at end of file From 39367676d7c1f2ac3fc39d72e643a45104872b45 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Thu, 12 Dec 2019 15:18:48 +0100 Subject: [PATCH 05/13] implemented DedupRecord factory with the merge of project --- .../eu/dnetlib/dhp/schema/oaf/Dataset.java | 24 +++- .../dnetlib/dhp/schema/oaf/GeoLocation.java | 25 ++++ .../java/eu/dnetlib/dhp/schema/oaf/Oaf.java | 20 +++ .../eu/dnetlib/dhp/schema/oaf/OafEntity.java | 10 +- .../dnetlib/dhp/schema/oaf/Organization.java | 31 +++-- .../eu/dnetlib/dhp/schema/oaf/Project.java | 35 +++++ .../dnetlib/dhp/schema/oaf/Publication.java | 3 +- .../eu/dnetlib/dhp/schema/oaf/Result.java | 16 ++- .../eu/dnetlib/dedup/DedupRecordFactory.java | 128 ++++++++++-------- .../java/eu/dnetlib/dedup/DedupUtility.java | 83 +++--------- .../java/eu/dnetlib/dedup/OafComparator.java | 15 -- .../main/java/eu/dnetlib/dedup/OafKey.java | 31 ----- .../java/eu/dnetlib/dedup/OafPartitioner.java | 59 -------- .../dnetlib/dhp/dedup/oozie_app/workflow.xml | 2 +- 14 files changed, 224 insertions(+), 258 deletions(-) delete mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java delete mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java delete mode 100644 dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java index 44d5226e9..27bee998e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java @@ -19,7 +19,7 @@ public class Dataset extends Result implements Serializable { private List geolocation; - public Field getStoragedate() { + public Field getStoragedate() { return storagedate; } @@ -74,4 +74,26 @@ public class Dataset extends Result implements Serializable { public void setGeolocation(List geolocation) { this.geolocation = geolocation; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + final Dataset d = (Dataset) e; + + storagedate = d.getStoragedate() != null && compareTrust(this, e)<0? d.getStoragedate() : storagedate; + + device= d.getDevice() != null && compareTrust(this, e)<0? d.getDevice() : device; + + size= d.getSize() != null && compareTrust(this, e)<0? d.getSize() : size; + + version= d.getVersion() != null && compareTrust(this, e)<0? d.getVersion() : version; + + lastmetadataupdate= d.getLastmetadataupdate() != null && compareTrust(this, e)<0? d.getLastmetadataupdate() :lastmetadataupdate; + + metadataversionnumber= d.getMetadataversionnumber() != null && compareTrust(this, e)<0? d.getMetadataversionnumber() : metadataversionnumber; + + geolocation = mergeLists(geolocation, d.getGeolocation()); + + mergeOAFDataInfo(d); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java index 567254a23..a0ce32353 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java @@ -33,4 +33,29 @@ public class GeoLocation implements Serializable { public void setPlace(String place) { this.place = place; } + + + public String toComparableString() { + return String.format("%s::%s%s", point != null ? point.toLowerCase() : "", box != null ? box.toLowerCase() : "",place != null ? place.toLowerCase() : ""); + } + + @Override + public int hashCode() { + return toComparableString().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + + GeoLocation other = (GeoLocation) obj; + + return toComparableString() + .equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java index 352ebbc6e..cc2ab8428 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java @@ -23,4 +23,24 @@ public abstract class Oaf implements Serializable { public void setLastupdatetimestamp(Long lastupdatetimestamp) { this.lastupdatetimestamp = lastupdatetimestamp; } + + + public void mergeOAFDataInfo(Oaf e) { + if (e.getDataInfo()!= null && compareTrust(this,e)<0) + dataInfo = e.getDataInfo(); + } + + protected String extractTrust(Oaf e) { + if (e == null || e.getDataInfo()== null || e.getDataInfo().getTrust()== null) + return "0.0"; + return e.getDataInfo().getTrust(); + + + + } + + protected int compareTrust(Oaf a, Oaf b) { + return extractTrust(a).compareTo(extractTrust(b)); + + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java index 955ddfd01..8a86f822d 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java @@ -86,6 +86,7 @@ public abstract class OafEntity extends Oaf implements Serializable { this.oaiprovenance = oaiprovenance; } + public void mergeFrom(OafEntity e) { if (e == null) @@ -97,13 +98,16 @@ public abstract class OafEntity extends Oaf implements Serializable { pid = mergeLists(pid, e.getPid()); - dateofcollection = e.getDateofcollection(); + if (e.getDateofcollection() != null && compareTrust(this, e) < 0) + dateofcollection = e.getDateofcollection(); - dateoftransformation = e.getDateoftransformation(); + if (e.getDateoftransformation() != null && compareTrust(this, e) < 0) + dateoftransformation = e.getDateoftransformation(); extraInfo = mergeLists(extraInfo, e.getExtraInfo()); - oaiprovenance = e.getOaiprovenance(); + if (e.getOaiprovenance() != null && compareTrust(this, e) < 0) + oaiprovenance = e.getOaiprovenance(); } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java index c3e9a7007..b0dffb485 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java @@ -170,21 +170,22 @@ public class Organization extends OafEntity implements Serializable { public void mergeFrom(OafEntity e) { super.mergeFrom(e); final Organization o = (Organization) e; - legalshortname = o.getLegalshortname() != null ? o.getLegalshortname() : legalshortname; - legalname = o.getLegalname() != null ? o.getLegalname() : legalname; + legalshortname = o.getLegalshortname() != null && compareTrust(this, e)<0? o.getLegalshortname() : legalshortname; + legalname = o.getLegalname() != null && compareTrust(this, e)<0 ? o.getLegalname() : legalname; alternativeNames = mergeLists(o.getAlternativeNames(), alternativeNames); - websiteurl = o.getWebsiteurl() != null ? o.getWebsiteurl() : websiteurl; - logourl = o.getLogourl() != null ? o.getLogourl() : logourl; - eclegalbody = o.getEclegalbody() != null ? o.getEclegalbody() : eclegalbody; - eclegalperson = o.getEclegalperson() != null ? o.getEclegalperson() : eclegalperson; - ecnonprofit = o.getEcnonprofit() != null ? o.getEcnonprofit() : ecnonprofit; - ecresearchorganization = o.getEcresearchorganization() != null ? o.getEcresearchorganization() : ecresearchorganization; - echighereducation = o.getEchighereducation() != null ? o.getEchighereducation() : echighereducation; - ecinternationalorganizationeurinterests = o.getEcinternationalorganizationeurinterests() != null ? o.getEcinternationalorganizationeurinterests() : ecinternationalorganizationeurinterests; - ecinternationalorganization = o.getEcinternationalorganization() != null ? o.getEcinternationalorganization() : ecinternationalorganization; - ecenterprise = o.getEcenterprise() != null ? o.getEcenterprise() :ecenterprise; - ecsmevalidated = o.getEcsmevalidated() != null ? o.getEcsmevalidated() :ecsmevalidated; - ecnutscode = o.getEcnutscode() != null ? o.getEcnutscode() :ecnutscode; - country = o.getCountry() != null ? o.getCountry() :country; + websiteurl = o.getWebsiteurl() != null && compareTrust(this, e)<0? o.getWebsiteurl() : websiteurl; + logourl = o.getLogourl() != null && compareTrust(this, e)<0? o.getLogourl() : logourl; + eclegalbody = o.getEclegalbody() != null && compareTrust(this, e)<0? o.getEclegalbody() : eclegalbody; + eclegalperson = o.getEclegalperson() != null && compareTrust(this, e)<0? o.getEclegalperson() : eclegalperson; + ecnonprofit = o.getEcnonprofit() != null && compareTrust(this, e)<0? o.getEcnonprofit() : ecnonprofit; + ecresearchorganization = o.getEcresearchorganization() != null && compareTrust(this, e)<0? o.getEcresearchorganization() : ecresearchorganization; + echighereducation = o.getEchighereducation() != null && compareTrust(this, e)<0? o.getEchighereducation() : echighereducation; + ecinternationalorganizationeurinterests = o.getEcinternationalorganizationeurinterests() != null && compareTrust(this, e)<0? o.getEcinternationalorganizationeurinterests() : ecinternationalorganizationeurinterests; + ecinternationalorganization = o.getEcinternationalorganization() != null && compareTrust(this, e)<0? o.getEcinternationalorganization() : ecinternationalorganization; + ecenterprise = o.getEcenterprise() != null && compareTrust(this, e)<0? o.getEcenterprise() :ecenterprise; + ecsmevalidated = o.getEcsmevalidated() != null && compareTrust(this, e)<0? o.getEcsmevalidated() :ecsmevalidated; + ecnutscode = o.getEcnutscode() != null && compareTrust(this, e)<0? o.getEcnutscode() :ecnutscode; + country = o.getCountry() != null && compareTrust(this, e)<0 ? o.getCountry() :country; + mergeOAFDataInfo(o); } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java index 65f22da37..0bc11bb41 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java @@ -264,4 +264,39 @@ public class Project extends OafEntity implements Serializable { public void setFundedamount(Float fundedamount) { this.fundedamount = fundedamount; } + + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + Project p = (Project)e; + + websiteurl= p.getWebsiteurl()!= null && compareTrust(this,e)<0?p.getWebsiteurl():websiteurl; + code= p.getCode()!=null && compareTrust(this,e)<0?p.getCode():code; + acronym= p.getAcronym()!= null && compareTrust(this,e)<0?p.getAcronym():acronym; + title= p.getTitle()!= null && compareTrust(this,e)<0?p.getTitle():title; + startdate= p.getStartdate()!=null && compareTrust(this,e)<0?p.getStartdate():startdate; + enddate= p.getEnddate()!=null && compareTrust(this,e)<0?p.getEnddate():enddate; + callidentifier= p.getCallidentifier()!=null && compareTrust(this,e)<0?p.getCallidentifier():callidentifier; + keywords= p.getKeywords()!=null && compareTrust(this,e)<0?p.getKeywords():keywords; + duration= p.getDuration()!=null && compareTrust(this,e)<0?p.getDuration():duration; + ecsc39= p.getEcsc39()!=null && compareTrust(this,e)<0?p.getEcsc39():ecsc39; + oamandatepublications= p.getOamandatepublications()!=null && compareTrust(this,e)<0?p.getOamandatepublications():oamandatepublications; + ecarticle29_3= p.getEcarticle29_3()!=null && compareTrust(this,e)<0?p.getEcarticle29_3():ecarticle29_3; + subjects= mergeLists(subjects, p.getSubjects()); + fundingtree= mergeLists(fundingtree, p.getFundingtree()); + contracttype= p.getContracttype()!=null && compareTrust(this,e)<0?p.getContracttype():contracttype; + optional1= p.getOptional1()!=null && compareTrust(this,e)<0?p.getOptional1():optional1; + optional2= p.getOptional2()!=null && compareTrust(this,e)<0?p.getOptional2():optional2; + jsonextrainfo= p.getJsonextrainfo()!=null && compareTrust(this,e)<0?p.getJsonextrainfo():jsonextrainfo; + contactfullname= p.getContactfullname()!=null && compareTrust(this,e)<0?p.getContactfullname():contactfullname; + contactfax= p.getContactfax()!=null && compareTrust(this,e)<0?p.getContactfax():contactfax; + contactphone= p.getContactphone()!=null && compareTrust(this,e)<0?p.getContactphone():contactphone; + contactemail= p.getContactemail()!=null && compareTrust(this,e)<0?p.getContactemail():contactemail; + summary= p.getSummary()!=null && compareTrust(this,e)<0?p.getSummary():summary; + currency= p.getCurrency()!=null && compareTrust(this,e)<0?p.getCurrency():currency; + totalcost= p.getTotalcost()!=null && compareTrust(this,e)<0?p.getTotalcost():totalcost; + fundedamount= p.getFundedamount()!= null && compareTrust(this,e)<0?p.getFundedamount():fundedamount; + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java index 181062f32..bb6990c1d 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java @@ -21,8 +21,9 @@ public class Publication extends Result implements Serializable { Publication p = (Publication) e; - if (p.getJournal() != null) + if (p.getJournal() != null && compareTrust(this, e)<0) journal = p.getJournal(); + mergeOAFDataInfo(e); } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 7fb7aef10..10339178d 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -252,10 +252,10 @@ public abstract class Result extends OafEntity implements Serializable { instance = mergeLists(instance, r.getInstance()); - if (r.getResulttype() != null) + if (r.getResulttype() != null && compareTrust(this, r)<0) resulttype = r.getResulttype(); - if (r.getLanguage() != null) + if (r.getLanguage() != null && compareTrust(this, r)<0) language = r.getLanguage(); country = mergeLists(country, r.getCountry()); @@ -268,10 +268,10 @@ public abstract class Result extends OafEntity implements Serializable { description = longestLists(description, r.getDescription()); - if (r.getPublisher() != null) + if (r.getPublisher() != null && compareTrust(this, r)<0) publisher = r.getPublisher(); - if (r.getEmbargoenddate() != null) + if (r.getEmbargoenddate() != null && compareTrust(this, r)<0) embargoenddate = r.getEmbargoenddate(); source = mergeLists(source, r.getSource()); @@ -287,21 +287,23 @@ public abstract class Result extends OafEntity implements Serializable { coverage = mergeLists(coverage, r.getCoverage()); - if (r.getRefereed() != null) + if (r.getRefereed() != null && compareTrust(this, r)<0) refereed = r.getRefereed(); context = mergeLists(context, r.getContext()); - if (r.getProcessingchargeamount() != null) + if (r.getProcessingchargeamount() != null && compareTrust(this, r)<0) processingchargeamount = r.getProcessingchargeamount(); - if (r.getProcessingchargecurrency() != null) + if (r.getProcessingchargecurrency() != null && compareTrust(this, r)<0) processingchargecurrency = r.getProcessingchargecurrency(); externalReference = mergeLists(externalReference, r.getExternalReference()); } + + private List> longestLists(List> a, List> b) { if(a == null || b == null) return a==null?b:a; diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index c6e3efbc4..86c1651a2 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -24,6 +24,8 @@ public class DedupRecordFactory { public static JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf) { + + long ts = System.currentTimeMillis(); // final JavaPairRDD inputJsonEntities = sc.textFile(entitiesInputPath) .mapToPair((PairFunction) it -> @@ -43,57 +45,31 @@ public class DedupRecordFactory { // final JavaPairRDD joinResult = mergeRels.join(inputJsonEntities).mapToPair((PairFunction>, String, String>) Tuple2::_2); - JavaPairRDD keyJson = joinResult.mapToPair((PairFunction, OafKey, String>) json -> { - - String idValue = json._1(); - - String trust = ""; - try { - trust = MapDocumentUtil.getJPathString("$.dataInfo.trust", json._2()); - } catch (Throwable e) { - - } - - //TODO remember to replace this with the actual trust retrieving - if (StringUtils.isBlank(trust)) { - Random generator = new Random(); - int number = generator.nextInt(20); - double result = (number / 100.0) + 0.80; - trust = "" + result; - } - - return new Tuple2(new OafKey(idValue, trust), json._2()); - }); - - OafComparator c = new OafComparator(); - // - JavaPairRDD> sortedJoinResult = keyJson.repartitionAndSortWithinPartitions(new OafPartitioner(keyJson.getNumPartitions()), c) - .mapToPair((PairFunction, String, String>) t -> new Tuple2(t._1().getDedupId(), t._2())) - .groupByKey(); - + JavaPairRDD> sortedJoinResult = joinResult.groupByKey(); switch (entityType) { case publication: - return sortedJoinResult.map(DedupRecordFactory::publicationMerger); + + return sortedJoinResult.map(p->DedupRecordFactory.publicationMerger(p, ts)); case dataset: - return sortedJoinResult.map(DedupRecordFactory::datasetMerger); + return sortedJoinResult.map(d->DedupRecordFactory.datasetMerger(d,ts)); case project: - return sortedJoinResult.map(DedupRecordFactory::projectMerger); + return sortedJoinResult.map(p->DedupRecordFactory.projectMerger(p,ts)); case software: - return sortedJoinResult.map(DedupRecordFactory::softwareMerger); + return sortedJoinResult.map(s->DedupRecordFactory.softwareMerger(s,ts)); case datasource: - return sortedJoinResult.map(DedupRecordFactory::datasourceMerger); + return sortedJoinResult.map(d->DedupRecordFactory.datasourceMerger(d,ts)); case organization: - return sortedJoinResult.map(DedupRecordFactory::organizationMerger); + return sortedJoinResult.map(o->DedupRecordFactory.organizationMerger(o,ts)); case otherresearchproduct: - return sortedJoinResult.map(DedupRecordFactory::otherresearchproductMerger); + return sortedJoinResult.map(o->DedupRecordFactory.otherresearchproductMerger(o,ts)); default: return null; } } - private static Publication publicationMerger(Tuple2> e) { + private static Publication publicationMerger(Tuple2> e, final long ts) { Publication p = new Publication(); //the result of the merge, to be returned at the end @@ -103,19 +79,11 @@ public class DedupRecordFactory { final Collection dateofacceptance = Lists.newArrayList(); - - StringBuilder trust = new StringBuilder("0.0"); - if (e._2() != null) e._2().forEach(pub -> { try { Publication publication = mapper.readValue(pub, Publication.class); - final String currentTrust = publication.getDataInfo().getTrust(); - if (!"1.0".equals(currentTrust)) { - trust.setLength(0); - trust.append(currentTrust); - } p.mergeFrom(publication); p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); //add to the list if they are not null @@ -126,30 +94,74 @@ public class DedupRecordFactory { } }); p.setDateofacceptance(DatePicker.pick(dateofacceptance)); + p.getDataInfo().setTrust("0.9"); + p.setLastupdatetimestamp(ts); return p; } - private static Dataset datasetMerger(Tuple2> e) { + private static Dataset datasetMerger(Tuple2> e, final long ts) { + + Dataset d = new Dataset(); //the result of the merge, to be returned at the end + + d.setId(e._1()); + + final ObjectMapper mapper = new ObjectMapper(); + + final Collection dateofacceptance = Lists.newArrayList(); + + if (e._2() != null) + e._2().forEach(dat -> { + try { + Dataset dataset = mapper.readValue(dat, Dataset.class); + + d.mergeFrom(dataset); + d.setAuthor(DedupUtility.mergeAuthor(d.getAuthor(), dataset.getAuthor())); + //add to the list if they are not null + if (dataset.getDateofacceptance() != null) + dateofacceptance.add(dataset.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + d.setDateofacceptance(DatePicker.pick(dateofacceptance)); + d.getDataInfo().setTrust("0.9"); + d.setLastupdatetimestamp(ts); + return d; + } + + private static Project projectMerger(Tuple2> e, final long ts) { + + Project p = new Project(); //the result of the merge, to be returned at the end + + p.setId(e._1()); + + final ObjectMapper mapper = new ObjectMapper(); + if (e._2() != null) + e._2().forEach(proj -> { + try { + Project project = mapper.readValue(proj, Project.class); + + p.mergeFrom(project); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + p.getDataInfo().setTrust("0.9"); + p.setLastupdatetimestamp(ts); + return p; + } + + private static Software softwareMerger(Tuple2> e, final long ts) { throw new NotImplementedException(); } - private static Project projectMerger(Tuple2> e) { + private static Datasource datasourceMerger(Tuple2> e, final long ts) { throw new NotImplementedException(); } - private static Software softwareMerger(Tuple2> e) { - - throw new NotImplementedException(); - } - - private static Datasource datasourceMerger(Tuple2> e) { - - throw new NotImplementedException(); - } - - private static Organization organizationMerger(Tuple2> e) { + private static Organization organizationMerger(Tuple2> e, final long ts) { Organization o = new Organization(); //the result of the merge, to be returned at the end @@ -180,7 +192,7 @@ public class DedupRecordFactory { return o; } - private static OtherResearchProduct otherresearchproductMerger(Tuple2> e) { + private static OtherResearchProduct otherresearchproductMerger(Tuple2> e, final long ts) { throw new NotImplementedException(); } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java index 388ab9b69..3bed74f86 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupUtility.java @@ -109,58 +109,26 @@ public class DedupUtility { int sa = authorsSize(a); int sb = authorsSize(b); - if(pa == pb){ - base = sa>sb?a:b; - enrich = sa>sb?b:a; + if (pa == pb) { + base = sa > sb ? a : b; + enrich = sa > sb ? b : a; } else { - base = pa>pb?a:b; - enrich = pa>pb?b:a; + base = pa > pb ? a : b; + enrich = pa > pb ? b : a; } enrichPidFromList(base, enrich); return base; - - - -// //if both have no authors with pids -// if (pa < 1 && pb < 1) { -// //B is bigger than A -// if (sa < sb) -// return b; -// //A is bigger than B -// else -// return a; -// } -// //If A has author with pids -// if (pa > 0) { -// //B has no author with pid -// if (pb < 1) -// return a; -// //B has author with pid -// else { -// enrichPidFromList(a, b); -// return a; -// } -// } -// //If B has author with pids -// //A has no author with pid -// if (pa < 1) -// return b; -// //A has author with pid -// else { -// enrichPidFromList(b, a); -// return b; -// } } private static void enrichPidFromList(List base, List enrich) { - if(base==null || enrich == null) + if (base == null || enrich == null) return; final Map basePidAuthorMap = base.stream() .filter(a -> a.getPid() != null && a.getPid().size() > 0) .flatMap(a -> a.getPid() .stream() .map(p -> new Tuple2<>(p.toComparableString(), a)) - ).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); + ).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); final List> pidToEnrich = enrich .stream() @@ -171,7 +139,7 @@ public class DedupUtility { pidToEnrich.forEach(a -> { Optional> simAuhtor = base.stream().map(ba -> new Tuple2<>(sim(ba, a._2()), ba)).max(Comparator.comparing(Tuple2::_1)); - if (simAuhtor.isPresent() && simAuhtor.get()._1()> THRESHOLD) { + if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) { Author r = simAuhtor.get()._2(); r.getPid().add(a._1()); } @@ -179,15 +147,15 @@ public class DedupUtility { } public static String createEntityPath(final String basePath, final String entityType) { - return String.format("%s/%s", basePath,entityType); + return String.format("%s/%s", basePath, entityType); } public static String createSimRelPath(final String basePath, final String entityType) { - return String.format("%s/%s_simRel", basePath,entityType); + return String.format("%s/%s_simRel", basePath, entityType); } public static String createMergeRelPath(final String basePath, final String entityType) { - return String.format("%s/%s_mergeRel", basePath,entityType); + return String.format("%s/%s_mergeRel", basePath, entityType); } private static Double sim(Author a, Author b) { @@ -220,6 +188,7 @@ public class DedupUtility { private static String nfd(final String s) { return Normalizer.normalize(s, Normalizer.Form.NFD); } + private static Person parse(Author author) { if (StringUtils.isNotBlank(author.getSurname())) { return new Person(author.getSurname() + ", " + author.getName(), false); @@ -233,7 +202,7 @@ public class DedupUtility { if (authors == null) return 0; - return (int) authors.stream().map(DedupUtility::extractAuthorPid).filter(Objects::nonNull).filter(StringUtils::isNotBlank).count(); + return (int) authors.stream().filter(DedupUtility::hasPid).count(); } private static int authorsSize(List authors) { @@ -242,29 +211,9 @@ public class DedupUtility { return authors.size(); } - - private static boolean isAccurate(final Author a) { - return StringUtils.isNotBlank(a.getName()) && StringUtils.isNotBlank(a.getSurname()); - } - - private static String extractAuthorPid(Author a) { - + private static boolean hasPid(Author a) { if (a == null || a.getPid() == null || a.getPid().size() == 0) - return null; - - StringBuilder mainPid = new StringBuilder(); - - a.getPid().forEach(pid -> { - if (pid.getQualifier().getClassid().equalsIgnoreCase("orcid")) { - mainPid.setLength(0); - mainPid.append(pid.getValue()); - } else { - if (mainPid.length() == 0) - mainPid.append(pid.getValue()); - } - }); - - return mainPid.toString(); - + return false; + return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); } } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java deleted file mode 100644 index 2ab78db7c..000000000 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafComparator.java +++ /dev/null @@ -1,15 +0,0 @@ -package eu.dnetlib.dedup; -import com.google.common.collect.ComparisonChain; -import java.io.Serializable; -import java.util.Comparator; - -public class OafComparator implements Comparator, Serializable { - - @Override - public int compare(OafKey a, OafKey b) { - return ComparisonChain.start() - .compare(a.getDedupId(), b.getDedupId()) - .compare(a.getTrust(), b.getTrust()) - .result(); - } -} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java deleted file mode 100644 index f66b0457e..000000000 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafKey.java +++ /dev/null @@ -1,31 +0,0 @@ -package eu.dnetlib.dedup; - -import java.io.Serializable; -public class OafKey implements Serializable { - - private String dedupId; - private String trust; - - public OafKey(String dedupId, String trust) { - this.dedupId = dedupId; - this.trust = trust; - } - public OafKey() { - } - public String getDedupId() { - return dedupId; - } - public void setDedupId(String dedupId) { - this.dedupId = dedupId; - } - public String getTrust() { - return trust; - } - public void setTrust(String trust) { - this.trust = trust; - } - @Override - public String toString(){ - return String.format("%s->%d", dedupId,trust); - } -} diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java deleted file mode 100644 index 20885fd0b..000000000 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/OafPartitioner.java +++ /dev/null @@ -1,59 +0,0 @@ -package eu.dnetlib.dedup; - -import org.apache.spark.Partitioner; - -import java.io.Serializable; - -public class OafPartitioner extends Partitioner implements Serializable { - - private final int numPartitions; - - public OafPartitioner(int partitions) { - assert (partitions > 0); - this.numPartitions = partitions; - } - - @Override - public int numPartitions() { - return numPartitions; - } - - @Override - public int getPartition(Object key) { - if (key instanceof OafKey) { - @SuppressWarnings("unchecked") - OafKey item = (OafKey) key; - return Math.abs(item.getDedupId().hashCode() % numPartitions); - } else { - throw new IllegalArgumentException("Unexpected Key"); - } - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + numPartitions; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (!(obj instanceof OafPartitioner)) { - return false; - } - // - OafPartitioner other = (OafPartitioner) obj; - if (numPartitions != other.numPartitions) { - return false; - } - // - return true; - } -} diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml index 09dd3a315..0e75aa072 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml @@ -30,7 +30,7 @@ - + From 545e940007ac2003d61d06dc018419ff153b7d01 Mon Sep 17 00:00:00 2001 From: miconis Date: Thu, 12 Dec 2019 15:36:41 +0100 Subject: [PATCH 06/13] implementation of the mergeFrom for the Datasources --- .../eu/dnetlib/dhp/schema/oaf/Datasource.java | 65 ++++++++++++++++++- 1 file changed, 64 insertions(+), 1 deletion(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java index 21408a5ec..f52a500fe 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java @@ -78,7 +78,7 @@ public class Datasource extends OafEntity implements Serializable { private Field certificates; - private List< KeyValue> policies; + private List policies; private Journal journal; @@ -361,4 +361,67 @@ public class Datasource extends OafEntity implements Serializable { public void setJournal(Journal journal) { this.journal = journal; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + Datasource d = (Datasource)e; + + datasourcetype = d.getDatasourcetype() != null && compareTrust(this, e)<0? d.getDatasourcetype() : datasourcetype; + openairecompatibility = d.getOpenairecompatibility() != null && compareTrust(this, e)<0? d.getOpenairecompatibility() : openairecompatibility; + officialname = d.getOfficialname() != null && compareTrust(this, e)<0? d.getOfficialname() : officialname; + englishname = d.getEnglishname() != null && compareTrust(this, e)<0? d.getEnglishname() : officialname; + websiteurl = d.getWebsiteurl() != null && compareTrust(this, e)<0? d.getWebsiteurl() : websiteurl; + logourl = d.getLogourl() != null && compareTrust(this, e)<0? d.getLogourl() : getLogourl(); + contactemail = d.getContactemail() != null && compareTrust(this, e)<0? d.getContactemail() : contactemail; + namespaceprefix = d.getNamespaceprefix() != null && compareTrust(this, e)<0? d.getNamespaceprefix() : namespaceprefix; + latitude = d.getLatitude() != null && compareTrust(this, e)<0? d.getLatitude() : latitude; + longitude = d.getLongitude() != null && compareTrust(this, e)<0? d.getLongitude() : longitude; + dateofvalidation = d.getDateofvalidation() != null && compareTrust(this, e)<0? d.getDateofvalidation() : dateofvalidation; + description = d.getDescription() != null && compareTrust(this, e)<0? d.getDescription() : description; + subjects = mergeLists(subjects, d.getSubjects()); + + // opendoar specific fields (od*) + odnumberofitems = d.getOdnumberofitems() != null && compareTrust(this, e)<0? d.getOdnumberofitems() : odnumberofitems; + odnumberofitemsdate = d.getOdnumberofitemsdate() != null && compareTrust(this, e)<0? d.getOdnumberofitemsdate() : odnumberofitemsdate; + odpolicies = d.getOdpolicies() != null && compareTrust(this, e)<0? d.getOdpolicies() : odpolicies; + odlanguages = mergeLists(odlanguages, d.getOdlanguages()); + odcontenttypes = mergeLists(odcontenttypes, d.getOdcontenttypes()); + accessinfopackage = mergeLists(accessinfopackage, d.getAccessinfopackage()); + + // re3data fields + releasestartdate = d.getReleasestartdate() != null && compareTrust(this, e)<0? d.getReleasestartdate() : releasestartdate; + releaseenddate = d.getReleaseenddate() != null && compareTrust(this, e)<0? d.getReleaseenddate() : releaseenddate; + missionstatementurl = d.getMissionstatementurl() != null && compareTrust(this, e)<0? d.getMissionstatementurl() : missionstatementurl; + dataprovider = d.getDataprovider() != null && compareTrust(this, e)<0? d.getDataprovider() : dataprovider; + serviceprovider = d.getServiceprovider() != null && compareTrust(this, e)<0? d.getServiceprovider() : serviceprovider; + + // {open, restricted or closed} + databaseaccesstype = d.getDatabaseaccesstype() != null && compareTrust(this, e)<0? d.getDatabaseaccesstype() : databaseaccesstype; + + // {open, restricted or closed} + datauploadtype = d.getDatauploadtype() != null && compareTrust(this, e)<0? d.getDatauploadtype() : datauploadtype; + + // {feeRequired, registration, other} + databaseaccessrestriction = d.getDatabaseaccessrestriction() != null && compareTrust(this, e)<0? d.getDatabaseaccessrestriction() : databaseaccessrestriction; + + // {feeRequired, registration, other} + datauploadrestriction = d.getDatauploadrestriction() != null && compareTrust(this, e)<0? d.getDatauploadrestriction() : datauploadrestriction; + + versioning = d.getVersioning() != null && compareTrust(this, e)<0? d.getVersioning() : versioning; + citationguidelineurl = d.getCitationguidelineurl() != null && compareTrust(this, e)<0? d.getCitationguidelineurl() : citationguidelineurl; + + //{yes, no, unknown} + qualitymanagementkind = d.getQualitymanagementkind() != null && compareTrust(this, e)<0? d.getQualitymanagementkind() : qualitymanagementkind; + pidsystems = d.getPidsystems() != null && compareTrust(this, e)<0? d.getPidsystems() : pidsystems; + + certificates = d.getCertificates() != null && compareTrust(this, e)<0? d.getCertificates() : certificates; + + policies = mergeLists(policies, d.getPolicies()); + + journal = d.getJournal() != null && compareTrust(this, e)<0? d.getJournal() : journal; + + mergeOAFDataInfo(e); + } } From b4392f9f439aabf2166c80bed03c5686cc3b30eb Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Fri, 13 Dec 2019 09:40:02 +0100 Subject: [PATCH 07/13] implemented DedupRecord factory for missing entities --- .../eu/dnetlib/dhp/schema/oaf/Context.java | 2 +- .../java/eu/dnetlib/dhp/schema/oaf/Field.java | 8 +- .../dnetlib/dhp/schema/oaf/GeoLocation.java | 10 ++- .../eu/dnetlib/dhp/schema/oaf/Instance.java | 3 + .../eu/dnetlib/dhp/schema/oaf/KeyValue.java | 8 +- .../dhp/schema/oaf/OtherResearchProduct.java | 12 +++ .../eu/dnetlib/dhp/schema/oaf/Qualifier.java | 11 ++- .../eu/dnetlib/dhp/schema/oaf/Result.java | 25 +++--- .../eu/dnetlib/dhp/schema/oaf/Software.java | 15 ++++ .../eu/dnetlib/dedup/DedupRecordFactory.java | 86 ++++++++++++++++--- 10 files changed, 143 insertions(+), 37 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java index 8f46a0e23..809200463 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java @@ -26,7 +26,7 @@ public class Context implements Serializable { @Override public int hashCode() { - return id.hashCode(); + return id ==null? 0 : id.hashCode(); } @Override diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java index b645d275f..2ab0b4d3c 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java @@ -25,8 +25,8 @@ public class Field implements Serializable { } @Override - public int hashCode(){ - return getValue().hashCode(); + public int hashCode() { + return getValue() == null ? 0 : getValue().hashCode(); } @Override @@ -37,11 +37,7 @@ public class Field implements Serializable { return false; if (getClass() != obj.getClass()) return false; - Field other = (Field) obj; - return getValue().equals(other.getValue()); } - - } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java index a0ce32353..43af60286 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java @@ -1,5 +1,7 @@ package eu.dnetlib.dhp.schema.oaf; +import org.apache.commons.lang3.StringUtils; + import java.io.Serializable; public class GeoLocation implements Serializable { @@ -35,8 +37,14 @@ public class GeoLocation implements Serializable { } + public boolean isBlank() { + return StringUtils.isBlank(point) && + StringUtils.isBlank(box) && + StringUtils.isBlank(place); + } + public String toComparableString() { - return String.format("%s::%s%s", point != null ? point.toLowerCase() : "", box != null ? box.toLowerCase() : "",place != null ? place.toLowerCase() : ""); + return isBlank()?"":String.format("%s::%s%s", point != null ? point.toLowerCase() : "", box != null ? box.toLowerCase() : "", place != null ? place.toLowerCase() : ""); } @Override diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java index dc5ac61e8..ceaaad6b0 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java @@ -84,6 +84,9 @@ public class Instance implements Serializable { public void setDateofacceptance(Field dateofacceptance) { this.dateofacceptance = dateofacceptance; } + + + public String toComparableString(){ return String.format("%s::%s::%s::%s", hostedby != null && hostedby.getKey()!= null ? hostedby.getKey().toLowerCase() : "", diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java index fd394e188..74d9f77bd 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java @@ -1,5 +1,7 @@ package eu.dnetlib.dhp.schema.oaf; +import org.apache.commons.lang3.StringUtils; + import java.io.Serializable; public class KeyValue implements Serializable { @@ -35,7 +37,11 @@ public class KeyValue implements Serializable { } public String toComparableString() { - return String.format("%s::%s", key != null ? key.toLowerCase() : "", value != null ? value.toLowerCase() : ""); + return isBlank()?"":String.format("%s::%s", key != null ? key.toLowerCase() : "", value != null ? value.toLowerCase() : ""); + } + + public boolean isBlank() { + return StringUtils.isBlank(key) && StringUtils.isBlank(value); } @Override diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java index 5f32b8895..72bec727e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java @@ -34,4 +34,16 @@ public class OtherResearchProduct extends Result implements Serializable { public void setTool(List> tool) { this.tool = tool; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + OtherResearchProduct o = (OtherResearchProduct)e; + + contactperson = mergeLists(contactperson, o.getContactperson()); + contactgroup = mergeLists(contactgroup, o.getContactgroup()); + tool = mergeLists(tool, o.getTool()); + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java index 9c52d7310..7e4660f4b 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java @@ -1,5 +1,7 @@ package eu.dnetlib.dhp.schema.oaf; +import org.apache.commons.lang3.StringUtils; + import java.io.Serializable; public class Qualifier implements Serializable { @@ -42,13 +44,18 @@ public class Qualifier implements Serializable { } public String toComparableString() { - return String.format("%s::%s::%s::%s", + return isBlank()?"": String.format("%s::%s::%s::%s", classid != null ? classid : "", classname != null ? classname : "", schemeid != null ? schemeid : "", schemename != null ? schemename : ""); } - + public boolean isBlank() { + return StringUtils.isBlank(classid) && + StringUtils.isBlank(classname) && + StringUtils.isBlank(schemeid) && + StringUtils.isBlank(schemename); + } @Override public int hashCode() { return toComparableString().hashCode(); diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 10339178d..eb5572ce1 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -252,10 +252,10 @@ public abstract class Result extends OafEntity implements Serializable { instance = mergeLists(instance, r.getInstance()); - if (r.getResulttype() != null && compareTrust(this, r)<0) + if (r.getResulttype() != null && compareTrust(this, r) < 0) resulttype = r.getResulttype(); - if (r.getLanguage() != null && compareTrust(this, r)<0) + if (r.getLanguage() != null && compareTrust(this, r) < 0) language = r.getLanguage(); country = mergeLists(country, r.getCountry()); @@ -268,10 +268,10 @@ public abstract class Result extends OafEntity implements Serializable { description = longestLists(description, r.getDescription()); - if (r.getPublisher() != null && compareTrust(this, r)<0) + if (r.getPublisher() != null && compareTrust(this, r) < 0) publisher = r.getPublisher(); - if (r.getEmbargoenddate() != null && compareTrust(this, r)<0) + if (r.getEmbargoenddate() != null && compareTrust(this, r) < 0) embargoenddate = r.getEmbargoenddate(); source = mergeLists(source, r.getSource()); @@ -287,15 +287,15 @@ public abstract class Result extends OafEntity implements Serializable { coverage = mergeLists(coverage, r.getCoverage()); - if (r.getRefereed() != null && compareTrust(this, r)<0) + if (r.getRefereed() != null && compareTrust(this, r) < 0) refereed = r.getRefereed(); context = mergeLists(context, r.getContext()); - if (r.getProcessingchargeamount() != null && compareTrust(this, r)<0) + if (r.getProcessingchargeamount() != null && compareTrust(this, r) < 0) processingchargeamount = r.getProcessingchargeamount(); - if (r.getProcessingchargecurrency() != null && compareTrust(this, r)<0) + if (r.getProcessingchargecurrency() != null && compareTrust(this, r) < 0) processingchargecurrency = r.getProcessingchargecurrency(); externalReference = mergeLists(externalReference, r.getExternalReference()); @@ -303,16 +303,15 @@ public abstract class Result extends OafEntity implements Serializable { } - private List> longestLists(List> a, List> b) { - if(a == null || b == null) - return a==null?b:a; - if (a.size()== b.size()) { + if (a == null || b == null) + return a == null ? b : a; + if (a.size() == b.size()) { int msa = a.stream().filter(i -> i.getValue() != null).map(i -> i.getValue().length()).max(Comparator.naturalOrder()).orElse(0); int msb = b.stream().filter(i -> i.getValue() != null).map(i -> i.getValue().length()).max(Comparator.naturalOrder()).orElse(0); - return msa>msb?a:b; + return msa > msb ? a : b; } - return a.size()> b.size()?a:b; + return a.size() > b.size() ? a : b; } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java index e696cdb52..7f835fdf8 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java @@ -44,4 +44,19 @@ public class Software extends Result implements Serializable { public void setProgrammingLanguage(Qualifier programmingLanguage) { this.programmingLanguage = programmingLanguage; } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + final Software s = (Software) e; + documentationUrl = mergeLists(documentationUrl, s.getDocumentationUrl()); + + license = mergeLists(license, s.getLicense()); + + codeRepositoryUrl = s.getCodeRepositoryUrl()!= null && compareTrust(this, s)<0?s.getCodeRepositoryUrl():codeRepositoryUrl; + + programmingLanguage= s.getProgrammingLanguage()!= null && compareTrust(this, s)<0?s.getProgrammingLanguage():programmingLanguage; + + mergeOAFDataInfo(e); + } } diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index 86c1651a2..704ef918d 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -23,8 +23,6 @@ import static java.util.stream.Collectors.toMap; public class DedupRecordFactory { public static JavaRDD createDedupRecord(final JavaSparkContext sc, final SparkSession spark, final String mergeRelsInputPath, final String entitiesInputPath, final OafEntityType entityType, final DedupConfig dedupConf) { - - long ts = System.currentTimeMillis(); // final JavaPairRDD inputJsonEntities = sc.textFile(entitiesInputPath) @@ -49,20 +47,19 @@ public class DedupRecordFactory { switch (entityType) { case publication: - - return sortedJoinResult.map(p->DedupRecordFactory.publicationMerger(p, ts)); + return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts)); case dataset: - return sortedJoinResult.map(d->DedupRecordFactory.datasetMerger(d,ts)); + return sortedJoinResult.map(d -> DedupRecordFactory.datasetMerger(d, ts)); case project: - return sortedJoinResult.map(p->DedupRecordFactory.projectMerger(p,ts)); + return sortedJoinResult.map(p -> DedupRecordFactory.projectMerger(p, ts)); case software: - return sortedJoinResult.map(s->DedupRecordFactory.softwareMerger(s,ts)); + return sortedJoinResult.map(s -> DedupRecordFactory.softwareMerger(s, ts)); case datasource: - return sortedJoinResult.map(d->DedupRecordFactory.datasourceMerger(d,ts)); + return sortedJoinResult.map(d -> DedupRecordFactory.datasourceMerger(d, ts)); case organization: - return sortedJoinResult.map(o->DedupRecordFactory.organizationMerger(o,ts)); + return sortedJoinResult.map(o -> DedupRecordFactory.organizationMerger(o, ts)); case otherresearchproduct: - return sortedJoinResult.map(o->DedupRecordFactory.otherresearchproductMerger(o,ts)); + return sortedJoinResult.map(o -> DedupRecordFactory.otherresearchproductMerger(o, ts)); default: return null; } @@ -153,12 +150,48 @@ public class DedupRecordFactory { private static Software softwareMerger(Tuple2> e, final long ts) { - throw new NotImplementedException(); + Software s = new Software(); //the result of the merge, to be returned at the end + + s.setId(e._1()); + final ObjectMapper mapper = new ObjectMapper(); + final Collection dateofacceptance = Lists.newArrayList(); + if (e._2() != null) + e._2().forEach(soft -> { + try { + Software software = mapper.readValue(soft, Software.class); + + s.mergeFrom(software); + s.setAuthor(DedupUtility.mergeAuthor(s.getAuthor(), software.getAuthor())); + //add to the list if they are not null + if (software.getDateofacceptance() != null) + dateofacceptance.add(software.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + s.setDateofacceptance(DatePicker.pick(dateofacceptance)); + s.getDataInfo().setTrust("0.9"); + s.setLastupdatetimestamp(ts); + return s; } private static Datasource datasourceMerger(Tuple2> e, final long ts) { + Datasource d = new Datasource(); //the result of the merge, to be returned at the end + d.setId(e._1()); + final ObjectMapper mapper = new ObjectMapper(); + if (e._2() != null) + e._2().forEach(dat -> { + try { + Datasource datasource = mapper.readValue(dat, Datasource.class); - throw new NotImplementedException(); + d.mergeFrom(datasource); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + d.getDataInfo().setTrust("0.9"); + d.setLastupdatetimestamp(ts); + return d; } private static Organization organizationMerger(Tuple2> e, final long ts) { @@ -188,13 +221,40 @@ public class DedupRecordFactory { throw new RuntimeException(exc); } }); + o.getDataInfo().setTrust("0.9"); + o.setLastupdatetimestamp(ts); return o; } private static OtherResearchProduct otherresearchproductMerger(Tuple2> e, final long ts) { - throw new NotImplementedException(); + OtherResearchProduct o = new OtherResearchProduct(); //the result of the merge, to be returned at the end + + o.setId(e._1()); + + final ObjectMapper mapper = new ObjectMapper(); + + final Collection dateofacceptance = Lists.newArrayList(); + + if (e._2() != null) + e._2().forEach(orp -> { + try { + OtherResearchProduct otherResearchProduct = mapper.readValue(orp, OtherResearchProduct.class); + + o.mergeFrom(otherResearchProduct); + o.setAuthor(DedupUtility.mergeAuthor(o.getAuthor(), otherResearchProduct.getAuthor())); + //add to the list if they are not null + if (otherResearchProduct.getDateofacceptance() != null) + dateofacceptance.add(otherResearchProduct.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + o.setDateofacceptance(DatePicker.pick(dateofacceptance)); + o.getDataInfo().setTrust("0.9"); + o.setLastupdatetimestamp(ts); + return o; } } From 76efcde4fddadb75b64ce102bb57adaf2ea802d1 Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Fri, 13 Dec 2019 12:20:35 +0100 Subject: [PATCH 08/13] using new branch decisionTreeDedup --- .../eu/dnetlib/dedup/DedupRecordFactory.java | 5 + .../dnetlib/dedup/SparkCreateDedupTest.java | 4 +- .../eu/dnetlib/dedup/conf/org.curr.conf.json | 1972 ++--------------- .../dnetlib/dedup/conf/pub_dt.curr.conf.json | 334 +++ 4 files changed, 575 insertions(+), 1740 deletions(-) create mode 100644 dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index 704ef918d..ed3ac8231 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -221,6 +221,11 @@ public class DedupRecordFactory { throw new RuntimeException(exc); } }); + + if (o.getDataInfo() == null) + { + o.setDataInfo(new DataInfo()); + } o.getDataInfo().setTrust("0.9"); o.setLastupdatetimestamp(ts); diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index 7aa8a4302..19522e275 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -16,11 +16,11 @@ import java.util.List; public class SparkCreateDedupTest { String configuration; - String entity = "organization"; + String entity = "publication"; @Before public void setUp() throws IOException { - configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json")); + configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json")); } diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json index 2aeb4c582..43003c2e0 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json @@ -1,1753 +1,249 @@ { - "wf": { - "threshold": "0.9", - "dedupRun": "001", - "entityType": "organization", - "orderField": "legalname", - "queueMaxSize": "2000", - "groupMaxSize": "50", - "slidingWindowSize": "200", + "wf" : { + "threshold" : "0.99", + "dedupRun" : "001", + "entityType" : "organization", + "orderField" : "legalname", + "queueMaxSize" : "2000", + "groupMaxSize" : "50", + "slidingWindowSize" : "200", + "rootBuilder" : [ "organization", "projectOrganization_participation_isParticipant", "datasourceOrganization_provision_isProvidedBy" ], + "includeChildren" : "true", "idPath": "$.id", - "rootBuilder": [ - "organization", - "projectOrganization_participation_isParticipant", - "datasourceOrganization_provision_isProvidedBy" - ], - "includeChildren": "true" + "maxIterations": "20" }, - "pace": { - "clustering": [ - { - "name": "sortedngrampairs", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "ngramLen": "3" - } - }, - { - "name": "suffixprefix", - "fields": [ - "legalname" - ], - "params": { - "max": 1, - "len": "3" - } - }, - { - "name": "urlclustering", - "fields": [ - "websiteurl" - ], - "params": {} - }, - { - "name": "keywordsclustering", - "fields": [ - "legalname" - ], - "params": { - "max": 2, - "windowSize": 4 - } - } + "pace" : { + "clustering" : [ + { "name" : "sortedngrampairs", "fields" : [ "legalname" ], "params" : { "max" : 2, "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "legalname" ], "params" : { "max" : 1, "len" : "3" } }, + { "name" : "urlclustering", "fields" : [ "websiteurl" ], "params" : { } }, + { "name" : "keywordsclustering", "fields" : [ "legalname" ], "params" : { "max": 2, "windowSize": 4} } ], - "strictConditions": [ - { - "name": "exactMatch", + "decisionTree" : { + "start": { "fields": [ - "gridid" - ] - } - ], - "conditions": [ - { - "name": "DomainExactMatch", + { + "field": "gridid", + "comparator": "exactMatch", + "weight": 1, + "countIfUndefined": "false", + "params": {} + } + ], + "threshold": 1, + "aggregation": "SC", + "positive": "MATCH", + "negative": "NO_MATCH", + "undefined": "layer2", + "ignoreUndefined": "false" + }, + "layer2": { "fields": [ - "websiteurl" - ] + { + "field": "websiteurl", + "comparator": "domainExactMatch", + "weight": 1, + "countIfUndefined": "false", + "params": {} + }, + { + "field": "country", + "comparator": "exactMatch", + "weight": 1, + "countIfUndefined": "true", + "params": {} + } + ], + "threshold": 1, + "aggregation": "NC", + "positive": "layer3", + "negative": "NO_MATCH", + "undefined": "layer3", + "ignoreUndefined": "true" }, - { - "name": "exactMatch", + "layer3": { "fields": [ - "country" - ] + { + "field": "legalname", + "comparator": "cityMatch", + "weight": 1.0, + "countIfUndefined": "true", + "params": { + "windowSize": "4", + "threshold": "0.0" + } + } + ], + "threshold": 1.0, + "aggregation": "W_MEAN", + "positive": "layer4", + "negative": "NO_MATCH", + "undefined": "NO_MATCH", + "ignoreUndefined": "true" + }, + "layer4": { + "fields": [ + { + "field": "legalname", + "comparator": "keywordMatch", + "weight": 1.0, + "countIfUndefined": "false", + "params": { + "windowSize": "4", + "threshold": "0.7" + } + } + ], + "threshold": 1.0, + "aggregation": "W_MEAN", + "positive": "layer5", + "negative": "NO_MATCH", + "undefined": "layer5", + "ignoreUndefined": "false" + }, + "layer5": { + "fields": [ + { + "field": "legalname", + "comparator": "jaroWinklerNormalizedName", + "weight": 0.9, + "countIfUndefined": "true", + "params": { + "windowSize": "4" + } + }, + { + "field": "legalshortname", + "comparator": "jaroWinklerNormalizedName", + "weight": 0.1, + "countIfUndefined": "false", + "params": {} + } + ], + "threshold": 0.9, + "aggregation": "W_MEAN", + "positive": "MATCH", + "negative": "NO_MATCH", + "undefined": "NO_MATCH", + "ignoreUndefined": "true" } + }, + "model" : [ + { "name" : "country", "type" : "String", "path" : "$.country.classid"}, + { "name" : "legalshortname", "type" : "String", "path" : "$.legalshortname.value"}, + { "name" : "legalname", "type" : "String", "path" : "$.legalname.value" }, + { "name" : "websiteurl", "type" : "URL", "path" : "$.websiteurl.value" }, + { "name" : "gridid", "type" : "String", "path" : "$.pid[?(@.qualifier.classid ==\"grid\")].value"} ], - "model": [ - { - "name": "country", - "algo": "Null", - "type": "String", - "weight": "0", - "ignoreMissing": "false", - "path": "$.country.classid" - }, - { - "name": "legalshortname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.1", - "ignoreMissing": "true", - "path": "$.legalshortname.value" - }, - { - "name": "legalname", - "algo": "JaroWinklerNormalizedName", - "type": "String", - "weight": "0.9", - "ignoreMissing": "false", - "path": "$.legalname.value", - "params": { - "windowSize": 4, - "threshold": 0.7 - } - }, - { - "name": "websiteurl", - "algo": "Null", - "type": "URL", - "weight": "0", - "ignoreMissing": "true", - "path": "$.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" - } - ], - "blacklists": { - "legalname": [] + "blacklists" : { + "legalname" : [] }, "synonyms": { - "key::1": [ - "university", - "università", - "università studi", - "universitario", - "universitaria", - "université", - "universitaire", - "universitaires", - "universidad", - "universitade", - "Universität", - "universitaet", - "Uniwersytet", - "университет", - "universiteit", - "πανεπιστήμιο", - "universitesi", - "universiteti" - ], - "key::2": [ - "studies", - "studi", - "études", - "estudios", - "estudos", - "Studien", - "studia", - "исследования", - "studies", - "σπουδές" - ], - "key::3": [ - "advanced", - "superiore", - "supérieur", - "supérieure", - "supérieurs", - "supérieures", - "avancado", - "avancados", - "fortgeschrittene", - "fortgeschritten", - "zaawansowany", - "передовой", - "gevorderd", - "gevorderde", - "προχωρημένος", - "προχωρημένη", - "προχωρημένο", - "προχωρημένες", - "προχωρημένα", - "wyzsza" - ], - "key::4": [ - "institute", - "istituto", - "institut", - "instituto", - "instituto", - "Institut", - "instytut", - "институт", - "instituut", - "ινστιτούτο" - ], - "key::5": [ - "hospital", - "ospedale", - "hôpital", - "hospital", - "hospital", - "Krankenhaus", - "szpital", - "больница", - "ziekenhuis", - "νοσοκομείο" - ], - "key::6": [ - "research", - "ricerca", - "recherche", - "investigacion", - "pesquisa", - "Forschung", - "badania", - "исследования", - "onderzoek", - "έρευνα", - "erevna", - "erevnas" - ], - "key::7": [ - "college", - "collegio", - "université", - "colegio", - "faculdade", - "Hochschule", - "Szkoła Wyższa", - "Высшая школа", - "universiteit", - "κολλέγιο" - ], - "key::8": [ - "foundation", - "fondazione", - "fondation", - "fundación", - "fundação", - "Stiftung", - "Fundacja", - "фонд", - "stichting", - "ίδρυμα", - "idryma" - ], - "key::9": [ - "center", - "centro", - "centre", - "centro", - "centro", - "zentrum", - "centrum", - "центр", - "centrum", - "κέντρο" - ], - "key::10": [ - "national", - "nazionale", - "national", - "nationale", - "nationaux", - "nationales", - "nacional", - "nacional", - "national", - "krajowy", - "национальный", - "nationaal", - "nationale", - "εθνικό" - ], - "key::11": [ - "association", - "associazione", - "association", - "asociación", - "associação", - "Verein", - "verband", - "stowarzyszenie", - "ассоциация", - "associatie" - ], - "key::12": [ - "society", - "societa", - "société", - "sociedad", - "sociedade", - "gesellschaft", - "społeczeństwo", - "общество", - "maatschappij", - "κοινωνία" - ], - "key::13": [ - "international", - "internazionale", - "international", - "internacional", - "internacional", - "international", - "międzynarodowy", - "Международный", - "internationaal", - "internationale", - "διεθνής", - "διεθνή", - "διεθνές" - ], - "key::14": [ - "community", - "comunita", - "communauté", - "comunidad", - "comunidade", - "Gemeinschaft", - "społeczność", - "сообщество", - "gemeenschap", - "κοινότητα" - ], - "key::15": [ - "school", - "scuola", - "école", - "escuela", - "escola", - "schule", - "Szkoła", - "школа", - "school", - "σχολείο" - ], - "key::16": [ - "education", - "educazione", - "éducation", - "educacion", - "Educação", - "Bildung", - "Edukacja", - "образование", - "opleiding", - "εκπαίδευση" - ], - "key::17": [ - "academy", - "accademia", - "académie", - "academia", - "academia", - "Akademie", - "akademie", - "академия", - "academie", - "ακαδημία" - ], - "key::18": [ - "public", - "pubblico", - "public", - "publique", - "publics", - "publiques", - "publico", - "publico", - "Öffentlichkeit", - "publiczny", - "публичный", - "publiek", - "publieke", - "δημόσιος", - "δημόσια", - "δημόσιο" - ], - "key::19": [ - "museum", - "museo", - "musée", - "mueso", - "museu", - "museum", - "muzeum", - "музей", - "museum", - "μουσείο" - ], - "key::20": [ - "group", - "gruppo", - "groupe", - "grupo", - "grupo", - "gruppe", - "grupa", - "группа", - "groep", - "ομάδα", - "όμιλος" - ], - "key::21": [ - "department", - "dipartimento", - "département", - "departamento", - "departamento", - "abteilung", - "departament", - "отдел", - "afdeling", - "τμήμα" - ], - "key::22": [ - "council", - "consiglio", - "conseil", - "Consejo", - "conselho", - "gesellschaft", - "rada", - "совет", - "raad", - "συμβούλιο" - ], - "key::23": [ - "library", - "biblioteca", - "bibliothèque", - "biblioteca", - "biblioteca", - "Bibliothek", - "biblioteka", - "библиотека", - "bibliotheek", - "βιβλιοθήκη" - ], - "key::24": [ - "ministry", - "ministero", - "ministère", - "ministerio", - "ministério", - "Ministerium", - "ministerstwo", - "министерство", - "ministerie", - "υπουργείο" - ], - "key::25": [ - "services", - "servizi", - "services", - "servicios", - "Serviços", - "Dienstleistungen", - "usługi", - "услуги", - "diensten", - "υπηρεσίες" - ], - "key::26": [ - "central", - "centrale", - "central", - "centrale", - "centrales", - "central", - "central", - "zentral", - "centralny", - "цетральный", - "centraal", - "κεντρικός", - "κεντρική", - "κεντρικό", - "κεντρικά" - ], - "key::27": [ - "general", - "generale", - "général", - "générale", - "généraux", - "générales", - "general", - "geral", - "general", - "Allgemeines", - "general", - "общий", - "algemeen", - "algemene", - "γενικός", - "γενική", - "γενικό", - "γενικά" - ], - "key::28": [ - "applied", - "applicati", - "appliqué", - "appliquée", - "appliqués", - "appliquées", - "aplicado", - "aplicada", - "angewendet", - "stosowany", - "прикладной", - "toegepast", - "toegepaste", - "εφαρμοσμένος", - "εφαρμοσμένη", - "εφαρμοσμένο", - "εφαρμοσμένα" - ], - "key::29": [ - "european", - "europee", - "europea", - "européen", - "européenne", - "européens", - "européennes", - "europeo", - "europeu", - "europäisch", - "europejski", - "европейский", - "Europees", - "Europese", - "ευρωπαϊκός", - "ευρωπαϊκή", - "ευρωπαϊκό", - "ευρωπαϊκά" - ], - "key::30": [ - "agency", - "agenzia", - "agence", - "agencia", - "agencia", - "agentur", - "agencja", - "агенция", - "agentschap", - "πρακτορείο" - ], - "key::31": [ - "laboratory", - "laboratorio", - "laboratoire", - "laboratorio", - "laboratorio", - "labor", - "laboratorium", - "лаборатория", - "laboratorium", - "εργαστήριο" - ], - "key::32": [ - "industry", - "industria", - "industrie", - "индустрия", - "industrie", - "βιομηχανία" - ], - "key::33": [ - "industrial", - "industriale", - "industriel", - "industrielle", - "industriels", - "industrielles", - "индустриальный", - "industrieel", - "βιομηχανικός", - "βιομηχανική", - "βιομηχανικό", - "βιομηχανικά", - "βιομηχανικές" - ], - "key::34": [ - "consortium", - "consorzio", - "consortium", - "консорциум", - "consortium", - "κοινοπραξία" - ], - "key::35": [ - "organization", - "organizzazione", - "organisation", - "organización", - "organização", - "organizacja", - "организация", - "organisatie", - "οργανισμός" - ], - "key::36": [ - "authority", - "autorità", - "autorité", - "авторитет", - "autoriteit" - ], - "key::37": [ - "federation", - "federazione", - "fédération", - "федерация", - "federatie", - "ομοσπονδία" - ], - "key::38": [ - "observatory", - "osservatorio", - "observatoire", - "обсерватория", - "observatorium", - "αστεροσκοπείο" - ], - "key::39": [ - "bureau", - "ufficio", - "bureau", - "офис", - "bureau", - "γραφείο" - ], - "key::40": [ - "company", - "impresa", - "compagnie", - "société", - "компания", - "bedrijf", - "εταιρία" - ], - "key::41": [ - "polytechnic", - "politecnico", - "polytechnique", - "политехника", - "polytechnisch", - "πολυτεχνείο", - "universita politecnica", - "polytechnic university", - "universidad politecnica", - "universitat politecnica", - "politechnika", - "politechniki", - "university technology", - "university science technology" - ], - "key::42": [ - "coalition", - "coalizione", - "coalition", - "коалиция", - "coalitie", - "συνασπισμός" - ], - "key::43": [ - "initiative", - "iniziativa", - "initiative", - "инициатива", - "initiatief", - "πρωτοβουλία" - ], - "key::44": [ - "academic", - "accademico", - "académique", - "universitaire", - "акадеческий academisch", - "ακαδημαϊκός", - "ακαδημαϊκή", - "ακαδημαϊκό", - "ακαδημαϊκές", - "ακαδημαϊκοί" - ], - "key::45": [ - "institution", - "istituzione", - "institution", - "институциональный", - "instelling", - "ινστιτούτο" - ], - "key::46": [ - "division", - "divisione", - "division", - "отделение", - "divisie", - "τμήμα" - ], - "key::47": [ - "committee", - "comitato", - "comité", - "комитет", - "commissie", - "επιτροπή" - ], - "key::48": [ - "promotion", - "promozione", - "продвижение", - "proothisis", - "forderung" - ], - "key::49": [ - "medical", - "medicine", - "clinical", - "medicina", - "clinici", - "médico", - "medicina", - "clínica", - "médico", - "medicina", - "clínica", - "medizinisch", - "Medizin", - "klinisch", - "medisch", - "geneeskunde", - "klinisch", - "ιατρικός", - "ιατρική", - "ιατρικό", - "ιατρικά", - "κλινικός", - "κλινική", - "κλινικό", - "κλινικά", - "tıbbi", - "tıp", - "klinik", - "orvosi", - "orvostudomány", - "klinikai", - "zdravniški", - "medicinski", - "klinični", - "meditsiini", - "kliinik", - "kliiniline" - ], - "key::50": [ - "technology", - "technological", - "tecnologia", - "tecnologie", - "tecnología", - "tecnológico", - "tecnologia", - "tecnológico", - "Technologie", - "technologisch", - "technologie", - "technologisch", - "τεχνολογία", - "τεχνολογικός", - "τεχνολογική", - "τεχνολογικό", - "teknoloji", - "teknolojik", - "technológia", - "technológiai", - "tehnologija", - "tehnološki", - "tehnoloogia", - "tehnoloogiline", - "technologii", - "technical", - "texniki", - "teknik" - ], - "key::51": [ - "science", - "scientific", - "scienza", - "scientifiche", - "scienze", - "ciencia", - "científico", - "ciência", - "científico", - "Wissenschaft", - "wissenschaftlich", - "wetenschap", - "wetenschappelijk", - "επιστήμη", - "επιστημονικός", - "επιστημονική", - "επιστημονικό", - "επιστημονικά", - "bilim", - "bilimsel", - "tudomány", - "tudományos", - "znanost", - "znanstveni", - "teadus", - "teaduslik", - "" - ], - "key::52": [ - "engineering", - "ingegneria", - "ingeniería", - "engenharia", - "Ingenieurwissenschaft", - "ingenieurswetenschappen", - "bouwkunde", - "μηχανικός", - "μηχανική", - "μηχανικό", - "mühendislik", - "mérnöki", - "Inženirstvo", - "inseneeria", - "inseneri", - "" - ], - "key::53": [ - "management", - "gestione", - "gestionale", - "gestionali", - "gestión", - "administración", - "gestão", - "administração", - "Verwaltung", - "management", - "διαχείριση", - "yönetim", - "menedzsment", - "vodstvo", - "upravljanje", - "management", - "juhtkond", - "juhtimine", - "haldus", - "" - ], - "key::54": [ - "energy", - "energia", - "energía", - "energia", - "Energie", - "energie", - "ενέργεια", - "enerji", - "energia", - "energija", - "energia", - "" - ], - "key::55": [ - "agricultural", - "agriculture", - "agricoltura", - "agricole", - "agrícola", - "agricultura", - "agrícola", - "agricultura", - "landwirtschaftlich", - "Landwirtschaft", - "landbouwkundig", - "landbouw", - "αγροτικός", - "αγροτική", - "αγροτικό", - "γεωργικός", - "γεωργική", - "γεωργικό", - "γεωργία", - "tarımsal", - "tarım", - "mezőgazdasági", - "mezőgazdaság", - "poljedelski", - "poljedelstvo", - "põllumajandus", - "põllumajanduslik", - "" - ], - "key::56": [ - "information", - "informazione", - "información", - "informação", - "Information", - "informatie", - "πληροφορία", - "bilgi", - "információ", - "informacija", - "informatsioon", - "informatycznych", - "" - ], - "key::57": [ - "social", - "sociali", - "social", - "social", - "Sozial", - "sociaal", - "maatschappelijk", - "κοινωνικός", - "κοινωνική", - "κοινωνικό", - "κοινωνικά", - "sosyal", - "szociális", - "družbeni", - "sotsiaal", - "sotsiaalne", - "" - ], - "key::58": [ - "environmental", - "ambiente", - "medioambiental", - "ambiente", - "medioambiente", - "meioambiente", - "Umwelt", - "milieu", - "milieuwetenschap", - "milieukunde", - "περιβαλλοντικός", - "περιβαλλοντική", - "περιβαλλοντικό", - "περιβαλλοντικά", - "çevre", - "környezeti", - "okoliški", - "keskonna", - "" - ], - "key::59": [ - "business", - "economia", - "economiche", - "economica", - "negocio", - "empresa", - "negócio", - "Unternehmen", - "bedrijf", - "bedrijfskunde", - "επιχείρηση", - "iş", - "üzleti", - "posel", - "ettevõte/äri", - "" - ], - "key::60": [ - "pharmaceuticals", - "pharmacy", - "farmacia", - "farmaceutica", - "farmacéutica", - "farmacia", - "farmacêutica", - "farmácia", - "Pharmazeutika", - "Arzneimittelkunde", - "farmaceutica", - "geneesmiddelen", - "apotheek", - "φαρμακευτικός", - "φαρμακευτική", - "φαρμακευτικό", - "φαρμακευτικά", - "φαρμακείο", - "ilaç", - "eczane", - "gyógyszerészeti", - "gyógyszertár", - "farmacevtika", - "lekarništvo", - "farmaatsia", - "farmatseutiline", - "" - ], - "key::61": [ - "healthcare", - "health services", - "salute", - "atenciónmédica", - "cuidadodelasalud", - "cuidadoscomasaúde", - "Gesundheitswesen", - "gezondheidszorg", - "ιατροφαρμακευτικήπερίθαλψη", - "sağlıkhizmeti", - "egészségügy", - "zdravstvo", - "tervishoid", - "tervishoiu", - "" - ], - "key::62": [ - "history", - "storia", - "historia", - "história", - "Geschichte", - "geschiedenis", - "geschiedkunde", - "ιστορία", - "tarih", - "történelem", - "zgodovina", - "ajalugu", - "" - ], - "key::63": [ - "materials", - "materiali", - "materia", - "materiales", - "materiais", - "materialen", - "υλικά", - "τεκμήρια", - "malzemeler", - "anyagok", - "materiali", - "materjalid", - "vahendid", - "" - ], - "key::64": [ - "economics", - "economia", - "economiche", - "economica", - "economía", - "economia", - "Wirtschaft", - "economie", - "οικονομικά", - "οικονομικέςεπιστήμες", - "ekonomi", - "közgazdaságtan", - "gospodarstvo", - "ekonomija", - "majanduslik", - "majandus", - "" - ], - "key::65": [ - "therapeutics", - "terapeutica", - "terapéutica", - "terapêutica", - "therapie", - "θεραπευτική", - "tedavibilimi", - "gyógykezelés", - "terapevtika", - "terapeutiline", - "ravi", - "" - ], - "key::66": [ - "oncology", - "oncologia", - "oncologico", - "oncología", - "oncologia", - "Onkologie", - "oncologie", - "ογκολογία", - "onkoloji", - "onkológia", - "onkologija", - "onkoloogia", - "" - ], - "key::67": [ - "natural", - "naturali", - "naturale", - "natural", - "natural", - "natürlich", - "natuurlijk", - "φυσικός", - "φυσική", - "φυσικό", - "φυσικά", - "doğal", - "természetes", - "naraven", - "loodus", - "" - ], - "key::68": [ - "educational", - "educazione", - "pedagogia", - "educacional", - "educativo", - "educacional", - "pädagogisch", - "educatief", - "εκπαιδευτικός", - "εκπαιδευτική", - "εκπαιδευτικό", - "εκπαιδευτικά", - "eğitimsel", - "oktatási", - "izobraževalen", - "haridus", - "hariduslik", - "" - ], - "key::69": [ - "biomedical", - "biomedica", - "biomédico", - "biomédico", - "biomedizinisch", - "biomedisch", - "βιοιατρικός", - "βιοιατρική", - "βιοιατρικό", - "βιοιατρικά", - "biyomedikal", - "orvosbiológiai", - "biomedicinski", - "biomeditsiiniline", - "" - ], - "key::70": [ - "veterinary", - "veterinaria", - "veterinarie", - "veterinaria", - "veterinária", - "tierärtzlich", - "veterinair", - "veeartsenijlkunde", - "κτηνιατρικός", - "κτηνιατρική", - "κτηνιατρικό", - "κτηνιατρικά", - "veteriner", - "állatorvosi", - "veterinar", - "veterinarski", - "veterinaaria", - "" - ], - "key::71": [ - "chemistry", - "chimica", - "química", - "química", - "Chemie", - "chemie", - "scheikunde", - "χημεία", - "kimya", - "kémia", - "kemija", - "keemia", - "" - ], - "key::72": [ - "security", - "sicurezza", - "seguridad", - "segurança", - "Sicherheit", - "veiligheid", - "ασφάλεια", - "güvenlik", - "biztonsági", - "varnost", - "turvalisus", - "julgeolek", - "" - ], - "key::73": [ - "biotechnology", - "biotecnologia", - "biotecnologie", - "biotecnología", - "biotecnologia", - "Biotechnologie", - "biotechnologie", - "βιοτεχνολογία", - "biyoteknoloji", - "biotechnológia", - "biotehnologija", - "biotehnoloogia", - "" - ], - "key::74": [ - "military", - "militare", - "militari", - "militar", - "militar", - "Militär", - "militair", - "leger", - "στρατιωτικός", - "στρατιωτική", - "στρατιωτικό", - "στρατιωτικά", - "askeri", - "katonai", - "vojaški", - "vojni", - "militaar", - "wojskowa", - "" - ], - "key::75": [ - "theological", - "teologia", - "teologico", - "teológico", - "tecnológica", - "theologisch", - "theologisch", - "θεολογικός", - "θεολογική", - "θεολογικό", - "θεολογικά", - "teolojik", - "technológiai", - "teološki", - "teoloogia", - "usuteadus", - "teoloogiline", - "" - ], - "key::76": [ - "electronics", - "elettronica", - "electrónica", - "eletrônicos", - "Elektronik", - "elektronica", - "ηλεκτρονική", - "elektronik", - "elektronika", - "elektronika", - "elektroonika", - "" - ], - "key::77": [ - "forestry", - "forestale", - "forestali", - "silvicultura", - "forestal", - "floresta", - "Forstwirtschaft", - "bosbouw", - "δασοκομία", - "δασολογία", - "ormancılık", - "erdészet", - "gozdarstvo", - "metsandus", - "" - ], - "key::78": [ - "maritime", - "marittima", - "marittime", - "marittimo", - "marítimo", - "marítimo", - "maritiem", - "ναυτικός", - "ναυτική", - "ναυτικό", - "ναυτικά", - "ναυτιλιακός", - "ναυτιλιακή", - "ναυτιλιακό", - "ναυτιλιακά", - "θαλάσσιος", - "θαλάσσια", - "θαλάσσιο", - "denizcilik", - "tengeri", - "morski", - "mere", - "merendus", - "" - ], - "key::79": [ - "sports", - "sport", - "deportes", - "esportes", - "Sport", - "sport", - "sportwetenschappen", - "άθληση", - "γυμναστικήδραστηριότητα", - "spor", - "sport", - "šport", - "sport", - "spordi", - "" - ], - "key::80": [ - "surgery", - "chirurgia", - "chirurgiche", - "cirugía", - "cirurgia", - "Chirurgie", - "chirurgie", - "heelkunde", - "εγχείρηση", - "επέμβαση", - "χειρουργικήεπέμβαση", - "cerrahi", - "sebészet", - "kirurgija", - "kirurgia", - "" - ], - "key::81": [ - "cultural", - "culturale", - "culturali", - "cultura", - "cultural", - "cultural", - "kulturell", - "cultureel", - "πολιτιστικός", - "πολιτιστική", - "πολιτιστικό", - "πολιτισμικός", - "πολιτισμική", - "πολιτισμικό", - "kültürel", - "kultúrális", - "kulturni", - "kultuuri", - "kultuuriline", - "" - ], - "key::82": [ - "computerscience", - "informatica", - "ordenador", - "computadora", - "informática", - "computación", - "cienciasdelacomputación", - "ciênciadacomputação", - "Computer", - "computer", - "υπολογιστής", - "ηλεκτρονικόςυπολογιστής", - "bilgisayar", - "számítógép", - "računalnik", - "arvuti", - "" - ], - "key::83": [ - "finance", - "financial", - "finanza", - "finanziarie", - "finanza", - "financiero", - "finanças", - "financeiro", - "Finanzen", - "finanziell", - "financiën", - "financieel", - "χρηματοοικονομικά", - "χρηματοδότηση", - "finanse", - "finansal", - "pénzügy", - "pénzügyi", - "finance", - "finančni", - "finants", - "finantsiline", - "" - ], - "key::84": [ - "communication", - "comunicazione", - "comuniciación", - "comunicação", - "Kommunikation", - "communication", - "επικοινωνία", - "iletişim", - "kommunikáció", - "komuniciranje", - "kommunikatsioon", - "" - ], - "key::85": [ - "justice", - "giustizia", - "justicia", - "justiça", - "Recht", - "Justiz", - "justitie", - "gerechtigheid", - "δικαιοσύνη", - "υπουργείοδικαιοσύνης", - "δίκαιο", - "adalet", - "igazságügy", - "pravo", - "õigus", - "" - ], - "key::86": [ - "aerospace", - "aerospaziale", - "aerospaziali", - "aeroespacio", - "aeroespaço", - "Luftfahrt", - "luchtvaart", - "ruimtevaart", - "αεροπορικός", - "αεροπορική", - "αεροπορικό", - "αεροναυπηγικός", - "αεροναυπηγική", - "αεροναυπηγικό", - "αεροναυπηγικά", - "havacılıkveuzay", - "légtér", - "zrakoplovstvo", - "atmosfäär", - "kosmos", - "" - ], - "key::87": [ - "dermatology", - "dermatologia", - "dermatología", - "dermatologia", - "Dermatologie", - "dermatologie", - "δρματολογία", - "dermatoloji", - "bőrgyógyászat", - "dermatológia", - "dermatologija", - "dermatoloogia", - "" - ], - "key::88": [ - "architecture", - "architettura", - "arquitectura", - "arquitetura", - "Architektur", - "architectuur", - "αρχιτεκτονική", - "mimarlık", - "építészet", - "arhitektura", - "arhitektuur", - "" - ], - "key::89": [ - "mathematics", - "matematica", - "matematiche", - "matemáticas", - "matemáticas", - "Mathematik", - "wiskunde", - "mathematica", - "μαθηματικά", - "matematik", - "matematika", - "matematika", - "matemaatika", - "" - ], - "key::90": [ - "language", - "lingue", - "linguistica", - "linguistiche", - "lenguaje", - "idioma", - "língua", - "idioma", - "Sprache", - "taal", - "taalkunde", - "γλώσσα", - "dil", - "nyelv", - "jezik", - "keel", - "" - ], - "key::91": [ - "neuroscience", - "neuroscienza", - "neurociencia", - "neurociência", - "Neurowissenschaft", - "neurowetenschappen", - "νευροεπιστήμη", - "nörobilim", - "idegtudomány", - "nevroznanost", - "neuroteadused", - "" - ], - "key::92": [ - "automation", - "automazione", - "automatización", - "automação", - "Automatisierung", - "automatisering", - "αυτοματοποίηση", - "otomasyon", - "automatizálás", - "avtomatizacija", - "automatiseeritud", - "" - ], - "key::93": [ - "pediatric", - "pediatria", - "pediatriche", - "pediatrico", - "pediátrico", - "pediatría", - "pediátrico", - "pediatria", - "pädiatrisch", - "pediatrische", - "παιδιατρική", - "pediatrik", - "gyermekgyógyászat", - "pediatrija", - "pediaatria", - "" - ], - "key::94": [ - "photonics", - "fotonica", - "fotoniche", - "fotónica", - "fotônica", - "Photonik", - "fotonica", - "φωτονική", - "fotonik", - "fotonika", - "fotonika", - "fotoonika", - "" - ], - "key::95": [ - "mechanics", - "meccanica", - "meccaniche", - "mecánica", - "mecânica", - "Mechanik", - "Maschinenbau", - "mechanica", - "werktuigkunde", - "μηχανικής", - "mekanik", - "gépészet", - "mehanika", - "mehaanika", - "" - ], - "key::96": [ - "psychiatrics", - "psichiatria", - "psichiatrica", - "psichiatriche", - "psiquiatría", - "psiquiatria", - "Psychiatrie", - "psychiatrie", - "ψυχιατρική", - "psikiyatrik", - "pszihiátria", - "psihiatrija", - "psühhaatria", - "" - ], - "key::97": [ - "psychology", - "fisiologia", - "psicología", - "psicologia", - "Psychologie", - "psychologie", - "ψυχολογία", - "psikoloji", - "pszihológia", - "psihologija", - "psühholoogia", - "" - ], - "key::98": [ - "automotive", - "industriaautomobilistica", - "industriadelautomóvil", - "automotriz", - "industriaautomotriz", - "automotivo", - "Automobilindustrie", - "autoindustrie", - "αυτοκίνητος", - "αυτοκίνητη", - "αυτοκίνητο", - "αυτοκινούμενος", - "αυτοκινούμενη", - "αυτοκινούμενο", - "αυτοκινητιστικός", - "αυτοκινητιστική", - "αυτοκινητιστικό", - "otomotiv", - "autóipari", - "samogiben", - "avtomobilskaindustrija", - "auto-", - "" - ], - "key::99": [ - "neurology", - "neurologia", - "neurologiche", - "neurología", - "neurologia", - "Neurologie", - "neurologie", - "zenuwleer", - "νευρολογία", - "nöroloji", - "neurológia", - "ideggyógyászat", - "nevrologija", - "neuroloogia", - "" - ], - "key::100": [ - "geology", - "geologia", - "geologiche", - "geología", - "geologia", - "Geologie", - "geologie", - "aardkunde", - "γεωλογία", - "jeoloji", - "geológia", - "földtudomány", - "geologija", - "geoloogia", - "" - ], - "key::101": [ - "microbiology", - "microbiologia", - "micro-biologia", - "microbiologiche", - "microbiología", - "microbiologia", - "Mikrobiologie", - "microbiologie", - "μικροβιολογία", - "mikrobiyoloji", - "mikrobiológia", - "mikrobiologija", - "mikrobioloogia", - "" - ], - "key::102": [ - "informatics", - "informatica", - "informática", - "informática", - "informatica", - "" - ], - "key::103": [ - "forschungsgemeinschaft", - "comunita ricerca", - "research community", - "research foundation", - "research association" - ], - "key::104": [ - "commerce", - "ticaret", - "ticarət", - "commercio", - "trade", - "handel", - "comercio" - ] + "key::1": ["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti", "universiti"], + "key::2": ["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"], + "key::3": ["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"], + "key::4": ["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"], + "key::5": ["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"], + "key::6": ["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"], + "key::7": ["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"], + "key::8": ["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"], + "key::9": ["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"], + "key::10": ["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"], + "key::11": ["association","associazione","association","asociación","associação","Verein","verband","stowarzyszenie","ассоциация","associatie"], + "key::12": ["society","societa","société","sociedad","sociedade","gesellschaft","społeczeństwo","общество","maatschappij","κοινωνία"], + "key::13": ["international","internazionale","international","internacional","internacional","international","międzynarodowy","Международный","internationaal","internationale","διεθνής","διεθνή","διεθνές"], + "key::14": ["community","comunita","communauté","comunidad","comunidade","Gemeinschaft","społeczność","сообщество","gemeenschap","κοινότητα"], + "key::15": ["school","scuola","école","escuela","escola","schule","Szkoła","школа","school","σχολείο"], + "key::16": ["education","educazione","éducation","educacion","Educação","Bildung","Edukacja","образование","opleiding","εκπαίδευση"], + "key::17": ["academy","accademia","académie","academia","academia","Akademie","akademie","академия","academie","ακαδημία"], + "key::18": ["public","pubblico","public","publique","publics","publiques","publico","publico","Öffentlichkeit","publiczny","публичный","publiek","publieke","δημόσιος","δημόσια","δημόσιο"], + "key::19": ["museum","museo","musée","mueso","museu","museum","muzeum","музей","museum","μουσείο"], + "key::20": ["group","gruppo","groupe","grupo","grupo","gruppe","grupa","группа","groep","ομάδα","όμιλος"], + "key::21": ["department","dipartimento","département","departamento","departamento","abteilung","departament","отдел","afdeling","τμήμα"], + "key::22": ["council","consiglio","conseil","Consejo","conselho","gesellschaft","rada","совет","raad","συμβούλιο"], + "key::23": ["library","biblioteca","bibliothèque","biblioteca","biblioteca","Bibliothek","biblioteka","библиотека","bibliotheek","βιβλιοθήκη"], + "key::24": ["ministry","ministero","ministère","ministerio","ministério","Ministerium","ministerstwo","министерство","ministerie","υπουργείο"], + "key::25": ["services","servizi","services","servicios","Serviços","Dienstleistungen","usługi","услуги","diensten","υπηρεσίες"], + "key::26": ["central","centrale","central","centrale","centrales","central","central","zentral","centralny","цетральный","centraal","κεντρικός","κεντρική","κεντρικό","κεντρικά"], + "key::27": ["general","generale","général","générale","généraux","générales","general","geral","general","Allgemeines","general","общий","algemeen","algemene","γενικός","γενική","γενικό","γενικά"], + "key::28": ["applied","applicati","appliqué","appliquée","appliqués","appliquées","aplicado","aplicada","angewendet","stosowany","прикладной","toegepast","toegepaste","εφαρμοσμένος","εφαρμοσμένη","εφαρμοσμένο","εφαρμοσμένα"], + "key::29": ["european","europee","europea","européen","européenne","européens","européennes","europeo","europeu","europäisch","europejski","европейский","Europees","Europese","ευρωπαϊκός","ευρωπαϊκή","ευρωπαϊκό","ευρωπαϊκά"], + "key::30": ["agency","agenzia","agence","agencia","agencia","agentur","agencja","агенция","agentschap","πρακτορείο"], + "key::31": ["laboratory","laboratorio","laboratoire","laboratorio","laboratorio","labor","laboratorium","лаборатория","laboratorium","εργαστήριο"], + "key::32": ["industry","industria","industrie","индустрия","industrie","βιομηχανία"], + "key::33": ["industrial","industriale","industriel","industrielle","industriels","industrielles","индустриальный","industrieel","βιομηχανικός","βιομηχανική","βιομηχανικό","βιομηχανικά","βιομηχανικές"], + "key::34": ["consortium","consorzio","consortium","консорциум","consortium","κοινοπραξία"], + "key::35": ["organization","organizzazione","organisation","organización","organização","organizacja","организация","organisatie","οργανισμός"], + "key::36": ["authority","autorità","autorité","авторитет","autoriteit"], + "key::37": ["federation","federazione","fédération","федерация","federatie","ομοσπονδία"], + "key::38": ["observatory","osservatorio","observatoire","обсерватория","observatorium","αστεροσκοπείο"], + "key::39": ["bureau","ufficio","bureau","офис","bureau","γραφείο"], + "key::40": ["company","impresa","compagnie","société","компания","bedrijf","εταιρία"], + "key::41": ["polytechnic","politecnico","polytechnique","политехника","polytechnisch","πολυτεχνείο","universita politecnica","polytechnic university","universidad politecnica","universitat politecnica","politechnika","politechniki","university technology","university science technology"], + "key::42": ["coalition","coalizione","coalition","коалиция","coalitie","συνασπισμός"], + "key::43": ["initiative","iniziativa","initiative","инициатива","initiatief","πρωτοβουλία"], + "key::44": ["academic","accademico","académique","universitaire","акадеческий academisch","ακαδημαϊκός","ακαδημαϊκή","ακαδημαϊκό","ακαδημαϊκές","ακαδημαϊκοί"], + "key::45": ["institution","istituzione","institution","институциональный","instelling","ινστιτούτο"], + "key::46": ["division","divisione","division","отделение","divisie","τμήμα"], + "key::47": ["committee","comitato","comité","комитет","commissie","επιτροπή"], + "key::48": ["promotion","promozione","продвижение","proothisis","forderung"], + "key::49": ["medical","medicine","clinical","medicina","clinici","médico","medicina","clínica","médico","medicina","clínica","medizinisch","Medizin","klinisch","medisch","geneeskunde","klinisch","ιατρικός","ιατρική","ιατρικό","ιατρικά","κλινικός","κλινική","κλινικό","κλινικά","tıbbi","tıp","klinik","orvosi","orvostudomány","klinikai","zdravniški","medicinski","klinični","meditsiini","kliinik","kliiniline"], + "key::50": ["technology","technological","tecnologia","tecnologie","tecnología","tecnológico","tecnologia","tecnológico","Technologie","technologisch","technologie","technologisch","τεχνολογία","τεχνολογικός","τεχνολογική","τεχνολογικό","teknoloji","teknolojik","technológia","technológiai","tehnologija","tehnološki","tehnoloogia","tehnoloogiline","technologii","technical","texniki","teknik"], + "key::51": ["science","scientific","scienza","scientifiche","scienze","ciencia","científico","ciência","científico","Wissenschaft","wissenschaftlich","wetenschap","wetenschappelijk","επιστήμη","επιστημονικός","επιστημονική","επιστημονικό","επιστημονικά","bilim","bilimsel","tudomány","tudományos","znanost","znanstveni","teadus","teaduslik",""], + "key::52": ["engineering","ingegneria","ingeniería","engenharia","Ingenieurwissenschaft","ingenieurswetenschappen","bouwkunde","μηχανικός","μηχανική","μηχανικό","mühendislik","mérnöki","Inženirstvo","inseneeria","inseneri",""], + "key::53": ["management","gestione","gestionale","gestionali","gestión","administración","gestão","administração","Verwaltung","management","διαχείριση","yönetim","menedzsment","vodstvo","upravljanje","management","juhtkond","juhtimine","haldus",""], + "key::54": ["energy","energia","energía","energia","Energie","energie","ενέργεια","enerji","energia","energija","energia",""], + "key::55": ["agricultural","agriculture","agricoltura","agricole","agrícola","agricultura","agrícola","agricultura","landwirtschaftlich","Landwirtschaft","landbouwkundig","landbouw","αγροτικός","αγροτική","αγροτικό","γεωργικός","γεωργική","γεωργικό","γεωργία","tarımsal","tarım","mezőgazdasági","mezőgazdaság","poljedelski","poljedelstvo","põllumajandus","põllumajanduslik",""], + "key::56": ["information","informazione","información","informação","Information","informatie","πληροφορία","bilgi","információ","informacija","informatsioon","informatycznych",""], + "key::57": ["social","sociali","social","social","Sozial","sociaal","maatschappelijk","κοινωνικός","κοινωνική","κοινωνικό","κοινωνικά","sosyal","szociális","družbeni","sotsiaal","sotsiaalne",""], + "key::58": ["environmental","ambiente","medioambiental","ambiente","medioambiente","meioambiente","Umwelt","milieu","milieuwetenschap","milieukunde","περιβαλλοντικός","περιβαλλοντική","περιβαλλοντικό","περιβαλλοντικά","çevre","környezeti","okoliški","keskonna",""], + "key::59": ["business","economia","economiche","economica","negocio","empresa","negócio","Unternehmen","bedrijf","bedrijfskunde","επιχείρηση","iş","üzleti","posel","ettevõte/äri",""], + "key::60": ["pharmaceuticals","pharmacy","farmacia","farmaceutica","farmacéutica","farmacia","farmacêutica","farmácia","Pharmazeutika","Arzneimittelkunde","farmaceutica","geneesmiddelen","apotheek","φαρμακευτικός","φαρμακευτική","φαρμακευτικό","φαρμακευτικά","φαρμακείο","ilaç","eczane","gyógyszerészeti","gyógyszertár","farmacevtika","lekarništvo","farmaatsia","farmatseutiline",""], + "key::61": ["healthcare","health services","salute","atenciónmédica","cuidadodelasalud","cuidadoscomasaúde","Gesundheitswesen","gezondheidszorg","ιατροφαρμακευτικήπερίθαλψη","sağlıkhizmeti","egészségügy","zdravstvo","tervishoid","tervishoiu",""], + "key::62": ["history","storia","historia","história","Geschichte","geschiedenis","geschiedkunde","ιστορία","tarih","történelem","zgodovina","ajalugu",""], + "key::63": ["materials","materiali","materia","materiales","materiais","materialen","υλικά","τεκμήρια","malzemeler","anyagok","materiali","materjalid","vahendid",""], + "key::64": ["economics","economia","economiche","economica","economía","economia","Wirtschaft","economie","οικονομικά","οικονομικέςεπιστήμες","ekonomi","közgazdaságtan","gospodarstvo","ekonomija","majanduslik","majandus",""], + "key::65": ["therapeutics","terapeutica","terapéutica","terapêutica","therapie","θεραπευτική","tedavibilimi","gyógykezelés","terapevtika","terapeutiline","ravi",""], + "key::66": ["oncology","oncologia","oncologico","oncología","oncologia","Onkologie","oncologie","ογκολογία","onkoloji","onkológia","onkologija","onkoloogia",""], + "key::67": ["natural","naturali","naturale","natural","natural","natürlich","natuurlijk","φυσικός","φυσική","φυσικό","φυσικά","doğal","természetes","naraven","loodus",""], + "key::68": ["educational","educazione","pedagogia","educacional","educativo","educacional","pädagogisch","educatief","εκπαιδευτικός","εκπαιδευτική","εκπαιδευτικό","εκπαιδευτικά","eğitimsel","oktatási","izobraževalen","haridus","hariduslik",""], + "key::69": ["biomedical","biomedica","biomédico","biomédico","biomedizinisch","biomedisch","βιοιατρικός","βιοιατρική","βιοιατρικό","βιοιατρικά","biyomedikal","orvosbiológiai","biomedicinski","biomeditsiiniline",""], + "key::70": ["veterinary","veterinaria","veterinarie","veterinaria","veterinária","tierärtzlich","veterinair","veeartsenijlkunde","κτηνιατρικός","κτηνιατρική","κτηνιατρικό","κτηνιατρικά","veteriner","állatorvosi","veterinar","veterinarski","veterinaaria",""], + "key::71": ["chemistry","chimica","química","química","Chemie","chemie","scheikunde","χημεία","kimya","kémia","kemija","keemia",""], + "key::72": ["security","sicurezza","seguridad","segurança","Sicherheit","veiligheid","ασφάλεια","güvenlik","biztonsági","varnost","turvalisus","julgeolek",""], + "key::73": ["biotechnology","biotecnologia","biotecnologie","biotecnología","biotecnologia","Biotechnologie","biotechnologie","βιοτεχνολογία","biyoteknoloji","biotechnológia","biotehnologija","biotehnoloogia",""], + "key::74": ["military","militare","militari","militar","militar","Militär","militair","leger","στρατιωτικός","στρατιωτική","στρατιωτικό","στρατιωτικά","askeri","katonai","vojaški","vojni","militaar","wojskowa",""], + "key::75": ["theological","teologia","teologico","teológico","tecnológica","theologisch","theologisch","θεολογικός","θεολογική","θεολογικό","θεολογικά","teolojik","technológiai","teološki","teoloogia","usuteadus","teoloogiline",""], + "key::76": ["electronics","elettronica","electrónica","eletrônicos","Elektronik","elektronica","ηλεκτρονική","elektronik","elektronika","elektronika","elektroonika",""], + "key::77": ["forestry","forestale","forestali","silvicultura","forestal","floresta","Forstwirtschaft","bosbouw","δασοκομία","δασολογία","ormancılık","erdészet","gozdarstvo","metsandus",""], + "key::78": ["maritime","marittima","marittime","marittimo","marítimo","marítimo","maritiem","ναυτικός","ναυτική","ναυτικό","ναυτικά","ναυτιλιακός","ναυτιλιακή","ναυτιλιακό","ναυτιλιακά","θαλάσσιος","θαλάσσια","θαλάσσιο","denizcilik","tengeri","morski","mere","merendus",""], + "key::79": ["sports","sport","deportes","esportes","Sport","sport","sportwetenschappen","άθληση","γυμναστικήδραστηριότητα","spor","sport","šport","sport","spordi",""], + "key::80": ["surgery","chirurgia","chirurgiche","cirugía","cirurgia","Chirurgie","chirurgie","heelkunde","εγχείρηση","επέμβαση","χειρουργικήεπέμβαση","cerrahi","sebészet","kirurgija","kirurgia",""], + "key::81": ["cultural","culturale","culturali","cultura","cultural","cultural","kulturell","cultureel","πολιτιστικός","πολιτιστική","πολιτιστικό","πολιτισμικός","πολιτισμική","πολιτισμικό","kültürel","kultúrális","kulturni","kultuuri","kultuuriline",""], + "key::82": ["computerscience","informatica","ordenador","computadora","informática","computación","cienciasdelacomputación","ciênciadacomputação","Computer","computer","υπολογιστής","ηλεκτρονικόςυπολογιστής","bilgisayar","számítógép","računalnik","arvuti",""], + "key::83": ["finance","financial","finanza","finanziarie","finanza","financiero","finanças","financeiro","Finanzen","finanziell","financiën","financieel","χρηματοοικονομικά","χρηματοδότηση","finanse","finansal","pénzügy","pénzügyi","finance","finančni","finants","finantsiline",""], + "key::84": ["communication","comunicazione","comuniciación","comunicação","Kommunikation","communication","επικοινωνία","iletişim","kommunikáció","komuniciranje","kommunikatsioon",""], + "key::85": ["justice","giustizia","justicia","justiça","Recht","Justiz","justitie","gerechtigheid","δικαιοσύνη","υπουργείοδικαιοσύνης","δίκαιο","adalet","igazságügy","pravo","õigus",""], + "key::86": ["aerospace","aerospaziale","aerospaziali","aeroespacio","aeroespaço","Luftfahrt","luchtvaart","ruimtevaart","αεροπορικός","αεροπορική","αεροπορικό","αεροναυπηγικός","αεροναυπηγική","αεροναυπηγικό","αεροναυπηγικά","havacılıkveuzay","légtér","zrakoplovstvo","atmosfäär","kosmos",""], + "key::87": ["dermatology","dermatologia","dermatología","dermatologia","Dermatologie","dermatologie","δρματολογία","dermatoloji","bőrgyógyászat","dermatológia","dermatologija","dermatoloogia",""], + "key::88": ["architecture","architettura","arquitectura","arquitetura","Architektur","architectuur","αρχιτεκτονική","mimarlık","építészet","arhitektura","arhitektuur",""], + "key::89": ["mathematics","matematica","matematiche","matemáticas","matemáticas","Mathematik","wiskunde","mathematica","μαθηματικά","matematik","matematika","matematika","matemaatika",""], + "key::90": ["language","lingue","linguistica","linguistiche","lenguaje","idioma","língua","idioma","Sprache","taal","taalkunde","γλώσσα","dil","nyelv","jezik","keel",""], + "key::91": ["neuroscience","neuroscienza","neurociencia","neurociência","Neurowissenschaft","neurowetenschappen","νευροεπιστήμη","nörobilim","idegtudomány","nevroznanost","neuroteadused",""], + "key::92": ["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""], + "key::93": ["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""], + "key::94": ["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""], + "key::95": ["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""], + "key::96": ["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""], + "key::97": ["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""], + "key::98": ["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""], + "key::99": ["neurology","neurologia","neurologiche","neurología","neurologia","Neurologie","neurologie","zenuwleer","νευρολογία","nöroloji","neurológia","ideggyógyászat","nevrologija","neuroloogia",""], + "key::100": ["geology","geologia","geologiche","geología","geologia","Geologie","geologie","aardkunde","γεωλογία","jeoloji","geológia","földtudomány","geologija","geoloogia",""], + "key::101": ["microbiology","microbiologia","micro-biologia","microbiologiche","microbiología","microbiologia","Mikrobiologie","microbiologie","μικροβιολογία","mikrobiyoloji","mikrobiológia","mikrobiologija","mikrobioloogia",""], + "key::102": ["informatics","informatica","informática","informática","informatica",""], + "key::103": ["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"], + "key::104": ["commerce","ticaret","ticarət","commercio","trade","handel","comercio"], + "key::105" : ["state", "stato", "etade", "statale", "etat", "zustand", "estado"] } } } \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json new file mode 100644 index 000000000..568b0e962 --- /dev/null +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json @@ -0,0 +1,334 @@ +{ + "wf" : { + "threshold" : "0.99", + "dedupRun" : "001", + "entityType" : "result", + "subEntityType" : "resulttype", + "subEntityValue" : "publication", + "orderField" : "title", + "queueMaxSize" : "2000", + "groupMaxSize" : "100", + "maxChildren" : "100", + "idPath": "$.id", + "slidingWindowSize" : "200", + "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_isAffiliatedWith", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], + "includeChildren" : "true" + }, + "pace" : { + "clustering" : [ + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } + ], + + "decisionTree" : { + "start": { + "fields": [ + { + "field": "pid", + "comparator": "pidMatch", + "weight": 1, + "countIfUndefined": "false", + "params": {} + } + ], + "threshold": 1, + "aggregation": "SC", + "positive": "MATCH", + "negative": "NO_MATCH", + "undefined": "layer2", + "ignoreUndefined": "false" + }, + "layer2": { + "fields": [ + { + "field": "title", + "comparator": "titleVersionMatch", + "weight": 1, + "countIfUndefined": "true", + "params": {} + }, + { + "field": "authors", + "comparator": "sizeMatch", + "weight": 1, + "countIfUndefined": "false", + "params": {} + } + ], + "threshold": 1, + "aggregation": "NC", + "positive": "layer3", + "negative": "NO_MATCH", + "undefined": "layer3", + "ignoreUndefined": "false" + }, + "layer3": { + "fields": [ + { + "field": "title", + "comparator": "LevensteinTitle", + "weight": 1, + "countIfUndefined": "true", + "params": {} + } + ], + "threshold": 0.99, + "aggregation": "W_MEAN", + "positive": "MATCH", + "negative": "NO_MATCH", + "undefined": "NO_MATCH", + "ignoreUndefined": "false" + } + }, + "model" : [ + { "name" : "doi", "type" : "String", "path" : "$.pid[?(@.qualifier.classid ==\"doi\")].value" }, + { "name" : "pid", "type" : "JSON","path" : "$.pid", "overrideMatch" : "true" }, + { "name" : "title", "type" : "String", "path" : "$.title[?(@.qualifier.classid ==\"main title\")].value", "length" : 250, "size" : 5 }, + { "name" : "authors", "type" : "List", "path" : "$.author[*].fullname", "size" : 200 }, + { "name" : "resulttype", "type" : "String", "path" : "$.resulttype.classid" } + ], + "synonyms": {}, + "blacklists" : { + "title" : [ + "^Inside Front Cover$", + "(?i)^Poster presentations$", + "^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$", + "^Problems with perinatal pathology\\.?$", + "(?i)^Cases? of Puerperal Convulsions$", + "(?i)^Operative Gyna?ecology$", + "(?i)^Mind the gap\\!?\\:?$", + "^Chronic fatigue syndrome\\.?$", + "^Cartas? ao editor Letters? to the Editor$", + "^Note from the Editor$", + "^Anesthesia Abstract$", + + "^Annual report$", + "(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$", + "(?i)^Graph and Table of Infectious Diseases?$", + "^Presentation$", + "(?i)^Reviews and Information on Publications$", + "(?i)^PUBLIC HEALTH SERVICES?$", + "(?i)^COMBINED TEXT-?BOOK OF OBSTETRICS AND GYN(Æ|ae)COLOGY$", + "(?i)^Adrese autora$", + "(?i)^Systematic Part .*\\. Catalogus Fossilium Austriae, Band 2: Echinoidea neogenica$", + "(?i)^Acknowledgement to Referees$", + "(?i)^Behçet's disease\\.?$", + "(?i)^Isolation and identification of restriction endonuclease.*$", + "(?i)^CEREBROVASCULAR DISEASES?.?$", + "(?i)^Screening for abdominal aortic aneurysms?\\.?$", + "^Event management$", + "(?i)^Breakfast and Crohn's disease.*\\.?$", + "^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$", + "(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$", + "^Gushi hakubutsugaku$", + + "^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$", + "^Intestinal spirocha?etosis$", + "^Treatment of Rodent Ulcer$", + "(?i)^\\W*Cloud Computing\\W*$", + "^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$", + "^Free Communications, Poster Presentations: Session [A-F]$", + + "^“The Historical Aspects? of Quackery\\.?”$", + "^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$", + "^P(er|re)-Mile Premiums for Auto Insurance\\.?$", + "(?i)^Case Report$", + "^Boletín Informativo$", + "(?i)^Glioblastoma Multiforme$", + "(?i)^Nuevos táxones animales descritos en la península Ibérica y Macaronesia desde 1994 \\(.*\\)$", + "^Zaměstnanecké výhody$", + "(?i)^The Economics of Terrorism and Counter-Terrorism: A Survey \\(Part .*\\)$", + "(?i)^Carotid body tumours?\\.?$", + "(?i)^\\[Españoles en Francia : La condición Emigrante.*\\]$", + "^Avant-propos$", + "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Head(s)? and Capital(s)?$", + "(?i)^St\\. Patrick's Cathedral, Dublin, County Dublin - Bases?$", + "(?i)^PUBLIC HEALTH VERSUS THE STATE$", + "^Viñetas de Cortázar$", + "(?i)^Search for heavy neutrinos and W(\\[|_|\\(|_\\{|-)?R(\\]|\\)|\\})? bosons with right-handed couplings in a left-right symmetric model in pp collisions at.*TeV(\\.)?$", + "(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$", + "(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$", + "(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$", + + "(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$", + "^Aus der AGMB$", + + "^Znanstveno-stručni prilozi$", + "(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$", + "(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$", + "(?i)^Hodnocení finanční situace podniku a návrhy na její zlepšení$", + "^Finanční analýza podniku$", + "^Financial analysis( of business)?$", + "(?i)^Textbook of Gyn(a)?(Æ)?(e)?cology$", + "^Jikken nihon shūshinsho$", + "(?i)^CORONER('|s)(s|') INQUESTS$", + "(?i)^(Μελέτη παραγόντων )?risk management( για ανάπτυξη και εφαρμογή ενός πληροφοριακού συστήματος| και ανάπτυξη συστήματος)?$", + "(?i)^Consultants' contract(s)?$", + "(?i)^Upute autorima$", + "(?i)^Bijdrage tot de Kennis van den Godsdienst der Dajaks van Lan(d|f)ak en Tajan$", + "^Joshi shin kokubun$", + "^Kōtō shōgaku dokuhon nōson'yō$", + "^Jinjō shōgaku shōka$", + "^Shōgaku shūjichō$", + "^Nihon joshi dokuhon$", + "^Joshi shin dokuhon$", + "^Chūtō kanbun dokuhon$", + "^Wabun dokuhon$", + "(?i)^(Analysis of economy selected village or town|Rozbor hospodaření vybrané obce či města)$", + "(?i)^cardiac rehabilitation$", + "(?i)^Analytical summary$", + "^Thesaurus resolutionum Sacrae Congregationis Concilii$", + "(?i)^Sumario analítico(\\s{1})?(Analitic summary)?$", + "^Prikazi i osvrti$", + "^Rodinný dům s provozovnou$", + "^Family house with an establishment$", + "^Shinsei chūtō shin kokugun$", + "^Pulmonary alveolar proteinosis(\\.?)$", + "^Shinshū kanbun$", + "^Viñeta(s?) de Rodríguez$", + "(?i)^RUBRIKA UREDNIKA$", + "^A Matching Model of the Academic Publication Market$", + "^Yōgaku kōyō$", + + "^Internetový marketing$", + "^Internet marketing$", + "^Chūtō kokugo dokuhon$", + "^Kokugo dokuhon$", + "^Antibiotic Cover for Dental Extraction(s?)$", + "^Strategie podniku$", + "^Strategy of an Enterprise$", + "(?i)^respiratory disease(s?)(\\.?)$", + "^Award(s?) for Gallantry in Civil Defence$", + "^Podniková kultura$", + "^Corporate Culture$", + "^Severe hyponatraemia in hospital inpatient(s?)(\\.?)$", + "^Pracovní motivace$", + "^Work Motivation$", + "^Kaitei kōtō jogaku dokuhon$", + "^Konsolidovaná účetní závěrka$", + "^Consolidated Financial Statements$", + "(?i)^intracranial tumour(s?)$", + "^Climate Change Mitigation Options and Directed Technical Change: A Decentralized Equilibrium Analysis$", + "^\\[CERVECERIAS MAHOU(\\.|\\:) INTERIOR\\] \\[Material gráfico\\]$", + "^Housing Market Dynamics(\\:|\\.) On the Contribution of Income Shocks and Credit Constraint(s?)$", + "^\\[Funciones auxiliares de la música en Radio París,.*\\]$", + "^Úroveň motivačního procesu jako způsobu vedení lidí$", + "^The level of motivation process as a leadership$", + "^Pay-beds in N(\\.?)H(\\.?)S(\\.?) Hospitals$", + "(?i)^news and events$", + "(?i)^NOVOSTI I DOGAĐAJI$", + "^Sansū no gakushū$", + "^Posouzení informačního systému firmy a návrh změn$", + "^Information System Assessment and Proposal for ICT Modification$", + "^Stresové zatížení pracovníků ve vybrané profesi$", + "^Stress load in a specific job$", + + "^Sunday: Poster Sessions, Pt.*$", + "^Monday: Poster Sessions, Pt.*$", + "^Wednesday: Poster Sessions, Pt.*", + "^Tuesday: Poster Sessions, Pt.*$", + + "^Analýza reklamy$", + "^Analysis of advertising$", + + "^Shōgaku shūshinsho$", + "^Shōgaku sansū$", + "^Shintei joshi kokubun$", + "^Taishō joshi kokubun dokuhon$", + "^Joshi kokubun$", + + "^Účetní uzávěrka a účetní závěrka v ČR$", + "(?i)^The \"?Causes\"? of Cancer$", + "^Normas para la publicación de artículos$", + "^Editor('|s)(s|') [Rr]eply$", + "^Editor(’|s)(s|’) letter$", + "^Redaktoriaus žodis$", + "^DISCUSSION ON THE PRECEDING PAPER$", + "^Kōtō shōgaku shūshinsho jidōyō$", + "^Shōgaku nihon rekishi$", + "^(Theory of the flow of action currents in isolated myelinated nerve fibers).*$", + "^Préface$", + "^Occupational [Hh]ealth [Ss]ervices.$", + "^In Memoriam Professor Toshiyuki TAKESHIMA$", + "^Účetní závěrka ve vybraném podniku.*$", + "^Financial statements in selected company$", + "^Abdominal [Aa]ortic [Aa]neurysms.*$", + "^Pseudomyxoma peritonei$", + "^Kazalo autora$", + + "(?i)^uvodna riječ$", + "^Motivace jako způsob vedení lidí$", + "^Motivation as a leadership$", + "^Polyfunkční dům$", + "^Multi\\-funkcional building$", + "^Podnikatelský plán$", + "(?i)^Podnikatelský záměr$", + "(?i)^Business Plan$", + "^Oceňování nemovitostí$", + "^Marketingová komunikace$", + "^Marketing communication$", + "^Sumario Analítico$", + "^Riječ uredništva$", + "^Savjetovanja i priredbe$", + "^Índice$", + "^(Starobosanski nadpisi).*$", + "^Vzdělávání pracovníků v organizaci$", + "^Staff training in organization$", + "^(Life Histories of North American Geometridae).*$", + "^Strategická analýza podniku$", + "^Strategic Analysis of an Enterprise$", + "^Sadržaj$", + "^Upute suradnicima$", + "^Rodinný dům$", + "(?i)^Fami(l)?ly house$", + "^Upute autorima$", + "^Strategic Analysis$", + "^Finanční analýza vybraného podniku$", + "^Finanční analýza$", + "^Riječ urednika$", + "(?i)^Content(s?)$", + "(?i)^Inhalt$", + "^Jinjō shōgaku shūshinsho jidōyō$", + "(?i)^Index$", + "^Chūgaku kokubun kyōkasho$", + "^Retrato de una mujer$", + "^Retrato de un hombre$", + "^Kōtō shōgaku dokuhon$", + "^Shotōka kokugo$", + "^Shōgaku dokuhon$", + "^Jinjō shōgaku kokugo dokuhon$", + "^Shinsei kokugo dokuhon$", + "^Teikoku dokuhon$", + "^Instructions to Authors$", + "^KİTAP TAHLİLİ$", + "^PRZEGLĄD PIŚMIENNICTWA$", + "(?i)^Presentación$", + "^İçindekiler$", + "(?i)^Tabl?e of contents$", + "^(CODICE DEL BEATO DE LOS REYES FERNANDO I Y SANCHA).*$", + "^(\\[MADRID\\. BIBL\\. NAC\\. N.*KING FERDINAND I.*FROM SAN ISIDORO DE LEON\\. FACUNDUS SCRIPSIT DATED.*\\]).*", + "^Editorial( Board)?$", + "(?i)^Editorial \\(English\\)$", + "^Editörden$", + "^(Corpus Oral Dialectal \\(COD\\)\\.).*$", + "^(Kiri Karl Morgensternile).*$", + "^(\\[Eksliibris Aleksandr).*\\]$", + "^(\\[Eksliibris Aleksandr).*$", + "^(Eksliibris Aleksandr).*$", + "^(Kiri A\\. de Vignolles).*$", + "^(2 kirja Karl Morgensternile).*$", + "^(Pirita kloostri idaosa arheoloogilised).*$", + "^(Kiri tundmatule).*$", + "^(Kiri Jenaer Allgemeine Literaturzeitung toimetusele).*$", + "^(Eksliibris Nikolai Birukovile).*$", + "^(Eksliibris Nikolai Issakovile).*$", + "^(WHP Cruise Summary Information of section).*$", + "^(Measurement of the top quark\\-pair production cross section with ATLAS in pp collisions at).*$", + "^(Measurement of the spin\\-dependent structure function).*", + "(?i)^.*authors['’′]? reply\\.?$", + "(?i)^.*authors['’′]? response\\.?$" + ] + } + } +} \ No newline at end of file From dd21db7036d23b1d8ae6132f9b39335e7006f65c Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Wed, 18 Dec 2019 16:28:22 +0100 Subject: [PATCH 09/13] fixed stuff --- .../eu/dnetlib/dedup/DedupRecordFactory.java | 14 ++ .../main/java/eu/dnetlib/dedup/Deduper.java | 13 +- .../dedup/SparkCreateConnectedComponent.java | 21 ++- .../dnetlib/dhp/dedup/oozie_app/workflow.xml | 4 +- .../dnetlib/dedup/SparkCreateDedupTest.java | 31 ++++ .../dnetlib/dedup/conf/pub_dt.curr.conf.json | 169 ++++++++++++------ 6 files changed, 176 insertions(+), 76 deletions(-) diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index ed3ac8231..5f81669e9 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -91,6 +91,8 @@ public class DedupRecordFactory { } }); p.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (p.getDataInfo() == null) + p.setDataInfo(new DataInfo()); p.getDataInfo().setTrust("0.9"); p.setLastupdatetimestamp(ts); return p; @@ -121,6 +123,8 @@ public class DedupRecordFactory { } }); d.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (d.getDataInfo() == null) + d.setDataInfo(new DataInfo()); d.getDataInfo().setTrust("0.9"); d.setLastupdatetimestamp(ts); return d; @@ -143,6 +147,8 @@ public class DedupRecordFactory { throw new RuntimeException(exc); } }); + if (p.getDataInfo() == null) + p.setDataInfo(new DataInfo()); p.getDataInfo().setTrust("0.9"); p.setLastupdatetimestamp(ts); return p; @@ -170,6 +176,8 @@ public class DedupRecordFactory { } }); s.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (s.getDataInfo() == null) + s.setDataInfo(new DataInfo()); s.getDataInfo().setTrust("0.9"); s.setLastupdatetimestamp(ts); return s; @@ -189,6 +197,8 @@ public class DedupRecordFactory { throw new RuntimeException(exc); } }); + if (d.getDataInfo() == null) + d.setDataInfo(new DataInfo()); d.getDataInfo().setTrust("0.9"); d.setLastupdatetimestamp(ts); return d; @@ -226,6 +236,8 @@ public class DedupRecordFactory { { o.setDataInfo(new DataInfo()); } + if (o.getDataInfo() == null) + o.setDataInfo(new DataInfo()); o.getDataInfo().setTrust("0.9"); o.setLastupdatetimestamp(ts); @@ -256,6 +268,8 @@ public class DedupRecordFactory { throw new RuntimeException(exc); } }); + if (o.getDataInfo() == null) + o.setDataInfo(new DataInfo()); o.setDateofacceptance(DatePicker.pick(dateofacceptance)); o.getDataInfo().setTrust("0.9"); o.setLastupdatetimestamp(ts); diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java index 51b991da5..7206f892f 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/Deduper.java @@ -119,7 +119,7 @@ public class Deduper implements Serializable { .reduceByKey((Function2, List, List>) (v1, v2) -> { v1.addAll(v2); v1.sort(Comparator.comparing(a -> a.getFieldMap().get(of).stringValue())); - if (v1.size()> maxQueueSize) + if (v1.size() > maxQueueSize) return new ArrayList<>(v1.subList(0, maxQueueSize)); return v1; }); @@ -146,10 +146,13 @@ public class Deduper implements Serializable { Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); return blocks.flatMapToPair((PairFlatMapFunction>, String, String>) it -> { - final SparkReporter reporter = new SparkReporter(accumulators); - new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter); - return reporter.getRelations().iterator(); - + try { + final SparkReporter reporter = new SparkReporter(accumulators); + new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter); + return reporter.getRelations().iterator(); + } catch (Exception e) { + throw new RuntimeException(it._2().get(0).getIdentifier(), e); + } }).mapToPair( (PairFunction, String, Tuple2>) item -> new Tuple2>(item._1() + item._2(), item)) diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index 9783e93d6..16e112c25 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -1,5 +1,9 @@ package eu.dnetlib.dedup; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; import eu.dnetlib.dedup.graph.ConnectedComponent; import eu.dnetlib.dedup.graph.GraphProcessor; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -43,18 +47,12 @@ public class SparkCreateConnectedComponent { final JavaPairRDD vertexes = sc.textFile(inputPath + "/" + entity) .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) .mapToPair((PairFunction) - s -> new Tuple2((long) s.hashCode(), s) + s -> new Tuple2(getHashcode(s), s) ); final Dataset similarityRelations = spark.read().load(DedupUtility.createSimRelPath(targetPath,entity)).as(Encoders.bean(Relation.class)); - - - final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(it.getSource().hashCode(), it.getTarget().hashCode(), it.getRelClass())).rdd(); - - - final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, 20).toJavaRDD(); - - + final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())).rdd(); + final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()).toJavaRDD(); final Dataset mergeRelation = spark.createDataset(cc.filter(k->k.getDocIds().size()>1).flatMap((FlatMapFunction) c -> c.getDocIds() .stream() @@ -72,9 +70,10 @@ public class SparkCreateConnectedComponent { tmp.add(r); return tmp.stream(); }).iterator()).rdd(), Encoders.bean(Relation.class)); - mergeRelation.write().mode("overwrite").save(DedupUtility.createMergeRelPath(targetPath,entity)); + } - + public static long getHashcode(final String id) { + return Hashing.murmur3_128().hashUnencodedChars(id).asLong(); } } diff --git a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml index 0e75aa072..5a00a5967 100644 --- a/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup/src/main/resources/eu/dnetlib/dhp/dedup/oozie_app/workflow.xml @@ -30,7 +30,7 @@ - + @@ -103,7 +103,7 @@ ${nameNode} yarn-cluster cluster - Create Connected Components + Create Dedup Record eu.dnetlib.dedup.SparkCreateDedupRecord dhp-dedup-${projectVersion}.jar --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index 19522e275..f294b10fe 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -1,6 +1,8 @@ package eu.dnetlib.dedup; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Publication; import org.apache.commons.io.FileUtils; @@ -67,5 +69,34 @@ public class SparkCreateDedupTest { } +// [20|grid________::6031f94bef015a37783268ec1e75f17f, 20|nsf_________::b12be9edf414df8ee66b4c52a2d8da46] +// [20|grid________::672e1e5cef49e68f124d3da5225a7357, 20|grid________::7a402604c3853c7a0af14f88f56bf7e1] +// [20|grid________::2fc05b35e11d915b220a66356053eae2, 20|grid________::b02fb3176eb38f6c572722550c07e7ab] +// [20|grid________::bc86248ab2b8d7955dcaf592ba342262, 20|corda_______::45a8ec964029278fb938805182e247a8] +// [20|doajarticles::74551f800ad1c81a6cd31c5162887b7f, 20|rcuk________::86dc9a83df05a58917f38ca09f814617] +// [20|nsf_________::5e837d8e6444cc298db314ea54ad2f4a, 20|snsf________::7b54715f0ec5c6a0a44672f45d98be8d] +// [20|corda__h2020::7ee7e57bad06b92c1a568dd61e10ba8c, 20|snsf________::2d4a2695221a3ce0c749ee34e064c0b3] +// [20|corda_______::25220a523550176dac9e5432dac43596, 20|grid________::9782f16a46650cbbfaaa2315109507d1] +// [20|nih_________::88c3b664dcc7af9e827f94ac964cd66c, 20|grid________::238d3ac0a7d119d5c8342a647f5245f5] +// [20|rcuk________::0582c20fcfb270f9ec1b19b0f0dcd881, 20|nsf_________::9afa48ddf0bc2cd4f3c41dc41daabcdb] +// [20|rcuk________::fbc445f8d24e569bc8b640dba86ae978, 20|corda_______::5a8a4094f1b68a88fc56e65cea7ebfa0] +// [20|rcuk________::7485257cd5caaf6316ba8062feea801d, 20|grid________::dded811e5f5a4c9f7ca8f9955e52ade7] +// [20|nih_________::0576dd270d29d5b7c23dd15a827ccdb9, 20|corda_______::10ca69f6a4a121f75fdde1feee226ce0] +// [20|corda__h2020::0429f6addf10e9b2939d65c6fb097ffd, 20|grid________::6563ec73057624d5ccc0cd050b302181] + + @Test + public void testHashCode() { + final String s1 = "20|grid________::6031f94bef015a37783268ec1e75f17f"; + final String s2 = "20|nsf_________::b12be9edf414df8ee66b4c52a2d8da46"; + + final HashFunction hashFunction = Hashing.murmur3_128(); + + System.out.println( s1.hashCode()); + System.out.println(hashFunction.hashUnencodedChars(s1).asLong()); + System.out.println( s2.hashCode()); + System.out.println(hashFunction.hashUnencodedChars(s2).asLong()); + + } + } diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json index 568b0e962..18b048e9e 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json @@ -1,62 +1,102 @@ { - "wf" : { - "threshold" : "0.99", - "dedupRun" : "001", - "entityType" : "result", - "subEntityType" : "resulttype", - "subEntityValue" : "publication", - "orderField" : "title", - "queueMaxSize" : "2000", - "groupMaxSize" : "100", - "maxChildren" : "100", - "idPath": "$.id", - "slidingWindowSize" : "200", - "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_isAffiliatedWith", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], - "includeChildren" : "true" - }, - "pace" : { - "clustering" : [ - { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, - { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } + "wf": { + "threshold": "0.99", + "dedupRun": "001", + "entityType": "result", + "subEntityType": "resulttype", + "subEntityValue": "publication", + "orderField": "title", + "queueMaxSize": "2000", + "groupMaxSize": "100", + "maxChildren": "100", + "slidingWindowSize": "200", + "rootBuilder": [ + "result", + "resultProject_outcome_isProducedBy", + "resultResult_publicationDataset_isRelatedTo", + "resultResult_similarity_isAmongTopNSimilarDocuments", + "resultResult_similarity_hasAmongTopNSimilarDocuments", + "resultOrganization_affiliation_isAffiliatedWith", + "resultResult_part_hasPart", + "resultResult_part_isPartOf", + "resultResult_supplement_isSupplementTo", + "resultResult_supplement_isSupplementedBy", + "resultResult_version_isVersionOf" ], - - "decisionTree" : { + "includeChildren": "true", + "maxIterations": 20, + "idPath": "$.id" + }, + "pace": { + "clustering": [ + { + "name": "ngrampairs", + "fields": [ + "title" + ], + "params": { + "max": "1", + "ngramLen": "3" + } + }, + { + "name": "suffixprefix", + "fields": [ + "title" + ], + "params": { + "max": "1", + "len": "3" + } + }, + { + "name": "lowercase", + "fields": [ + "doi" + ], + "params": {} + } + ], + "decisionTree": { "start": { "fields": [ { "field": "pid", - "comparator": "pidMatch", - "weight": 1, + "comparator": "jsonListMatch", + "weight": 1.0, "countIfUndefined": "false", - "params": {} + "params": { + "threshold": "0.5", + "jpath_value": "$.value", + "jpath_classid": "$.qualifier.classid" + } } ], - "threshold": 1, - "aggregation": "SC", + "threshold": 1.0, + "aggregation": "MAX", "positive": "MATCH", - "negative": "NO_MATCH", + "negative": "layer2", "undefined": "layer2", - "ignoreUndefined": "false" + "ignoreUndefined": "true" }, "layer2": { "fields": [ { "field": "title", "comparator": "titleVersionMatch", - "weight": 1, - "countIfUndefined": "true", + "weight": 1.0, + "countIfUndefined": "false", "params": {} }, { "field": "authors", "comparator": "sizeMatch", - "weight": 1, + "weight": 1.0, "countIfUndefined": "false", "params": {} } ], - "threshold": 1, + "threshold": 1.0, "aggregation": "NC", "positive": "layer3", "negative": "NO_MATCH", @@ -67,30 +107,53 @@ "fields": [ { "field": "title", - "comparator": "LevensteinTitle", - "weight": 1, + "comparator": "levensteinTitle", + "weight": 1.0, "countIfUndefined": "true", "params": {} } ], "threshold": 0.99, - "aggregation": "W_MEAN", + "aggregation": "SUM", "positive": "MATCH", "negative": "NO_MATCH", "undefined": "NO_MATCH", - "ignoreUndefined": "false" + "ignoreUndefined": "true" } }, - "model" : [ - { "name" : "doi", "type" : "String", "path" : "$.pid[?(@.qualifier.classid ==\"doi\")].value" }, - { "name" : "pid", "type" : "JSON","path" : "$.pid", "overrideMatch" : "true" }, - { "name" : "title", "type" : "String", "path" : "$.title[?(@.qualifier.classid ==\"main title\")].value", "length" : 250, "size" : 5 }, - { "name" : "authors", "type" : "List", "path" : "$.author[*].fullname", "size" : 200 }, - { "name" : "resulttype", "type" : "String", "path" : "$.resulttype.classid" } + "model": [ + { + "name": "doi", + "type": "String", + "path": "$.pid[?(@.qualifier.classid == 'doi')].value" + }, + { + "name": "pid", + "type": "JSON", + "path": "$.pid", + "overrideMatch": "true" + }, + { + "name": "title", + "type": "String", + "path": "$.title[?(@.qualifier.classid == 'main title')].value", + "length": 250, + "size": 5 + }, + { + "name": "authors", + "type": "List", + "path": "$.author[*].fullname", + "size": 200 + }, + { + "name": "resulttype", + "type": "String", + "path": "$.resulttype.classid" + } ], - "synonyms": {}, - "blacklists" : { - "title" : [ + "blacklists": { + "title": [ "^Inside Front Cover$", "(?i)^Poster presentations$", "^THE ASSOCIATION AND THE GENERAL MEDICAL COUNCIL$", @@ -102,7 +165,6 @@ "^Cartas? ao editor Letters? to the Editor$", "^Note from the Editor$", "^Anesthesia Abstract$", - "^Annual report$", "(?i)^“?THE RADICAL PREVENTION OF VENEREAL DISEASE\\.?”?$", "(?i)^Graph and Table of Infectious Diseases?$", @@ -122,14 +184,12 @@ "^Cálculo de concentraciones en disoluciones acuosas. Ejercicio interactivo\\..*\\.$", "(?i)^Genetic and functional analyses of SHANK2 mutations suggest a multiple hit model of Autism spectrum disorders?\\.?$", "^Gushi hakubutsugaku$", - "^Starobosanski nadpisi u Bosni i Hercegovini \\(.*\\)$", "^Intestinal spirocha?etosis$", "^Treatment of Rodent Ulcer$", "(?i)^\\W*Cloud Computing\\W*$", "^Compendio mathematico : en que se contienen todas las materias mas principales de las Ciencias que tratan de la cantidad$", "^Free Communications, Poster Presentations: Session [A-F]$", - "^“The Historical Aspects? of Quackery\\.?”$", "^A designated centre for people with disabilities operated by St John of God Community Services (Limited|Ltd), Louth$", "^P(er|re)-Mile Premiums for Auto Insurance\\.?$", @@ -150,10 +210,8 @@ "(?i)^Measurement of the pseudorapidity and centrality dependence of the transverse energy density in Pb(-?)Pb collisions at.*tev(\\.?)$", "(?i)^Search for resonances decaying into top-quark pairs using fully hadronic decays in pp collisions with ATLAS at.*TeV$", "(?i)^Search for neutral minimal supersymmetric standard model Higgs bosons decaying to tau pairs in pp collisions at.*tev$", - "(?i)^Relatório de Estágio (de|em) Angiologia e Cirurgia Vascular$", "^Aus der AGMB$", - "^Znanstveno-stručni prilozi$", "(?i)^Zhodnocení finanční situace podniku a návrhy na zlepšení$", "(?i)^Evaluation of the Financial Situation in the Firm and Proposals to its Improvement$", @@ -190,7 +248,6 @@ "(?i)^RUBRIKA UREDNIKA$", "^A Matching Model of the Academic Publication Market$", "^Yōgaku kōyō$", - "^Internetový marketing$", "^Internet marketing$", "^Chūtō kokugo dokuhon$", @@ -223,21 +280,17 @@ "^Information System Assessment and Proposal for ICT Modification$", "^Stresové zatížení pracovníků ve vybrané profesi$", "^Stress load in a specific job$", - "^Sunday: Poster Sessions, Pt.*$", "^Monday: Poster Sessions, Pt.*$", "^Wednesday: Poster Sessions, Pt.*", "^Tuesday: Poster Sessions, Pt.*$", - "^Analýza reklamy$", "^Analysis of advertising$", - "^Shōgaku shūshinsho$", "^Shōgaku sansū$", "^Shintei joshi kokubun$", "^Taishō joshi kokubun dokuhon$", "^Joshi kokubun$", - "^Účetní uzávěrka a účetní závěrka v ČR$", "(?i)^The \"?Causes\"? of Cancer$", "^Normas para la publicación de artículos$", @@ -256,7 +309,6 @@ "^Abdominal [Aa]ortic [Aa]neurysms.*$", "^Pseudomyxoma peritonei$", "^Kazalo autora$", - "(?i)^uvodna riječ$", "^Motivace jako způsob vedení lidí$", "^Motivation as a leadership$", @@ -329,6 +381,7 @@ "(?i)^.*authors['’′]? reply\\.?$", "(?i)^.*authors['’′]? response\\.?$" ] - } + }, + "synonyms": {} } } \ No newline at end of file From 7229fecbcf9c7d34fbe4b4e5cf1df6beeaad8e7f Mon Sep 17 00:00:00 2001 From: Michele Artini Date: Fri, 20 Dec 2019 13:41:08 +0100 Subject: [PATCH 10/13] fix warnings in poms --- dhp-build/dhp-build-assembly-resources/pom.xml | 1 + .../dhp-build-properties-maven-plugin/pom.xml | 1 + dhp-workflows/dhp-aggregation/pom.xml | 2 ++ dhp-workflows/pom.xml | 1 + pom.xml | 17 ++++------------- 5 files changed, 9 insertions(+), 13 deletions(-) diff --git a/dhp-build/dhp-build-assembly-resources/pom.xml b/dhp-build/dhp-build-assembly-resources/pom.xml index f1dd5f631..834af77fa 100644 --- a/dhp-build/dhp-build-assembly-resources/pom.xml +++ b/dhp-build/dhp-build-assembly-resources/pom.xml @@ -17,6 +17,7 @@ org.apache.maven.plugins maven-compiler-plugin + ${maven.compiler.plugin.version} diff --git a/dhp-build/dhp-build-properties-maven-plugin/pom.xml b/dhp-build/dhp-build-properties-maven-plugin/pom.xml index d561ce5dc..4f99d5298 100644 --- a/dhp-build/dhp-build-properties-maven-plugin/pom.xml +++ b/dhp-build/dhp-build-properties-maven-plugin/pom.xml @@ -53,6 +53,7 @@ org.apache.maven.plugins maven-compiler-plugin + ${maven.compiler.plugin.version} org.apache.maven.plugins diff --git a/dhp-workflows/dhp-aggregation/pom.xml b/dhp-workflows/dhp-aggregation/pom.xml index 10d4ff79a..328e783c4 100644 --- a/dhp-workflows/dhp-aggregation/pom.xml +++ b/dhp-workflows/dhp-aggregation/pom.xml @@ -7,6 +7,8 @@ 1.0.5-SNAPSHOT dhp-aggregation + + diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index bd2ca9704..cf71190a4 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -525,6 +525,7 @@ org.apache.maven.plugins maven-failsafe-plugin + ${maven.failsave.plugin.version} default-integration-test diff --git a/pom.xml b/pom.xml index cc39e46f2..aedf5ebff 100644 --- a/pom.xml +++ b/pom.xml @@ -184,16 +184,6 @@ provided - - com.fasterxml.jackson.core - jackson-core - ${dhp.jackson.version} - provided - - - - - com.fasterxml.jackson.core jackson-annotations @@ -268,7 +258,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.6.0 + ${maven.compiler.plugin.version} 1.8 1.8 @@ -422,14 +412,15 @@ UTF-8 UTF-8 - + 3.6.0 + 2.22.2 cdh5.9.2 2.6.0-${dhp.cdh.version} 4.1.0-${dhp.cdh.version} 2.4.0.cloudera2 2.9.6 3.5 - 2.11.8 + 2.11.12 From 9bdcb02179496aa2f3c989adfe330160deba16f3 Mon Sep 17 00:00:00 2001 From: miconis Date: Tue, 14 Jan 2020 15:01:03 +0200 Subject: [PATCH 11/13] minor changes and update of the configuration for publications --- .../dedup/SparkCreateConnectedComponent.java | 2 +- .../eu/dnetlib/dedup/SparkCreateSimRels.java | 40 ++++----------- .../dnetlib/dedup/SparkCreateDedupTest.java | 28 +++------- .../eu/dnetlib/dedup/conf/org.curr.conf.json | 51 ++++++++++++------- .../dnetlib/dedup/conf/pub_dt.curr.conf.json | 9 ++-- 5 files changed, 54 insertions(+), 76 deletions(-) diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index 16e112c25..e0549dfbf 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -52,7 +52,7 @@ public class SparkCreateConnectedComponent { final Dataset similarityRelations = spark.read().load(DedupUtility.createSimRelPath(targetPath,entity)).as(Encoders.bean(Relation.class)); final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())).rdd(); - final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()).toJavaRDD(); + final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, 20).toJavaRDD(); final Dataset mergeRelation = spark.createDataset(cc.filter(k->k.getDocIds().size()>1).flatMap((FlatMapFunction) c -> c.getDocIds() .stream() diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java index 48d442d04..831e45daf 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java @@ -1,7 +1,6 @@ package eu.dnetlib.dedup; -import eu.dnetlib.dedup.graph.ConnectedComponent; -import eu.dnetlib.dedup.graph.GraphProcessor; +import com.google.common.hash.Hashing; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.pace.config.DedupConfig; @@ -12,8 +11,6 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.graphx.Edge; -import org.apache.spark.rdd.RDD; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import scala.Tuple2; @@ -44,30 +41,23 @@ public class SparkCreateSimRels { final String inputPath = parser.get("sourcePath"); final String entity = parser.get("entity"); final String targetPath = parser.get("targetPath"); -// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf2.json"))); +// final DedupConfig dedupConf = DedupConfig.load(IOUtils.toString(SparkCreateSimRels.class.getResourceAsStream("/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); - final long total = sc.textFile(inputPath + "/" + entity).count(); - JavaPairRDD vertexes = sc.textFile(inputPath + "/" + entity) - .map(s->{ + JavaPairRDD mapDocument = sc.textFile(inputPath + "/" + entity) + .mapToPair(s->{ MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf,s); - return new Tuple2<>(d.getIdentifier(), d);}) - .mapToPair((PairFunction, Object, MapDocument>) t -> new Tuple2((long) t._1().hashCode(), t._2())); - - - - - JavaPairRDD mapDocument = vertexes.mapToPair((PairFunction, String, MapDocument>) item -> new Tuple2(item._2().getIdentifier(), item._2())); + return new Tuple2<>(d.getIdentifier(), d);}); //create blocks for deduplication - JavaPairRDD> blocks = Deduper.createsortedBlocks(sc,mapDocument, dedupConf); - + JavaPairRDD> blocks = Deduper.createsortedBlocks(sc, mapDocument, dedupConf); +// JavaPairRDD> blocks = Deduper.createBlocks(sc, mapDocument, dedupConf); //create relations by comparing only elements in the same group final JavaPairRDD dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf); - +// final JavaPairRDD dedupRels = Deduper.computeRelations(sc, blocks, dedupConf); final JavaRDD isSimilarToRDD = dedupRels.map(simRel -> { final Relation r = new Relation(); @@ -79,17 +69,5 @@ public class SparkCreateSimRels { spark.createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)).write().mode("overwrite").save( DedupUtility.createSimRelPath(targetPath,entity)); - - - - - - - - - } - - - -} +} \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java index f294b10fe..f93703e37 100644 --- a/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java +++ b/dhp-workflows/dhp-dedup/src/test/java/eu/dnetlib/dedup/SparkCreateDedupTest.java @@ -18,11 +18,11 @@ import java.util.List; public class SparkCreateDedupTest { String configuration; - String entity = "publication"; + String entity = "organization"; @Before public void setUp() throws IOException { - configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json")); + configuration = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dedup/conf/org.curr.conf.json")); } @@ -31,7 +31,7 @@ public class SparkCreateDedupTest { public void createSimRelsTest() throws Exception { SparkCreateSimRels.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", @@ -44,7 +44,7 @@ public class SparkCreateDedupTest { SparkCreateConnectedComponent.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-t", "/tmp/dedup", @@ -56,7 +56,7 @@ public class SparkCreateDedupTest { public void dedupRecordTest() throws Exception { SparkCreateDedupRecord.main(new String[] { "-mt", "local[*]", - "-s", "/home/sandro/betadump", + "-s", "/Users/miconis/dumps", "-e", entity, "-c", ArgumentApplicationParser.compressArgument(configuration), "-d", "/tmp/dedup", @@ -64,26 +64,10 @@ public class SparkCreateDedupTest { } @Test - public void printCC() throws Exception { + public void printConfiguration() throws Exception { System.out.println(ArgumentApplicationParser.compressArgument(configuration)); } - -// [20|grid________::6031f94bef015a37783268ec1e75f17f, 20|nsf_________::b12be9edf414df8ee66b4c52a2d8da46] -// [20|grid________::672e1e5cef49e68f124d3da5225a7357, 20|grid________::7a402604c3853c7a0af14f88f56bf7e1] -// [20|grid________::2fc05b35e11d915b220a66356053eae2, 20|grid________::b02fb3176eb38f6c572722550c07e7ab] -// [20|grid________::bc86248ab2b8d7955dcaf592ba342262, 20|corda_______::45a8ec964029278fb938805182e247a8] -// [20|doajarticles::74551f800ad1c81a6cd31c5162887b7f, 20|rcuk________::86dc9a83df05a58917f38ca09f814617] -// [20|nsf_________::5e837d8e6444cc298db314ea54ad2f4a, 20|snsf________::7b54715f0ec5c6a0a44672f45d98be8d] -// [20|corda__h2020::7ee7e57bad06b92c1a568dd61e10ba8c, 20|snsf________::2d4a2695221a3ce0c749ee34e064c0b3] -// [20|corda_______::25220a523550176dac9e5432dac43596, 20|grid________::9782f16a46650cbbfaaa2315109507d1] -// [20|nih_________::88c3b664dcc7af9e827f94ac964cd66c, 20|grid________::238d3ac0a7d119d5c8342a647f5245f5] -// [20|rcuk________::0582c20fcfb270f9ec1b19b0f0dcd881, 20|nsf_________::9afa48ddf0bc2cd4f3c41dc41daabcdb] -// [20|rcuk________::fbc445f8d24e569bc8b640dba86ae978, 20|corda_______::5a8a4094f1b68a88fc56e65cea7ebfa0] -// [20|rcuk________::7485257cd5caaf6316ba8062feea801d, 20|grid________::dded811e5f5a4c9f7ca8f9955e52ade7] -// [20|nih_________::0576dd270d29d5b7c23dd15a827ccdb9, 20|corda_______::10ca69f6a4a121f75fdde1feee226ce0] -// [20|corda__h2020::0429f6addf10e9b2939d65c6fb097ffd, 20|grid________::6563ec73057624d5ccc0cd050b302181] - @Test public void testHashCode() { final String s1 = "20|grid________::6031f94bef015a37783268ec1e75f17f"; diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json index 43003c2e0..2d0905562 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/org.curr.conf.json @@ -7,9 +7,9 @@ "queueMaxSize" : "2000", "groupMaxSize" : "50", "slidingWindowSize" : "200", + "idPath":"$.id", "rootBuilder" : [ "organization", "projectOrganization_participation_isParticipant", "datasourceOrganization_provision_isProvidedBy" ], "includeChildren" : "true", - "idPath": "$.id", "maxIterations": "20" }, "pace" : { @@ -31,7 +31,7 @@ } ], "threshold": 1, - "aggregation": "SC", + "aggregation": "AVG", "positive": "MATCH", "negative": "NO_MATCH", "undefined": "layer2", @@ -52,10 +52,24 @@ "weight": 1, "countIfUndefined": "true", "params": {} + }, + { + "field": "legalname", + "comparator": "numbersMatch", + "weight": 1, + "countIfUndefined": "true", + "params": {} + }, + { + "field": "legalname", + "comparator": "romansMatch", + "weight": 1, + "countIfUndefined": "true", + "params": {} } ], "threshold": 1, - "aggregation": "NC", + "aggregation": "AND", "positive": "layer3", "negative": "NO_MATCH", "undefined": "layer3", @@ -69,12 +83,11 @@ "weight": 1.0, "countIfUndefined": "true", "params": { - "windowSize": "4", - "threshold": "0.0" + "windowSize": "4" } } ], - "threshold": 1.0, + "threshold": 0.7, "aggregation": "W_MEAN", "positive": "layer4", "negative": "NO_MATCH", @@ -87,19 +100,18 @@ "field": "legalname", "comparator": "keywordMatch", "weight": 1.0, - "countIfUndefined": "false", + "countIfUndefined": "true", "params": { - "windowSize": "4", - "threshold": "0.7" + "windowSize": "4" } } ], - "threshold": 1.0, - "aggregation": "W_MEAN", + "threshold": 0.9, + "aggregation": "AVG", "positive": "layer5", "negative": "NO_MATCH", "undefined": "layer5", - "ignoreUndefined": "false" + "ignoreUndefined": "true" }, "layer5": { "fields": [ @@ -133,19 +145,20 @@ { "name" : "legalshortname", "type" : "String", "path" : "$.legalshortname.value"}, { "name" : "legalname", "type" : "String", "path" : "$.legalname.value" }, { "name" : "websiteurl", "type" : "URL", "path" : "$.websiteurl.value" }, - { "name" : "gridid", "type" : "String", "path" : "$.pid[?(@.qualifier.classid ==\"grid\")].value"} + { "name" : "gridid", "type" : "String", "path" : "$.pid[?(@.qualifier.classid =='grid.ac')].value"}, + { "name" : "originalId", "type" : "String", "path" : "$.id" } ], "blacklists" : { "legalname" : [] }, "synonyms": { - "key::1": ["university","università","università studi","universitario","universitaria","université","universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti", "universiti"], + "key::1": ["university","università","università studi","universitario","universitaria","université", "universite", "universitaire","universitaires","universidad","universitade","Universität","universitaet","Uniwersytet","университет","universiteit","πανεπιστήμιο","universitesi","universiteti", "universiti"], "key::2": ["studies","studi","études","estudios","estudos","Studien","studia","исследования","studies","σπουδές"], "key::3": ["advanced","superiore","supérieur","supérieure","supérieurs","supérieures","avancado","avancados","fortgeschrittene","fortgeschritten","zaawansowany","передовой","gevorderd","gevorderde","προχωρημένος","προχωρημένη","προχωρημένο","προχωρημένες","προχωρημένα","wyzsza"], "key::4": ["institute","istituto","institut","instituto","instituto","Institut","instytut","институт","instituut","ινστιτούτο"], "key::5": ["hospital","ospedale","hôpital","hospital","hospital","Krankenhaus","szpital","больница","ziekenhuis","νοσοκομείο"], "key::6": ["research","ricerca","recherche","investigacion","pesquisa","Forschung","badania","исследования","onderzoek","έρευνα","erevna","erevnas"], - "key::7": ["college","collegio","université","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","universiteit","κολλέγιο"], + "key::7": ["college","collegio","colegio","faculdade","Hochschule","Szkoła Wyższa","Высшая школа","κολλέγιο"], "key::8": ["foundation","fondazione","fondation","fundación","fundação","Stiftung","Fundacja","фонд","stichting","ίδρυμα","idryma"], "key::9": ["center","centro","centre","centro","centro","zentrum","centrum","центр","centrum","κέντρο"], "key::10": ["national","nazionale","national","nationale","nationaux","nationales","nacional","nacional","national","krajowy","национальный","nationaal","nationale","εθνικό"], @@ -233,7 +246,7 @@ "key::92": ["automation","automazione","automatización","automação","Automatisierung","automatisering","αυτοματοποίηση","otomasyon","automatizálás","avtomatizacija","automatiseeritud",""], "key::93": ["pediatric","pediatria","pediatriche","pediatrico","pediátrico","pediatría","pediátrico","pediatria","pädiatrisch","pediatrische","παιδιατρική","pediatrik","gyermekgyógyászat","pediatrija","pediaatria",""], "key::94": ["photonics","fotonica","fotoniche","fotónica","fotônica","Photonik","fotonica","φωτονική","fotonik","fotonika","fotonika","fotoonika",""], - "key::95": ["mechanics","meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""], + "key::95": ["mechanics", "mechanical", "meccanica","meccaniche","mecánica","mecânica","Mechanik","Maschinenbau","mechanica","werktuigkunde","μηχανικής","mekanik","gépészet","mehanika","mehaanika",""], "key::96": ["psychiatrics","psichiatria","psichiatrica","psichiatriche","psiquiatría","psiquiatria","Psychiatrie","psychiatrie","ψυχιατρική","psikiyatrik","pszihiátria","psihiatrija","psühhaatria",""], "key::97": ["psychology","fisiologia","psicología","psicologia","Psychologie","psychologie","ψυχολογία","psikoloji","pszihológia","psihologija","psühholoogia",""], "key::98": ["automotive","industriaautomobilistica","industriadelautomóvil","automotriz","industriaautomotriz","automotivo","Automobilindustrie","autoindustrie","αυτοκίνητος","αυτοκίνητη","αυτοκίνητο","αυτοκινούμενος","αυτοκινούμενη","αυτοκινούμενο","αυτοκινητιστικός","αυτοκινητιστική","αυτοκινητιστικό","otomotiv","autóipari","samogiben","avtomobilskaindustrija","auto-",""], @@ -243,7 +256,11 @@ "key::102": ["informatics","informatica","informática","informática","informatica",""], "key::103": ["forschungsgemeinschaft","comunita ricerca","research community","research foundation","research association"], "key::104": ["commerce","ticaret","ticarət","commercio","trade","handel","comercio"], - "key::105" : ["state", "stato", "etade", "statale", "etat", "zustand", "estado"] + "key::105" : ["state", "stato", "etade", "estado", "statale", "etat", "zustand", "estado"], + "key::106" : ["seminary", "seminario", "seminaire", "seminar"], + "key::107" : ["agricultural forestry", "af", "a f"], + "key::108" : ["agricultural mechanical", "am", "a m"], + "key::109" : ["catholic", "catholique", "katholische", "catolica", "cattolica", "catolico"] } } } \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json index 18b048e9e..6ca0ecd53 100644 --- a/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json +++ b/dhp-workflows/dhp-dedup/src/test/resources/eu/dnetlib/dedup/conf/pub_dt.curr.conf.json @@ -66,14 +66,13 @@ "weight": 1.0, "countIfUndefined": "false", "params": { - "threshold": "0.5", "jpath_value": "$.value", "jpath_classid": "$.qualifier.classid" } } ], - "threshold": 1.0, - "aggregation": "MAX", + "threshold": 0.5, + "aggregation": "AVG", "positive": "MATCH", "negative": "layer2", "undefined": "layer2", @@ -97,7 +96,7 @@ } ], "threshold": 1.0, - "aggregation": "NC", + "aggregation": "AND", "positive": "layer3", "negative": "NO_MATCH", "undefined": "layer3", @@ -114,7 +113,7 @@ } ], "threshold": 0.99, - "aggregation": "SUM", + "aggregation": "AVG", "positive": "MATCH", "negative": "NO_MATCH", "undefined": "NO_MATCH", From f61adfc2bb7a0e32cd9015884dab9b402033e5bb Mon Sep 17 00:00:00 2001 From: miconis Date: Tue, 14 Jan 2020 15:03:27 +0200 Subject: [PATCH 12/13] minor changes --- .../java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index e0549dfbf..16e112c25 100644 --- a/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -52,7 +52,7 @@ public class SparkCreateConnectedComponent { final Dataset similarityRelations = spark.read().load(DedupUtility.createSimRelPath(targetPath,entity)).as(Encoders.bean(Relation.class)); final RDD> edgeRdd = similarityRelations.javaRDD().map(it -> new Edge<>(getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())).rdd(); - final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, 20).toJavaRDD(); + final JavaRDD cc = GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()).toJavaRDD(); final Dataset mergeRelation = spark.createDataset(cc.filter(k->k.getDocIds().size()>1).flatMap((FlatMapFunction) c -> c.getDocIds() .stream() From 749b0660abbb44b0fa96eff27fd986e15b0daeeb Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 17 Jan 2020 14:22:15 +0100 Subject: [PATCH 13/13] instance URLs must be repeatable --- .../src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java index ceaaad6b0..8f852af65 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; +import java.util.List; public class Instance implements Serializable { @@ -12,7 +13,7 @@ public class Instance implements Serializable { private KeyValue hostedby; - private String url; + private List url; // other research products specifc private String distributionlocation; @@ -53,11 +54,11 @@ public class Instance implements Serializable { this.hostedby = hostedby; } - public String getUrl() { + public List getUrl() { return url; } - public void setUrl(String url) { + public void setUrl(List url) { this.url = url; }