From cc63706347ff65fc1c57c63a673175473680770f Mon Sep 17 00:00:00 2001 From: "sandro.labruzzo" Date: Fri, 6 Dec 2019 13:38:00 +0100 Subject: [PATCH] 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} + +