From c7bc73aedf804f15c5a5faa3436b04dbb94d7645 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 11:44:48 +0100 Subject: [PATCH 001/259] country propagation for results collected from institutional repositories --- dhp-workflows/dhp-bulktag/pom.xml | 15 + dhp-workflows/dhp-propagation/pom.xml | 39 +++ .../SparkCountryPropagationJob.java | 308 ++++++++++++++++++ .../dhp/countrypropagation/TypedRow.java | 70 ++++ .../countrypropagation-ovverride.properties | 3 + .../input_countrypropagation_parameters.json | 5 + .../oozie_app/config-default.xml | 18 + .../countrypropagation/oozie_app/workflow.xml | 55 ++++ dhp-workflows/pom.xml | 2 + 9 files changed, 515 insertions(+) create mode 100644 dhp-workflows/dhp-bulktag/pom.xml create mode 100644 dhp-workflows/dhp-propagation/pom.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-bulktag/pom.xml new file mode 100644 index 000000000..5b6a21027 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/pom.xml @@ -0,0 +1,15 @@ + + + + dhp-workflows + eu.dnetlib.dhp + 1.0.5-SNAPSHOT + + 4.0.0 + + dhp-bulktag + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml new file mode 100644 index 000000000..10fcd38bb --- /dev/null +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -0,0 +1,39 @@ + + + + dhp-workflows + eu.dnetlib.dhp + 1.0.5-SNAPSHOT + + 4.0.0 + + dhp-propagation + + + + 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} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java new file mode 100644 index 000000000..abcec9dbb --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -0,0 +1,308 @@ +package eu.dnetlib.dhp.countrypropagation; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +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.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.*; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Stream; + +public class SparkCountryPropagationJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCountryPropagationJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = parser.get("outputPath"); + + + List whitelist = new ArrayList<>(); + List allowedtypes = new ArrayList<>(); + + + + // JavaPairRDD results = getResults(sc, inputPath); +// sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) +// .map(oaf -> new TypedRow().setType("dataset").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) +// .mapToPair(toPair()) +// .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) +// .map(oaf -> new TypedRow().setType("otherresearchproduct").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) +// .mapToPair(toPair())) +// .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) +// .map(oaf -> new TypedRow().setType("software").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) +// .mapToPair(toPair())) +// .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) +// .map(oaf -> new TypedRow().setType("publication").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) +// .mapToPair(toPair())); +// +// + JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) + .filter(org -> !org.getDataInfo().getDeletedbyinference()) + .map(org -> new TypedRow().setSourceId(org.getId()).setCountry(org.getCountry().getClassid())) + .mapToPair(toPair()); + + JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> "datasourceOrganization".equals(r.getRelClass()) && "isProvidedBy".equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) + .filter(ds -> whitelist.contains(ds.getId()) || allowedtypes.contains(ds.getDatasourcetype().getClassid())) + .map(ds -> new TypedRow().setSourceId(ds.getId())) + .mapToPair(toPair()); + + + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + + JavaPairRDD datasource_results = publications + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + }) + .union(datasets + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(software + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(other + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + JavaPairRDD datasource_country = organizations.join(organization_datasource) + .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) + .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) + + + JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) + .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())); + + + JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) + .map(u -> u._2()._2().setCountry(u._2()._1().getCountry())) + .mapToPair(toPair()) + .reduceByKey((a, p) -> { + if (a == null) { + return p; + } + if (p == null) { + return a; + } + HashSet countries = new HashSet(); + countries.addAll(Arrays.asList(a.getCountry().split(";"))); + countries.addAll(Arrays.asList(p.getCountry().split(";"))); + String country = new String(); + for (String c : countries) { + country += c + ";"; + } + + return a.setCountry(country); + }); + + updateResult(pubs, toupdateresult, outputPath, "publication"); + updateResult(dss, toupdateresult, outputPath, "dataset"); + updateResult(sfw, toupdateresult, outputPath, "software"); + updateResult(orp, toupdateresult, outputPath, "otherresearchproduct"); + //we use leftOuterJoin because we want to rebuild the entire structure + + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(c -> { + OafEntity oaf = c._2()._1(); + List qualifierList = null; + if (oaf.getClass() == Publication.class) { + qualifierList = ((Publication) oaf).getCountry(); + + } + if (oaf.getClass() == Dataset.class){ + qualifierList = ((Dataset) oaf).getCountry(); + } + + if (oaf.getClass() == Software.class){ + qualifierList = ((Software) oaf).getCountry(); + } + + if (oaf.getClass() == OtherResearchProduct.class){ + qualifierList = ((OtherResearchProduct) oaf).getCountry(); + } + + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : qualifierList) { + countries.add(country.getClassid()); + } + TypedRow t = c._2()._2().get(); + + for (String country : t.getCountry().split(";")) { + if (!countries.contains(country)) { + Qualifier q = new Qualifier(); + q.setClassid(country); + qualifierList.add(q); + } + + } + if (oaf.getClass() == Publication.class) { + ((Publication) oaf).setCountry(qualifierList); + return (Publication) oaf; + + } + if (oaf.getClass() == Dataset.class){ + ((Dataset) oaf).setCountry(qualifierList); + return (Dataset) oaf; + } + + if (oaf.getClass() == Software.class){ + ((Software) oaf).setCountry(qualifierList); + return (Software) oaf; + } + + if (oaf.getClass() == OtherResearchProduct.class){ + ((OtherResearchProduct) oaf).setCountry(qualifierList); + return (OtherResearchProduct) oaf; + } + } + + + return null; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + private static List getTypedRows(OafEntity oaf) { + List lst = new ArrayList<>(); + Set datasources_provenance = new HashSet<>(); + List instanceList = null; + String type = ""; + if (oaf.getClass() == Publication.class) { + instanceList = ((Publication) oaf).getInstance(); + type = "publication"; + } + if (oaf.getClass() == Dataset.class){ + instanceList = ((Dataset)oaf).getInstance(); + type = "dataset"; + } + + if (oaf.getClass() == Software.class){ + instanceList = ((Software)oaf).getInstance(); + type = "software"; + } + + if (oaf.getClass() == OtherResearchProduct.class){ + instanceList = ((OtherResearchProduct)oaf).getInstance(); + type = "otherresearchproduct"; + } + + + for (Instance i : instanceList) { + datasources_provenance.add(i.getCollectedfrom().getKey()); + datasources_provenance.add(i.getHostedby().getKey()); + } + for (String dsId : datasources_provenance) { + lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); + } + return lst; + } + + + private static JavaPairRDD getResults(JavaSparkContext sc , String inputPath){ + + return + sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) + .filter(ds -> !ds.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("dataset").setSourceId(oaf.getId())) + .mapToPair(toPair()) + .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) + .filter(o -> !o.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("otherresearchproduct").setSourceId(oaf.getId())) + .mapToPair(toPair())) + .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) + .filter(s -> !s.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("software").setSourceId(oaf.getId())) + .mapToPair(toPair())) + .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("publication").setSourceId(oaf.getId())) + .mapToPair(toPair())); + + + } + + + + private static PairFunction toPair() { + return e -> new Tuple2<>( e.getSourceId(), e); + + }; + + +} + diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java new file mode 100644 index 000000000..f006a320b --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java @@ -0,0 +1,70 @@ +package eu.dnetlib.dhp.countrypropagation; + + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class TypedRow implements Serializable { + private String sourceId; + private String targetId; + private String type; + private String country; + + public List getAccumulator() { + return accumulator; + } + + public TypedRow setAccumulator(List accumulator) { + this.accumulator = accumulator; + return this; + } + + private List accumulator; + + + public void add(String a){ + if (accumulator == null){ + accumulator = new ArrayList<>(); + } + accumulator.add(a); + } + + public Iterator getAccumulatorIterator(){ + return accumulator.iterator(); + } + + public String getCountry() { + return country; + } + + public TypedRow setCountry(String country) { + this.country = country; + return this; + } + + public String getSourceId() { + return sourceId; + } + public TypedRow setSourceId(String sourceId) { + this.sourceId = sourceId; + return this; + } + public String getTargetId() { + return targetId; + } + public TypedRow setTargetId(String targetId) { + this.targetId = targetId; + return this; + } + + public String getType() { + return type; + } + public TypedRow setType(String type) { + this.type = type; + return this; + } + +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties new file mode 100644 index 000000000..7754151e8 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties @@ -0,0 +1,3 @@ +sourcePath=/tmp/db_openaireplus_services.export_dhp.2020.02.03 +sparkDriverMemory=15G +sparkExecutorMemory=15G \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json new file mode 100644 index 000000000..03f3dd8ff --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -0,0 +1,5 @@ +[ + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true}, + {"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the path of the sequential file to write", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml new file mode 100644 index 000000000..1fef8b17c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + sourcePath + the source path + + + outputPath + the output 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 + CountryPropagation + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob + dhp-graph-countrypropagation-${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" + -mt yarn-cluster + --sourcePath${sourcePath} + --outputPath${outputPath} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index cf71190a4..2a9a72f30 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -18,6 +18,8 @@ dhp-distcp dhp-graph-mapper dhp-dedup + dhp-bulktag + dhp-propagation From be2421d5d8acf98d4aa931785c74221431af4473 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 12:07:26 +0100 Subject: [PATCH 002/259] removed wrongly pushed file --- .../countrypropagation/countrypropagation-ovverride.properties | 3 --- 1 file changed, 3 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties deleted file mode 100644 index 7754151e8..000000000 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/countrypropagation-ovverride.properties +++ /dev/null @@ -1,3 +0,0 @@ -sourcePath=/tmp/db_openaireplus_services.export_dhp.2020.02.03 -sparkDriverMemory=15G -sparkExecutorMemory=15G \ No newline at end of file From 9abde5cfac4f7aa54535ba0030d8385d7d7d99d9 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 14:19:53 +0100 Subject: [PATCH 003/259] removed outputPath from job parameters --- .../input_countrypropagation_parameters.json | 3 +-- .../eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 03f3dd8ff..6cc21c544 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -1,5 +1,4 @@ [ {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true}, - {"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the path of the sequential file to write", "paramRequired": true} + {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 1fef8b17c..dd30e3e0a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -45,7 +45,6 @@ --conf spark.sql.warehouse.dir="/user/hive/warehouse" -mt yarn-cluster --sourcePath${sourcePath} - --outputPath${outputPath} From a5517eee358029d301f82eb7b892dbba398f1277 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 14:20:42 +0100 Subject: [PATCH 004/259] adding the mkdirs for creation of propagation folder under provision on tmp --- .../countrypropagation/SparkCountryPropagationJob.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index abcec9dbb..919c69015 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -16,6 +16,7 @@ import org.apache.spark.rdd.RDD; import org.apache.spark.sql.*; import scala.Tuple2; +import java.io.File; import java.util.*; import java.util.stream.Stream; @@ -33,8 +34,13 @@ public class SparkCountryPropagationJob { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); - final String outputPath = parser.get("outputPath"); + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + File directory = new File(outputPath); + + if(!directory.exists()){ + directory.mkdirs(); + } List whitelist = new ArrayList<>(); List allowedtypes = new ArrayList<>(); From 3a9d723655318ceb49352d2e8cf54d148def1fb0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 16:30:52 +0100 Subject: [PATCH 005/259] adding default parameters in code --- .../SparkCountryPropagationJob.java | 25 +++---------------- ...arkResultToOrganizationFromIstRepoJob.java | 4 +++ 2 files changed, 7 insertions(+), 22 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 919c69015..d395d85be 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -42,30 +42,11 @@ public class SparkCountryPropagationJob { directory.mkdirs(); } - List whitelist = new ArrayList<>(); - List allowedtypes = new ArrayList<>(); + //TODO: add as Job Parameters + List whitelist = Arrays.asList("10|opendoar____::300891a62162b960cf02ce3827bb363c"); + List allowedtypes = Arrays.asList("pubsrepository::institutional"); - - // JavaPairRDD results = getResults(sc, inputPath); -// sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) -// .map(oaf -> new TypedRow().setType("dataset").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) -// .mapToPair(toPair()) -// .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) -// .map(oaf -> new TypedRow().setType("otherresearchproduct").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) -// .mapToPair(toPair())) -// .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) -// .map(oaf -> new TypedRow().setType("software").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) -// .mapToPair(toPair())) -// .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) -// .map(oaf -> new TypedRow().setType("publication").setDeleted(oaf.getDataInfo().getDeletedbyinference()).setOaf(oaf.toString()).setSourceId(oaf.getId())) -// .mapToPair(toPair())); -// -// JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) .filter(org -> !org.getDataInfo().getDeletedbyinference()) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java new file mode 100644 index 000000000..1ea62d62b --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +public class SparkResultToOrganizationFromIstRepoJob { +} From bd0e504b42d695167731d9ab3fb25257e90de03d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 18:04:15 +0100 Subject: [PATCH 006/259] changes to the wf configuration --- .../SparkCountryPropagationJob.java | 9 +- ...arkResultToOrganizationFromIstRepoJob.java | 326 ++++++++++++++++++ .../input_countrypropagation_parameters.json | 4 - .../input_countrypropagation_parameters.json | 26 ++ .../oozie_app/config-default.xml | 0 .../oozie_app/workflow.xml | 18 +- 6 files changed, 366 insertions(+), 17 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{countrypropagation => graph}/oozie_app/config-default.xml (100%) rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{countrypropagation => graph}/oozie_app/workflow.xml (76%) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index d395d85be..073c80d1b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -9,16 +9,12 @@ import org.apache.hadoop.io.Text; 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.Function2; import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.rdd.RDD; import org.apache.spark.sql.*; import scala.Tuple2; import java.io.File; import java.util.*; -import java.util.stream.Stream; public class SparkCountryPropagationJob { public static void main(String[] args) throws Exception { @@ -42,9 +38,8 @@ public class SparkCountryPropagationJob { directory.mkdirs(); } - //TODO: add as Job Parameters - List whitelist = Arrays.asList("10|opendoar____::300891a62162b960cf02ce3827bb363c"); - List allowedtypes = Arrays.asList("pubsrepository::institutional"); + List whitelist = Arrays.asList(parser.get("whitelist").split(";")); + List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index 1ea62d62b..bfb2562d4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -1,4 +1,330 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import org.apache.commons.io.IOUtils; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +import java.io.File; + public class SparkResultToOrganizationFromIstRepoJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_resulttoorganizationfrominstrepo_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } + } + } +/* +package eu.dnetlib.dhp.graph; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +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.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.*; +import scala.Tuple2; + +import java.io.File; +import java.util.*; +import java.util.stream.Stream; + +public class SparkCountryPropagationJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCountryPropagationJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + + File directory = new File(outputPath); + + if(!directory.exists()){ + directory.mkdirs(); + } + + //TODO: add as Job Parameters + List whitelist = Arrays.asList("10|opendoar____::300891a62162b960cf02ce3827bb363c"); + List allowedtypes = Arrays.asList("pubsrepository::institutional"); + + + JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) + .filter(org -> !org.getDataInfo().getDeletedbyinference()) + .map(org -> new TypedRow().setSourceId(org.getId()).setCountry(org.getCountry().getClassid())) + .mapToPair(toPair()); + + JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> "datasourceOrganization".equals(r.getRelClass()) && "isProvidedBy".equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) + .filter(ds -> whitelist.contains(ds.getId()) || allowedtypes.contains(ds.getDatasourcetype().getClassid())) + .map(ds -> new TypedRow().setSourceId(ds.getId())) + .mapToPair(toPair()); + + + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + + JavaPairRDD datasource_results = publications + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + }) + .union(datasets + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(software + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(other + .map(oaf -> getTypedRows(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + JavaPairRDD datasource_country = organizations.join(organization_datasource) + .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) + .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) + + + JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) + .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())); + + + JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) + .map(u -> u._2()._2().setCountry(u._2()._1().getCountry())) + .mapToPair(toPair()) + .reduceByKey((a, p) -> { + if (a == null) { + return p; + } + if (p == null) { + return a; + } + HashSet countries = new HashSet(); + countries.addAll(Arrays.asList(a.getCountry().split(";"))); + countries.addAll(Arrays.asList(p.getCountry().split(";"))); + String country = new String(); + for (String c : countries) { + country += c + ";"; + } + + return a.setCountry(country); + }); + + updateResult(pubs, toupdateresult, outputPath, "publication"); + updateResult(dss, toupdateresult, outputPath, "dataset"); + updateResult(sfw, toupdateresult, outputPath, "software"); + updateResult(orp, toupdateresult, outputPath, "otherresearchproduct"); + //we use leftOuterJoin because we want to rebuild the entire structure + + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(c -> { + OafEntity oaf = c._2()._1(); + List qualifierList = null; + if (oaf.getClass() == Publication.class) { + qualifierList = ((Publication) oaf).getCountry(); + + } + if (oaf.getClass() == Dataset.class){ + qualifierList = ((Dataset) oaf).getCountry(); + } + + if (oaf.getClass() == Software.class){ + qualifierList = ((Software) oaf).getCountry(); + } + + if (oaf.getClass() == OtherResearchProduct.class){ + qualifierList = ((OtherResearchProduct) oaf).getCountry(); + } + + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : qualifierList) { + countries.add(country.getClassid()); + } + TypedRow t = c._2()._2().get(); + + for (String country : t.getCountry().split(";")) { + if (!countries.contains(country)) { + Qualifier q = new Qualifier(); + q.setClassid(country); + qualifierList.add(q); + } + + } + if (oaf.getClass() == Publication.class) { + ((Publication) oaf).setCountry(qualifierList); + return (Publication) oaf; + + } + if (oaf.getClass() == Dataset.class){ + ((Dataset) oaf).setCountry(qualifierList); + return (Dataset) oaf; + } + + if (oaf.getClass() == Software.class){ + ((Software) oaf).setCountry(qualifierList); + return (Software) oaf; + } + + if (oaf.getClass() == OtherResearchProduct.class){ + ((OtherResearchProduct) oaf).setCountry(qualifierList); + return (OtherResearchProduct) oaf; + } + } + + + return null; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + private static List getTypedRows(OafEntity oaf) { + List lst = new ArrayList<>(); + Set datasources_provenance = new HashSet<>(); + List instanceList = null; + String type = ""; + if (oaf.getClass() == Publication.class) { + instanceList = ((Publication) oaf).getInstance(); + type = "publication"; + } + if (oaf.getClass() == Dataset.class){ + instanceList = ((Dataset)oaf).getInstance(); + type = "dataset"; + } + + if (oaf.getClass() == Software.class){ + instanceList = ((Software)oaf).getInstance(); + type = "software"; + } + + if (oaf.getClass() == OtherResearchProduct.class){ + instanceList = ((OtherResearchProduct)oaf).getInstance(); + type = "otherresearchproduct"; + } + + + for (Instance i : instanceList) { + datasources_provenance.add(i.getCollectedfrom().getKey()); + datasources_provenance.add(i.getHostedby().getKey()); + } + for (String dsId : datasources_provenance) { + lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); + } + return lst; + } + + + private static JavaPairRDD getResults(JavaSparkContext sc , String inputPath){ + + return + sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) + .filter(ds -> !ds.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("dataset").setSourceId(oaf.getId())) + .mapToPair(toPair()) + .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) + .filter(o -> !o.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("otherresearchproduct").setSourceId(oaf.getId())) + .mapToPair(toPair())) + .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) + .filter(s -> !s.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("software").setSourceId(oaf.getId())) + .mapToPair(toPair())) + .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .map(oaf -> new TypedRow().setType("publication").setSourceId(oaf.getId())) + .mapToPair(toPair())); + + + } + + + + private static PairFunction toPair() { + return e -> new Tuple2<>( e.getSourceId(), e); + + }; + + } + + + */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json deleted file mode 100644 index 6cc21c544..000000000 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ /dev/null @@ -1,4 +0,0 @@ -[ - {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, - {"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true} -] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json new file mode 100644 index 000000000..cbafdcd46 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true, + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true, + }, + { + "paramName":"wl", + "paramLongName":"whitelist", + "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", + "paramRequired": true + }, + { + "paramName":"at", + "paramLongName":"allowedtypes", + "paramDescription": "the types of the allowed datasources. Split by ;", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml similarity index 76% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml index dd30e3e0a..af88c6fbb 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml @@ -1,12 +1,16 @@ - + sourcePath the source path - outputPath - the output path + whitelist + the white list + + + allowedtypes + the allowed types sparkDriverMemory @@ -22,7 +26,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -36,15 +40,17 @@ cluster CountryPropagation eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-graph-countrypropagation-${projectVersion}.jar + dhp-propagation-${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" + -mt yarn-cluster --sourcePath${sourcePath} + --withelist${whitelist} + --allowedtypes${allowedtypes} From 18e4092d5c289bc384677d8e90455ac4edbd8103 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 17 Feb 2020 18:07:06 +0100 Subject: [PATCH 007/259] change name of properties dir --- .../SparkResultToOrganizationFromIstRepoJob.java | 4 ++-- .../input_countrypropagation_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../dhp/{graph => countrypropagation}/oozie_app/workflow.xml | 0 4 files changed, 2 insertions(+), 2 deletions(-) rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{graph => countrypropagation}/input_countrypropagation_parameters.json (100%) rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{graph => countrypropagation}/oozie_app/config-default.xml (100%) rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{graph => countrypropagation}/oozie_app/workflow.xml (100%) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index bfb2562d4..5b9504fec 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -31,7 +31,7 @@ public class SparkResultToOrganizationFromIstRepoJob { } } /* -package eu.dnetlib.dhp.graph; +package eu.dnetlib.dhp.countrypropagation; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -56,7 +56,7 @@ import java.util.stream.Stream; public class SparkCountryPropagationJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); parser.parseArgument(args); final SparkSession spark = SparkSession .builder() diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/input_countrypropagation_parameters.json rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/config-default.xml rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/graph/oozie_app/workflow.xml rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml From e0a777028ababa1cb48cb566d8192e4b410176c7 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:23:34 +0100 Subject: [PATCH 008/259] fix problem in parameters --- .../dnetlib/dhp/graph/MappingUtilsTest.java | 66 ------------------- .../dhp/graph/XmlRecordFactoryTest.java | 55 ---------------- .../eu/dnetlib/dhp/PropagationConstant.java | 4 ++ .../{countrypropagation => }/TypedRow.java | 0 .../input_countrypropagation_parameters.json | 4 +- .../countrypropagation/oozie_app/workflow.xml | 2 +- ...sulaffiliationfrominstrepo_parameters.json | 26 ++++++++ .../oozie_app/config-default.xml | 18 +++++ .../oozie_app/workflow.xml | 60 +++++++++++++++++ 9 files changed, 111 insertions(+), 124 deletions(-) delete mode 100644 dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/MappingUtilsTest.java delete mode 100644 dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/XmlRecordFactoryTest.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/{countrypropagation => }/TypedRow.java (100%) create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/MappingUtilsTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/MappingUtilsTest.java deleted file mode 100644 index a9d696bea..000000000 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/MappingUtilsTest.java +++ /dev/null @@ -1,66 +0,0 @@ -package eu.dnetlib.dhp.graph; - -import eu.dnetlib.dhp.graph.model.EntityRelEntity; -import eu.dnetlib.dhp.graph.model.RelatedEntity; -import eu.dnetlib.dhp.graph.utils.GraphMappingUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.io.InputStreamReader; - -public class MappingUtilsTest { - - private GraphMappingUtils utils; - - @Before - public void setUp() { - utils = new GraphMappingUtils(); - } - - @Test - public void testOafMappingDatasource() throws IOException { - - final InputStreamReader in = new InputStreamReader(getClass().getResourceAsStream("datasource.json")); - final EntityRelEntity e = new ObjectMapper().readValue(in, EntityRelEntity.class); - e.getSource().setType("datasource"); - - final EntityRelEntity out = utils.asRelatedEntity(e); - System.out.println(out); - - } - - //@Test - public void testOafMappingResult() throws IOException { - - final InputStreamReader in = new InputStreamReader(getClass().getResourceAsStream("result.json")); - final EntityRelEntity e = new ObjectMapper().readValue(in, EntityRelEntity.class); - - final EntityRelEntity out = utils.asRelatedEntity(e); - System.out.println(out); - - } - - @Test - public void testOafMappingSoftware() throws IOException { - - final InputStreamReader in = new InputStreamReader(getClass().getResourceAsStream("software.json")); - final EntityRelEntity e = new ObjectMapper().readValue(in, EntityRelEntity.class); - - final EntityRelEntity out = utils.asRelatedEntity(e); - System.out.println(out); - - } - - - @Test - public void testParseRelatedEntity() throws IOException { - - final InputStreamReader in = new InputStreamReader(getClass().getResourceAsStream("related_entity.json")); - final RelatedEntity e = new ObjectMapper().readValue(in, RelatedEntity.class); - - System.out.println(e); - - } -} diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/XmlRecordFactoryTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/XmlRecordFactoryTest.java deleted file mode 100644 index 2a3c343ec..000000000 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/graph/XmlRecordFactoryTest.java +++ /dev/null @@ -1,55 +0,0 @@ -package eu.dnetlib.dhp.graph; - -import eu.dnetlib.dhp.graph.utils.ContextMapper; -import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.spark.sql.SparkSession; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -public class XmlRecordFactoryTest { - - private static final Log log = LogFactory.getLog(XmlRecordFactoryTest.class); - - private Path testDir; - - @Before - public void setup() throws IOException { - testDir = Files.createTempDirectory(getClass().getSimpleName()); - log.info("created test directory " + testDir.toString()); - } - - @After - public void tearDown() throws IOException { - FileUtils.deleteDirectory(testDir.toFile()); - log.info("deleted test directory " + testDir.toString()); - } - - @Test - public void testXmlSerialization() throws Exception { - - final SparkSession spark = SparkSession - .builder() - .appName(SparkXmlRecordBuilderJob.class.getSimpleName()) - .master("local[*]") - .getOrCreate(); - - final String inputDir = testDir.toString() + "/3_joined_entities"; - FileUtils.forceMkdir(new File(inputDir)); - FileUtils.copyFile(new File("/Users/claudio/Downloads/joined_entities-part-00000"), new File(inputDir + "/joined_entities-part-00000")); - - final ContextMapper ctx = ContextMapper.fromIS("https://dev-openaire.d4science.org:443/is/services/isLookUp"); - - final GraphJoiner g = new GraphJoiner(spark, ctx, inputDir, testDir.toString()); - - g.asXML(); - } - -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java new file mode 100644 index 000000000..eed336d5b --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp; + +public class PropagationConstant { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/TypedRow.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index cbafdcd46..090f3f152 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -3,13 +3,13 @@ "paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", - "paramRequired": true, + "paramRequired": true }, { "paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", - "paramRequired": true, + "paramRequired": true }, { "paramName":"wl", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index af88c6fbb..f24417bc0 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -49,7 +49,7 @@ -mt yarn-cluster --sourcePath${sourcePath} - --withelist${whitelist} + --whitelist${whitelist} --allowedtypes${allowedtypes} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json new file mode 100644 index 000000000..090f3f152 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"wl", + "paramLongName":"whitelist", + "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", + "paramRequired": true + }, + { + "paramName":"at", + "paramLongName":"allowedtypes", + "paramDescription": "the types of the allowed datasources. Split by ;", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml new file mode 100644 index 000000000..f24417bc0 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -0,0 +1,60 @@ + + + + sourcePath + the source path + + + whitelist + the white list + + + allowedtypes + the allowed types + + + 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 + CountryPropagation + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + --whitelist${whitelist} + --allowedtypes${allowedtypes} + + + + + + + \ No newline at end of file From c0022fec9f0067ff036b9bd92ca1c4356c88c0f2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:24:11 +0100 Subject: [PATCH 009/259] moved on upper package to serve other propagations --- .../dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java index f006a320b..d08b0e33d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp.countrypropagation; +package eu.dnetlib.dhp; import java.io.Serializable; From 2688a89c21455eef99a28d259eac1981b5355643 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:24:40 +0100 Subject: [PATCH 010/259] changed relclass and reltype in relation specification --- .../eu/dnetlib/dhp/PropagationConstant.java | 112 ++++++++++++++++++ 1 file changed, 112 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index eed336d5b..6957490bc 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -1,4 +1,116 @@ package eu.dnetlib.dhp; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.spark.api.java.function.PairFunction; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + public class PropagationConstant { + public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional"; + + public final static String PROPAGATION_DATA_INFO_TYPE = "propagation"; + + + public final static String DNET_COUNTRY_SCHEMA = "dnet:countries"; + public final static String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public final static String DNET_SCHEMA_ID = "dnet:provenanceActions"; + + public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; + public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; + public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; + public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; + + public final static String RELATION_DATASOURCEORGANIZATION_REL_TYPE = "datasourceOrganization"; + public final static String RELATION_DATASOURCEORGANIZATION_SUBREL_TYPE = "provision"; + public final static String RELATION_ORGANIZATION_DATASOURCE_REL_CLASS = "isProvidedBy"; + public final static String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; + + public final static String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; + public final static String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; + public final static String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; + public final static String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; + + + public static Country getCountry(String country){ + Country nc = new Country(); + nc.setClassid(country); + nc.setClassname(country); + nc.setSchemename(DNET_COUNTRY_SCHEMA); + nc.setSchemeid(DNET_COUNTRY_SCHEMA); + nc.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME)); + return nc; + } + + public static DataInfo getDataInfo(String inference_provenance, String inference_class_id, String inference_class_name){ + DataInfo di = new DataInfo(); + di.setInferred(true); + di.setInferenceprovenance(inference_provenance); + di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); + return di; + } + + public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { + Qualifier pa = new Qualifier(); + pa.setClassid(inference_class_id); + pa.setClassname(inference_class_name); + pa.setSchemeid(DNET_SCHEMA_ID); + pa.setSchemename(DNET_SCHEMA_NAME); + return pa; + } + + + public static Relation getRelation(String source, String target, String rel_class, String rel_type, String subrel_type, String inference_provenance, String inference_class_id, String inference_class_name){ + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(rel_class); + r.setRelType(rel_type); + r.setSubRelType(subrel_type); + r.setDataInfo(getDataInfo(inference_provenance, inference_class_id, inference_class_name)); + return r; +} + + public static PairFunction toPair() { + return e -> new Tuple2<>( e.getSourceId(), e); + + } + + public static List getTypedRowsDatasourceResult(OafEntity oaf) { + List lst = new ArrayList<>(); + Set datasources_provenance = new HashSet<>(); + List instanceList = null; + String type = ""; + if (oaf.getClass() == Publication.class) { + instanceList = ((Publication) oaf).getInstance(); + type = "publication"; + } + if (oaf.getClass() == Dataset.class){ + instanceList = ((Dataset)oaf).getInstance(); + type = "dataset"; + } + + if (oaf.getClass() == Software.class){ + instanceList = ((Software)oaf).getInstance(); + type = "software"; + } + + if (oaf.getClass() == OtherResearchProduct.class){ + instanceList = ((OtherResearchProduct)oaf).getInstance(); + type = "otherresearchproduct"; + } + + + for (Instance i : instanceList) { + datasources_provenance.add(i.getCollectedfrom().getKey()); + datasources_provenance.add(i.getHostedby().getKey()); + } + for (String dsId : datasources_provenance) { + lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); + } + return lst; + } } From ed262293a69b5785421d8e3afcc8a137122e3f39 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:25:32 +0100 Subject: [PATCH 011/259] aligned to new snapshot version 1.1.6 --- dhp-workflows/dhp-propagation/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml index 10fcd38bb..0e7ebe804 100644 --- a/dhp-workflows/dhp-propagation/pom.xml +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.0.5-SNAPSHOT + 1.1.6-SNAPSHOT 4.0.0 From b736a9581c8dee7a215c01f19ddf5e14a8cfab1c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:27:28 +0100 Subject: [PATCH 012/259] changed relclass and reltype in reelation specification for country propagation and implementation of propagation of result affiliation through institutional repositories --- .../SparkCountryPropagationJob.java | 78 ++--- ...arkResultToOrganizationFromIstRepoJob.java | 305 +++--------------- 2 files changed, 73 insertions(+), 310 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 073c80d1b..3a6e7e8e2 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.countrypropagation; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Dataset; @@ -9,13 +10,14 @@ import org.apache.hadoop.io.Text; 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.sql.*; import scala.Tuple2; import java.io.File; import java.util.*; +import static eu.dnetlib.dhp.PropagationConstant.*; + public class SparkCountryPropagationJob { public static void main(String[] args) throws Exception { @@ -51,9 +53,10 @@ public class SparkCountryPropagationJob { JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> "datasourceOrganization".equals(r.getRelClass()) && "isProvidedBy".equals(r.getRelType())) + .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_ORGANIZATION_DATASOURCE_REL_CLASS.equals(r.getRelType())) .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); + .mapToPair(toPair()); //id is the organization identifier + JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) @@ -72,7 +75,7 @@ public class SparkCountryPropagationJob { .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); JavaPairRDD datasource_results = publications - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -81,7 +84,7 @@ public class SparkCountryPropagationJob { return ret.iterator(); }) .union(datasets - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -90,7 +93,7 @@ public class SparkCountryPropagationJob { return ret.iterator(); })) .union(software - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -99,7 +102,7 @@ public class SparkCountryPropagationJob { return ret.iterator(); })) .union(other - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -155,55 +158,53 @@ public class SparkCountryPropagationJob { results.leftOuterJoin(toupdateresult) .map(c -> { OafEntity oaf = c._2()._1(); - List qualifierList = null; + List countryList = null; if (oaf.getClass() == Publication.class) { - qualifierList = ((Publication) oaf).getCountry(); + countryList = ((Publication) oaf).getCountry(); } if (oaf.getClass() == Dataset.class){ - qualifierList = ((Dataset) oaf).getCountry(); + countryList = ((Dataset) oaf).getCountry(); } if (oaf.getClass() == Software.class){ - qualifierList = ((Software) oaf).getCountry(); + countryList = ((Software) oaf).getCountry(); } if (oaf.getClass() == OtherResearchProduct.class){ - qualifierList = ((OtherResearchProduct) oaf).getCountry(); + countryList = ((OtherResearchProduct) oaf).getCountry(); } if (c._2()._2().isPresent()) { HashSet countries = new HashSet<>(); - for (Qualifier country : qualifierList) { + for (Qualifier country : countryList) { countries.add(country.getClassid()); } TypedRow t = c._2()._2().get(); for (String country : t.getCountry().split(";")) { if (!countries.contains(country)) { - Qualifier q = new Qualifier(); - q.setClassid(country); - qualifierList.add(q); + countryList.add(getCountry(country)); } } if (oaf.getClass() == Publication.class) { - ((Publication) oaf).setCountry(qualifierList); + ((Publication) oaf).setCountry(countryList); return (Publication) oaf; } if (oaf.getClass() == Dataset.class){ - ((Dataset) oaf).setCountry(qualifierList); + ((Dataset) oaf).setCountry(countryList); return (Dataset) oaf; } if (oaf.getClass() == Software.class){ - ((Software) oaf).setCountry(qualifierList); + ((Software) oaf).setCountry(countryList); return (Software) oaf; } if (oaf.getClass() == OtherResearchProduct.class){ - ((OtherResearchProduct) oaf).setCountry(qualifierList); + ((OtherResearchProduct) oaf).setCountry(countryList); return (OtherResearchProduct) oaf; } } @@ -215,40 +216,7 @@ public class SparkCountryPropagationJob { .saveAsTextFile(outputPath+"/"+type); } - private static List getTypedRows(OafEntity oaf) { - List lst = new ArrayList<>(); - Set datasources_provenance = new HashSet<>(); - List instanceList = null; - String type = ""; - if (oaf.getClass() == Publication.class) { - instanceList = ((Publication) oaf).getInstance(); - type = "publication"; - } - if (oaf.getClass() == Dataset.class){ - instanceList = ((Dataset)oaf).getInstance(); - type = "dataset"; - } - if (oaf.getClass() == Software.class){ - instanceList = ((Software)oaf).getInstance(); - type = "software"; - } - - if (oaf.getClass() == OtherResearchProduct.class){ - instanceList = ((OtherResearchProduct)oaf).getInstance(); - type = "otherresearchproduct"; - } - - - for (Instance i : instanceList) { - datasources_provenance.add(i.getCollectedfrom().getKey()); - datasources_provenance.add(i.getHostedby().getKey()); - } - for (String dsId : datasources_provenance) { - lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); - } - return lst; - } private static JavaPairRDD getResults(JavaSparkContext sc , String inputPath){ @@ -280,10 +248,6 @@ public class SparkCountryPropagationJob { - private static PairFunction toPair() { - return e -> new Tuple2<>( e.getSourceId(), e); - - }; } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index 5b9504fec..c8dd27dd1 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -1,11 +1,22 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; 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.sql.SparkSession; - +import org.apache.hadoop.io.Text; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; +import java.util.ArrayList; +import java.util.List; + +import eu.dnetlib.dhp.schema.oaf.*; +import scala.Tuple2; + +import static eu.dnetlib.dhp.PropagationConstant.*; public class SparkResultToOrganizationFromIstRepoJob { public static void main(String[] args) throws Exception { @@ -28,78 +39,28 @@ public class SparkResultToOrganizationFromIstRepoJob { if (!directory.exists()) { directory.mkdirs(); } - } - } -/* -package eu.dnetlib.dhp.countrypropagation; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; -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.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.*; -import scala.Tuple2; - -import java.io.File; -import java.util.*; -import java.util.stream.Stream; - -public class SparkCountryPropagationJob { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = SparkSession - .builder() - .appName(SparkCountryPropagationJob.class.getSimpleName()) - .master(parser.get("master")) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; - - File directory = new File(outputPath); - - if(!directory.exists()){ - directory.mkdirs(); - } - - //TODO: add as Job Parameters - List whitelist = Arrays.asList("10|opendoar____::300891a62162b960cf02ce3827bb363c"); - List allowedtypes = Arrays.asList("pubsrepository::institutional"); - - - JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) - .filter(org -> !org.getDataInfo().getDeletedbyinference()) - .map(org -> new TypedRow().setSourceId(org.getId()).setCountry(org.getCountry().getClassid())) - .mapToPair(toPair()); - - JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> "datasourceOrganization".equals(r.getRelClass()) && "isProvidedBy".equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); + //get the institutional repositories JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) - .filter(ds -> whitelist.contains(ds.getId()) || allowedtypes.contains(ds.getDatasourcetype().getClassid())) + .filter(ds -> INSTITUTIONAL_REPO_TYPE.equals(ds.getDatasourcetype().getClassid())) .map(ds -> new TypedRow().setSourceId(ds.getId())) .mapToPair(toPair()); + JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + JavaPairRDD instdatasource_organization = datasources.join(rel_datasource_organization) + .map(x -> x._2()._2()) + .mapToPair(toPair()); + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)); JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) @@ -110,7 +71,7 @@ public class SparkCountryPropagationJob { .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); JavaPairRDD datasource_results = publications - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -119,7 +80,7 @@ public class SparkCountryPropagationJob { return ret.iterator(); }) .union(datasets - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -128,16 +89,16 @@ public class SparkCountryPropagationJob { return ret.iterator(); })) .union(software - .map(oaf -> getTypedRows(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) .union(other - .map(oaf -> getTypedRows(oaf)) + .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { ArrayList> ret = new ArrayList<>(); for (TypedRow t : f) { @@ -146,185 +107,23 @@ public class SparkCountryPropagationJob { return ret.iterator(); })); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - JavaPairRDD datasource_country = organizations.join(organization_datasource) - .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) - .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) - - - JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) - .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())); - - - JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) - .map(u -> u._2()._2().setCountry(u._2()._1().getCountry())) - .mapToPair(toPair()) - .reduceByKey((a, p) -> { - if (a == null) { - return p; - } - if (p == null) { - return a; - } - HashSet countries = new HashSet(); - countries.addAll(Arrays.asList(a.getCountry().split(";"))); - countries.addAll(Arrays.asList(p.getCountry().split(";"))); - String country = new String(); - for (String c : countries) { - country += c + ";"; - } - - return a.setCountry(country); + JavaRDD newRels = instdatasource_organization.join(datasource_results) + .flatMap(c -> { + List rels = new ArrayList(); + rels.add(getRelation(c._2()._1().getTargetId(), c._2()._2().getTargetId(), RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + rels.add(getRelation(c._2()._2().getTargetId(), c._2()._1().getTargetId(), RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + return rels.iterator(); }); + newRels.map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation_new"); - updateResult(pubs, toupdateresult, outputPath, "publication"); - updateResult(dss, toupdateresult, outputPath, "dataset"); - updateResult(sfw, toupdateresult, outputPath, "software"); - updateResult(orp, toupdateresult, outputPath, "otherresearchproduct"); - //we use leftOuterJoin because we want to rebuild the entire structure - + newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation"); } - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { - results.leftOuterJoin(toupdateresult) - .map(c -> { - OafEntity oaf = c._2()._1(); - List qualifierList = null; - if (oaf.getClass() == Publication.class) { - qualifierList = ((Publication) oaf).getCountry(); - - } - if (oaf.getClass() == Dataset.class){ - qualifierList = ((Dataset) oaf).getCountry(); - } - - if (oaf.getClass() == Software.class){ - qualifierList = ((Software) oaf).getCountry(); - } - - if (oaf.getClass() == OtherResearchProduct.class){ - qualifierList = ((OtherResearchProduct) oaf).getCountry(); - } - - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : qualifierList) { - countries.add(country.getClassid()); - } - TypedRow t = c._2()._2().get(); - - for (String country : t.getCountry().split(";")) { - if (!countries.contains(country)) { - Qualifier q = new Qualifier(); - q.setClassid(country); - qualifierList.add(q); - } - - } - if (oaf.getClass() == Publication.class) { - ((Publication) oaf).setCountry(qualifierList); - return (Publication) oaf; - - } - if (oaf.getClass() == Dataset.class){ - ((Dataset) oaf).setCountry(qualifierList); - return (Dataset) oaf; - } - - if (oaf.getClass() == Software.class){ - ((Software) oaf).setCountry(qualifierList); - return (Software) oaf; - } - - if (oaf.getClass() == OtherResearchProduct.class){ - ((OtherResearchProduct) oaf).setCountry(qualifierList); - return (OtherResearchProduct) oaf; - } - } - - - return null; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - - private static List getTypedRows(OafEntity oaf) { - List lst = new ArrayList<>(); - Set datasources_provenance = new HashSet<>(); - List instanceList = null; - String type = ""; - if (oaf.getClass() == Publication.class) { - instanceList = ((Publication) oaf).getInstance(); - type = "publication"; - } - if (oaf.getClass() == Dataset.class){ - instanceList = ((Dataset)oaf).getInstance(); - type = "dataset"; - } - - if (oaf.getClass() == Software.class){ - instanceList = ((Software)oaf).getInstance(); - type = "software"; - } - - if (oaf.getClass() == OtherResearchProduct.class){ - instanceList = ((OtherResearchProduct)oaf).getInstance(); - type = "otherresearchproduct"; - } - - - for (Instance i : instanceList) { - datasources_provenance.add(i.getCollectedfrom().getKey()); - datasources_provenance.add(i.getHostedby().getKey()); - } - for (String dsId : datasources_provenance) { - lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); - } - return lst; - } - - - private static JavaPairRDD getResults(JavaSparkContext sc , String inputPath){ - - return - sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) - .filter(ds -> !ds.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("dataset").setSourceId(oaf.getId())) - .mapToPair(toPair()) - .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) - .filter(o -> !o.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("otherresearchproduct").setSourceId(oaf.getId())) - .mapToPair(toPair())) - .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) - .filter(s -> !s.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("software").setSourceId(oaf.getId())) - .mapToPair(toPair())) - .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("publication").setSourceId(oaf.getId())) - .mapToPair(toPair())); - - - } - - - - private static PairFunction toPair() { - return e -> new Tuple2<>( e.getSourceId(), e); - - }; - } - - - */ \ No newline at end of file From b81e6af429338087dcd2b96cf4d458cde69ccc24 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 18 Feb 2020 17:30:44 +0100 Subject: [PATCH 013/259] added config for new propagation --- ...esulaffiliationfrominstrepo_parameters.json | 12 ------------ .../oozie_app/workflow.xml | 18 ++++-------------- 2 files changed, 4 insertions(+), 26 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json index 090f3f152..2aa5332b2 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json @@ -10,17 +10,5 @@ "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true - }, - { - "paramName":"wl", - "paramLongName":"whitelist", - "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", - "paramRequired": true - }, - { - "paramName":"at", - "paramLongName":"allowedtypes", - "paramDescription": "the types of the allowed datasources. Split by ;", - "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index f24417bc0..ed634b37d 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -1,17 +1,9 @@ - + sourcePath the source path - - whitelist - the white list - - - allowedtypes - the allowed types - sparkDriverMemory memory for driver process @@ -26,20 +18,20 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + ${jobTracker} ${nameNode} yarn-cluster cluster CountryPropagation - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-memory ${sparkExecutorMemory} --executor-cores ${sparkExecutorCores} @@ -49,8 +41,6 @@ -mt yarn-cluster --sourcePath${sourcePath} - --whitelist${whitelist} - --allowedtypes${allowedtypes} From 7167673a58ada940619b4a42545bdf3c6e3918b6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 14:54:18 +0100 Subject: [PATCH 014/259] implementation and configuration for propagation of project to result through semantic relation: P -> R1 and R1 -> supplemented by -> R2 => P -> R2 --- .../SparkResultToProjectThroughSemRelJob.java | 120 ++++++++++++++++++ .../input_projecttoresult_parameters.json | 21 +++ .../oozie_app/config-default.xml | 18 +++ .../projecttoresult/oozie_app/workflow.xml | 55 ++++++++ 4 files changed, 214 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java new file mode 100644 index 000000000..1d99b3ec8 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java @@ -0,0 +1,120 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.toPair; + +public class SparkResultToProjectThroughSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToProjectThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToProjectThroughSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/projecttoresult"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + + JavaPairRDD result_result = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + JavaPairRDD result_project = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result + JavaPairRDD project_result = result_project.join(result_result) + .map(c -> { + String projectId = c._2()._1().getTargetId(); + String resultId = c._2()._2().getTargetId(); + return new TypedRow().setSourceId(projectId).setTargetId(resultId); + }) + .mapToPair(toPair()); + + //relationships from project to result. One Pair for each project => project id list of results related to the project + JavaPairRDD project_results = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()) + .reduceByKey((a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + a.addAll(b.getAccumulator()); + return a; + }); + + + + JavaRDD newRels = project_result.join(project_results) + .flatMap(c -> { + String resId = c._2()._1().getTargetId(); + + if (c._2()._2().getAccumulator().contains(resId)) { + return null; + } + String progId = c._2()._1().getSourceId(); + List rels = new ArrayList(); + + rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + return rels.iterator(); + }) + .cache(); + + newRels.map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation_new"); + + newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation"); + + } + + +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json new file mode 100644 index 000000000..695dc176c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json @@ -0,0 +1,21 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + + { + "paramName":"asr", + "paramLongName":"allowedsemrels", + "paramDescription": "the types of the allowed datasources. Split by ;", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml new file mode 100644 index 000000000..4c073f0a2 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + sourcePath + the source path + + + allowedsemrels + the allowed semantics + + + 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 + ProjectToResultPropagation + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + + + + + + + \ No newline at end of file From 8aa3b4d7c048592668f5b3dac2ee3437d1ce7c90 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 14:55:54 +0100 Subject: [PATCH 015/259] adding to propagation constants the ones needed for propagation of project to result and addition of new accumulator Set in typed row to collect values of a type --- .../eu/dnetlib/dhp/PropagationConstant.java | 11 ++++++++ .../main/java/eu/dnetlib/dhp/TypedRow.java | 26 ++++++++++--------- 2 files changed, 25 insertions(+), 12 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 6957490bc..7cd8c54d1 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -23,6 +23,9 @@ public class PropagationConstant { public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; + public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; + public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; + public final static String RELATION_DATASOURCEORGANIZATION_REL_TYPE = "datasourceOrganization"; public final static String RELATION_DATASOURCEORGANIZATION_SUBREL_TYPE = "provision"; @@ -34,6 +37,14 @@ public class PropagationConstant { public final static String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; public final static String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; + public static final String RELATION_RESULTRESULT_REL_TYPE = "resultResult"; + public static final String RELATION_RESULTRESULT_SUBREL_TYPE = "supplement"; + + public static final String RELATION_RESULTPROJECT_REL_TYPE = "resultProject"; + public static final String RELATION_RESULTPROJECT_SUBREL_TYPE = "outcome"; + public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; + public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; + public static Country getCountry(String country){ Country nc = new Country(); diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java index d08b0e33d..13f3ca611 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java @@ -2,31 +2,33 @@ package eu.dnetlib.dhp; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; +import java.util.*; public class TypedRow implements Serializable { private String sourceId; private String targetId; private String type; - private String country; + private String value; + private Set accumulator; - public List getAccumulator() { + public Set getAccumulator() { return accumulator; } - public TypedRow setAccumulator(List accumulator) { + public TypedRow setAccumulator(Set accumulator) { this.accumulator = accumulator; return this; } - private List accumulator; + + public void addAll(Set toadd){ + this.accumulator.addAll(toadd); + } public void add(String a){ if (accumulator == null){ - accumulator = new ArrayList<>(); + accumulator = new HashSet<>(); } accumulator.add(a); } @@ -35,12 +37,12 @@ public class TypedRow implements Serializable { return accumulator.iterator(); } - public String getCountry() { - return country; + public String getValue() { + return value; } - public TypedRow setCountry(String country) { - this.country = country; + public TypedRow setValue(String value) { + this.value = value; return this; } From 9e1678ccf85a8c44df8d39fbd37c70fec77ea3ed Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 14:59:24 +0100 Subject: [PATCH 016/259] fix error in workflow name --- .../oozie_app/workflow.xml | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index ed634b37d..b2f6e5919 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -1,50 +1,50 @@ - - - sourcePath - the source path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - + + + sourcePath + the source 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())}] - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - CountryPropagation - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${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" - - -mt yarn-cluster - --sourcePath${sourcePath} - - - - + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + AffiliatioPropagation + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + + + + - + \ No newline at end of file From bb0fdf5e0aa3aad7d458631df48684a0592eb88a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 15:00:46 +0100 Subject: [PATCH 017/259] fix wrong source target in new relation --- .../SparkResultToOrganizationFromIstRepoJob.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index c8dd27dd1..d0ef02a4b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -110,10 +110,12 @@ public class SparkResultToOrganizationFromIstRepoJob { JavaRDD newRels = instdatasource_organization.join(datasource_results) .flatMap(c -> { List rels = new ArrayList(); - rels.add(getRelation(c._2()._1().getTargetId(), c._2()._2().getTargetId(), RELATION_ORGANIZATION_RESULT_REL_CLASS, + String orgId = c._2()._1().getTargetId(); + String resId = c._2()._2().getTargetId(); + rels.add(getRelation(orgId, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - rels.add(getRelation(c._2()._2().getTargetId(), c._2()._1().getTargetId(), RELATION_ORGANIZATION_RESULT_REL_CLASS, + rels.add(getRelation(resId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); return rels.iterator(); From ab84163bb38a92d31623ea40216fdaf5f11f003a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 15:02:50 +0100 Subject: [PATCH 018/259] added set accumulator in TypedRow and used it to acucmulate country information in Country Propagation --- .../main/java/eu/dnetlib/dhp/TypedRow.java | 26 ++++++++++--------- .../SparkCountryPropagationJob.java | 17 ++++-------- 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java index d08b0e33d..13f3ca611 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java @@ -2,31 +2,33 @@ package eu.dnetlib.dhp; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; +import java.util.*; public class TypedRow implements Serializable { private String sourceId; private String targetId; private String type; - private String country; + private String value; + private Set accumulator; - public List getAccumulator() { + public Set getAccumulator() { return accumulator; } - public TypedRow setAccumulator(List accumulator) { + public TypedRow setAccumulator(Set accumulator) { this.accumulator = accumulator; return this; } - private List accumulator; + + public void addAll(Set toadd){ + this.accumulator.addAll(toadd); + } public void add(String a){ if (accumulator == null){ - accumulator = new ArrayList<>(); + accumulator = new HashSet<>(); } accumulator.add(a); } @@ -35,12 +37,12 @@ public class TypedRow implements Serializable { return accumulator.iterator(); } - public String getCountry() { - return country; + public String getValue() { + return value; } - public TypedRow setCountry(String country) { - this.country = country; + public TypedRow setValue(String value) { + this.value = value; return this; } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 3a6e7e8e2..ca126ac1a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -47,7 +47,7 @@ public class SparkCountryPropagationJob { JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) .filter(org -> !org.getDataInfo().getDeletedbyinference()) - .map(org -> new TypedRow().setSourceId(org.getId()).setCountry(org.getCountry().getClassid())) + .map(org -> new TypedRow().setSourceId(org.getId()).setValue(org.getCountry().getClassid())) .mapToPair(toPair()); JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) @@ -126,7 +126,7 @@ public class SparkCountryPropagationJob { JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) - .map(u -> u._2()._2().setCountry(u._2()._1().getCountry())) + .map(u -> u._2()._2().setValue(u._2()._1().getValue())) .mapToPair(toPair()) .reduceByKey((a, p) -> { if (a == null) { @@ -135,15 +135,8 @@ public class SparkCountryPropagationJob { if (p == null) { return a; } - HashSet countries = new HashSet(); - countries.addAll(Arrays.asList(a.getCountry().split(";"))); - countries.addAll(Arrays.asList(p.getCountry().split(";"))); - String country = new String(); - for (String c : countries) { - country += c + ";"; - } - - return a.setCountry(country); + a.addAll(p.getAccumulator()); + return a; }); updateResult(pubs, toupdateresult, outputPath, "publication"); @@ -182,7 +175,7 @@ public class SparkCountryPropagationJob { } TypedRow t = c._2()._2().get(); - for (String country : t.getCountry().split(";")) { + for (String country : t.getAccumulator()) { if (!countries.contains(country)) { countryList.add(getCountry(country)); } From 79ff79b0cdbe098b9cd31280dd7cb91a9cd8530c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 17:02:39 +0100 Subject: [PATCH 019/259] propagation of result to community through semantic relation: C -> R and R -> isSupplementedBy R1 => C -> R1 --- ...parkResultToCommunityThroughSemRelJob.java | 262 ++++++++++++++++++ 1 file changed, 262 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java new file mode 100644 index 000000000..a096ef13f --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -0,0 +1,262 @@ +package eu.dnetlib.dhp.communitytoresultthroughsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME; + +public class SparkResultToCommunityThroughSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/communitytoresultthroughsemrel/input_communitytoresultthroughsemrel_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/communitytoresultthroughsemrel"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); + final List communityIdList = Arrays.asList(parser.get("communityidlist").split(";")); + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } +/* + //get the institutional repositories + JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) + .filter(ds -> INSTITUTIONAL_REPO_TYPE.equals(ds.getDatasourcetype().getClassid())) + .map(ds -> new TypedRow().setSourceId(ds.getId())) + .mapToPair(toPair()); + + + JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + JavaPairRDD instdatasource_organization = datasources.join(rel_datasource_organization) + .map(x -> x._2()._2()) + .mapToPair(toPair()); + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)); + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + + JavaPairRDD datasource_results = publications + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + }) + .union(datasets + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(software + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })) + .union(other + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + })); + + JavaRDD newRels = instdatasource_organization.join(datasource_results) + .flatMap(c -> { + List rels = new ArrayList(); + String orgId = c._2()._1().getTargetId(); + String resId = c._2()._2().getTargetId(); + rels.add(getRelation(orgId, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + rels.add(getRelation(resId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + return rels.iterator(); + }); + newRels.map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation_new"); + + newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/relation");*/ + } +} +/* +package eu.dnetlib.data.mapreduce.hbase.propagation.communitytoresult; + +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import eu.dnetlib.data.mapreduce.hbase.propagation.Value; +import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; +import eu.dnetlib.data.proto.OafProtos; +import eu.dnetlib.data.proto.ResultProtos; +import eu.dnetlib.data.proto.TypeProtos; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.TableMapper; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.stream.Collectors; + +import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.COUNTER_PROPAGATION; +import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.DEFAULT_COMMUNITY_RELATION_SET; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; + + +public class CommunityToResultMapper extends TableMapper { + + private Text keyOut; + private Text valueOut; + private String[] sem_rels; + private String trust; + CommunityList idCommunityList; + + @Override + protected void setup(final Context context) throws IOException, InterruptedException { + + idCommunityList = new Gson().fromJson(context.getConfiguration().get("community.id.list"), CommunityList.class); + keyOut = new Text(); + valueOut = new Text(); + + sem_rels = context.getConfiguration().getStrings("propagatetocommunity.semanticrelations", DEFAULT_COMMUNITY_RELATION_SET); + trust = context.getConfiguration().get("propagatetocommunity.trust","0.85"); + + } + + @Override + protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { + + final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); + + //If the type is not result I do not need to process it + if (!type.equals(TypeProtos.Type.result)) { + return; + } + + //verify if entity is valid + final OafProtos.OafEntity entity = getEntity(value, type); + if (entity == null) { + context.getCounter(COUNTER_PROPAGATION, "Del by inference or null body for result").increment(1); + return; + } + final Set toemitrelations = new HashSet<>(); + //verify if we have some relation + for (String sem_rel : sem_rels) + toemitrelations.addAll(getRelationTarget(value, sem_rel, context, COUNTER_PROPAGATION)); + + if (toemitrelations.isEmpty()) { + context.getCounter(COUNTER_PROPAGATION, "No allowed semantic relation present in result").increment(1); + return; + } + + //verify if we have a relation to a context in the body + Set contextIds = entity.getResult().getMetadata().getContextList() + .stream() + .map(ResultProtos.Result.Context::getId) + .collect(Collectors.toSet()); + + //verify if we have a relation to a context in the update part made by the inference + NavigableMap map = value.getFamilyMap(Bytes.toBytes(TypeProtos.Type.result.toString())); + + final Map stringMap = Maps.newHashMap(); + for (Map.Entry e : map.entrySet()) { + stringMap.put(Bytes.toString(e.getKey()), e.getValue()); + } + + // we fetch all the body updates + for (final String o : stringMap.keySet()) { + if (o.startsWith("update_")) { + final OafProtos.Oaf update = OafProtos.Oaf.parseFrom(stringMap.get(o)); + contextIds.addAll(update.getEntity().getResult().getMetadata().getContextList() + .stream() + .map(ResultProtos.Result.Context::getId) + .map(s -> s.split("::")[0]) + .collect(Collectors.toSet())); + } + } + + //we verify if we have something + if (contextIds.isEmpty()) { + context.getCounter(COUNTER_PROPAGATION, "No context in the body and in the update of the result").increment(1); + return; + } + + //verify if some of the context collected for the result are associated to a community in the communityIdList + for (String id : idCommunityList) { + if (contextIds.contains(id)) { + for (String target : toemitrelations) { + keyOut.set(target); + valueOut.set(Value.newInstance(id).setTrust(trust).toJson()); + context.write(keyOut, valueOut); + context.getCounter(COUNTER_PROPAGATION, "Emit propagation for " + id).increment(1); + } + } + } + + } + +} + + + */ \ No newline at end of file From 5f63ab1416d98164f6993651ddbd67d01f1ff54b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 17:59:02 +0100 Subject: [PATCH 020/259] to query the information system to get the list of comunities up to now. It will have a more general usage when introducing bulk tagging --- .../dnetlib/dhp/QueryInformationSystem.java | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java new file mode 100644 index 000000000..b1ec7726e --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java @@ -0,0 +1,23 @@ +package eu.dnetlib.dhp; + +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + +import java.util.List; + + +public class QueryInformationSystem { + private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" + + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" + + " return $x//CONFIGURATION/context/@id/string()"; + + public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + return isLookUp.quickSearchProfile(XQUERY); + + } + + +} From d0279af6305bbafbb01c29b08fa3b6bf660fe624 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 17:59:24 +0100 Subject: [PATCH 021/259] start to implement the business logic --- .../SparkResultToCommunityThroughSemRelJob.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java index a096ef13f..b99d9098f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.communitytoresultthroughsemrel; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob; @@ -39,7 +40,7 @@ public class SparkResultToCommunityThroughSemRelJob { final String outputPath = "/tmp/provision/propagation/communitytoresultthroughsemrel"; final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); - final List communityIdList = Arrays.asList(parser.get("communityidlist").split(";")); + final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); File directory = new File(outputPath); @@ -47,12 +48,7 @@ public class SparkResultToCommunityThroughSemRelJob { directory.mkdirs(); } /* - //get the institutional repositories - JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) - .filter(ds -> INSTITUTIONAL_REPO_TYPE.equals(ds.getDatasourcetype().getClassid())) - .map(ds -> new TypedRow().setSourceId(ds.getId())) - .mapToPair(toPair()); + JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) From a153a07997653d5312924deb0292fee1b961e3fc Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 19 Feb 2020 18:03:13 +0100 Subject: [PATCH 022/259] none --- ...parkResultToCommunityThroughSemRelJob.java | 127 +++++++++--------- 1 file changed, 62 insertions(+), 65 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java index b99d9098f..b2178f1fb 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -48,86 +48,83 @@ public class SparkResultToCommunityThroughSemRelJob { directory.mkdirs(); } /* - - JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + + JavaPairRDD result_result = relations .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelType())) + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) .mapToPair(toPair()); - JavaPairRDD instdatasource_organization = datasources.join(rel_datasource_organization) - .map(x -> x._2()._2()) + JavaPairRDD result_project = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) .mapToPair(toPair()); - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); - - JavaPairRDD datasource_results = publications - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); + //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result + JavaPairRDD project_result = result_project.join(result_result) + .map(c -> { + String projectId = c._2()._1().getTargetId(); + String resultId = c._2()._2().getTargetId(); + return new TypedRow().setSourceId(projectId).setTargetId(resultId); }) - .union(datasets - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(software - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(other - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })); + .mapToPair(toPair()); - JavaRDD newRels = instdatasource_organization.join(datasource_results) - .flatMap(c -> { - List rels = new ArrayList(); - String orgId = c._2()._1().getTargetId(); - String resId = c._2()._2().getTargetId(); - rels.add(getRelation(orgId, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - rels.add(getRelation(resId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - return rels.iterator(); + //relationships from project to result. One Pair for each project => project id list of results related to the project + JavaPairRDD project_results = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()) + .reduceByKey((a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + a.addAll(b.getAccumulator()); + return a; }); + + + + JavaRDD newRels = project_result.join(project_results) + .flatMap(c -> { + String resId = c._2()._1().getTargetId(); + + if (c._2()._2().getAccumulator().contains(resId)) { + return null; + } + String progId = c._2()._1().getSourceId(); + List rels = new ArrayList(); + + rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + return rels.iterator(); + }) + .cache(); + newRels.map(p -> new ObjectMapper().writeValueAsString(p)) .saveAsTextFile(outputPath + "/relation_new"); newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation");*/ + .saveAsTextFile(outputPath + "/relation"); + + } + + +} +*/ } } /* From a86426776ad034147b84cc80c926592aeff0a47f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:20:19 +0100 Subject: [PATCH 023/259] Changed from Oaf to Result the type of the updateResult method parameter, not to be forced to cast each time --- ...parkResultToCommunityThroughSemRelJob.java | 285 ++++++------------ .../SparkOrcidToReseltFromSemRelJob.java | 78 +++++ .../SparkOrcidToResultFromSemRelJob.java | 123 ++++++++ 3 files changed, 291 insertions(+), 195 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java index b2178f1fb..707ac9eea 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -15,10 +15,8 @@ import org.apache.spark.sql.SparkSession; import scala.Tuple2; import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; +import java.util.*; +import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME; @@ -26,7 +24,7 @@ import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RELATION_RESULT_ORG public class SparkResultToCommunityThroughSemRelJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/communitytoresultthroughsemrel/input_communitytoresultthroughsemrel_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); parser.parseArgument(args); final SparkSession spark = SparkSession .builder() @@ -39,7 +37,7 @@ public class SparkResultToCommunityThroughSemRelJob { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/communitytoresultthroughsemrel"; - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); File directory = new File(outputPath); @@ -47,209 +45,106 @@ public class SparkResultToCommunityThroughSemRelJob { if (!directory.exists()) { directory.mkdirs(); } -/* + JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, + sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class))); + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); - - JavaPairRDD result_result = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); - - JavaPairRDD result_project = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); - - //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result - JavaPairRDD project_result = result_project.join(result_result) - .map(c -> { - String projectId = c._2()._1().getTargetId(); - String resultId = c._2()._2().getTargetId(); - return new TypedRow().setSourceId(projectId).setTargetId(resultId); - }) - .mapToPair(toPair()); - - //relationships from project to result. One Pair for each project => project id list of results related to the project - JavaPairRDD project_results = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + JavaPairRDD resultLinkedToCommunities = publications + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) + .filter(p -> !(p == null)) .mapToPair(toPair()) - .reduceByKey((a, b) -> { - if (a == null) { - return b; + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); + + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResult(pubs, to_add_result_communities, outputPath, "publication"); + updateResult(dss, to_add_result_communities, outputPath, "dataset"); + updateResult(sfw, to_add_result_communities, outputPath, "software"); + updateResult(orp, to_add_result_communities, outputPath, "otherresearchproduct"); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla + + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + Set communityList = p._2()._2().get().getAccumulator(); + for(Context c: r.getContext()){ + if (communityList.contains(c.getId())){ + //verify if the datainfo for this context contains propagation + if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + //community id already in the context of the result. Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } + } + List cc = r.getContext(); + for(String cId: communityList){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); } - if (b == null) { - return a; - } - a.addAll(b.getAccumulator()); - return a; - }); - - - - JavaRDD newRels = project_result.join(project_results) - .flatMap(c -> { - String resId = c._2()._1().getTargetId(); - - if (c._2()._2().getAccumulator().contains(resId)) { - return null; - } - String progId = c._2()._1().getSourceId(); - List rels = new ArrayList(); - - rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - return rels.iterator(); + return r; }) - .cache(); - - newRels.map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation_new"); - - newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation"); - + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); } -} -*/ - } -} -/* -package eu.dnetlib.data.mapreduce.hbase.propagation.communitytoresult; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import eu.dnetlib.data.mapreduce.hbase.propagation.Value; -import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; -import eu.dnetlib.data.proto.OafProtos; -import eu.dnetlib.data.proto.ResultProtos; -import eu.dnetlib.data.proto.TypeProtos; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.mapreduce.TableMapper; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.Text; - -import java.io.IOException; -import java.util.HashSet; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Set; -import java.util.stream.Collectors; - -import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.COUNTER_PROPAGATION; -import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.DEFAULT_COMMUNITY_RELATION_SET; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; - - -public class CommunityToResultMapper extends TableMapper { - - private Text keyOut; - private Text valueOut; - private String[] sem_rels; - private String trust; - CommunityList idCommunityList; - - @Override - protected void setup(final Context context) throws IOException, InterruptedException { - - idCommunityList = new Gson().fromJson(context.getConfiguration().get("community.id.list"), CommunityList.class); - keyOut = new Text(); - valueOut = new Text(); - - sem_rels = context.getConfiguration().getStrings("propagatetocommunity.semanticrelations", DEFAULT_COMMUNITY_RELATION_SET); - trust = context.getConfiguration().get("propagatetocommunity.trust","0.85"); - - } - - @Override - protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { - - final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); - - //If the type is not result I do not need to process it - if (!type.equals(TypeProtos.Type.result)) { - return; - } - - //verify if entity is valid - final OafProtos.OafEntity entity = getEntity(value, type); - if (entity == null) { - context.getCounter(COUNTER_PROPAGATION, "Del by inference or null body for result").increment(1); - return; - } - final Set toemitrelations = new HashSet<>(); - //verify if we have some relation - for (String sem_rel : sem_rels) - toemitrelations.addAll(getRelationTarget(value, sem_rel, context, COUNTER_PROPAGATION)); - - if (toemitrelations.isEmpty()) { - context.getCounter(COUNTER_PROPAGATION, "No allowed semantic relation present in result").increment(1); - return; - } - - //verify if we have a relation to a context in the body - Set contextIds = entity.getResult().getMetadata().getContextList() + private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { + Set result_communities = context .stream() - .map(ResultProtos.Result.Context::getId) + .map(c -> c.getId()) .collect(Collectors.toSet()); - - //verify if we have a relation to a context in the update part made by the inference - NavigableMap map = value.getFamilyMap(Bytes.toBytes(TypeProtos.Type.result.toString())); - - final Map stringMap = Maps.newHashMap(); - for (Map.Entry e : map.entrySet()) { - stringMap.put(Bytes.toString(e.getKey()), e.getValue()); - } - - // we fetch all the body updates - for (final String o : stringMap.keySet()) { - if (o.startsWith("update_")) { - final OafProtos.Oaf update = OafProtos.Oaf.parseFrom(stringMap.get(o)); - contextIds.addAll(update.getEntity().getResult().getMetadata().getContextList() - .stream() - .map(ResultProtos.Result.Context::getId) - .map(s -> s.split("::")[0]) - .collect(Collectors.toSet())); + TypedRow tp = new TypedRow(); + tp.setSourceId(id); + tp.setType(type); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + tp.add(communityId); } } - - //we verify if we have something - if (contextIds.isEmpty()) { - context.getCounter(COUNTER_PROPAGATION, "No context in the body and in the update of the result").increment(1); - return; + if (tp.getAccumulator() != null) { + return tp; } - - //verify if some of the context collected for the result are associated to a community in the communityIdList - for (String id : idCommunityList) { - if (contextIds.contains(id)) { - for (String target : toemitrelations) { - keyOut.set(target); - valueOut.set(Value.newInstance(id).setTrust(trust).toJson()); - context.write(keyOut, valueOut); - context.getCounter(COUNTER_PROPAGATION, "Emit propagation for " + id).increment(1); - } - } - } - + return null; } - } - - - */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java new file mode 100644 index 000000000..682bf2200 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java @@ -0,0 +1,78 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +public class SparkOrcidToReseltFromSemRelJob { +} + +/* +public class PropagationOrcidToResultMapper extends TableMapper { + private static final Log log = LogFactory.getLog(PropagationOrcidToResultMapper.class); // NOPMD by marko on 11/24/08 5:02 PM + private Text valueOut; + private ImmutableBytesWritable keyOut; + private String[] sem_rels; + private String trust; + + @Override + protected void setup(final Context context) throws IOException, InterruptedException { + super.setup(context); + valueOut = new Text(); + keyOut = new ImmutableBytesWritable(); + + sem_rels = context.getConfiguration().getStrings("propagatetoorcid.semanticrelations", DEFAULT_RESULT_RELATION_SET); + trust = context.getConfiguration().get("propagatetoorcid.trust","0.85"); + + } + + @Override + protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { + final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); + final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference + + + if (entity != null) { + + if (type == TypeProtos.Type.result){ + Set result_result = new HashSet<>(); + //verifico se il risultato ha una relazione semantica verso uno o piu' risultati. + //per ogni risultato linkato con issupplementto o issupplementedby emetto: + // id risultato linkato come chiave, + // id risultato oggetto del mapping e lista degli autori del risultato oggetto del mapper come value + for(String sem : sem_rels){ + result_result.addAll(getRelationTarget(value, sem, context, COUNTER_PROPAGATION)); + } + if(!result_result.isEmpty()){ + List authorlist = getAuthorList(entity.getResult().getMetadata().getAuthorList()); + Emit e = new Emit(); + e.setId(Bytes.toString(keyIn.get())); + e.setAuthor_list(authorlist); + valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), + trust, + Type.fromsemrel).toJson()); + for (String result: result_result){ + keyOut.set(Bytes.toBytes(result)); + context.write(keyOut,valueOut); + context.getCounter(COUNTER_PROPAGATION,"emit for sem_rel").increment(1); + } + + //emetto anche id dell'oggetto del mapper come chiave e lista degli autori come valore + e.setId(keyIn.toString()); + e.setAuthor_list(authorlist); + valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), trust, Type.fromresult).toJson()); + context.write(keyIn, valueOut); + context.getCounter(COUNTER_PROPAGATION,"emit for result with orcid").increment(1); + + } + } + + } + } + + private List getAuthorList(List author_list){ + + return author_list.stream().map(a -> new JsonFormat().printToString(a)).collect(Collectors.toList()); + + } + + + +} + */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java new file mode 100644 index 000000000..3632cf745 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java @@ -0,0 +1,123 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +import java.io.File; +import java.util.Arrays; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.getResultResultSemRel; + +public class SparkOrcidToResultFromSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcid" + + "toresult_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkOrcidToResultFromSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/orcidtoresult"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + + JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); + } +} +/* +public class PropagationOrcidToResultMapper extends TableMapper { + private static final Log log = LogFactory.getLog(PropagationOrcidToResultMapper.class); // NOPMD by marko on 11/24/08 5:02 PM + private Text valueOut; + private ImmutableBytesWritable keyOut; + private String[] sem_rels; + private String trust; + + @Override + protected void setup(final Context context) throws IOException, InterruptedException { + super.setup(context); + valueOut = new Text(); + keyOut = new ImmutableBytesWritable(); + + sem_rels = context.getConfiguration().getStrings("propagatetoorcid.semanticrelations", DEFAULT_RESULT_RELATION_SET); + trust = context.getConfiguration().get("propagatetoorcid.trust","0.85"); + + } + + @Override + protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { + final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); + final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference + + + if (entity != null) { + + if (type == TypeProtos.Type.result){ + Set result_result = new HashSet<>(); + //verifico se il risultato ha una relazione semantica verso uno o piu' risultati. + //per ogni risultato linkato con issupplementto o issupplementedby emetto: + // id risultato linkato come chiave, + // id risultato oggetto del mapping e lista degli autori del risultato oggetto del mapper come value + for(String sem : sem_rels){ + result_result.addAll(getRelationTarget(value, sem, context, COUNTER_PROPAGATION)); + } + if(!result_result.isEmpty()){ + List authorlist = getAuthorList(entity.getResult().getMetadata().getAuthorList()); + Emit e = new Emit(); + e.setId(Bytes.toString(keyIn.get())); + e.setAuthor_list(authorlist); + valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), + trust, + Type.fromsemrel).toJson()); + for (String result: result_result){ + keyOut.set(Bytes.toBytes(result)); + context.write(keyOut,valueOut); + context.getCounter(COUNTER_PROPAGATION,"emit for sem_rel").increment(1); + } + + //emetto anche id dell'oggetto del mapper come chiave e lista degli autori come valore + e.setId(keyIn.toString()); + e.setAuthor_list(authorlist); + valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), trust, Type.fromresult).toJson()); + context.write(keyIn, valueOut); + context.getCounter(COUNTER_PROPAGATION,"emit for result with orcid").increment(1); + + } + } + + } + } + + private List getAuthorList(List author_list){ + + return author_list.stream().map(a -> new JsonFormat().printToString(a)).collect(Collectors.toList()); + + } + + + +} + */ \ No newline at end of file From 833c83c69434339dd54bf3560c593877ef96ee6a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:21:01 +0100 Subject: [PATCH 024/259] Wrong file name --- .../SparkOrcidToReseltFromSemRelJob.java | 78 ------------------- 1 file changed, 78 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java deleted file mode 100644 index 682bf2200..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToReseltFromSemRelJob.java +++ /dev/null @@ -1,78 +0,0 @@ -package eu.dnetlib.dhp.orcidtoresultfromsemrel; - -public class SparkOrcidToReseltFromSemRelJob { -} - -/* -public class PropagationOrcidToResultMapper extends TableMapper { - private static final Log log = LogFactory.getLog(PropagationOrcidToResultMapper.class); // NOPMD by marko on 11/24/08 5:02 PM - private Text valueOut; - private ImmutableBytesWritable keyOut; - private String[] sem_rels; - private String trust; - - @Override - protected void setup(final Context context) throws IOException, InterruptedException { - super.setup(context); - valueOut = new Text(); - keyOut = new ImmutableBytesWritable(); - - sem_rels = context.getConfiguration().getStrings("propagatetoorcid.semanticrelations", DEFAULT_RESULT_RELATION_SET); - trust = context.getConfiguration().get("propagatetoorcid.trust","0.85"); - - } - - @Override - protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { - final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); - final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference - - - if (entity != null) { - - if (type == TypeProtos.Type.result){ - Set result_result = new HashSet<>(); - //verifico se il risultato ha una relazione semantica verso uno o piu' risultati. - //per ogni risultato linkato con issupplementto o issupplementedby emetto: - // id risultato linkato come chiave, - // id risultato oggetto del mapping e lista degli autori del risultato oggetto del mapper come value - for(String sem : sem_rels){ - result_result.addAll(getRelationTarget(value, sem, context, COUNTER_PROPAGATION)); - } - if(!result_result.isEmpty()){ - List authorlist = getAuthorList(entity.getResult().getMetadata().getAuthorList()); - Emit e = new Emit(); - e.setId(Bytes.toString(keyIn.get())); - e.setAuthor_list(authorlist); - valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), - trust, - Type.fromsemrel).toJson()); - for (String result: result_result){ - keyOut.set(Bytes.toBytes(result)); - context.write(keyOut,valueOut); - context.getCounter(COUNTER_PROPAGATION,"emit for sem_rel").increment(1); - } - - //emetto anche id dell'oggetto del mapper come chiave e lista degli autori come valore - e.setId(keyIn.toString()); - e.setAuthor_list(authorlist); - valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), trust, Type.fromresult).toJson()); - context.write(keyIn, valueOut); - context.getCounter(COUNTER_PROPAGATION,"emit for result with orcid").increment(1); - - } - } - - } - } - - private List getAuthorList(List author_list){ - - return author_list.stream().map(a -> new JsonFormat().printToString(a)).collect(Collectors.toList()); - - } - - - -} - */ \ No newline at end of file From 2b7b05fb291d926371e44b447080feabd7e0e0b5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:22:41 +0100 Subject: [PATCH 025/259] New propagation of ORCID to result exploiting the semantic relation connecting them. R has author with orcid o, R is bounf by strong semantic relationship with R1 that has the same author withouth orcid, then o is also associated to the author in R1 --- .../SparkOrcidToResultFromSemRelJob.java | 247 +++++++++++++----- 1 file changed, 187 insertions(+), 60 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java index 3632cf745..adc92aa75 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java @@ -3,19 +3,26 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.Text; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.SparkSession; +import scala.Tuple2; import java.io.File; import java.util.Arrays; import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; -import static eu.dnetlib.dhp.PropagationConstant.getResultResultSemRel; +import static eu.dnetlib.dhp.PropagationConstant.*; public class SparkOrcidToResultFromSemRelJob { public static void main(String[] args) throws Exception { @@ -46,78 +53,198 @@ public class SparkOrcidToResultFromSemRelJob { .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - } -} -/* -public class PropagationOrcidToResultMapper extends TableMapper { - private static final Log log = LogFactory.getLog(PropagationOrcidToResultMapper.class); // NOPMD by marko on 11/24/08 5:02 PM - private Text valueOut; - private ImmutableBytesWritable keyOut; - private String[] sem_rels; - private String trust; - @Override - protected void setup(final Context context) throws IOException, InterruptedException { - super.setup(context); - valueOut = new Text(); - keyOut = new ImmutableBytesWritable(); + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); - sem_rels = context.getConfiguration().getStrings("propagatetoorcid.semanticrelations", DEFAULT_RESULT_RELATION_SET); - trust = context.getConfiguration().get("propagatetoorcid.trust","0.85"); + //get the results having at least one author pid we are interested in + JavaPairRDD resultswithorcid = publications.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + .union(datasets.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())) + .union(software.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())) + .union(other.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())); + + + JavaPairRDD to_add_orcid_to_result = resultswithorcid.join(result_result) + .map(p -> p._2()._1().setSourceId(p._2()._2().getTargetId())) //associate the pid of the result (target) which should get the orcid to the typed row containing the authors with the orcid from the result(source) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); + updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); + updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); + updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); } - @Override - protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { - final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); - final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference + private static Author enrichAutor(Author autoritative_author, Author author) { + boolean toaddpid = false; - if (entity != null) { + if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { + if (StringUtils.isNoneEmpty(author.getSurname())) { + if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { - if (type == TypeProtos.Type.result){ - Set result_result = new HashSet<>(); - //verifico se il risultato ha una relazione semantica verso uno o piu' risultati. - //per ogni risultato linkato con issupplementto o issupplementedby emetto: - // id risultato linkato come chiave, - // id risultato oggetto del mapping e lista degli autori del risultato oggetto del mapper come value - for(String sem : sem_rels){ - result_result.addAll(getRelationTarget(value, sem, context, COUNTER_PROPAGATION)); - } - if(!result_result.isEmpty()){ - List authorlist = getAuthorList(entity.getResult().getMetadata().getAuthorList()); - Emit e = new Emit(); - e.setId(Bytes.toString(keyIn.get())); - e.setAuthor_list(authorlist); - valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), - trust, - Type.fromsemrel).toJson()); - for (String result: result_result){ - keyOut.set(Bytes.toBytes(result)); - context.write(keyOut,valueOut); - context.getCounter(COUNTER_PROPAGATION,"emit for sem_rel").increment(1); + //have the same surname. Check the name + if (StringUtils.isNoneEmpty(autoritative_author.getName())) { + if (StringUtils.isNoneEmpty(author.getName())) { + if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { + toaddpid = true; + } + //they could be differently written (i.e. only the initials of the name in one of the two + if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { + toaddpid = true; + } + } } - - //emetto anche id dell'oggetto del mapper come chiave e lista degli autori come valore - e.setId(keyIn.toString()); - e.setAuthor_list(authorlist); - valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), trust, Type.fromresult).toJson()); - context.write(keyIn, valueOut); - context.getCounter(COUNTER_PROPAGATION,"emit for result with orcid").increment(1); - } } - } - } - - private List getAuthorList(List author_list){ - - return author_list.stream().map(a -> new JsonFormat().printToString(a)).collect(Collectors.toList()); - + if (toaddpid){ + StructuredProperty pid = new StructuredProperty(); + for(StructuredProperty sp : autoritative_author.getPid()){ + if (PROPAGATION_AUTHOR_PID.equals(sp.getQualifier().getClassid())){ + pid.setValue(sp.getValue()); + pid.setQualifier(getQualifier(sp.getQualifier().getClassid(),sp.getQualifier().getClassname() )); + pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + if(author.getPid() == null){ + author.setPid(Arrays.asList(pid)); + }else{ + author.getPid().add(pid); + } + } + } + return author; + } + return null; } + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + List autoritative_authors = p._2()._2().get().getAuthors(); + List to_enrich_authors = r.getAuthor(); + //.stream().filter(a -> !containsAllowedPid(a)) + //.collect(Collectors.toList()); + + r.setAuthor(to_enrich_authors + .stream() + .map(a -> { + if (containsAllowedPid(a)) { + return a; + } + + List lst = autoritative_authors.stream() + .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); + if(lst.size() == 0){ + return a; + } + return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people + + }).collect(Collectors.toList())); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + private static TypedRow getTypedRow(Result p) { + TypedRow tp = new TypedRow(); + tp.setSourceId(p.getId()); + List authorList = p.getAuthor() + .stream() + .map(a -> { + if (a.getPid().stream().map(pid -> { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return a; + } + return null; + }).filter(aut -> !(aut == null)).collect(Collectors.toList()).size() > 0){ + return a; + } + return null; + }).filter(a -> !(a == null)).collect(Collectors.toList()); + tp.setAuthors(authorList); + if(authorList.size() > 0){ + return tp; + } + return null; + + + } + + private static boolean containsAllowedPid(Author a){ + + + return (a.getPid().stream().map(pid -> { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return true; + } + return false; + }).filter(aut -> (aut == true)).collect(Collectors.toList()).size()) > 0; + } + } - */ \ No newline at end of file + + +/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ + ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); + boolean updated = false; + + for (FieldTypeProtos.Author a: author_list){ + FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); + if(author != null){ + updated = true; + metadataBuilder.addAuthor(author); + }else{ + metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); + } + } + if(updated) + return metadataBuilder; + return null; + } + private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ + if(containsOrcid(a.getPidList())) + return null; + for(FieldTypeProtos.Author autoritative_author : author_list) { + if (equals(autoritative_author, a)) { + if(!containsOrcid(a.getPidList())) + return update(a, autoritative_author); + } + } + return null; + + } + + private boolean containsOrcid(List pidList){ + if(pidList == null) + return false; + return pidList + .stream() + .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) + .collect(Collectors.toList()).size() > 0; + } + */ \ No newline at end of file From 841f5523fe3b5d9f18eaa9ce28c43d61b84324a9 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:23:16 +0100 Subject: [PATCH 026/259] Added information and methods for the new propagation of orcid to result through semrel --- .../eu/dnetlib/dhp/PropagationConstant.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 7cd8c54d1..d95736ca4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -1,6 +1,10 @@ package eu.dnetlib.dhp; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; @@ -21,11 +25,18 @@ public class PropagationConstant { public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; + public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; + public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; + public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = "propagation:community:productsthroughsemrel"; + public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = " Propagation of result belonging to community through semantic relation"; + + public final static String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "propagation:orcid:result"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of ORCID through result linked by isSupplementedBy or isSupplementTo semantic relations"; public final static String RELATION_DATASOURCEORGANIZATION_REL_TYPE = "datasourceOrganization"; public final static String RELATION_DATASOURCEORGANIZATION_SUBREL_TYPE = "provision"; @@ -45,6 +56,7 @@ public class PropagationConstant { public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; + public static final String PROPAGATION_AUTHOR_PID = "ORCID"; public static Country getCountry(String country){ Country nc = new Country(); @@ -90,6 +102,16 @@ public class PropagationConstant { } + public static JavaPairRDD getResultResultSemRel(List allowedsemrel, JavaRDD relations) { + return relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + } + + + public static List getTypedRowsDatasourceResult(OafEntity oaf) { List lst = new ArrayList<>(); Set datasources_provenance = new HashSet<>(); From b098ee0baec88398cceaa21cf5268b35957cf068 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:23:51 +0100 Subject: [PATCH 027/259] Changed the structure of typed row to conatain also list of authors with orcid --- .../main/java/eu/dnetlib/dhp/TypedRow.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java index 13f3ca611..8ca6eb7ed 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp; +import eu.dnetlib.dhp.schema.oaf.Author; + import java.io.Serializable; import java.util.*; @@ -10,6 +12,23 @@ public class TypedRow implements Serializable { private String type; private String value; private Set accumulator; + private List authors; + + public List getAuthors() { + return authors; + } + + public TypedRow setAuthors(List authors) { + this.authors = authors; + return this; + } + + public void addAuthor(Author a){ + if(authors == null){ + authors = new ArrayList<>(); + } + authors.add(a); + } public Set getAccumulator() { return accumulator; From 550cb21c23559e7025b28be6483244cd04aef21c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:24:39 +0100 Subject: [PATCH 028/259] None --- .../SparkResultToProjectThroughSemRelJob.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java index 1d99b3ec8..1798aa67a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java @@ -48,11 +48,7 @@ public class SparkResultToProjectThroughSemRelJob { JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); - JavaPairRDD result_result = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); + JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); JavaPairRDD result_project = relations .filter(r -> !r.getDataInfo().getDeletedbyinference()) @@ -117,4 +113,6 @@ public class SparkResultToProjectThroughSemRelJob { } + + } From b50166b9ad9ae7b92789ebb0dbb4231b3d1df22f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:25:28 +0100 Subject: [PATCH 029/259] None --- .../SparkCountryPropagationJob.java | 53 ++++--------------- 1 file changed, 9 insertions(+), 44 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index ca126ac1a..0261e3887 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -111,10 +111,10 @@ public class SparkCountryPropagationJob { return ret.iterator(); })); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); JavaPairRDD datasource_country = organizations.join(organization_datasource) .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) @@ -147,27 +147,11 @@ public class SparkCountryPropagationJob { } - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { results.leftOuterJoin(toupdateresult) .map(c -> { - OafEntity oaf = c._2()._1(); - List countryList = null; - if (oaf.getClass() == Publication.class) { - countryList = ((Publication) oaf).getCountry(); - - } - if (oaf.getClass() == Dataset.class){ - countryList = ((Dataset) oaf).getCountry(); - } - - if (oaf.getClass() == Software.class){ - countryList = ((Software) oaf).getCountry(); - } - - if (oaf.getClass() == OtherResearchProduct.class){ - countryList = ((OtherResearchProduct) oaf).getCountry(); - } - + Result oaf = c._2()._1(); + List countryList = oaf.getCountry(); if (c._2()._2().isPresent()) { HashSet countries = new HashSet<>(); for (Qualifier country : countryList) { @@ -181,29 +165,10 @@ public class SparkCountryPropagationJob { } } - if (oaf.getClass() == Publication.class) { - ((Publication) oaf).setCountry(countryList); - return (Publication) oaf; - - } - if (oaf.getClass() == Dataset.class){ - ((Dataset) oaf).setCountry(countryList); - return (Dataset) oaf; - } - - if (oaf.getClass() == Software.class){ - ((Software) oaf).setCountry(countryList); - return (Software) oaf; - } - - if (oaf.getClass() == OtherResearchProduct.class){ - ((OtherResearchProduct) oaf).setCountry(countryList); - return (OtherResearchProduct) oaf; - } + oaf.setCountry(countryList); } - - return null; + return oaf; }) .map(p -> new ObjectMapper().writeValueAsString(p)) .saveAsTextFile(outputPath+"/"+type); From 3a4ccb26c0ec9663c5d6c00334cc52ed68682c54 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 28 Feb 2020 18:26:04 +0100 Subject: [PATCH 030/259] New properties for the orcid to result propagation through semantic relation --- .../input_orcidtoresult_parameters.json | 20 +++++++ .../oozie_app/config-default.xml | 18 ++++++ .../oozie_app/workflow.xml | 55 +++++++++++++++++++ ...nresultcommunityfromsemrel_parameters.json | 20 +++++++ .../oozie_app/config-default.xml | 18 ++++++ .../oozie_app/workflow.xml | 55 +++++++++++++++++++ 6 files changed, 186 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json new file mode 100644 index 000000000..ffb314cdf --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json @@ -0,0 +1,20 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"as", + "paramLongName":"allowedsemrels", + "paramDescription": "the allowed sematinc relations for propagation", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml new file mode 100644 index 000000000..15065b35a --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + 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 + AffiliatioPropagation + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json new file mode 100644 index 000000000..ffb314cdf --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json @@ -0,0 +1,20 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"as", + "paramLongName":"allowedsemrels", + "paramDescription": "the allowed sematinc relations for propagation", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml new file mode 100644 index 000000000..45c3b6854 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + 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 + AffiliatioPropagation + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + + + + + + + \ No newline at end of file From 3d63f35dcb4a0eeed127542fc2630e61a9c9b2fe Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 2 Mar 2020 16:39:03 +0100 Subject: [PATCH 031/259] implementation of new propagation. Result to community for results linked to given organization. We exploit the hasAuthorInstitution semantic link to discover which results are related to institutions --- .../OrganizationMap.java | 20 ++ ...parkResultToCommunityFromOrganization.java | 243 ++++++++++++++++++ 2 files changed, 263 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java new file mode 100644 index 000000000..9e9c2030a --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java @@ -0,0 +1,20 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +public class OrganizationMap extends HashMap> { + + public OrganizationMap(){ + super(); + } + + public List get(String key){ + + if (super.get(key) == null){ + return new ArrayList<>(); + } + return super.get(key); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java new file mode 100644 index 000000000..1f634a693 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java @@ -0,0 +1,243 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.io.File; +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToCommunityFromOrganization { + + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityFromOrganization.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityFromOrganization.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/communitytoresult"; + final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); + + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } + + JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + + + //relations between organziations and results + JavaPairRDD organization_result = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) + .map(r -> new TypedRow().setSourceId(r.getTarget()).setTargetId(r.getSource() )) + .mapToPair(toPair()); + + //relations between representative organization and merged Id. One relation per merged organization + JavaPairRDD organization_organization = relations + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) + .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .mapToPair(toPair()); + + //get the original id of the organizations to be checked against the id associated to the communities + JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) + .map(c -> { + if (!c._2()._2().isPresent()) + return c._2()._1(); + return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); + }) + .mapToPair(toPair()); + + //associates to each result connected to an organization the list of communities related to that organization + JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { + List communityList = organizationMap.get(o._1()); + if (communityList.size() == 0) + return null; + TypedRow tp = o._2(); + tp.setAccumulator(new HashSet<>(communityList)).setSourceId(tp.getTargetId()); + return tp; + }) + .filter(r -> !(r == null)) + .mapToPair(toPair()) + .reduceByKey((a, b) -> { + if (a == null) + return b; + if (b == null) + return a; + a.addAll(b.getAccumulator()); + return a; + }); + + + JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); + JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); + JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); + JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + + } + + // private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { +// results.leftOuterJoin(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// if (p._2()._2().isPresent()){ +// Set communityList = p._2()._2().get().getAccumulator(); +// for(Context c: r.getContext()){ +// if (communityList.contains(c.getId())){ +// //verify if the datainfo for this context contains propagation +// if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ +// c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME)); +// //community id already in the context of the result. Remove it from the set that has to be added +// communityList.remove(c.getId()); +// } +// } +// } +// List cc = r.getContext(); +// for(String cId: communityList){ +// Context context = new Context(); +// context.setId(cId); +// context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); +// cc.add(context); +// } +// r.setContext(cc); +// } +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath+"/"+type); +// } + + +} + + +/* +package eu.dnetlib.data.mapreduce.hbase.propagation.communitythroughorganization; + +import com.google.gson.Gson; +import eu.dnetlib.data.mapreduce.hbase.propagation.Value; +import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; +import eu.dnetlib.data.proto.OafProtos; +import eu.dnetlib.data.proto.TypeProtos; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.TableMapper; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; + +import java.io.IOException; +import java.util.Set; + +import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.*; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; + +public class PropagationCommunityThroughOrganizationMapper extends TableMapper { + private Text valueOut; + private ImmutableBytesWritable keyOut; + private OrganizationMap organizationMap; + + //seleziono il tipo della entry: + //Result: + //se non e' deleted by inference ed ha organizzazioni a cui e' associato, + // //emetto id della relazione ed id del risultato per ogni organizzazione a cui e' associato + //ORGANIZATION: + //se non e' deleted by inference e non e' deduplicata emetto l'id della organizzazione + //se non e' deleted by inference ed e' deduplicata, emetto id della organizzazione ed id del deduplicato e lista delle organizzazioni mergiate + @Override + protected void setup(final Context context) throws IOException, InterruptedException { + super.setup(context); + valueOut = new Text(); + keyOut = new ImmutableBytesWritable(); + organizationMap = new Gson().fromJson(context.getConfiguration().get("organizationtoresult.community.map"), OrganizationMap.class); + System.out.println("got organizationtoresult map: " + new Gson().toJson(organizationMap)) ; + } + + @Override + protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { + final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); + final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference + if (entity != null) { + switch (type) { + case organization: + DedupedList communityList = getCommunityList(Bytes.toString(keyIn.get()), + getRelationTarget(value, DEDUP_RELATION_ORGANIZATION + REL_DEDUP_REPRESENTATIVE_RESULT, context, COUNTER_PROPAGATION), context); + if (communityList.size() > 0){ + valueOut.set(Value.newInstance( + new Gson().toJson( + communityList, //search for organizationtoresult it merges + DedupedList.class), + ORGANIZATION_COMMUNITY_TRUST, + Type.fromorganization).toJson()); + context.write(keyIn, valueOut); + context.getCounter(COUNTER_PROPAGATION, "emit for organizationtoresult ").increment(1); + }else{ + context.getCounter(COUNTER_PROPAGATION, "community list size = 0 ").increment(1); + } + + break; + case result: + Set result_organization = getRelationTarget(value, RELATION_ORGANIZATION + REL_RESULT_ORGANIZATION, context, COUNTER_PROPAGATION); + for(String org: result_organization) + emit(org, Bytes.toString(keyIn.get()), context); + break; + } + } + } + + private DedupedList getCommunityList(String organizationId, Set relationTarget, Context context) { + DedupedList communityList = new DedupedList(); + relationTarget.stream().forEach(org -> communityList.addAll(organizationMap.get(StringUtils.substringAfter(org, "|")))); + communityList.addAll(organizationMap.get(StringUtils.substringAfter(organizationId,"|"))); + communityList.stream().forEach(c->context.getCounter(COUNTER_PROPAGATION,"found organizationtoresult for " + c).increment(1)); + return communityList; + } + + private void emit(String org, String resId, Context context) throws IOException, InterruptedException { + keyOut.set(Bytes.toBytes(org)); + valueOut.set(Value.newInstance(resId, ORGANIZATION_COMMUNITY_TRUST, Type.fromresult).toJson()); + context.write(keyOut,valueOut); + context.getCounter(COUNTER_PROPAGATION, "emit for result").increment(1); + } + +} + */ \ No newline at end of file From 95f8c3092f6956ca377df373a021ee0d97222c25 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 2 Mar 2020 16:40:17 +0100 Subject: [PATCH 032/259] update for new propagation implementation and moving of updateResult for community business logic since the same can be used for result to community from organization and result to community from semrel --- .../eu/dnetlib/dhp/PropagationConstant.java | 52 +++++++++++++++++-- 1 file changed, 47 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index d95736ca4..46271f7e9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -8,10 +8,8 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; +import java.util.stream.Collectors; public class PropagationConstant { public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional"; @@ -32,9 +30,12 @@ public class PropagationConstant { public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; - public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = "propagation:community:productsthroughsemrel"; + public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = "result:community:semrel"; public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = " Propagation of result belonging to community through semantic relation"; + public final static String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = "result:community:organization"; + public final static String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = " Propagation of result belonging to community through organization"; + public final static String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "propagation:orcid:result"; public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of ORCID through result linked by isSupplementedBy or isSupplementTo semantic relations"; @@ -56,6 +57,15 @@ public class PropagationConstant { public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; + + public static final String RELATION_RESULT_REPRESENTATIVERESULT_REL_CLASS = "isMergedIn"; + public static final String RELATION_REPRESENTATIVERESULT_RESULT_CLASS = "merges"; + + + public static final String RELATION_ORGANIZATIONORGANIZATION_REL_TYPE = "organizationOrganization"; + + public static final String RELATION_DEDUPORGANIZATION_SUBREL_TYPE = "dedup"; + public static final String PROPAGATION_AUTHOR_PID = "ORCID"; public static Country getCountry(String country){ @@ -146,4 +156,36 @@ public class PropagationConstant { } return lst; } + + public static void updateResultForCommunity(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type, String class_id, String class_name) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + Set communityList = p._2()._2().get().getAccumulator(); + for(Context c: r.getContext()){ + if (communityList.contains(c.getId())){ + //verify if the datainfo for this context contains propagation + if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); + //community id already in the context of the result. Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } + } + List cc = r.getContext(); + for(String cId: communityList){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + cc.add(context); + } + r.setContext(cc); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + } From 02815dd2cf71c305b656bfe605ee7d0b74af6b18 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 2 Mar 2020 16:40:56 +0100 Subject: [PATCH 033/259] update result for community moved in propagationconstants --- .../SparkResultToCommunityThroughSemRelJob.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java index 707ac9eea..ad3cb95b4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -87,10 +87,10 @@ public class SparkResultToCommunityThroughSemRelJob { JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - updateResult(pubs, to_add_result_communities, outputPath, "publication"); - updateResult(dss, to_add_result_communities, outputPath, "dataset"); - updateResult(sfw, to_add_result_communities, outputPath, "software"); - updateResult(orp, to_add_result_communities, outputPath, "otherresearchproduct"); + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla From 50080c1b3ca86166bc7b219fb19611834fb4be42 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 2 Mar 2020 16:41:37 +0100 Subject: [PATCH 034/259] changed the implementation of addAll method. Before adding all the items in a collection, we check if the accumulator set is not empty --- .../src/main/java/eu/dnetlib/dhp/TypedRow.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java index 8ca6eb7ed..56d519509 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java @@ -40,7 +40,10 @@ public class TypedRow implements Serializable { } - public void addAll(Set toadd){ + public void addAll(Collection toadd){ + if(accumulator == null){ + accumulator = new HashSet<>(); + } this.accumulator.addAll(toadd); } From e80f80ca9390c8d6636c8c358b80d04f60dc227b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 2 Mar 2020 17:03:31 +0100 Subject: [PATCH 035/259] properties and workflow for new propagation --- .../input_communitytoresult_parameters.json | 21 +++++++ .../oozie_app/config-default.xml | 18 ++++++ .../oozie_app/workflow.xml | 55 +++++++++++++++++++ 3 files changed, 94 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json new file mode 100644 index 000000000..ec2549d04 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json @@ -0,0 +1,21 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + + { + "paramName":"ocm", + "paramLongName":"organizationtoresultcommunitymap", + "paramDescription": "the map in json format assocaition the organization original id with the list of communities", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml new file mode 100644 index 000000000..cb85030d8 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -0,0 +1,55 @@ + + + + sourcePath + the source path + + + organizationtoresultcommunitymap + organization community map + + + 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 + ProjectToResultPropagation + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + --organizationtoresultcommunitymap${organizationtoresultcommunitymap} + + + + + + + \ No newline at end of file From d9d206056130fbda5e34830eb81ec24454faa654 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 3 Mar 2020 16:38:50 +0100 Subject: [PATCH 036/259] implementation for bulk tagging --- dhp-workflows/dhp-bulktag/pom.xml | 37 +++- .../main/java/eu/dnetlib/dhp/Community.java | 64 +++++++ .../dnetlib/dhp/CommunityConfiguration.java | 170 +++++++++++++++++ .../dhp/CommunityConfigurationFactory.java | 146 +++++++++++++++ .../main/java/eu/dnetlib/dhp/Constraint.java | 62 +++++++ .../main/java/eu/dnetlib/dhp/Constraints.java | 79 ++++++++ .../main/java/eu/dnetlib/dhp/Datasource.java | 65 +++++++ .../src/main/java/eu/dnetlib/dhp/Pair.java | 38 ++++ .../main/java/eu/dnetlib/dhp/ProtoMap.java | 10 + .../dnetlib/dhp/QueryInformationSystem.java | 66 +++++++ .../java/eu/dnetlib/dhp/ResultTagger.java | 175 ++++++++++++++++++ .../eu/dnetlib/dhp/SelectionConstraints.java | 49 +++++ .../java/eu/dnetlib/dhp/SparkBulkTagJob.java | 71 +++++++ .../java/eu/dnetlib/dhp/TagginConstants.java | 27 +++ .../java/eu/dnetlib/dhp/ZenodoCommunity.java | 46 +++++ .../dhp/selectioncriteria/ContainsVerb.java | 27 +++ .../dhp/selectioncriteria/EqualVerb.java | 29 +++ .../selectioncriteria/InterfaceAdapter.java | 37 ++++ .../selectioncriteria/NotContainsVerb.java | 27 +++ .../dhp/selectioncriteria/NotEqualVerb.java | 29 +++ .../dhp/selectioncriteria/Selection.java | 6 + .../dhp/selectioncriteria/VerbClass.java | 13 ++ .../dhp/selectioncriteria/VerbResolver.java | 25 +++ .../VerbResolverFactory.java | 10 + 24 files changed, 1307 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-bulktag/pom.xml index 5b6a21027..3fa311315 100644 --- a/dhp-workflows/dhp-bulktag/pom.xml +++ b/dhp-workflows/dhp-bulktag/pom.xml @@ -5,11 +5,46 @@ dhp-workflows eu.dnetlib.dhp - 1.0.5-SNAPSHOT + 1.1.6-SNAPSHOT 4.0.0 dhp-bulktag + + + 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} + + + dom4j + dom4j + + + com.jayway.jsonpath + json-path + + + org.reflections + reflections + 0.9.11 + compile + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java new file mode 100644 index 000000000..eebeb05b3 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java @@ -0,0 +1,64 @@ +package eu.dnetlib.dhp; + +import com.google.gson.Gson; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * Created by miriam on 01/08/2018. + */ +public class Community { + + private static final Log log = LogFactory.getLog(Community.class); + + private String id; + private List subjects = new ArrayList<>(); + private List datasources = new ArrayList<>(); + private List zenodoCommunities = new ArrayList<>(); + + + public String toJson() { + final Gson g = new Gson(); + return g.toJson(this); + } + + public boolean isValid() { + return !getSubjects().isEmpty() || !getDatasources().isEmpty() || !getZenodoCommunities().isEmpty(); + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public List getSubjects() { + return subjects; + } + + public void setSubjects(List subjects) { + this.subjects = subjects; + } + + public List getDatasources() { + return datasources; + } + + public void setDatasources(List datasources) { + this.datasources = datasources; + } + + public List getZenodoCommunities() { + return zenodoCommunities; + } + + public void setZenodoCommunities(List zenodoCommunities) { + this.zenodoCommunities = zenodoCommunities; + } + +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java new file mode 100644 index 000000000..6bf5a1a80 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java @@ -0,0 +1,170 @@ +package eu.dnetlib.dhp; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; + +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; +import eu.dnetlib.dhp.selectioncriteria.Selection; +import org.apache.commons.lang.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Created by miriam on 02/08/2018. + */ +public class CommunityConfiguration { + + private static final Log log = LogFactory.getLog(CommunityConfiguration.class); + + + private Map communities; + + + //map subject -> communityid + private transient Map>> subjectMap = new HashMap<>(); + //map datasourceid -> communityid + private transient Map>> datasourceMap = new HashMap<>(); + //map zenodocommunityid -> communityid + private transient Map>> zenodocommunityMap = new HashMap<>(); + + CommunityConfiguration(final Map communities) { + this.communities = communities; + init(); + } + + void init() { + + if (subjectMap == null) { + subjectMap = Maps.newHashMap(); + } + if (datasourceMap == null) { + datasourceMap = Maps.newHashMap(); + } + if (zenodocommunityMap == null) { + zenodocommunityMap = Maps.newHashMap(); + } + + + for(Community c : getCommunities().values()) { + //get subjects + final String id = c.getId(); + for(String sbj : c.getSubjects()){ + Pair p = new Pair<>(id,new SelectionConstraints()); + add(sbj.toLowerCase().trim() , p, subjectMap); + } + //get datasources + for(Datasource d: c.getDatasources()){ + + add(d.getOpenaireId(),new Pair<>(id,d.getSelectionConstraints()),datasourceMap); + } + //get zenodo communities + for(ZenodoCommunity zc : c.getZenodoCommunities()){ + add(zc.getZenodoCommunityId(),new Pair<>(id,zc.getSelCriteria()),zenodocommunityMap); + } + + + } + } + + private void add(String key, Pair value, Map>> map){ + List> values = map.get(key); + + if (values == null){ + values = new ArrayList<>(); + map.put(key,values); + } + values.add(value); + } + + public List> getCommunityForSubject(String sbj){ + return subjectMap.get(sbj); + } + + public List> getCommunityForDatasource(String dts){ + return datasourceMap.get(dts); + } + + + public List getCommunityForDatasource(final String dts, final Map> param) { + List> lp = datasourceMap.get(dts); + if (lp==null) + return Lists.newArrayList(); + + return lp.stream().map(p -> { + if (p.getSnd() == null) + return p.getFst(); + if (((SelectionConstraints) p.getSnd()).verifyCriteria(param)) + return p.getFst(); + else + return null; + }).filter(st->(st!=null)).collect(Collectors.toList()); + + + } + + public List> getCommunityForZenodoCommunity(String zc){ + return zenodocommunityMap.get(zc); + } + + public List getCommunityForSubjectValue(String value) { + + return getContextIds(subjectMap.get(value)); + } + + public List getCommunityForDatasourceValue(String value) { + + return getContextIds(datasourceMap.get(value.toLowerCase())); + } + + public List getCommunityForZenodoCommunityValue(String value){ + + return getContextIds(zenodocommunityMap.get(value.toLowerCase())); + } + + private List getContextIds(List> list) { + if (list != null) { + return list.stream().map(p -> p.getFst()).collect(Collectors.toList()); + } + return Lists.newArrayList(); + } + + + public Map getCommunities() { + return communities; + } + + public void setCommunities(Map communities) { + this.communities = communities; + } + + public String toJson() { + GsonBuilder builder = new GsonBuilder(); + builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); + Gson gson = builder.create(); + + return gson.toJson(this); + } + + public int size() { + return communities.keySet().size(); + } + + public Community getCommunityById(String id){ + return communities.get(id); + } + + public List getCommunityList() { + return Lists.newLinkedList(communities.values()); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java new file mode 100644 index 000000000..98a2437fb --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java @@ -0,0 +1,146 @@ +package eu.dnetlib.dhp; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; +import eu.dnetlib.dhp.selectioncriteria.Selection; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import eu.dnetlib.dhp.selectioncriteria.VerbResolverFactory; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.Node; +import org.dom4j.io.SAXReader; + + +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Created by miriam on 03/08/2018. + */ +public class CommunityConfigurationFactory { + + private static final Log log = LogFactory.getLog(CommunityConfigurationFactory.class); + + private static VerbResolver resolver = VerbResolverFactory.newInstance(); + + public static CommunityConfiguration newInstance(final String xml) throws DocumentException { + + log.debug(String.format("parsing community configuration from:\n%s", xml)); + + final Document doc = new SAXReader().read(new StringReader(xml)); + + final Map communities = Maps.newHashMap(); + + for(final Object o : doc.selectNodes("//community")) { + + final Node node = (Node) o; + + final Community community = parseCommunity(node); + + if (community.isValid()) { + communities.put(community.getId(), community); + } + } + + log.info(String.format("loaded %s community configuration profiles", communities.size())); + log.debug(String.format("loaded community configuration:\n%s", communities.toString())); + + + return new CommunityConfiguration(communities); + } + + public static CommunityConfiguration fromJson(final String json) { + GsonBuilder builder = new GsonBuilder(); + builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); + Gson gson = builder.create(); + final CommunityConfiguration conf = gson.fromJson(json, CommunityConfiguration.class); + log.info(String.format("loaded %s community configuration profiles", conf.size())); + conf.init(); + log.info("created inverse maps"); + + return conf; + } + + private static Community parseCommunity(final Node node) { + + final Community c = new Community(); + + c.setId(node.valueOf("./@id")); + + log.info(String.format("community id: %s", c.getId())); + + c.setSubjects(parseSubjects(node)); + c.setDatasources(parseDatasources(node)); + c.setZenodoCommunities(parseZenodoCommunities(node)); + return c; + } + + private static List parseSubjects(final Node node) { + + final List subjects = Lists.newArrayList(); + + final List list = node.selectNodes("./subjects/subject"); + + for(Node n : list){ + log.debug("text of the node " + n.getText()); + subjects.add(StringUtils.trim(n.getText())); + } + log.info("size of the subject list " + subjects.size()); + return subjects; + } + + + private static List parseDatasources(final Node node) { + final List list = node.selectNodes("./datasources/datasource"); + final List datasourceList = new ArrayList<>(); + for(Node n : list){ + Datasource d = new Datasource(); + d.setOpenaireId(n.selectSingleNode("./openaireId").getText()); + d.setSelCriteria(n.selectSingleNode("./selcriteria"), resolver); + datasourceList.add(d); + } + log.info("size of the datasource list " + datasourceList.size()); + return datasourceList; + } + + private static List parseZenodoCommunities(final Node node) { + final Node oacommunitynode = node.selectSingleNode("./oacommunity"); + String oacommunity = null; + if (oacommunitynode != null){ + String tmp = oacommunitynode.getText(); + if(StringUtils.isNotBlank(tmp)) + oacommunity = tmp; + } + + + final List list = node.selectNodes("./zenodocommunities/zenodocommunity"); + final List zenodoCommunityList = new ArrayList<>(); + for(Node n : list){ + ZenodoCommunity zc = new ZenodoCommunity(); + zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); + zc.setSelCriteria(n.selectSingleNode("./selcriteria")); + + zenodoCommunityList.add(zc); + } + if(oacommunity != null){ + ZenodoCommunity zc = new ZenodoCommunity(); + zc.setZenodoCommunityId(oacommunity); + zenodoCommunityList.add(zc); + } + log.info("size of the zenodo community list " + zenodoCommunityList.size()); + return zenodoCommunityList; + } + + + + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java new file mode 100644 index 000000000..6df127046 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java @@ -0,0 +1,62 @@ +package eu.dnetlib.dhp; + + + +import eu.dnetlib.dhp.selectioncriteria.Selection; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; + + +public class Constraint implements Serializable { + private String verb; + private String field; + private String value; + private Selection selection; + + public Constraint() { + } + + public String getVerb() { + return verb; + } + + public void setVerb(String verb) { + this.verb = verb; + } + + public String getField() { + return field; + } + + public void setField(String field) { + this.field = field; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + + + public void setSelection(Selection sel){ + selection = sel; + } + + public void setSelection(VerbResolver resolver) throws InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException { + selection = resolver.getSelectionCriteria(verb,value); + } + + + public boolean verifyCriteria(String metadata){ + return selection.apply(metadata); + } + + + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java new file mode 100644 index 000000000..27572d8ae --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java @@ -0,0 +1,79 @@ +package eu.dnetlib.dhp; + + +import com.google.gson.Gson; + +import com.google.gson.reflect.TypeToken; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Type; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +/** + * Created by miriam on 02/08/2018. + */ +public class Constraints implements Serializable { + private static final Log log = LogFactory.getLog(Constraints.class); + //private ConstraintEncapsulator ce; + private List constraint; + + + public Constraints() { + } + public List getConstraint() { + return constraint; + } + + public void setConstraint(List constraint) { + this.constraint = constraint; + } + + public void setSc(String json){ + Type collectionType = new TypeToken>(){}.getType(); + constraint = new Gson().fromJson(json, collectionType); + + } + + void setSelection(VerbResolver resolver) { + for(Constraint st: constraint){ + + try { + st.setSelection(resolver); + } catch (NoSuchMethodException e) { + log.error(e.getMessage()); + } catch (IllegalAccessException e) { + log.error(e.getMessage()); + } catch (InvocationTargetException e) { + log.error(e.getMessage()); + } catch (InstantiationException e) { + log.error(e.getMessage()); + } + } + + } + + + //Constraint in and + public boolean verifyCriteria(final Map> param) { + + for(Constraint sc : constraint) { + boolean verified = false; + for(String value : param.get(sc.getField())){ + if (sc.verifyCriteria(value.trim())){ + verified = true; + } + } + if(!verified) + return verified; + } + return true; + } + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java new file mode 100644 index 000000000..a4a254f33 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java @@ -0,0 +1,65 @@ +package eu.dnetlib.dhp; + + +import com.google.gson.Gson; + +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.dom4j.Node; + +/** + * Created by miriam on 01/08/2018. + */ +public class Datasource { + private static final Log log = LogFactory.getLog(Datasource.class); + + private String openaireId; + + private SelectionConstraints selectionConstraints; + + + public SelectionConstraints getSelCriteria() { + return selectionConstraints; + } + + public SelectionConstraints getSelectionConstraints() { + return selectionConstraints; + } + + public void setSelectionConstraints(SelectionConstraints selectionConstraints) { + this.selectionConstraints = selectionConstraints; + } + + public void setSelCriteria(SelectionConstraints selCriteria) { + this.selectionConstraints = selCriteria; + } + + public String getOpenaireId() { + return openaireId; + } + + public void setOpenaireId(String openaireId) { + this.openaireId = openaireId; + } + + private void setSelCriteria(String json, VerbResolver resolver){ + log.info("Selection constraints for datasource = " + json); + selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class); + + selectionConstraints.setSelection(resolver); + } + + public void setSelCriteria(Node n, VerbResolver resolver){ + try{ + setSelCriteria(n.getText(),resolver); + }catch(Exception e) { + log.info("not set selection criteria... "); + selectionConstraints =null; + } + + } + + + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java new file mode 100644 index 000000000..4a1fece43 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java @@ -0,0 +1,38 @@ +package eu.dnetlib.dhp; + +import com.google.gson.Gson; + +/** + * Created by miriam on 03/08/2018. + */ +public class Pair { + private A fst; + private B snd; + + public A getFst() { + return fst; + } + + public Pair setFst(A fst) { + this.fst = fst; + return this; + } + + public B getSnd() { + return snd; + } + + public Pair setSnd(B snd) { + this.snd = snd; + return this; + } + + public Pair(A a, B b){ + fst = a; + snd = b; + } + + public String toJson(){ + return new Gson().toJson(this); + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java new file mode 100644 index 000000000..01f4b8a37 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java @@ -0,0 +1,10 @@ +package eu.dnetlib.dhp; + +import java.util.HashMap; + +public class ProtoMap extends HashMap { + + public ProtoMap(){ + super(); + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java new file mode 100644 index 000000000..1794e4928 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java @@ -0,0 +1,66 @@ +package eu.dnetlib.dhp; + +import com.google.common.base.Joiner; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.dom4j.DocumentException; + +import java.util.List; + + +public class QueryInformationSystem { + private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " + + " let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() " + + " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " + + " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " + + " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " + + " return " + + " " + + " { $x//CONFIGURATION/context/@id} " + + " " + + " {for $y in tokenize($subj,',') " + + " return " + + " {$y}} " + + " " + + " " + + " {for $d in $datasources " + + " where $d/param[./@name='enabled']/text()='true' " + + " return " + + " " + + " " + + " {$d//param[./@name='openaireId']/text()} " + + " " + + " " + + " {$d/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " " + + " {for $zc in $communities " + + " return " + + " " + + " " + + " {$zc/param[./@name='zenodoid']/text()} " + + " " + + " " + + " {$zc/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " "; + + public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) throws ISLookUpException, DocumentException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + final List res = isLookUp.quickSearchProfile(XQUERY); + + final String xmlConf = "" + Joiner.on(" ").join(res) + ""; + + + return CommunityConfigurationFactory.newInstance(xmlConf); + + } + + +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java new file mode 100644 index 000000000..c6c30c9d7 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java @@ -0,0 +1,175 @@ +package eu.dnetlib.dhp; +import com.google.gson.Gson; +import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.JsonPath; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.lang3.StringUtils; + +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static eu.dnetlib.dhp.TagginConstants.*; + + +/** + * Created by miriam on 02/08/2018. + */ +public class ResultTagger { + + + private String trust = "0.8"; + + + private boolean clearContext(Result result){ + int tmp = result.getContext().size(); + List clist = result.getContext().stream() + .filter(c -> (!c.getId().contains(ZENODO_COMMUNITY_INDICATOR))).collect(Collectors.toList()); + result.setContext(clist); + return (tmp != clist.size()); + } + + private Map> getParamMap(final Result result, Map params) { + Map> param = new HashMap<>(); + String json = new Gson().toJson(result,Result.class); + DocumentContext jsonContext = JsonPath.parse(json); + if (params == null){ + params = new HashMap<>(); + } + for(String key : params.keySet()) { + try { + param.put(key, jsonContext.read(params.get(key))); + } catch (com.jayway.jsonpath.PathNotFoundException e) { + param.put(key, new ArrayList<>()); + // throw e; + } + } + return param; + + } + + + public Result enrichContextCriteria(final Result result, final CommunityConfiguration conf, final Map criteria) { + final Map> param = getParamMap(result, criteria); + + //Verify if the entity is deletedbyinference. In case verify if to clean the context list from all the zenodo communities + if(result.getDataInfo().getDeletedbyinference()){ + return result; + } + + //communities contains all the communities to be added as context for the result + final Set communities = new HashSet<>(); + + + //tagging for Subject + final Set subjects = new HashSet<>(); + result.getSubject().stream() + .map(subject -> subject.getValue()) + .filter(StringUtils::isNotBlank) + .map(String::toLowerCase) + .map(String::trim) + .collect(Collectors.toCollection(HashSet::new)) + .forEach(s -> subjects.addAll(conf.getCommunityForSubjectValue(s))); + + communities.addAll(subjects); + + + //Tagging for datasource + final Set datasources = new HashSet<>(); + final Set tmp = new HashSet<>(); + + for(Instance i : result.getInstance()){ + tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(),"|")); + tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(),"|")); + } + + result.getInstance() + .stream() + .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) + .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) + .map(s -> StringUtils.substringAfter(s, "|")) + .collect(Collectors.toCollection(HashSet::new)) + .forEach(dsId -> datasources.addAll(conf.getCommunityForDatasource(dsId,param))); + + communities.addAll(datasources); + + /*Tagging for Zenodo Communities*/ + final Set czenodo = new HashSet<>(); + //final ResultProtos.Result.Metadata.Builder mBuilder = builder.getEntityBuilder().getResultBuilder().getMetadataBuilder(); + result.getContext() + .stream() + .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) + .collect(Collectors.toList()) + .forEach(c->czenodo.addAll(conf.getCommunityForZenodoCommunityValue(c.getId().substring(c.getId().lastIndexOf("/")+1).trim()))); + + communities.addAll(czenodo); + + clearContext(result); + + /*Verify if there is something to bulktag*/ + if(communities.isEmpty()){ + return result; + + } + + result.getContext() + .stream() + .map(c -> { + if(communities.contains(c.getId())){ + List dataInfoList = c.getDataInfo(); + if (subjects.contains(c.getId())) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_SUBJECT, CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c.getId())) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_DATASOURCE, CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c.getId())) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_CZENODO, CLASS_NAME_BULKTAG_ZENODO)); + } + return c; + }) + .collect(Collectors.toList()); + + + communities.removeAll(result.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet())); + + if(communities.isEmpty()) + return result; + + List toaddcontext = communities + .stream() + .map(c -> { + Context context = new Context(); + context.setId(c); + List dataInfoList = Arrays.asList(); + if (subjects.contains(c)) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_SUBJECT, CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c)) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_DATASOURCE, CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c)) + dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_CZENODO, CLASS_NAME_BULKTAG_ZENODO)); + context.setDataInfo(dataInfoList); + return context; + }) + .collect(Collectors.toList()); + + result.getContext().addAll(toaddcontext); + return result; + } + + public static DataInfo getDataInfo(String inference_provenance, String inference_class_id, String inference_class_name){ + DataInfo di = new DataInfo(); + di.setInferred(true); + di.setInferenceprovenance(inference_provenance); + di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); + return di; + } + + public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { + Qualifier pa = new Qualifier(); + pa.setClassid(inference_class_id); + pa.setClassname(inference_class_name); + pa.setSchemeid(DNET_SCHEMA_ID); + pa.setSchemename(DNET_SCHEMA_NAME); + return pa; + } + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java new file mode 100644 index 000000000..2890986d3 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java @@ -0,0 +1,49 @@ +package eu.dnetlib.dhp; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + + +import java.io.Serializable; +import java.lang.reflect.Type; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +public class SelectionConstraints implements Serializable { + private List criteria; + + public SelectionConstraints() { + } + + + public List getCriteria() { + return criteria; + } + + public void setCriteria(List criteria) { + this.criteria = criteria; + } + + public void setSc(String json){ + Type collectionType = new TypeToken>(){}.getType(); + criteria = new Gson().fromJson(json, collectionType); + } + + //Constraints in or + public boolean verifyCriteria(final Map> param){ + for(Constraints selc : criteria) { + if(selc.verifyCriteria(param)){ + return true; + } + } + return false; + } + public void setSelection(VerbResolver resolver) { + + for(Constraints cs : criteria){ + cs.setSelection(resolver); + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java new file mode 100644 index 000000000..755afd2ed --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java @@ -0,0 +1,71 @@ +package eu.dnetlib.dhp; + +import java.io.File; +import java.util.Arrays; +import java.util.List; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + + + +public class SparkBulkTagJob { + + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkBulkTagJob.class.getResourceAsStream("/eu/dnetlib/dhp/input_bulktag_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkBulkTagJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/bulktagging"; + + final ResultTagger resultTagger = new ResultTagger(); + ProtoMap protoMappingParams = new Gson().fromJson(parser.get("mappingProto"),ProtoMap.class);; + + File directory = new File(outputPath); + + if (!directory.exists()) { + directory.mkdirs(); + } + + CommunityConfiguration cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); + + + sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) + .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/publication"); + sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) + .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/dataset"); + sc.sequenceFile(inputPath + "/software", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) + .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/software"); + sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) + .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) + .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/otherresearchproduct"); + + + + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java new file mode 100644 index 000000000..b089d9fd5 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java @@ -0,0 +1,27 @@ +package eu.dnetlib.dhp; + + +public class TagginConstants { + + + public final static String BULKTAG_DATA_INFO_TYPE = "bulktagging"; + + public final static String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public final static String DNET_SCHEMA_ID = "dnet:provenanceActions"; + + public final static String CLASS_ID_SUBJECT = "bulktagging:community:subject"; + public final static String CLASS_ID_DATASOURCE = "bulktagging:community:datasource"; + public final static String CLASS_ID_CZENODO = "bulktagging:community:zenodocommunity"; + + public final static String SCHEMA_ID = "dnet:provenanceActions"; + public final static String COUNTER_GROUP = "Bulk Tagging"; + + public final static String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; + + public final static String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; + public final static String CLASS_NAME_BULKTAG_DATASOURCE = "Bulktagging for Community - Datasource"; + public final static String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; + + + +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java new file mode 100644 index 000000000..af39d18e3 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java @@ -0,0 +1,46 @@ +package eu.dnetlib.dhp; + +import com.google.gson.Gson; +import org.dom4j.Node; + + +/** + * Created by miriam on 01/08/2018. + */ +public class ZenodoCommunity { + + private String zenodoCommunityId; + + private SelectionConstraints selCriteria; + + public String getZenodoCommunityId() { + return zenodoCommunityId; + } + + public void setZenodoCommunityId(String zenodoCommunityId) { + this.zenodoCommunityId = zenodoCommunityId; + } + + public SelectionConstraints getSelCriteria() { + return selCriteria; + } + + public void setSelCriteria(SelectionConstraints selCriteria) { + this.selCriteria = selCriteria; + } + + private void setSelCriteria(String json){ + //Type collectionType = new TypeToken>(){}.getType(); + selCriteria = new Gson().fromJson(json, SelectionConstraints.class); + + } + + public void setSelCriteria(Node n){ + if (n==null){ + selCriteria = null; + }else{ + setSelCriteria(n.getText()); + } + } + +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java new file mode 100644 index 000000000..eb7f059d8 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java @@ -0,0 +1,27 @@ +package eu.dnetlib.dhp.selectioncriteria; + +@VerbClass("contains") +public class ContainsVerb implements Selection { + + private String param; + + public ContainsVerb() { + } + + public ContainsVerb(final String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return value.contains(param); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java new file mode 100644 index 000000000..9b35e9583 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java @@ -0,0 +1,29 @@ +package eu.dnetlib.dhp.selectioncriteria; + + +@VerbClass("equals") +public class EqualVerb implements Selection { + + private String param; + + public EqualVerb() { + } + + public EqualVerb(final String param) { + this.param = param; + } + + + @Override + public boolean apply(String value) { + return value.equalsIgnoreCase(param); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java new file mode 100644 index 000000000..7cd261b85 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java @@ -0,0 +1,37 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import com.google.gson.*; + +import java.lang.reflect.Type; + +public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { + + private static final String CLASSNAME = "CLASSNAME"; + private static final String DATA = "DATA"; + + public Object deserialize(JsonElement jsonElement, Type type, + JsonDeserializationContext jsonDeserializationContext) throws JsonParseException { + + JsonObject jsonObject = jsonElement.getAsJsonObject(); + JsonPrimitive prim = (JsonPrimitive) jsonObject.get(CLASSNAME); + String className = prim.getAsString(); + Class klass = getObjectClass(className); + return jsonDeserializationContext.deserialize(jsonObject.get(DATA), klass); + } + public JsonElement serialize(Object jsonElement, Type type, JsonSerializationContext jsonSerializationContext) { + JsonObject jsonObject = new JsonObject(); + jsonObject.addProperty(CLASSNAME, jsonElement.getClass().getName()); + jsonObject.add(DATA, jsonSerializationContext.serialize(jsonElement)); + return jsonObject; + } + /****** Helper method to get the className of the object to be deserialized *****/ + public Class getObjectClass(String className) { + try { + return Class.forName(className); + } catch (ClassNotFoundException e) { + //e.printStackTrace(); + throw new JsonParseException(e.getMessage()); + } + } +} + diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java new file mode 100644 index 000000000..ecfabd7de --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java @@ -0,0 +1,27 @@ +package eu.dnetlib.dhp.selectioncriteria; + +@VerbClass("not_contains") +public class NotContainsVerb implements Selection { + + private String param; + + public NotContainsVerb() { + } + + public NotContainsVerb(final String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return !value.contains(param); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java new file mode 100644 index 000000000..c2c9e73ad --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java @@ -0,0 +1,29 @@ +package eu.dnetlib.dhp.selectioncriteria; + + +@VerbClass("not_equals") +public class NotEqualVerb implements Selection { + + private String param; + + + public NotEqualVerb(final String param) { + this.param = param; + } + + public NotEqualVerb() { + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return !value.equalsIgnoreCase(param); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java new file mode 100644 index 000000000..cd9888a7e --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java @@ -0,0 +1,6 @@ +package eu.dnetlib.dhp.selectioncriteria; + +public interface Selection { + + boolean apply(String value); +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java new file mode 100644 index 000000000..9f519f091 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java @@ -0,0 +1,13 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface VerbClass { + + public String value(); +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java new file mode 100644 index 000000000..a8df69ea6 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java @@ -0,0 +1,25 @@ +package eu.dnetlib.dhp.selectioncriteria; + + +import org.reflections.Reflections; + +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.util.Map; +import java.util.stream.Collectors; + +public class VerbResolver implements Serializable { + private final Map> map; + + public VerbResolver(){ + this.map = new Reflections("eu.dnetlib").getTypesAnnotatedWith(VerbClass.class).stream() + .collect(Collectors.toMap(v -> v.getAnnotation(VerbClass.class).value(), v->(Class)v)); + } + + + public Selection getSelectionCriteria(String name, String param) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + + return map.get(name).getDeclaredConstructor((String.class)).newInstance(param); + + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java new file mode 100644 index 000000000..8879e2d3b --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java @@ -0,0 +1,10 @@ +package eu.dnetlib.dhp.selectioncriteria; + +public class VerbResolverFactory { + + public static VerbResolver newInstance(){ + + return new VerbResolver(); + } + +} \ No newline at end of file From c37f2bd1b57f1313786b1b376ba3910d917e7281 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 3 Mar 2020 16:42:23 +0100 Subject: [PATCH 037/259] moved some classes to package to make code clearer --- .../src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java | 6 ++++-- .../main/java/eu/dnetlib/dhp/{ => community}/Community.java | 2 +- .../dnetlib/dhp/{ => community}/CommunityConfiguration.java | 4 +--- .../dhp/{ => community}/CommunityConfigurationFactory.java | 2 +- .../java/eu/dnetlib/dhp/{ => community}/Constraint.java | 2 +- .../java/eu/dnetlib/dhp/{ => community}/Constraints.java | 2 +- .../java/eu/dnetlib/dhp/{ => community}/Datasource.java | 2 +- .../src/main/java/eu/dnetlib/dhp/{ => community}/Pair.java | 2 +- .../main/java/eu/dnetlib/dhp/{ => community}/ProtoMap.java | 2 +- .../dnetlib/dhp/{ => community}/QueryInformationSystem.java | 2 +- .../java/eu/dnetlib/dhp/{ => community}/ResultTagger.java | 4 ++-- .../dnetlib/dhp/{ => community}/SelectionConstraints.java | 2 +- .../eu/dnetlib/dhp/{ => community}/TagginConstants.java | 2 +- .../eu/dnetlib/dhp/{ => community}/ZenodoCommunity.java | 2 +- 14 files changed, 18 insertions(+), 18 deletions(-) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/Community.java (97%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/CommunityConfiguration.java (98%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/CommunityConfigurationFactory.java (99%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/Constraint.java (97%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/Constraints.java (98%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/Datasource.java (97%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/Pair.java (94%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/ProtoMap.java (79%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/QueryInformationSystem.java (98%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/ResultTagger.java (98%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/SelectionConstraints.java (97%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/TagginConstants.java (96%) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => community}/ZenodoCommunity.java (96%) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java index 755afd2ed..13540cefa 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java @@ -1,12 +1,14 @@ package eu.dnetlib.dhp; import java.io.File; -import java.util.Arrays; -import java.util.List; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.community.CommunityConfiguration; +import eu.dnetlib.dhp.community.ProtoMap; +import eu.dnetlib.dhp.community.QueryInformationSystem; +import eu.dnetlib.dhp.community.ResultTagger; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java index eebeb05b3..47e4b7a12 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Community.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; import org.apache.commons.logging.Log; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java similarity index 98% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java index 6bf5a1a80..aa1a9bb8c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfiguration.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -8,7 +8,6 @@ import com.google.gson.GsonBuilder; import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; import eu.dnetlib.dhp.selectioncriteria.Selection; -import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -18,7 +17,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import java.util.stream.Stream; /** * Created by miriam on 02/08/2018. diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java similarity index 99% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java index 98a2437fb..772aea51d 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java index 6df127046..271493ca8 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraint.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java similarity index 98% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java index 27572d8ae..a33f993f0 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Constraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java index a4a254f33..c0047a07e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Datasource.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java similarity index 94% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java index 4a1fece43..1cff8bfef 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/Pair.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java similarity index 79% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java index 01f4b8a37..6bbe50c86 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ProtoMap.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import java.util.HashMap; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java similarity index 98% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java index 1794e4928..00edf7fdc 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.common.base.Joiner; import eu.dnetlib.dhp.utils.ISLookupClientFactory; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java similarity index 98% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java index c6c30c9d7..b9962050c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ResultTagger.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.JsonPath; @@ -9,7 +9,7 @@ import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import static eu.dnetlib.dhp.TagginConstants.*; +import static eu.dnetlib.dhp.community.TagginConstants.*; /** diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java index 2890986d3..2c09ecaf1 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SelectionConstraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java similarity index 96% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java index b089d9fd5..494d955c8 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/TagginConstants.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; public class TagginConstants { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java similarity index 96% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java index af39d18e3..d785a3d94 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/ZenodoCommunity.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.community; import com.google.gson.Gson; import org.dom4j.Node; From b4652d018cc95f88ec1483d7860c9393e966e44d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 17 Mar 2020 16:31:24 +0100 Subject: [PATCH 038/259] moved the creation of new dir to common class. --- .../main/java/eu/dnetlib/dhp/PropagationConstant.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 46271f7e9..55ebc17b2 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -2,12 +2,15 @@ package eu.dnetlib.dhp; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; +import java.io.IOException; import java.util.*; import java.util.stream.Collectors; @@ -188,4 +191,11 @@ public class PropagationConstant { .saveAsTextFile(outputPath+"/"+type); } + public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { + if (fs.exists(new Path(outputPath))) { + fs.delete(new Path(outputPath), true); + fs.mkdirs(new Path(outputPath)); + } + } + } From 67ea3cf3edf4668f6f54450bd76bab55fb25331f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 17 Mar 2020 16:32:05 +0100 Subject: [PATCH 039/259] changed the way to read the file with info on resource or relation. From sequenceFile to textFile --- .../SparkCountryPropagationJob.java | 66 ++++++------------- 1 file changed, 21 insertions(+), 45 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 0261e3887..3fbba47ba 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -6,6 +6,8 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.Dataset; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -14,6 +16,7 @@ import org.apache.spark.sql.*; import scala.Tuple2; import java.io.File; +import java.io.IOException; import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; @@ -34,45 +37,43 @@ public class SparkCountryPropagationJob { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; - File directory = new File(outputPath); + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - if(!directory.exists()){ - directory.mkdirs(); - } List whitelist = Arrays.asList(parser.get("whitelist").split(";")); List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); - JavaPairRDD organizations = sc.sequenceFile(inputPath + "/organization", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Organization.class)) + JavaPairRDD organizations = sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)) .filter(org -> !org.getDataInfo().getDeletedbyinference()) .map(org -> new TypedRow().setSourceId(org.getId()).setValue(org.getCountry().getClassid())) .mapToPair(toPair()); - JavaPairRDD organization_datasource = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) + JavaPairRDD organization_datasource = + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) .filter(r -> !r.getDataInfo().getDeletedbyinference()) .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_ORGANIZATION_DATASOURCE_REL_CLASS.equals(r.getRelType())) .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) .mapToPair(toPair()); //id is the organization identifier - JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) + JavaPairRDD datasources = sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)) .filter(ds -> whitelist.contains(ds.getId()) || allowedtypes.contains(ds.getDatasourcetype().getClassid())) .map(ds -> new TypedRow().setSourceId(ds.getId())) .mapToPair(toPair()); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + JavaRDD publications = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD datasets = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD software = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); JavaPairRDD datasource_results = publications .map(oaf -> getTypedRowsDatasourceResult(oaf)) @@ -147,6 +148,8 @@ public class SparkCountryPropagationJob { } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { results.leftOuterJoin(toupdateresult) .map(c -> { @@ -177,33 +180,6 @@ public class SparkCountryPropagationJob { - private static JavaPairRDD getResults(JavaSparkContext sc , String inputPath){ - - return - sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) - .filter(ds -> !ds.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("dataset").setSourceId(oaf.getId())) - .mapToPair(toPair()) - .union(sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) - .filter(o -> !o.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("otherresearchproduct").setSourceId(oaf.getId())) - .mapToPair(toPair())) - .union(sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) - .filter(s -> !s.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("software").setSourceId(oaf.getId())) - .mapToPair(toPair())) - .union(sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .map(oaf -> new TypedRow().setType("publication").setSourceId(oaf.getId())) - .mapToPair(toPair())); - - - } - From 47561f35973fabf995892611877ff7d24f8ed1da Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 11:58:32 +0100 Subject: [PATCH 040/259] changed the implementation from rdd to dataset got from sql queries (on hive) --- .../eu/dnetlib/dhp/SerializableSupplier.java | 4 + .../SparkCountryPropagationJob.java | 430 ++++++++++++++++-- 2 files changed, 406 insertions(+), 28 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java new file mode 100644 index 000000000..b890c141d --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp; + +public class SerializableSupplier { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 3fbba47ba..75fceab63 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -1,20 +1,27 @@ package eu.dnetlib.dhp.countrypropagation; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.PropagationConstant; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.Dataset; +import net.sf.saxon.expr.ContextMappingFunction; +import net.sf.saxon.expr.flwor.Tuple; +import net.sf.saxon.om.Item; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; import scala.Tuple2; +import javax.sql.DataSource; +import java.beans.Encoder; import java.io.File; import java.io.IOException; import java.util.*; @@ -22,42 +29,298 @@ import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; public class SparkCountryPropagationJob { + public static void main(String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() .appName(SparkCountryPropagationJob.class.getSimpleName()) .master(parser.get("master")) + .config(conf) .enableHiveSupport() .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - List whitelist = Arrays.asList(parser.get("whitelist").split(";")); List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); - JavaPairRDD organizations = sc.textFile(inputPath + "/organization") + datasource(spark, whitelist, outputPath, inputPath); + + //rdd(spark,whitelist,allowedtypes, outputPath, inputPath); + + } + + private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath){ + String whitelisted = ""; + for (String i : whitelist){ + whitelisted += " OR id = '" + i + "'"; + } + + String query = "SELECT source ds, target org, country.classid country " + + "FROM ( SELECT id " + + "FROM openaire.datasource " + + "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + + "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + + "JOIN ( SELECT source, target " + + "FROM openaire.relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source " + + "JOIN (SELECT id, country " + + "FROM openaire.organization " + + "WHERE datainfo.deletedbyinference = false ) o " + + "ON o.id = rel.target"; + + Dataset rels = spark.sql(query); + rels.createOrReplaceTempView("rels"); + + + + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "openaire.software"); + createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/software"); + createUpdateForResultDatasetWrite(toupdateresultsoftware, outputPath, "update_software"); + + JavaRDD toupdateresultdataset = propagateOnResult(spark, "openaire.dataset"); + createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/dataset"); + createUpdateForResultDatasetWrite(toupdateresultdataset, outputPath, "update_dataset"); + + JavaRDD toupdateresultother = propagateOnResult(spark, "openaire.otherresearchproduct"); + createUpdateForOtherDataset(toupdateresultother, inputPath, spark) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/otherresearchproduct"); + createUpdateForResultDatasetWrite(toupdateresultother, outputPath, "update_other"); + + createUpdateForPublicationDataset(propagateOnResult(spark, "openaire.publication"), inputPath, spark) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/publication"); + + } + + private static JavaRDD createUpdateForOtherDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + OtherResearchProduct oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForPublicationDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + Publication oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + Software oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForDatasetDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)) + .mapToPair(d -> new Tuple2<>(d.getId(), d)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + eu.dnetlib.dhp.schema.oaf.Dataset oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD propagateOnResult(SparkSession spark, String table) { + String query; + query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM " + table + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + + return countryPropagationAssoc(spark, "cfhb").toJavaRDD(); + + } + + private static Dataset countryPropagationAssoc(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(country) country "+ + "FROM ( SELECT id, country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + + private static JavaPairRDD getStringResultJavaPairRDD(JavaRDD toupdateresult) { + return toupdateresult.map(c -> { + List countryList = new ArrayList<>(); + List tmp = c.getList(1); + for (String country : tmp) { + countryList.add(getCountry(country)); + } + Result r = new Result(); + r.setId(c.getString(0)); + r.setCountry(countryList); + return r; + }).mapToPair(r -> new Tuple2<>(r.getId(), r)); + } + + private static void createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type){ + toupdateresult.map(c -> { + List countryList = new ArrayList<>(); + List tmp = c.getList(1); + for (String country : tmp) { + countryList.add(getCountry(country)); + } + Result r = new Result(); + r.setId(c.getString(0)); + r.setCountry(countryList); + return r; + + }).map(r ->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath+"/"+type); + } + + private static Dataset countryPropagation(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(named_struct('classid', country, 'classname', country, " + + "'datainfo', named_struct( 'deletedbyinference', false, " + + "'inferenceprovenance','" + PROPAGATION_DATA_INFO_TYPE +"'," + + "'inferred',true,'invisible',false, " + + "'provenanceaction', named_struct('classid','" + PROPAGATION_COUNTRY_INSTREPO_CLASS_ID + "'," + + "'classname','" + PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME +"'," + + "'schemeid','" + DNET_SCHEMA_ID +"'," + + "'schemename','" + DNET_SCHEMA_NAME +"') , " + + "'trust','0.9') ,'schemeid','" + DNET_COUNTRY_SCHEMA +"','schemename','" + DNET_COUNTRY_SCHEMA + "')) country " + + "FROM ( SELECT id, country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + + private static void rdd(SparkSession spark, List whitelist, List allowedtypes, String outputPath, String inputPath) throws IOException { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaPairRDD organizations = sc.textFile(inputPath + "/organization") .map(item -> new ObjectMapper().readValue(item, Organization.class)) .filter(org -> !org.getDataInfo().getDeletedbyinference()) - .map(org -> new TypedRow().setSourceId(org.getId()).setValue(org.getCountry().getClassid())) + .map(org -> { + TypedRow tr = new TypedRow(); + tr.setSourceId(org.getId()); + tr.setValue(org.getCountry().getClassid()); + return tr; + }) .mapToPair(toPair()); JavaPairRDD organization_datasource = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_ORGANIZATION_DATASOURCE_REL_CLASS.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); //id is the organization identifier - + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelType()) && + RELATION_ORGANIZATION_DATASOURCE_REL_CLASS.equals(r.getRelClass())) + .map(r -> { + TypedRow tp = new TypedRow(); tp.setSourceId(r.getSource()); tp.setTargetId(r.getTarget()); + return tp; + }) + .mapToPair(toPair()); //id is the organization identifier JavaPairRDD datasources = sc.textFile(inputPath + "/datasource") .map(item -> new ObjectMapper().readValue(item, Datasource.class)) @@ -65,6 +328,97 @@ public class SparkCountryPropagationJob { .map(ds -> new TypedRow().setSourceId(ds.getId())) .mapToPair(toPair()); + JavaPairRDD datasource_country = organizations.join(organization_datasource) + .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) + .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) + + + JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) + .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())).cache(); + +// System.out.println("OUTPUT *** ORGANIZATION COUNT *** " + organizations.count()); +// System.out.println("OUTPUT *** ORGANIZATION DATASOURCE RELATIONS COUNT *** " + organization_datasource.count()); +// System.out.println("OUTPUT *** DATASOURCE COUNT *** " + datasources.count()); +// System.out.println("OUTPUT *** ALLOWED_DATASOURCE-COUNTRY COUNT *** " + alloweddatasources_country.count()); + +// alloweddatasources_country.map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath+"/datasource_country"); + + JavaRDD software = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + + JavaPairRDD datasource_software = software + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + }); + + datasource_software.map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/datasource_software"); + + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_software) + .map(u -> { + TypedRow tp = u._2()._2(); + tp.setValue(u._2()._1().getValue()); + return tp; + }) + .mapToPair(toPair()) + .reduceByKey((a, p) -> { + if (a == null) { + return p; + } + if (p == null) { + return a; + } + a.addAll(p.getAccumulator()); + return a; + }); + toupdateresult.map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/toupdateresult"); + //updateResult(sfw, toupdateresult, outputPath, "software"); + // createUpdateForResult(toupdateresult, outputPath, "software"); + + + + /* JavaRDD publications = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + + JavaPairRDD datasource_publication = publications + .map(oaf -> getTypedRowsDatasourceResult(oaf)) + .flatMapToPair(f -> { + ArrayList> ret = new ArrayList<>(); + for (TypedRow t : f) { + ret.add(new Tuple2<>(t.getSourceId(), t)); + } + return ret.iterator(); + }); + + JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_publication) + .map(u -> u._2()._2().setValue(u._2()._1().getValue())) + .mapToPair(toPair()) + .reduceByKey((a, p) -> { + if (a == null) { + return p; + } + if (p == null) { + return a; + } + a.addAll(p.getAccumulator()); + return a; + }); + + + + + + + JavaRDD publications = sc.textFile(inputPath + "/publication") .map(item -> new ObjectMapper().readValue(item, Publication.class)); @@ -75,6 +429,9 @@ public class SparkCountryPropagationJob { JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + + + JavaPairRDD datasource_results = publications .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { @@ -112,18 +469,8 @@ public class SparkCountryPropagationJob { return ret.iterator(); })); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - JavaPairRDD datasource_country = organizations.join(organization_datasource) - .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) - .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) - JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) - .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())); JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) @@ -140,15 +487,48 @@ public class SparkCountryPropagationJob { return a; }); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + updateResult(pubs, toupdateresult, outputPath, "publication"); updateResult(dss, toupdateresult, outputPath, "dataset"); updateResult(sfw, toupdateresult, outputPath, "software"); updateResult(orp, toupdateresult, outputPath, "otherresearchproduct"); //we use leftOuterJoin because we want to rebuild the entire structure +*/ + } + + private static void createUpdateForResult(JavaPairRDD toupdateresult, String outputPath, String type){ + toupdateresult.map(c -> { + List countryList = new ArrayList<>(); + for (String country : c._2.getAccumulator()) { + countryList.add(getCountry(country)); + } + switch(type ){ + case "software": + Software s = new Software(); + s.setId(c._1()); + s.setCountry(countryList); + return s; + case "publication": + break; + case "dataset": + break; + case "otherresearchproduct": + break; + + } + return null; + }).map(r ->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath+"/"+type); + } private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { results.leftOuterJoin(toupdateresult) @@ -176,13 +556,7 @@ public class SparkCountryPropagationJob { .map(p -> new ObjectMapper().writeValueAsString(p)) .saveAsTextFile(outputPath+"/"+type); } - - - - - - - + } From a440152b46556d4084c8e79d6507bfb1bb391baf Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 14:30:56 +0100 Subject: [PATCH 041/259] refactoring --- .../SparkCountryPropagationJob.java | 309 +----------------- 1 file changed, 16 insertions(+), 293 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 75fceab63..d2b50791d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -18,6 +18,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; +import org.codehaus.janino.Java; import scala.Tuple2; import javax.sql.DataSource; @@ -56,8 +57,6 @@ public class SparkCountryPropagationJob { datasource(spark, whitelist, outputPath, inputPath); - //rdd(spark,whitelist,allowedtypes, outputPath, inputPath); - } private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath){ @@ -83,33 +82,40 @@ public class SparkCountryPropagationJob { Dataset rels = spark.sql(query); rels.createOrReplaceTempView("rels"); - - + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "openaire.software"); + final JavaRDD toupdateresultdataset = propagateOnResult(spark, "openaire.dataset"); + final JavaRDD toupdateresultother = propagateOnResult(spark, "openaire.otherresearchproduct"); + final JavaRDD toupdateresultpublication = propagateOnResult(spark, "openaire.publication"); + + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) .map(s -> new ObjectMapper().writeValueAsString(s)) .saveAsTextFile(outputPath + "/software"); - createUpdateForResultDatasetWrite(toupdateresultsoftware, outputPath, "update_software"); - JavaRDD toupdateresultdataset = propagateOnResult(spark, "openaire.dataset"); createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) .map(d -> new ObjectMapper().writeValueAsString(d)) .saveAsTextFile(outputPath + "/dataset"); - createUpdateForResultDatasetWrite(toupdateresultdataset, outputPath, "update_dataset"); - JavaRDD toupdateresultother = propagateOnResult(spark, "openaire.otherresearchproduct"); createUpdateForOtherDataset(toupdateresultother, inputPath, spark) .map(o -> new ObjectMapper().writeValueAsString(o)) .saveAsTextFile(outputPath + "/otherresearchproduct"); - createUpdateForResultDatasetWrite(toupdateresultother, outputPath, "update_other"); - createUpdateForPublicationDataset(propagateOnResult(spark, "openaire.publication"), inputPath, spark) + createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) .map(p -> new ObjectMapper().writeValueAsString(p)) .saveAsTextFile(outputPath + "/publication"); } + private static void writeUpdates(JavaRDD software, JavaRDD dataset, JavaRDD other , JavaRDD publication, String outputPath){ + createUpdateForResultDatasetWrite(software, outputPath, "update_software"); + createUpdateForResultDatasetWrite(dataset, outputPath, "update_dataset"); + createUpdateForResultDatasetWrite(other, outputPath, "update_other"); + createUpdateForResultDatasetWrite(publication, outputPath, "update_publication"); + } + private static JavaRDD createUpdateForOtherDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); @@ -273,290 +279,7 @@ public class SparkCountryPropagationJob { .saveAsTextFile(outputPath+"/"+type); } - private static Dataset countryPropagation(SparkSession spark, String cfhbTable){ - String query = "SELECT id, collect_set(named_struct('classid', country, 'classname', country, " + - "'datainfo', named_struct( 'deletedbyinference', false, " + - "'inferenceprovenance','" + PROPAGATION_DATA_INFO_TYPE +"'," + - "'inferred',true,'invisible',false, " + - "'provenanceaction', named_struct('classid','" + PROPAGATION_COUNTRY_INSTREPO_CLASS_ID + "'," + - "'classname','" + PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME +"'," + - "'schemeid','" + DNET_SCHEMA_ID +"'," + - "'schemename','" + DNET_SCHEMA_NAME +"') , " + - "'trust','0.9') ,'schemeid','" + DNET_COUNTRY_SCHEMA +"','schemename','" + DNET_COUNTRY_SCHEMA + "')) country " + - "FROM ( SELECT id, country " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON cf = ds " + - "UNION ALL " + - "SELECT id , country " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON hb = ds ) tmp " + - "GROUP BY id"; - return spark.sql(query); - } - private static void rdd(SparkSession spark, List whitelist, List allowedtypes, String outputPath, String inputPath) throws IOException { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaPairRDD organizations = sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)) - .filter(org -> !org.getDataInfo().getDeletedbyinference()) - .map(org -> { - TypedRow tr = new TypedRow(); - tr.setSourceId(org.getId()); - tr.setValue(org.getCountry().getClassid()); - return tr; - }) - .mapToPair(toPair()); - - JavaPairRDD organization_datasource = - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelType()) && - RELATION_ORGANIZATION_DATASOURCE_REL_CLASS.equals(r.getRelClass())) - .map(r -> { - TypedRow tp = new TypedRow(); tp.setSourceId(r.getSource()); tp.setTargetId(r.getTarget()); - return tp; - }) - .mapToPair(toPair()); //id is the organization identifier - - JavaPairRDD datasources = sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)) - .filter(ds -> whitelist.contains(ds.getId()) || allowedtypes.contains(ds.getDatasourcetype().getClassid())) - .map(ds -> new TypedRow().setSourceId(ds.getId())) - .mapToPair(toPair()); - - JavaPairRDD datasource_country = organizations.join(organization_datasource) - .map(x -> x._2()._1().setSourceId(x._2()._2().getTargetId())) // (OrganizationId,(TypedRow for Organization, TypedRow for Relation) - .mapToPair(toPair()); //(DatasourceId, TypedRowforOrganziation) - - - JavaPairRDD alloweddatasources_country = datasources.join(datasource_country) - .mapToPair(ds -> new Tuple2<>(ds._1(), ds._2()._2())).cache(); - -// System.out.println("OUTPUT *** ORGANIZATION COUNT *** " + organizations.count()); -// System.out.println("OUTPUT *** ORGANIZATION DATASOURCE RELATIONS COUNT *** " + organization_datasource.count()); -// System.out.println("OUTPUT *** DATASOURCE COUNT *** " + datasources.count()); -// System.out.println("OUTPUT *** ALLOWED_DATASOURCE-COUNTRY COUNT *** " + alloweddatasources_country.count()); - -// alloweddatasources_country.map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath+"/datasource_country"); - - JavaRDD software = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - - JavaPairRDD datasource_software = software - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - }); - - datasource_software.map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/datasource_software"); - - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_software) - .map(u -> { - TypedRow tp = u._2()._2(); - tp.setValue(u._2()._1().getValue()); - return tp; - }) - .mapToPair(toPair()) - .reduceByKey((a, p) -> { - if (a == null) { - return p; - } - if (p == null) { - return a; - } - a.addAll(p.getAccumulator()); - return a; - }); - toupdateresult.map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/toupdateresult"); - //updateResult(sfw, toupdateresult, outputPath, "software"); - // createUpdateForResult(toupdateresult, outputPath, "software"); - - - - /* JavaRDD publications = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - - JavaPairRDD datasource_publication = publications - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - }); - - JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_publication) - .map(u -> u._2()._2().setValue(u._2()._1().getValue())) - .mapToPair(toPair()) - .reduceByKey((a, p) -> { - if (a == null) { - return p; - } - if (p == null) { - return a; - } - a.addAll(p.getAccumulator()); - return a; - }); - - - - - - - - - JavaRDD publications = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD datasets = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD software = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); - - - - - JavaPairRDD datasource_results = publications - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - }) - .union(datasets - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(software - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(other - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })); - - - - - - JavaPairRDD toupdateresult = alloweddatasources_country.join(datasource_results) - .map(u -> u._2()._2().setValue(u._2()._1().getValue())) - .mapToPair(toPair()) - .reduceByKey((a, p) -> { - if (a == null) { - return p; - } - if (p == null) { - return a; - } - a.addAll(p.getAccumulator()); - return a; - }); - - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResult(pubs, toupdateresult, outputPath, "publication"); - updateResult(dss, toupdateresult, outputPath, "dataset"); - updateResult(sfw, toupdateresult, outputPath, "software"); - updateResult(orp, toupdateresult, outputPath, "otherresearchproduct"); - //we use leftOuterJoin because we want to rebuild the entire structure - -*/ - - } - - - - private static void createUpdateForResult(JavaPairRDD toupdateresult, String outputPath, String type){ - toupdateresult.map(c -> { - List countryList = new ArrayList<>(); - for (String country : c._2.getAccumulator()) { - countryList.add(getCountry(country)); - } - switch(type ){ - case "software": - Software s = new Software(); - s.setId(c._1()); - s.setCountry(countryList); - return s; - case "publication": - break; - case "dataset": - break; - case "otherresearchproduct": - break; - - } - return null; - }).map(r ->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath+"/"+type); - } - - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { - results.leftOuterJoin(toupdateresult) - .map(c -> { - Result oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - TypedRow t = c._2()._2().get(); - - for (String country : t.getAccumulator()) { - if (!countries.contains(country)) { - countryList.add(getCountry(country)); - } - - } - oaf.setCountry(countryList); - } - - return oaf; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - } From 30d58fd98c5e9858e48658f745964cf02f8a292b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 14:32:49 +0100 Subject: [PATCH 042/259] change the configuration of the workflow --- .../input_countrypropagation_parameters.json | 6 ++++++ .../dhp/countrypropagation/oozie_app/config-default.xml | 8 ++++++++ .../dnetlib/dhp/countrypropagation/oozie_app/workflow.xml | 1 + 3 files changed, 15 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 090f3f152..ff1b79d9c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -22,5 +22,11 @@ "paramLongName":"allowedtypes", "paramDescription": "the types of the allowed datasources. Split by ;", "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml index 2e0ed9aee..fcab9dd00 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml @@ -15,4 +15,12 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hive_db_name + openaire + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index f24417bc0..62d454fd8 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -51,6 +51,7 @@ --sourcePath${sourcePath} --whitelist${whitelist} --allowedtypes${allowedtypes} + --hive_metastore_uris${hive_metastore_uris} From 8ab8b6b0bf49dec8d55d05200168502bdfeb1a28 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 14:35:23 +0100 Subject: [PATCH 043/259] minor --- .../dhp/countrypropagation/SparkCountryPropagationJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index d2b50791d..7cb3bd839 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -88,7 +88,7 @@ public class SparkCountryPropagationJob { final JavaRDD toupdateresultdataset = propagateOnResult(spark, "openaire.dataset"); final JavaRDD toupdateresultother = propagateOnResult(spark, "openaire.otherresearchproduct"); final JavaRDD toupdateresultpublication = propagateOnResult(spark, "openaire.publication"); - + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) From a7bf0373069e888b63a1246673a34fe4131f34e6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 14:36:43 +0100 Subject: [PATCH 044/259] remove unused class --- .../src/main/java/eu/dnetlib/dhp/SerializableSupplier.java | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java deleted file mode 100644 index b890c141d..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/SerializableSupplier.java +++ /dev/null @@ -1,4 +0,0 @@ -package eu.dnetlib.dhp; - -public class SerializableSupplier { -} From 9418e3d4fa7223d5a90837b24aeeb50711c08cd1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 17:09:27 +0100 Subject: [PATCH 045/259] read dataset from files instead of using hive tables --- .../SparkCountryPropagationJob.java | 86 +++++++++++++++---- 1 file changed, 69 insertions(+), 17 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 7cb3bd839..ff0fb9603 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -59,35 +59,87 @@ public class SparkCountryPropagationJob { } + private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath){ String whitelisted = ""; for (String i : whitelist){ whitelisted += " OR id = '" + i + "'"; } + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + + Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + + Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + + Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); + + Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); + + Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); + + Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); + + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); +// String query = "SELECT source ds, target org, country.classid country " + +// "FROM ( SELECT id " + +// "FROM openaire.datasource " + +// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + +// "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + +// "JOIN ( SELECT source, target " + +// "FROM openaire.relation " + +// "WHERE relclass = 'provides' " + +// "AND datainfo.deletedbyinference = false ) rel " + +// "ON d.id = rel.source " + +// "JOIN (SELECT id, country " + +// "FROM openaire.organization " + +// "WHERE datainfo.deletedbyinference = false ) o " + +// "ON o.id = rel.target"; String query = "SELECT source ds, target org, country.classid country " + - "FROM ( SELECT id " + - "FROM openaire.datasource " + - "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + - "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + - "JOIN ( SELECT source, target " + - "FROM openaire.relation " + - "WHERE relclass = 'provides' " + - "AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source " + - "JOIN (SELECT id, country " + - "FROM openaire.organization " + - "WHERE datainfo.deletedbyinference = false ) o " + - "ON o.id = rel.target"; + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + + "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source " + + "JOIN (SELECT id, country " + + "FROM organization " + + "WHERE datainfo.deletedbyinference = false ) o " + + "ON o.id = rel.target"; Dataset rels = spark.sql(query); rels.createOrReplaceTempView("rels"); + software.createOrReplaceTempView("software"); + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); - final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "openaire.software"); - final JavaRDD toupdateresultdataset = propagateOnResult(spark, "openaire.dataset"); - final JavaRDD toupdateresultother = propagateOnResult(spark, "openaire.otherresearchproduct"); - final JavaRDD toupdateresultpublication = propagateOnResult(spark, "openaire.publication"); + dataset.createOrReplaceTempView("dataset"); + final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); + + other.createOrReplaceTempView("other"); + final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); + + publication.createOrReplaceTempView("publication"); + final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); From f09cd1e91170cb355478a8d7d583cfd125f89934 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 17:10:14 +0100 Subject: [PATCH 046/259] removed unuseful variable in the configuration --- .../dhp/countrypropagation/oozie_app/config-default.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml index fcab9dd00..ea3a4d922 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml @@ -19,8 +19,4 @@ hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 - - hive_db_name - openaire - \ No newline at end of file From f1e9fe9752f5315c29edd58f019ce019b00f63dc Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 17:11:00 +0100 Subject: [PATCH 047/259] changed implementation using dataset and query on hive --- ...arkResultToOrganizationFromIstRepoJob.java | 151 +++++++++++++++++- 1 file changed, 144 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index d0ef02a4b..8b802cf80 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -3,14 +3,19 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.hadoop.io.Text; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import eu.dnetlib.dhp.schema.oaf.*; @@ -23,10 +28,13 @@ public class SparkResultToOrganizationFromIstRepoJob { final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_resulttoorganizationfrominstrepo_parameters.json"))); parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) .master(parser.get("master")) + .config(conf) .enableHiveSupport() .getOrCreate(); @@ -34,12 +42,67 @@ public class SparkResultToOrganizationFromIstRepoJob { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; - File directory = new File(outputPath); + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - if (!directory.exists()) { - directory.mkdirs(); - } + org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") + .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); + + String query = "SELECT source ds, target org " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + + "AND datainfo.deletedbyinference = false ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source "; + + org.apache.spark.sql.Dataset rels = spark.sql(query); + rels.createOrReplaceTempView("rels"); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); + + org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); + + + software.createOrReplaceTempView("software"); + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); + + dataset.createOrReplaceTempView("dataset"); + final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); + + other.createOrReplaceTempView("other"); + final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); + + publication.createOrReplaceTempView("publication"); + final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); + + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + + /* //get the institutional repositories JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) @@ -51,8 +114,11 @@ public class SparkResultToOrganizationFromIstRepoJob { JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelClass()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelType()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) + .map(r -> { + TypedRow tp = new TypedRow(); tp.setSourceId(r.getSource()); tp.setTargetId(r.getTarget()); + return tp; + }) .mapToPair(toPair()); JavaPairRDD instdatasource_organization = datasources.join(rel_datasource_organization) @@ -70,6 +136,7 @@ public class SparkResultToOrganizationFromIstRepoJob { JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + JavaPairRDD datasource_results = publications .map(oaf -> getTypedRowsDatasourceResult(oaf)) .flatMapToPair(f -> { @@ -124,8 +191,78 @@ public class SparkResultToOrganizationFromIstRepoJob { .saveAsTextFile(outputPath + "/relation_new"); newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation"); + .saveAsTextFile(outputPath + "/relation");*/ + } + + private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { + createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); + createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); + createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); + createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); + + } + + private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { + toupdaterelation.map(s -> { + List relationList = Arrays.asList(); + List orgs = s.getList(1); + String resId = s.getString(0); + for (String org : orgs) { + relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + + } + return relationList.iterator(); + }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); } + private static org.apache.spark.sql.Dataset instPropagationAssoc(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(org) org "+ + "FROM ( SELECT id, org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + + private static JavaRDD propagateOnResult(SparkSession spark, String table) { + String query; + query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM " + table + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + org.apache.spark.sql.Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + + return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); + + } + + private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(org) org "+ + "FROM ( SELECT id, org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + } From ad712f2d79b30f7f1c0340513d027144082ed24d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 23 Mar 2020 17:11:36 +0100 Subject: [PATCH 048/259] added the needed variables in the config and read the variables in the workflow --- .../oozie_app/config-default.xml | 4 ++++ .../resulttoorganizationfrominstrepo/oozie_app/workflow.xml | 3 ++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml index 2e0ed9aee..ea3a4d922 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml @@ -15,4 +15,8 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index b2f6e5919..08a07f497 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -30,7 +30,7 @@ ${nameNode} yarn-cluster cluster - AffiliatioPropagation + AffiliationPropagation eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-memory ${sparkExecutorMemory} @@ -41,6 +41,7 @@ -mt yarn-cluster --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} From 46094a3eec9667b7a7d215571820faf086b5defb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 24 Mar 2020 16:19:36 +0100 Subject: [PATCH 049/259] bug fixing for implementation with dataset --- ...arkResultToOrganizationFromIstRepoJob.java | 114 +----------------- 1 file changed, 4 insertions(+), 110 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index 8b802cf80..cd13ab9e5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -26,7 +26,7 @@ import static eu.dnetlib.dhp.PropagationConstant.*; public class SparkResultToOrganizationFromIstRepoJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_resulttoorganizationfrominstrepo_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); parser.parseArgument(args); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); @@ -101,97 +101,7 @@ public class SparkResultToOrganizationFromIstRepoJob { final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); - - /* - //get the institutional repositories - JavaPairRDD datasources = sc.sequenceFile(inputPath + "/datasource", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Datasource.class)) - .filter(ds -> INSTITUTIONAL_REPO_TYPE.equals(ds.getDatasourcetype().getClassid())) - .map(ds -> new TypedRow().setSourceId(ds.getId())) - .mapToPair(toPair()); - - - JavaPairRDD rel_datasource_organization = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_DATASOURCEORGANIZATION_REL_TYPE.equals(r.getRelType()) && RELATION_DATASOURCE_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) - .map(r -> { - TypedRow tp = new TypedRow(); tp.setSourceId(r.getSource()); tp.setTargetId(r.getTarget()); - return tp; - }) - .mapToPair(toPair()); - - JavaPairRDD instdatasource_organization = datasources.join(rel_datasource_organization) - .map(x -> x._2()._2()) - .mapToPair(toPair()); - - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); - - - JavaPairRDD datasource_results = publications - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - }) - .union(datasets - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(software - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })) - .union(other - .map(oaf -> getTypedRowsDatasourceResult(oaf)) - .flatMapToPair(f -> { - ArrayList> ret = new ArrayList<>(); - for (TypedRow t : f) { - ret.add(new Tuple2<>(t.getSourceId(), t)); - } - return ret.iterator(); - })); - - JavaRDD newRels = instdatasource_organization.join(datasource_results) - .flatMap(c -> { - List rels = new ArrayList(); - String orgId = c._2()._1().getTargetId(); - String resId = c._2()._2().getTargetId(); - rels.add(getRelation(orgId, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - rels.add(getRelation(resId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - return rels.iterator(); - }); - newRels.map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation_new"); - - newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation");*/ + } private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { @@ -203,8 +113,8 @@ public class SparkResultToOrganizationFromIstRepoJob { } private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { - toupdaterelation.map(s -> { - List relationList = Arrays.asList(); + toupdaterelation.flatMap(s -> { + List relationList = new ArrayList<>(); List orgs = s.getList(1); String resId = s.getString(0); for (String org : orgs) { @@ -220,22 +130,6 @@ public class SparkResultToOrganizationFromIstRepoJob { }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); } - - private static org.apache.spark.sql.Dataset instPropagationAssoc(SparkSession spark, String cfhbTable){ - String query = "SELECT id, collect_set(org) org "+ - "FROM ( SELECT id, org " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON cf = ds " + - "UNION ALL " + - "SELECT id , org " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON hb = ds ) tmp " + - "GROUP BY id"; - return spark.sql(query); - } - private static JavaRDD propagateOnResult(SparkSession spark, String table) { String query; query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + From ad24c8478f359477918abe48d75e0e1ba49c471b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 24 Mar 2020 16:19:59 +0100 Subject: [PATCH 050/259] added missing parameter --- ..._propagationresulaffiliationfrominstrepo_parameters.json | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json index 2aa5332b2..215f69fcc 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json @@ -10,5 +10,11 @@ "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true } ] \ No newline at end of file From 19d7f8b51dff30ba8cd116f68e398aa43a0ea1b2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 24 Mar 2020 16:49:46 +0100 Subject: [PATCH 051/259] decommented execution for some of the result type for testing purposes --- .../SparkResultToOrganizationFromIstRepoJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index cd13ab9e5..8d47bdd90 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -101,7 +101,7 @@ public class SparkResultToOrganizationFromIstRepoJob { final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); - + } private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { From b1af90a45f63a807379e485ed3d21f64ac3eed8e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 30 Mar 2020 10:50:03 +0200 Subject: [PATCH 052/259] to make it visible to Claudio --- ...parkResultToCommunityThroughSemRelJob.java | 379 ++++++++++++++++-- 1 file changed, 356 insertions(+), 23 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java index ad3cb95b4..acc411fd0 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -4,32 +4,37 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.io.File; import java.util.*; import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME; public class SparkResultToCommunityThroughSemRelJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils + .toString(SparkResultToCommunityThroughSemRelJob.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) .master(parser.get("master")) + .config(conf) .enableHiveSupport() .getOrCreate(); @@ -37,29 +42,125 @@ public class SparkResultToCommunityThroughSemRelJob { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/communitytoresultthroughsemrel"; - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + //final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + //final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); - File directory = new File(outputPath); + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - if (!directory.exists()) { - directory.mkdirs(); - } - JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, - sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class))); + JavaRDD all_publication_rdd = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD publication_rdd = all_publication_rdd + .filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); + JavaRDD all_dataset_rdd = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD dataset_rdd = all_dataset_rdd + .filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaPairRDD resultLinkedToCommunities = publications + JavaRDD all_orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD orp_rdd = all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD all_software_rdd = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD software_rdd = all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + + + org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), + Encoders.bean(Publication.class)); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), + Encoders.bean(Software.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), + Encoders.bean(Relation.class)); + + publication.createOrReplaceTempView("publication"); + relation.createOrReplaceTempView("relation"); + dataset.createOrReplaceTempView("dataset"); + software.createOrReplaceTempView("software"); + other.createOrReplaceTempView("other"); + + org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); + publication_context.createOrReplaceTempView("publication_context"); + + org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); + dataset_context.createOrReplaceTempView("dataset_context"); + + org.apache.spark.sql.Dataset software_context = getContext(spark, "software"); + software_context.createOrReplaceTempView("software_context"); + + org.apache.spark.sql.Dataset other_context = getContext(spark, "other"); + other_context.createOrReplaceTempView("other_context"); + + publication = spark.createDataset(all_publication_rdd.rdd(), + Encoders.bean(Publication.class)); + publication.createOrReplaceTempView("publication"); + + dataset = spark.createDataset(all_dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + dataset.createOrReplaceTempView("dataset"); + + other = spark.createDataset(all_orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + other.createOrReplaceTempView("other"); + + software = spark.createDataset(all_software_rdd.rdd(), + Encoders.bean(Software.class)); + software.createOrReplaceTempView("software"); + + + org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); + org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + + createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + + updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForDatasetDataset(toupdatesoftwareresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForOtherDataset(toupdatepublicationreresult.toJavaRDD(), other.toJavaRDD(), outputPath, "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + +/* + JavaPairRDD resultLinkedToCommunities = publication .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) .filter(p -> !(p == null)) .mapToPair(toPair()) @@ -93,7 +194,239 @@ public class SparkResultToCommunityThroughSemRelJob { updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla +*/ + } + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + String query = "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + table + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + table + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + table +" t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + table + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLORE(source_context) MyT as co " + + " GROUP BY target_id" ; + + return spark.sql(query); + } + + private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return toupdateresult.map(r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + contextList.add(newContext); + } + + } + + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }).filter(r -> r != null); + } + + private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r-> (Dataset)r) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Publication)r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r -> (OtherResearchProduct)r) + .map( o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); + } + + + + private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map(c -> { + if(! c._2()._2().isPresent()){ + return c._2()._1(); + } + + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for(Object cId: toAddContext){ + String id = (String)cId; + if (communityIdList.contains(id)){ + context_set.add(id); + } + } + for (Context context: c._2()._1().getContext()){ + if(context_set.contains(context)){ + context_set.remove(context); + } + } + + List contextList = context_set.stream().map(co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + return newContext; + + }).collect(Collectors.toList()); + + if(contextList.size() > 0 ){ + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }).filter(r -> r != null); + + +// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) +// .join(result) +// .map(c -> { +// List toAddContext = c._2()._1(); +// Set context_set = new HashSet<>(); +// for(Object cId: toAddContext){ +// String id = (String)cId; +// if (communityIdList.contains(id)){ +// context_set.add(id); +// } +// } +// for (Context context: c._2()._2().getContext()){ +// if(context_set.contains(context)){ +// context_set.remove(context); +// } +// } +// +// List contextList = context_set.stream().map(co -> { +// Context newContext = new Context(); +// newContext.setId(co); +// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); +// return newContext; +// +// }).collect(Collectors.toList()); +// +// if(contextList.size() > 0 ){ +// Result r = new Result(); +// r.setId(c._1()); +// r.setContext(contextList); +// return r; +// } +// return null; +// }) +// .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, + JavaRDD result, String class_id, String class_name) { + return result + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map(c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())){ + if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); + //community id already in the context of the result. Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for(String cId: contexts){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + cc.add(context); + } + oaf.setContext(cc); + + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair(c -> { + + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } + } + + return new Tuple2<>(c.getString(0) ,contextList); + }); + } + + + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ + String query = "SELECT source, context , target " + + "FROM " + table + + " JOIN relation " + + "ON id = source" ; + + return spark.sql(query); + } + + private static Boolean relatedToCommunities(Result r, List communityIdList) { + Set result_communities = r.getContext() + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + return true; + } + } + return false; } private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { From dd011f4a95873dd1eb1158d43d21dd8e260429bb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 30 Mar 2020 10:55:47 +0200 Subject: [PATCH 053/259] to make them visible to Claudio --- ...ationresultcommunityfromsemrel_parameters.json | 15 ++++++++++++++- .../oozie_app/config-default.xml | 4 ++++ .../oozie_app/workflow.xml | 10 ++++++++-- 3 files changed, 26 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json index ffb314cdf..034205b88 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json @@ -1,4 +1,10 @@ [ + { + "paramName":"is", + "paramLongName":"isLookupUrl", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true + }, { "paramName":"mt", "paramLongName":"master", @@ -14,7 +20,14 @@ { "paramName":"as", "paramLongName":"allowedsemrels", - "paramDescription": "the allowed sematinc relations for propagation", + "paramDescription": "the allowed semantic relations for propagation", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", "paramRequired": true } + ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml index 2e0ed9aee..ea3a4d922 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml @@ -15,4 +15,8 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index 45c3b6854..4312ec068 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -20,6 +20,10 @@ sparkExecutorCores number of cores used by single executor + + isLookupUrl + the isLookup service endpoint + @@ -34,7 +38,7 @@ ${nameNode} yarn-cluster cluster - AffiliatioPropagation + ResultToCommunitySemRelPropagation eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-memory ${sparkExecutorMemory} @@ -45,7 +49,9 @@ -mt yarn-cluster --sourcePath${sourcePath} - --allowedsemrels${allowedsemrels} + --allowedsemrels${allowedsemrels} + --hive_metastore_uris${hive_metastore_uris} + --isLookupUrl${isLookupUrl} From beebbcf66b761bf874333dc56503e2b11fa929e6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 10:31:29 +0200 Subject: [PATCH 054/259] new config for countrypropagation --- .../dnetlib/dhp/input_bulkTag_parameters.json | 27 + .../dnetlib/dhp/oozie_app/config-default.xml | 22 + .../eu/dnetlib/dhp/oozie_app/workflow.xml | 61 +++ .../dhp/orcidtoresultfromsemrel/Author.java | 4 + .../AutoritativeAuthor.java | 4 + .../ResultWithOrcid.java | 4 + .../SparkOrcidToResultFromSemRelJob2.java | 317 +++++++++++ ...SparkResultToProjectThroughSemRelJob2.java | 222 ++++++++ ...parkResultToCommunityThroughSemRelJob.java | 15 +- ...arkResultToCommunityThroughSemRelJob2.java | 495 ++++++++++++++++++ ...arkResultToCommunityThroughSemRelJob3.java | 484 +++++++++++++++++ .../input_countrypropagation_parameters.json | 12 + .../oozie_app/config-default.xml | 36 ++ .../countrypropagation/oozie_app/workflow.xml | 28 +- 14 files changed, 1722 insertions(+), 9 deletions(-) create mode 100644 dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json create mode 100644 dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/{communitytoresultthroughsemrel => resulttocommunityfromsemrel}/SparkResultToCommunityThroughSemRelJob.java (97%) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json new file mode 100644 index 000000000..3221924bf --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json @@ -0,0 +1,27 @@ +[ + { + "paramName":"is", + "paramLongName":"isLookupUrl", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true + }, + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName": "pm", + "paramLongName":"protoMap", + "paramDescription": "the json path associated to each selection field", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml new file mode 100644 index 000000000..ea3a4d922 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml @@ -0,0 +1,22 @@ + + + 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 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml new file mode 100644 index 000000000..1866bb0a0 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml @@ -0,0 +1,61 @@ + + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + isLookupUrl + the isLookup service endpoint + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + ResultToCommunitySemRelPropagation + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob + dhp-propagation-${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" + + -mt yarn-cluster + --sourcePath${sourcePath} + + --hive_metastore_uris${hive_metastore_uris} + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java new file mode 100644 index 000000000..18332bc8f --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +public class Author { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java new file mode 100644 index 000000000..7e496c7cf --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +public class AutoritativeAuthor { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java new file mode 100644 index 000000000..49fbea567 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +public class ResultWithOrcid { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java new file mode 100644 index 000000000..73b8895e1 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java @@ -0,0 +1,317 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.Text; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkOrcidToResultFromSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkOrcidToResultFromSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/orcidtoresult"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + JavaRDD relations = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).cache(); + + JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); + + JavaRDD publications = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD datasets = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD software = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + + //get the results having at least one author pid we are interested in + JavaPairRDD resultswithorcid = publications.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + .union(datasets.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())) + .union(software.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())) + .union(other.map(p -> getTypedRow(p)) + .filter(p -> !(p == null)) + .mapToPair(toPair())); + + + JavaPairRDD to_add_orcid_to_result = resultswithorcid.join(result_result) + .map(p -> p._2()._1().setSourceId(p._2()._2().getTargetId())) //associate the pid of the result (target) which should get the orcid to the typed row containing the authors with the orcid from the result(source) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + if(writeUpdate){ + writeResult(pubs, to_add_orcid_to_result, outputPath, "publication"); + writeResult(dss, to_add_orcid_to_result, outputPath, "dataset"); + writeResult(sfw, to_add_orcid_to_result, outputPath, "software"); + writeResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + } + + if (saveGraph){ + updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); + updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); + updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); + updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + } + + + } + + + private static Author enrichAutor(Author autoritative_author, Author author) { + boolean toaddpid = false; + + if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { + if (StringUtils.isNoneEmpty(author.getSurname())) { + if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { + + //have the same surname. Check the name + if (StringUtils.isNoneEmpty(autoritative_author.getName())) { + if (StringUtils.isNoneEmpty(author.getName())) { + if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { + toaddpid = true; + } + //they could be differently written (i.e. only the initials of the name in one of the two + if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { + toaddpid = true; + } + } + } + } + } + } + if (toaddpid){ + StructuredProperty pid = new StructuredProperty(); + for(StructuredProperty sp : autoritative_author.getPid()){ + if (PROPAGATION_AUTHOR_PID.equals(sp.getQualifier().getClassid())){ + pid.setValue(sp.getValue()); + pid.setQualifier(getQualifier(sp.getQualifier().getClassid(),sp.getQualifier().getClassname() )); + pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + if(author.getPid() == null){ + author.setPid(Arrays.asList(pid)); + }else{ + author.getPid().add(pid); + } + } + } + return author; + } + return null; + } + + + private static List enrichAutors(List autoritative_authors, List to_enrich_authors, boolean filter){ +// List autoritative_authors = p._2()._2().get().getAuthors(); +// List to_enrich_authors = r.getAuthor(); + + return to_enrich_authors + .stream() + .map(a -> { + if (filter) { + if (containsAllowedPid(a)) { + return a; + } + } + + List lst = autoritative_authors.stream() + .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); + if (lst.size() == 0) { + return a; + } + return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people + + }).collect(Collectors.toList()); + } + + private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, + String outputPath, String type) { + + results.join(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + + List autoritative_authors = p._2()._2().getAuthors(); + List to_enrich_authors = r.getAuthor(); + + r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); +// .stream() +// .map(a -> { +// if(filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } +// +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if(lst.size() == 0){ +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// +// }).collect(Collectors.toList())); + + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type + "_update"); + } + + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, + String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + List autoritative_authors = p._2()._2().get().getAuthors(); + List to_enrich_authors = r.getAuthor(); + + r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); +// .stream() +// .map(a -> { +// if(filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } +// +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if(lst.size() == 0){ +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// +// }).collect(Collectors.toList())); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + private static TypedRow getTypedRow(Result p) { + TypedRow tp = new TypedRow(); + tp.setSourceId(p.getId()); + List authorList = p.getAuthor() + .stream() + .map(a -> { + if (a.getPid().stream().map(pid -> { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return a; + } + return null; + }).filter(aut -> !(aut == null)).collect(Collectors.toList()).size() > 0){ + return a; + } + return null; + }).filter(a -> !(a == null)).collect(Collectors.toList()); + tp.setAuthors(authorList); + if(authorList.size() > 0){ + return tp; + } + return null; + + + } + + private static boolean containsAllowedPid(Author a){ + + + return (a.getPid().stream().map(pid -> { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return true; + } + return false; + }).filter(aut -> (aut == true)).collect(Collectors.toList()).size()) > 0; + } + +} + + +/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ + ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); + boolean updated = false; + + for (FieldTypeProtos.Author a: author_list){ + FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); + if(author != null){ + updated = true; + metadataBuilder.addAuthor(author); + }else{ + metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); + } + } + if(updated) + return metadataBuilder; + return null; + } + private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ + if(containsOrcid(a.getPidList())) + return null; + for(FieldTypeProtos.Author autoritative_author : author_list) { + if (equals(autoritative_author, a)) { + if(!containsOrcid(a.getPidList())) + return update(a, autoritative_author); + } + } + return null; + + } + + private boolean containsOrcid(List pidList){ + if(pidList == null) + return false; + return pidList + .stream() + .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) + .collect(Collectors.toList()).size() > 0; + } + */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java new file mode 100644 index 000000000..563fcb3bc --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java @@ -0,0 +1,222 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.io.File; +import java.util.*; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.toPair; + +public class SparkResultToProjectThroughSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToProjectThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToProjectThroughSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/projecttoresult"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + JavaRDD all_relations = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + + JavaRDD relations = all_relations.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); + + JavaRDD result_result = relations + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())); + + org.apache.spark.sql.Dataset resres_relation = spark.createDataset(result_result.rdd(), + Encoders.bean(Relation.class)); + + JavaRDD result_project = relations + .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) + && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())); + + org.apache.spark.sql.Dataset resproj_relation = spark.createDataset(result_project.rdd(), + Encoders.bean(Relation.class)); + + resres_relation.createOrReplaceTempView("resres_relation"); + resproj_relation.createOrReplaceTempView("resproj_relation"); + + String query ="SELECT proj, collect_set(r1target) result_set " + + "FROM (" + + " SELECT r1.source as sourcer, r1.relclass as r1rel, r1.target as r1target, r2.target as proj " + + " FROM resres_relation r1 " + + " JOIN resproj_relation r2 " + + " ON r1.source = r2.source " + + " ) tmp " + + "GROUP BY proj "; + + Dataset toaddrelations = spark.sql(query); + + + JavaPairRDD project_resultlist = relations + .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass())) + .map(r -> { + TypedRow tp = new TypedRow(); + tp.setSourceId(r.getSource()); + tp.add(r.getTarget()); + return tp; + }).mapToPair(toPair()) + .reduceByKey((a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + + a.addAll(b.getAccumulator()); + return a; + }).cache(); + + + JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + .leftOuterJoin(project_resultlist) + .flatMap(c -> { + List toAddRel = new ArrayList<>(); + toAddRel.addAll(c._2()._1()); + if (c._2()._2().isPresent()) { + Set originalRels = c._2()._2().get().getAccumulator(); + for (String o : originalRels) { + if (toAddRel.contains(o)) { + toAddRel.remove(o); + } + } + } + List relationList = new ArrayList<>(); + String projId = c._1(); + for (Object r : toAddRel) { + String rId = (String) r; + relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + + } + return relationList.iterator(); + }).cache(); + + toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/toupdaterelations"); + + new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/new_relations" ); + + all_relations.union(new_relations) + .map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/relation"); + + + //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); + +// JavaPairRDD result_project = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) +// && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) +// .map(rel ->{ +// +// TypedRow tr = new TypedRow(); +// tr.setSourceId(rel.getSource()); +// tr.setTargetId(rel.getTarget()); +// return tr; +// }) +// .mapToPair(toPair()); +// +// //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result +// JavaPairRDD project_result = result_project.join(result_result) +// .map(c -> { +// String projectId = c._2()._1().getTargetId(); +// String resultId = c._2()._2().getTargetId(); +// TypedRow tr = new TypedRow(); tr.setSourceId(projectId); tr.setTargetId(resultId); +// return tr; +// }) +// .mapToPair(toPair()); +// +// //relationships from project to result. One Pair for each project => project id list of results related to the project +// JavaPairRDD project_results = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) +// .map(r -> { +// TypedRow tr = new TypedRow(); tr.setSourceId(r.getSource()); tr.setTargetId(r.getTarget()); +// return tr; +// }) +// .mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) { +// return b; +// } +// if (b == null) { +// return a; +// } +// a.addAll(b.getAccumulator()); +// return a; +// }); +// +// +// +// JavaRDD newRels = project_result.join(project_results) +// .flatMap(c -> { +// String resId = c._2()._1().getTargetId(); +// +// if (c._2()._2().getAccumulator().contains(resId)) { +// return null; +// } +// String progId = c._2()._1().getSourceId(); +// List rels = new ArrayList(); +// +// rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, +// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); +// rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, +// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); +// return rels.iterator(); +// }) +// .cache(); +// +// newRels.map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/relation_new"); +// +// newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/relation"); + + } + + + + +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java similarity index 97% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java index acc411fd0..7739ff99d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/communitytoresultthroughsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -28,7 +28,12 @@ public class SparkResultToCommunityThroughSemRelJob { .toString(SparkResultToCommunityThroughSemRelJob.class .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); parser.parseArgument(args); - SparkConf conf = new SparkConf(); + + for(String key : parser.getObjectMap().keySet()){ + System.out.println(key + " = " + parser.get(key)); + } + + /* SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() @@ -152,10 +157,10 @@ public class SparkResultToCommunityThroughSemRelJob { updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - updateForDatasetDataset(toupdatesoftwareresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "software", + updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - updateForOtherDataset(toupdatepublicationreresult.toJavaRDD(), other.toJavaRDD(), outputPath, "publication", + updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); @@ -197,7 +202,7 @@ public class SparkResultToCommunityThroughSemRelJob { */ } - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + /* private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ String query = "SELECT target_id, collect_set(co.id) context_id " + " FROM (SELECT t.id target_id, s.context source_context " + " FROM context_software s " + @@ -479,5 +484,5 @@ public class SparkResultToCommunityThroughSemRelJob { return tp; } return null; - } + }*/ } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java new file mode 100644 index 000000000..2da8d648f --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java @@ -0,0 +1,495 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +import com.cloudera.com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.QueryInformationSystem; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToCommunityThroughSemRelJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils + .toString(SparkResultToCommunityThroughSemRelJob.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + parser.parseArgument(args); + + for(String key : parser.getObjectMap().keySet()){ + System.out.println(key + " = " + parser.get(key)); + } + + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; + + //final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + //final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + + JavaRDD all_publication_rdd = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD publication_rdd = all_publication_rdd + .filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD all_dataset_rdd = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD dataset_rdd = all_dataset_rdd + .filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD all_orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD orp_rdd = all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD all_software_rdd = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); + JavaRDD software_rdd = all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + + JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + + + org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), + Encoders.bean(Publication.class)); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), + Encoders.bean(Software.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), + Encoders.bean(Relation.class)); + + publication.createOrReplaceTempView("publication"); + relation.createOrReplaceTempView("relation"); + dataset.createOrReplaceTempView("dataset"); + software.createOrReplaceTempView("software"); + other.createOrReplaceTempView("other"); + +// org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); +// publication_context.createOrReplaceTempView("publication_context"); + + org.apache.spark.sql.Dataset publication_context = spark.sql( "SELECT relation.source, " + + "publication.context , relation.target " + + "FROM publication " + + " JOIN relation " + + "ON id = source"); + + org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); + dataset_context.createOrReplaceTempView("dataset_context"); + + org.apache.spark.sql.Dataset software_context = getContext(spark, "software"); + software_context.createOrReplaceTempView("software_context"); + + org.apache.spark.sql.Dataset other_context = getContext(spark, "other"); + other_context.createOrReplaceTempView("other_context"); + + publication = spark.createDataset(all_publication_rdd.rdd(), + Encoders.bean(Publication.class)); + publication.createOrReplaceTempView("publication"); + + dataset = spark.createDataset(all_dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + dataset.createOrReplaceTempView("dataset"); + + other = spark.createDataset(all_orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + other.createOrReplaceTempView("other"); + + software = spark.createDataset(all_software_rdd.rdd(), + Encoders.bean(Software.class)); + software.createOrReplaceTempView("software"); + + + org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); + org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + + createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + + updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + +/* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); + + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla +*/ + } + + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + String query = "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + table + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + table + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + table +" t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + table + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id" ; + + return spark.sql(query); + } + + private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return toupdateresult.map(r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + contextList.add(newContext); + } + + } + + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }).filter(r -> r != null); + } + + private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r-> (Dataset)r) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Publication)r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r -> (OtherResearchProduct)r) + .map( o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); + } + + + + private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map(c -> { + if(! c._2()._2().isPresent()){ + return c._2()._1(); + } + + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for(Object cId: toAddContext){ + String id = (String)cId; + if (communityIdList.contains(id)){ + context_set.add(id); + } + } + for (Context context: c._2()._1().getContext()){ + if(context_set.contains(context)){ + context_set.remove(context); + } + } + + List contextList = context_set.stream().map(co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + return newContext; + + }).collect(Collectors.toList()); + + if(contextList.size() > 0 ){ + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }).filter(r -> r != null); + + +// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) +// .join(result) +// .map(c -> { +// List toAddContext = c._2()._1(); +// Set context_set = new HashSet<>(); +// for(Object cId: toAddContext){ +// String id = (String)cId; +// if (communityIdList.contains(id)){ +// context_set.add(id); +// } +// } +// for (Context context: c._2()._2().getContext()){ +// if(context_set.contains(context)){ +// context_set.remove(context); +// } +// } +// +// List contextList = context_set.stream().map(co -> { +// Context newContext = new Context(); +// newContext.setId(co); +// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); +// return newContext; +// +// }).collect(Collectors.toList()); +// +// if(contextList.size() > 0 ){ +// Result r = new Result(); +// r.setId(c._1()); +// r.setContext(contextList); +// return r; +// } +// return null; +// }) +// .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, + JavaRDD result, String class_id, String class_name) { + return result + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map(c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())){ + if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); + //community id already in the context of the result. Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for(String cId: contexts){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + cc.add(context); + } + oaf.setContext(cc); + + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair(c -> { + + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } + } + + return new Tuple2<>(c.getString(0) ,contextList); + }); + } + + + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ + String query = "SELECT relation.source, " + table +".context , relation.target " + + "FROM " + table + + " JOIN relation " + + "ON id = source" ; + + return spark.sql(query); + } + + private static Boolean relatedToCommunities(Result r, List communityIdList) { + Set result_communities = r.getContext() + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + return true; + } + } + return false; + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + Set communityList = p._2()._2().get().getAccumulator(); + for(Context c: r.getContext()){ + if (communityList.contains(c.getId())){ + //verify if the datainfo for this context contains propagation + if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + //community id already in the context of the result. Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } + } + List cc = r.getContext(); + for(String cId: communityList){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + + + private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { + Set result_communities = context + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + TypedRow tp = new TypedRow(); + tp.setSourceId(id); + tp.setType(type); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + tp.add(communityId); + } + } + if (tp.getAccumulator() != null) { + return tp; + } + return null; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java new file mode 100644 index 000000000..c55c0e8ea --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java @@ -0,0 +1,484 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +import com.cloudera.com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.QueryInformationSystem; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToCommunityThroughSemRelJob2 { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils + .toString(SparkResultToCommunityThroughSemRelJob2.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + //final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + //final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + + JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + +// JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") +// .map(item -> new ObjectMapper().readValue(item, Dataset.class)); +// +// JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") +// .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); +// +// JavaRDD software_rdd = sc.textFile(inputPath + "/software") +// .map(item -> new ObjectMapper().readValue(item, Software.class)); + + JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + + + org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), + Encoders.bean(Publication.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), + Encoders.bean(Relation.class)); + +// org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), +// Encoders.bean(Dataset.class)); +// +// org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), +// Encoders.bean(OtherResearchProduct.class)); +// +// org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), +// Encoders.bean(Software.class)); +// +// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), +// Encoders.bean(Relation.class)); + + publication.createOrReplaceTempView("publication"); + relation.createOrReplaceTempView("relation"); +// relation.createOrReplaceTempView("relation"); +// dataset.createOrReplaceTempView("dataset"); +// software.createOrReplaceTempView("software"); +// other.createOrReplaceTempView("other"); + + String communitylist = getConstraintList(" co.id = '", communityIdList); + + String semrellist = getConstraintList(" relClass = '", allowedsemrel ); + + + String query = "Select source, community_context, target " + + "from (select id, collect_set(co.id) community_context " + + "from publication " + + "lateral view explode (context) c as co " + + "where datainfo.deletedbyinference = false "+ communitylist + + " group by id) p " + + "JOIN " + + "(select * " + + "from relation " + + "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " + + "ON p.id = r.source"; + + + org.apache.spark.sql.Dataset publication_context = spark.sql( query); + publication_context.createOrReplaceTempView("publication_context"); + + //( source, (mes, dh-ch-, ni), target ) + query = "select target , collect_set(co) " + + "from (select target, community_context " + + "from publication_context pc join publication p on " + + "p.id = pc.source) tmp " + + "lateral view explode (community_context) c as co " + + "group by target"; + + + + org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); + + +// org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); +// org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); +// org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); +// org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + +// createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + +// createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// +// updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// + +/* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); + + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla +*/ + } + + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + String query = "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + table + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + table + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + table +" t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + table + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id" ; + + return spark.sql(query); + } + + private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return toupdateresult.map(r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + contextList.add(newContext); + } + + } + + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }).filter(r -> r != null); + } + + private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r-> (Dataset)r) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Publication)r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r -> (OtherResearchProduct)r) + .map( o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); + } + + + + private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map(c -> { + if(! c._2()._2().isPresent()){ + return c._2()._1(); + } + + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for(Object cId: toAddContext){ + String id = (String)cId; + if (communityIdList.contains(id)){ + context_set.add(id); + } + } + for (Context context: c._2()._1().getContext()){ + if(context_set.contains(context)){ + context_set.remove(context); + } + } + + List contextList = context_set.stream().map(co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + return newContext; + + }).collect(Collectors.toList()); + + if(contextList.size() > 0 ){ + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }).filter(r -> r != null); + + +// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) +// .join(result) +// .map(c -> { +// List toAddContext = c._2()._1(); +// Set context_set = new HashSet<>(); +// for(Object cId: toAddContext){ +// String id = (String)cId; +// if (communityIdList.contains(id)){ +// context_set.add(id); +// } +// } +// for (Context context: c._2()._2().getContext()){ +// if(context_set.contains(context)){ +// context_set.remove(context); +// } +// } +// +// List contextList = context_set.stream().map(co -> { +// Context newContext = new Context(); +// newContext.setId(co); +// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); +// return newContext; +// +// }).collect(Collectors.toList()); +// +// if(contextList.size() > 0 ){ +// Result r = new Result(); +// r.setId(c._1()); +// r.setContext(contextList); +// return r; +// } +// return null; +// }) +// .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, + JavaRDD result, String class_id, String class_name) { + return result + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map(c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())){ + if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); + //community id already in the context of the result. Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for(String cId: contexts){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + cc.add(context); + } + oaf.setContext(cc); + + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair(c -> { + + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } + } + + return new Tuple2<>(c.getString(0) ,contextList); + }); + } + + + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ + String query = "SELECT relation.source, " + table +".context , relation.target " + + "FROM " + table + + " JOIN relation " + + "ON id = source" ; + + return spark.sql(query); + } + + private static Boolean relatedToCommunities(Result r, List communityIdList) { + Set result_communities = r.getContext() + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + return true; + } + } + return false; + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + Set communityList = p._2()._2().get().getAccumulator(); + for(Context c: r.getContext()){ + if (communityList.contains(c.getId())){ + //verify if the datainfo for this context contains propagation + if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + //community id already in the context of the result. Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } + } + List cc = r.getContext(); + for(String cId: communityList){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + + + private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { + Set result_communities = context + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + TypedRow tp = new TypedRow(); + tp.setSourceId(id); + tp.setType(type); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + tp.add(communityId); + } + } + if (tp.getAccumulator() != null) { + return tp; + } + return null; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index ff1b79d9c..81fead58f 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -28,5 +28,17 @@ "paramLongName":"hive_metastore_uris", "paramDescription": "the hive metastore uris", "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml index ea3a4d922..2744ea92b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml @@ -19,4 +19,40 @@ hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 62d454fd8..aa1e6dc78 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -24,6 +24,18 @@ sparkExecutorCores number of cores used by single executor + + sparkExecutorNumber + number of executors used + + + writeUpdate + writes the information found for the update. No double check done if the information is already present + + + saveGraph + writes new version of the graph after the propagation step + @@ -41,17 +53,25 @@ CountryPropagation eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob dhp-propagation-${projectVersion}.jar - --executor-memory ${sparkExecutorMemory} - --executor-cores ${sparkExecutorCores} + + --num-executors=${sparkExecutorNumber} + --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.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} -mt yarn-cluster --sourcePath${sourcePath} --whitelist${whitelist} --allowedtypes${allowedtypes} --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} From 2afe9718166aaf22da100c042c1342d4479ce1f4 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 10:49:09 +0200 Subject: [PATCH 055/259] new implementation for country propagatio --- .../SparkCountryPropagationJob.java | 77 +++++++++---------- 1 file changed, 36 insertions(+), 41 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index ff0fb9603..2ebd818cb 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -53,14 +53,18 @@ public class SparkCountryPropagationJob { List whitelist = Arrays.asList(parser.get("whitelist").split(";")); List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); + boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); +// datasource(spark, whitelist, outputPath, inputPath, "true".equals(parser.get("writeUpdate")), +// "true".equals(parser.get("saveGraph")), allowedtypes); +// +// } +// +// +// private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath, +// boolean writeUpdates, boolean saveGraph, List allowedtypes){ - datasource(spark, whitelist, outputPath, inputPath); - - } - - - private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath){ String whitelisted = ""; for (String i : whitelist){ whitelisted += " OR id = '" + i + "'"; @@ -78,7 +82,7 @@ public class SparkCountryPropagationJob { .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") @@ -96,26 +100,14 @@ public class SparkCountryPropagationJob { datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); -// String query = "SELECT source ds, target org, country.classid country " + -// "FROM ( SELECT id " + -// "FROM openaire.datasource " + -// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + -// "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + -// "JOIN ( SELECT source, target " + -// "FROM openaire.relation " + -// "WHERE relclass = 'provides' " + -// "AND datainfo.deletedbyinference = false ) rel " + -// "ON d.id = rel.source " + -// "JOIN (SELECT id, country " + -// "FROM openaire.organization " + -// "WHERE datainfo.deletedbyinference = false ) o " + -// "ON o.id = rel.target"; String query = "SELECT source ds, target org, country.classid country " + "FROM ( SELECT id " + "FROM datasource " + - "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + - "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + + "WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + + getConstraintList("datasourcetype.classid = '", allowedtypes) + + // "datasourcetype.classid = 'pubsrepository::institutional' " + + // "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + "JOIN ( SELECT source, target " + "FROM relation " + "WHERE relclass = 'provides' " + @@ -141,23 +133,27 @@ public class SparkCountryPropagationJob { publication.createOrReplaceTempView("publication"); final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); - writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + if(writeUpdates){ + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + } - createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/software"); + if(saveGraph){ + createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/software"); - createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/dataset"); + createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/dataset"); - createUpdateForOtherDataset(toupdateresultother, inputPath, spark) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); + createUpdateForOtherDataset(toupdateresultother, inputPath, spark) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/otherresearchproduct"); - createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); + createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/publication"); + } } @@ -276,9 +272,9 @@ public class SparkCountryPropagationJob { String query; query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + "FROM ( SELECT id, instance " + - "FROM " + table + - " WHERE datainfo.deletedbyinference = false) ds " + - "LATERAL VIEW EXPLODE(instance) i AS inst"; + "FROM " + table + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; Dataset cfhb = spark.sql(query); cfhb.createOrReplaceTempView("cfhb"); @@ -333,5 +329,4 @@ public class SparkCountryPropagationJob { -} - +} \ No newline at end of file From 8438702b3de0661c5d16804d093f9460ac69127b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 10:54:01 +0200 Subject: [PATCH 056/259] addition in propagation constants --- .../eu/dnetlib/dhp/PropagationConstant.java | 29 ++++++++++++++++--- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 55ebc17b2..fa0ab94ee 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -4,7 +4,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.PairFunction; @@ -19,6 +18,8 @@ public class PropagationConstant { public final static String PROPAGATION_DATA_INFO_TYPE = "propagation"; + public static final String TRUE = "true"; + public final static String DNET_COUNTRY_SCHEMA = "dnet:countries"; public final static String DNET_SCHEMA_NAME = "dnet:provenanceActions"; @@ -84,6 +85,8 @@ public class PropagationConstant { public static DataInfo getDataInfo(String inference_provenance, String inference_class_id, String inference_class_name){ DataInfo di = new DataInfo(); di.setInferred(true); + di.setDeletedbyinference(false); + di.setTrust("0.85"); di.setInferenceprovenance(inference_provenance); di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); return di; @@ -119,11 +122,25 @@ public class PropagationConstant { return relations .filter(r -> !r.getDataInfo().getDeletedbyinference()) .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) + .map(r -> { + TypedRow tr = new TypedRow(); + tr.setSourceId(r.getSource()); + tr.setTargetId(r.getTarget()); + return tr; + }) .mapToPair(toPair()); } + public static String getConstraintList(String text, List constraints){ + String ret = " and (" + text + constraints.get(0) + "'"; + for (int i =1; i < constraints.size(); i++){ + ret += " OR " + text + constraints.get(i) + "'"; + } + ret += ")"; + return ret; + } + public static List getTypedRowsDatasourceResult(OafEntity oaf) { List lst = new ArrayList<>(); @@ -155,7 +172,11 @@ public class PropagationConstant { datasources_provenance.add(i.getHostedby().getKey()); } for (String dsId : datasources_provenance) { - lst.add(new TypedRow().setSourceId(dsId).setTargetId(oaf.getId()).setType(type)); + TypedRow tr = new TypedRow(); + tr.setSourceId(dsId); + tr.setTargetId(oaf.getId()); + tr.setType(type); + lst.add(tr); } return lst; } @@ -194,8 +215,8 @@ public class PropagationConstant { public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { if (fs.exists(new Path(outputPath))) { fs.delete(new Path(outputPath), true); - fs.mkdirs(new Path(outputPath)); } + fs.mkdirs(new Path(outputPath)); } } From 540da4ab6124fc4e6c1aad2433550f24401f3b84 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 13:04:04 +0200 Subject: [PATCH 057/259] new busuness logic with prepared info before actual job run --- .../countrypropagation/DatasourceCountry.java | 24 ++ .../PrepareResultCountryAssociation.java | 132 ++++++++ .../SparkCountryPropagationJob2.java | 287 ++++++++++++++++++ 3 files changed, 443 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java new file mode 100644 index 000000000..460764e16 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java @@ -0,0 +1,24 @@ +package eu.dnetlib.dhp.countrypropagation; + +import java.io.Serializable; + +public class DatasourceCountry implements Serializable { + private String dataSourceId; + private String country; + + public String getDataSourceId() { + return dataSourceId; + } + + public void setDataSourceId(String dataSourceId) { + this.dataSourceId = dataSourceId; + } + + public String getCountry() { + return country; + } + + public void setCountry(String country) { + this.country = country; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java new file mode 100644 index 000000000..9572159ce --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java @@ -0,0 +1,132 @@ +package eu.dnetlib.dhp.countrypropagation; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.SaveMode; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.PropagationConstant.createOutputDirs; +import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +/** + * For the association of the country to the datasource + * The association is computed only for datasource of specific type or having whitelisted ids + * The country is registered in the Organization associated to the Datasource, so the + * relation provides between Datasource and Organization is exploited to get the country for the datasource + */ + +public class PrepareResultCountryAssociation { + private static final Logger log = LoggerFactory.getLogger(PrepareResultCountryAssociation.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils.toString(PrepareResultCountryAssociation.class + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + + String graphTableClassName = parser.get("graphTableClassName"); + log.info("graphTableClassName: {}", graphTableClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkSession(conf, isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + joinRelationEntity(spark, inputRelationsPath, inputEntityPath, entityClazz, outputPath); + }); + + + final SparkSession spark = SparkSession + .builder() + .appName(PrepareResultCountryAssociation.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + //todo add link to working dir + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + List whitelist = Arrays.asList(parser.get("whitelist").split(";")); + List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); + + String whitelisted = ""; + for (String i : whitelist){ + whitelisted += " OR id = '" + i + "'"; + } + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + + Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + + Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); + + String query = "SELECT source ds, country.classid country " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + + getConstraintList("datasourcetype.classid = '", allowedtypes) + ") d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source " + + "JOIN (SELECT id, country " + + "FROM organization " + + "WHERE datainfo.deletedbyinference = false " + + "AND length(country.classid)>0) o " + + "ON o.id = rel.target"; + + spark.sql(query) + .as(Encoders.bean(DatasourceCountry.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath + "/prepared_datasource_country"); + + + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java new file mode 100644 index 000000000..23595bada --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -0,0 +1,287 @@ +package eu.dnetlib.dhp.countrypropagation; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +public class SparkCountryPropagationJob2 { + + private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob2.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob2.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkCountryPropagationJob2.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + + // createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + + Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); + + Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)).rdd(), + Encoders.bean(OtherResearchProduct.class)); + + Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)).rdd(), + Encoders.bean(Software.class)); + + Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)).rdd(), + Encoders.bean(Publication.class)); + + + //todo broadcast + + + software.createOrReplaceTempView("software"); + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); + + dataset.createOrReplaceTempView("dataset"); + final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); + + other.createOrReplaceTempView("other"); + final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); + + publication.createOrReplaceTempView("publication"); + final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); + + if(writeUpdates){ + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + } + + if(saveGraph){ + createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/software"); + + createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/dataset"); + + createUpdateForOtherDataset(toupdateresultother, inputPath, spark) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/otherresearchproduct"); + + createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/publication"); + } + + } + + private static void writeUpdates(JavaRDD software, JavaRDD dataset, JavaRDD other , JavaRDD publication, String outputPath){ + createUpdateForResultDatasetWrite(software, outputPath, "update_software"); + createUpdateForResultDatasetWrite(dataset, outputPath, "update_dataset"); + createUpdateForResultDatasetWrite(other, outputPath, "update_other"); + createUpdateForResultDatasetWrite(publication, outputPath, "update_publication"); + } + + private static JavaRDD createUpdateForOtherDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + OtherResearchProduct oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForPublicationDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + Publication oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + Software oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD createUpdateForDatasetDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + return sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)) + .mapToPair(d -> new Tuple2<>(d.getId(), d)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) + .map(c -> { + eu.dnetlib.dhp.schema.oaf.Dataset oaf = c._2()._1(); + List countryList = oaf.getCountry(); + if (c._2()._2().isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : countryList) { + countries.add(country.getClassid()); + } + Result r = c._2()._2().get(); + for (Country country : r.getCountry()) { + if (!countries.contains(country.getClassid())) { + countryList.add(country); + } + } + oaf.setCountry(countryList); + } + return oaf; + }); + } + + private static JavaRDD propagateOnResult(SparkSession spark, String result_type) { + String query; + query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM " + result_type + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + + return countryPropagationAssoc(spark, "cfhb").toJavaRDD(); + + } + + private static Dataset countryPropagationAssoc(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(country) country "+ + "FROM ( SELECT id, country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , country " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + + private static JavaPairRDD getStringResultJavaPairRDD(JavaRDD toupdateresult) { + return toupdateresult.map(c -> { + List countryList = new ArrayList<>(); + List tmp = c.getList(1); + for (String country : tmp) { + countryList.add(getCountry(country)); + } + Result r = new Result(); + r.setId(c.getString(0)); + r.setCountry(countryList); + return r; + }).mapToPair(r -> new Tuple2<>(r.getId(), r)); + } + + private static void createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type){ + toupdateresult.map(c -> { + List countryList = new ArrayList<>(); + List tmp = c.getList(1); + for (String country : tmp) { + countryList.add(getCountry(country)); + } + Result r = new Result(); + r.setId(c.getString(0)); + r.setCountry(countryList); + return r; + + }).map(r ->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath+"/"+type); + } + + + +} \ No newline at end of file From fcfef4632ffd28924ab87a0400c94f58c0ea0e47 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 18:07:18 +0200 Subject: [PATCH 058/259] input parameters for country propagation preparation job --- .../input_prepare_dc_assoc.json | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json new file mode 100644 index 000000000..ff1b79d9c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json @@ -0,0 +1,32 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"wl", + "paramLongName":"whitelist", + "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", + "paramRequired": true + }, + { + "paramName":"at", + "paramLongName":"allowedtypes", + "paramDescription": "the types of the allowed datasources. Split by ;", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + } +] \ No newline at end of file From 03f7cb64028fc1f215d4c549bd16b46ea29f0e1b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 18:08:41 +0200 Subject: [PATCH 059/259] new parametrized implementation for country propagation --- .../PrepareResultCountryAssociation.java | 68 ++-- .../SparkCountryPropagationJob2.java | 298 ++++++------------ 2 files changed, 133 insertions(+), 233 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java index 9572159ce..b84c4afb5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java @@ -14,14 +14,13 @@ import org.apache.spark.sql.SaveMode; import java.util.Arrays; import java.util.List; -import java.util.Optional; + import static eu.dnetlib.dhp.PropagationConstant.createOutputDirs; import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; /** * For the association of the country to the datasource @@ -38,53 +37,46 @@ public class PrepareResultCountryAssociation { public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString(PrepareResultCountryAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); parser.parseArgument(args); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); - - String graphTableClassName = parser.get("graphTableClassName"); - log.info("graphTableClassName: {}", graphTableClassName); - SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkSession(conf, isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - joinRelationEntity(spark, inputRelationsPath, inputEntityPath, entityClazz, outputPath); - }); - - final SparkSession spark = SparkSession .builder() - .appName(PrepareResultCountryAssociation.class.getSimpleName()) + .appName(SparkCountryPropagationJob.class.getSimpleName()) .master(parser.get("master")) .config(conf) .enableHiveSupport() .getOrCreate(); - //todo add link to working dir + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + prepareDatasourceCountryAssociation(spark, + Arrays.asList(parser.get("whitelist").split(";")), + Arrays.asList(parser.get("allowedtypes").split(";")), + inputPath, + outputPath); - List whitelist = Arrays.asList(parser.get("whitelist").split(";")); - List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); + } + + private static void prepareDatasourceCountryAssociation(SparkSession spark, + List whitelist, + List allowedtypes, + String inputPath, + String outputPath) { String whitelisted = ""; for (String i : whitelist){ whitelisted += " OR id = '" + i + "'"; @@ -93,13 +85,13 @@ public class PrepareResultCountryAssociation { Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); @@ -107,18 +99,18 @@ public class PrepareResultCountryAssociation { String query = "SELECT source ds, country.classid country " + "FROM ( SELECT id " + - "FROM datasource " + - "WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + - getConstraintList("datasourcetype.classid = '", allowedtypes) + ") d " + + " FROM datasource " + + " WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + + getConstraintList("datasourcetype.classid = '", allowedtypes) + ") d " + "JOIN ( SELECT source, target " + - "FROM relation " + - "WHERE relclass = 'provides' " + - "AND datainfo.deletedbyinference = false ) rel " + + " FROM relation " + + " WHERE relclass = 'provides' " + + " AND datainfo.deletedbyinference = false ) rel " + "ON d.id = rel.source " + "JOIN (SELECT id, country " + - "FROM organization " + - "WHERE datainfo.deletedbyinference = false " + - "AND length(country.classid)>0) o " + + " FROM organization " + + " WHERE datainfo.deletedbyinference = false " + + " AND length(country.classid)>0) o " + "ON o.id = rel.target"; spark.sql(query) @@ -129,4 +121,6 @@ public class PrepareResultCountryAssociation { } + + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 23595bada..91b12b444 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -4,21 +4,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; +import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; import org.slf4j.Logger; @@ -32,8 +28,25 @@ public class SparkCountryPropagationJob2 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob2.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); + String jsonConfiguration = IOUtils.toString(SparkCountryPropagationJob2.class + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + parser.parseArgument(args); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + final String datasourcecountrypath = outputPath + "/prepared_datasource_country"; + final String resultClassName = parser.get("resultClazz"); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".")+1); + + Class resultClazz = (Class) Class.forName(resultClassName); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession @@ -44,242 +57,135 @@ public class SparkCountryPropagationJob2 { .enableHiveSupport() .getOrCreate(); - - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; - - // createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + final boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + final boolean saveGraph = TRUE.equals(parser.get("saveGraph")); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + //Load parque file with preprocessed association datasource - country + Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); + //broadcasting the result of the preparation step + Broadcast> broadcast_datasourcecountryassoc = sc.broadcast(datasourcecountryassoc); - Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)).rdd(), - Encoders.bean(OtherResearchProduct.class)); - - Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)).rdd(), - Encoders.bean(Software.class)); - - Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)).rdd(), - Encoders.bean(Publication.class)); - - - //todo broadcast - - - software.createOrReplaceTempView("software"); - final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); - - dataset.createOrReplaceTempView("dataset"); - final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); - - other.createOrReplaceTempView("other"); - final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); - - publication.createOrReplaceTempView("publication"); - final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); + Dataset potentialUpdates = getPotentialResultToUpdate(spark, inputPath, resultClazz, broadcast_datasourcecountryassoc); if(writeUpdates){ - writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + writeUpdates(potentialUpdates.toJavaRDD(), outputPath + "/update_" + resultType); } if(saveGraph){ - createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/software"); + updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath + "/" + resultType); - createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/dataset"); - - createUpdateForOtherDataset(toupdateresultother, inputPath, spark) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); - - createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); } } - private static void writeUpdates(JavaRDD software, JavaRDD dataset, JavaRDD other , JavaRDD publication, String outputPath){ - createUpdateForResultDatasetWrite(software, outputPath, "update_software"); - createUpdateForResultDatasetWrite(dataset, outputPath, "update_dataset"); - createUpdateForResultDatasetWrite(other, outputPath, "update_other"); - createUpdateForResultDatasetWrite(publication, outputPath, "update_publication"); - } + private static void updateResultTable(SparkSession spark, Dataset potentialUpdates, + String inputPath, + Class resultClazz, + String outputPath) { - private static JavaRDD createUpdateForOtherDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + Dataset result = readPathEntity(spark, inputPath, resultClazz); - return sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - OtherResearchProduct oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { + Dataset> result_pair = result + .map(r -> new Tuple2<>(r.getId(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); + + Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getString(0), pu.getList(1)), + Encoders.tuple(Encoders.STRING(), Encoders.bean(List.class))); + + Dataset new_table = result_pair + .joinWith(potential_update_pair, result_pair.col("_1").equalTo(potential_update_pair.col("_1")), "left") + .map((MapFunction, Tuple2>, R>) value -> { + R r = value._1()._2(); + Optional> potentialNewCountries = Optional.ofNullable(value._2()).map(Tuple2::_2); + if (potentialNewCountries != null) { HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { + for (Qualifier country : r.getCountry()) { countries.add(country.getClassid()); } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); + + for (Object country : potentialNewCountries.get()) { + if (!countries.contains(country)) { + r.getCountry().add(getCountry((String) country)); } } - oaf.setCountry(countryList); } - return oaf; - }); + return r; + + }, Encoders.bean(resultClazz)); + + + log.info("Saving graph table to path: {}", outputPath); + result + .toJSON() + .write() + .option("compression", "gzip") + .text(outputPath); + } + + + + private static Dataset getPotentialResultToUpdate(SparkSession spark, String inputPath, + Class resultClazz, + Broadcast> broadcast_datasourcecountryassoc) { + + Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + createCfHbforresult(spark); + return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); } - private static JavaRDD createUpdateForPublicationDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - return sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - Publication oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - Software oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD createUpdateForDatasetDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)) - .mapToPair(d -> new Tuple2<>(d.getId(), d)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - eu.dnetlib.dhp.schema.oaf.Dataset oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD propagateOnResult(SparkSession spark, String result_type) { + private static void createCfHbforresult(SparkSession spark) { String query; query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + "FROM ( SELECT id, instance " + - "FROM " + result_type + + "FROM result " + " WHERE datainfo.deletedbyinference = false) ds " + "LATERAL VIEW EXPLODE(instance) i AS inst"; Dataset cfhb = spark.sql(query); cfhb.createOrReplaceTempView("cfhb"); - - return countryPropagationAssoc(spark, "cfhb").toJavaRDD(); - } - private static Dataset countryPropagationAssoc(SparkSession spark, String cfhbTable){ + + private static Dataset countryPropagationAssoc(SparkSession spark, + Broadcast> broadcast_datasourcecountryassoc){ + Dataset datasource_country = broadcast_datasourcecountryassoc.value(); + datasource_country.createOrReplaceTempView("datasource_country"); + String query = "SELECT id, collect_set(country) country "+ "FROM ( SELECT id, country " + "FROM rels " + - "JOIN " + cfhbTable + + "JOIN cfhb " + " ON cf = ds " + "UNION ALL " + "SELECT id , country " + "FROM rels " + - "JOIN " + cfhbTable + + "JOIN cfhb " + " ON hb = ds ) tmp " + "GROUP BY id"; return spark.sql(query); } - private static JavaPairRDD getStringResultJavaPairRDD(JavaRDD toupdateresult) { - return toupdateresult.map(c -> { - List countryList = new ArrayList<>(); - List tmp = c.getList(1); - for (String country : tmp) { - countryList.add(getCountry(country)); - } - Result r = new Result(); - r.setId(c.getString(0)); - r.setCountry(countryList); - return r; - }).mapToPair(r -> new Tuple2<>(r.getId(), r)); + private static Dataset readPathEntity(SparkSession spark, String inputEntityPath, Class resultClazz) { + + log.info("Reading Graph table from: {}", inputEntityPath); + return spark + .read() + .textFile(inputEntityPath) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), Encoders.bean(resultClazz)); } - private static void createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type){ - toupdateresult.map(c -> { - List countryList = new ArrayList<>(); - List tmp = c.getList(1); - for (String country : tmp) { - countryList.add(getCountry(country)); - } - Result r = new Result(); - r.setId(c.getString(0)); - r.setCountry(countryList); - return r; + private static Dataset readAssocDatasourceCountry(SparkSession spark, String relationPath) { + return spark.read() + .load(relationPath) + .as(Encoders.bean(DatasourceCountry.class)); + } - }).map(r ->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath+"/"+type); + private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ + potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) + .saveAsTextFile(outputPath); } From 6dfdba9ef7752403c539c793c0fd7868925d44c2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 18:14:37 +0200 Subject: [PATCH 060/259] new parametrized implementation for country propagation --- ...ssociation.java => PrepareDatasourceCountryAssociation.java} | 0 .../dhp/countrypropagation/SparkCountryPropagationJob2.java | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/{PrepareResultCountryAssociation.java => PrepareDatasourceCountryAssociation.java} (100%) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountryAssociation.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 91b12b444..210d42f9c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -123,7 +123,7 @@ public class SparkCountryPropagationJob2 { .option("compression", "gzip") .text(outputPath); } - + private static Dataset getPotentialResultToUpdate(SparkSession spark, String inputPath, From a2d309545b2e02b764913d335e7a50d3dea8de84 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 19:12:59 +0200 Subject: [PATCH 061/259] new parametrized implementation for country propagation --- .../dhp/countrypropagation/Country.java | 24 +++++++++++++++ .../countrypropagation/DatasourceCountry.java | 6 ++-- .../PrepareDatasourceCountryAssociation.java | 8 ++--- .../countrypropagation/ResultCountrySet.java | 25 ++++++++++++++++ .../SparkCountryPropagationJob.java | 7 +++-- .../SparkCountryPropagationJob2.java | 30 ++++++++++--------- 6 files changed, 76 insertions(+), 24 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java new file mode 100644 index 000000000..accd278d7 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java @@ -0,0 +1,24 @@ +package eu.dnetlib.dhp.countrypropagation; + +import java.io.Serializable; + +public class Country implements Serializable { + private String classid; + private String classname; + + public String getClassid() { + return classid; + } + + public void setClassid(String classid) { + this.classid = classid; + } + + public String getClassname() { + return classname; + } + + public void setClassname(String classname) { + this.classname = classname; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java index 460764e16..3cce8f4bb 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java @@ -4,7 +4,7 @@ import java.io.Serializable; public class DatasourceCountry implements Serializable { private String dataSourceId; - private String country; + private Country country; public String getDataSourceId() { return dataSourceId; @@ -14,11 +14,11 @@ public class DatasourceCountry implements Serializable { this.dataSourceId = dataSourceId; } - public String getCountry() { + public Country getCountry() { return country; } - public void setCountry(String country) { + public void setCountry(Country country) { this.country = country; } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index b84c4afb5..887ddbc69 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -29,14 +29,14 @@ import org.slf4j.LoggerFactory; * relation provides between Datasource and Organization is exploited to get the country for the datasource */ -public class PrepareResultCountryAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareResultCountryAssociation.class); +public class PrepareDatasourceCountryAssociation { + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultCountryAssociation.class + String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -97,7 +97,7 @@ public class PrepareResultCountryAssociation { relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); - String query = "SELECT source ds, country.classid country " + + String query = "SELECT source ds, named_struct('classid', country.classid, 'classname', country.classname) country " + "FROM ( SELECT id " + " FROM datasource " + " WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java new file mode 100644 index 000000000..4978d9182 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java @@ -0,0 +1,25 @@ +package eu.dnetlib.dhp.countrypropagation; + +import java.io.Serializable; +import java.util.Set; + +public class ResultCountrySet implements Serializable { + private String resultId; + private Set countrySet; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public Set getCountrySet() { + return countrySet; + } + + public void setCountrySet(Set countrySet) { + this.countrySet = countrySet; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 2ebd818cb..cf80649b6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -100,7 +100,7 @@ public class SparkCountryPropagationJob { datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); - +//todo add length(country.classid)>0 String query = "SELECT source ds, target org, country.classid country " + "FROM ( SELECT id " + "FROM datasource " + @@ -118,6 +118,7 @@ public class SparkCountryPropagationJob { "WHERE datainfo.deletedbyinference = false ) o " + "ON o.id = rel.target"; + //todo broadcast Dataset rels = spark.sql(query); rels.createOrReplaceTempView("rels"); @@ -268,11 +269,11 @@ public class SparkCountryPropagationJob { }); } - private static JavaRDD propagateOnResult(SparkSession spark, String table) { + private static JavaRDD propagateOnResult(SparkSession spark, String result_type) { String query; query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + "FROM ( SELECT id, instance " + - "FROM " + table + + "FROM " + result_type + " WHERE datainfo.deletedbyinference = false) ds " + "LATERAL VIEW EXPLODE(instance) i AS inst"; Dataset cfhb = spark.sql(query); diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 210d42f9c..967c940b5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -5,7 +5,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; @@ -41,7 +40,7 @@ public class SparkCountryPropagationJob2 { final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; final String datasourcecountrypath = outputPath + "/prepared_datasource_country"; - final String resultClassName = parser.get("resultClazz"); + final String resultClassName = parser.get("resultTableName"); final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".")+1); @@ -67,7 +66,8 @@ public class SparkCountryPropagationJob2 { //broadcasting the result of the preparation step Broadcast> broadcast_datasourcecountryassoc = sc.broadcast(datasourcecountryassoc); - Dataset potentialUpdates = getPotentialResultToUpdate(spark, inputPath, resultClazz, broadcast_datasourcecountryassoc); + Dataset potentialUpdates = getPotentialResultToUpdate(spark, inputPath, resultClazz, + broadcast_datasourcecountryassoc).as(Encoders.bean(ResultCountrySet.class)); if(writeUpdates){ writeUpdates(potentialUpdates.toJavaRDD(), outputPath + "/update_" + resultType); @@ -80,7 +80,7 @@ public class SparkCountryPropagationJob2 { } - private static void updateResultTable(SparkSession spark, Dataset potentialUpdates, + private static void updateResultTable(SparkSession spark, Dataset potentialUpdates, String inputPath, Class resultClazz, String outputPath) { @@ -91,23 +91,24 @@ public class SparkCountryPropagationJob2 { .map(r -> new Tuple2<>(r.getId(), r), Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); - Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getString(0), pu.getList(1)), - Encoders.tuple(Encoders.STRING(), Encoders.bean(List.class))); + Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), + pu), + Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); Dataset new_table = result_pair .joinWith(potential_update_pair, result_pair.col("_1").equalTo(potential_update_pair.col("_1")), "left") - .map((MapFunction, Tuple2>, R>) value -> { + .map((MapFunction, Tuple2>, R>) value -> { R r = value._1()._2(); - Optional> potentialNewCountries = Optional.ofNullable(value._2()).map(Tuple2::_2); - if (potentialNewCountries != null) { + Optional potentialNewCountries = Optional.ofNullable(value._2()).map(Tuple2::_2); + if (potentialNewCountries.isPresent()) { HashSet countries = new HashSet<>(); for (Qualifier country : r.getCountry()) { countries.add(country.getClassid()); } - for (Object country : potentialNewCountries.get()) { - if (!countries.contains(country)) { - r.getCountry().add(getCountry((String) country)); + for (Country country : potentialNewCountries.get().getCountrySet()) { + if (!countries.contains(country.getClassid())) { + r.getCountry().add(getCountry(country.getClassid(),country.getClassname())); } } } @@ -117,7 +118,7 @@ public class SparkCountryPropagationJob2 { log.info("Saving graph table to path: {}", outputPath); - result + new_table .toJSON() .write() .option("compression", "gzip") @@ -151,6 +152,7 @@ public class SparkCountryPropagationJob2 { private static Dataset countryPropagationAssoc(SparkSession spark, Broadcast> broadcast_datasourcecountryassoc){ + Dataset datasource_country = broadcast_datasourcecountryassoc.value(); datasource_country.createOrReplaceTempView("datasource_country"); @@ -183,7 +185,7 @@ public class SparkCountryPropagationJob2 { .as(Encoders.bean(DatasourceCountry.class)); } - private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ + private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) .saveAsTextFile(outputPath); } From 9c63c4840d4f7c31b9b68aa913775e3528887bdb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 8 Apr 2020 19:13:42 +0200 Subject: [PATCH 062/259] new workflow and parameters for country propagation --- .../input_countrypropagation_parameters.json | 18 +-- .../countrypropagation/oozie_app/workflow.xml | 142 +++++++++++++++++- 2 files changed, 147 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 81fead58f..068c673ce 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -11,18 +11,6 @@ "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"wl", - "paramLongName":"whitelist", - "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", - "paramRequired": true - }, - { - "paramName":"at", - "paramLongName":"allowedtypes", - "paramDescription": "the types of the allowed datasources. Split by ;", - "paramRequired": true - }, { "paramName":"h", "paramLongName":"hive_metastore_uris", @@ -40,5 +28,11 @@ "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", "paramRequired": true + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index aa1e6dc78..d91207f46 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -38,12 +38,152 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + yarn + cluster + PrepareDatasourceCountryAssociation + eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation + dhp-graph-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + -mt yarn-cluster + --sourcePath${sourcePath} + --whitelist${whitelist} + --allowedtypes${allowedtypes} + --hive_metastore_uris${hive_metastore_uris} + + + + + + + + + + + + + + yarn + cluster + countryPropagationForPublications + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + dhp-graph-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + -mt yarn-cluster + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + + + + + + + + yarn + cluster + countryPropagationForDataset + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + dhp-graph-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + -mt yarn-cluster + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + + + + + + + + yarn + cluster + countryPropagationForORP + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + dhp-graph-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + -mt yarn-cluster + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Otherresearchproduct + + + + + + + + yarn + cluster + countryPropagationForSoftware + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + dhp-graph-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCoresForJoining} + --executor-memory=${sparkExecutorMemoryForJoining} + --driver-memory=${sparkDriverMemoryForJoining} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + -mt yarn-cluster + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + + + + + ${jobTracker} From 627ad58a8b8fa6c53908c3a6e9bceb819a681c23 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 9 Apr 2020 11:33:19 +0200 Subject: [PATCH 063/259] new wf definition --- .../{Country.java => CountrySbs.java} | 0 .../input_countrypropagation_parameters.json | 12 +++++++ .../input_prepare_dc_assoc.json | 32 ------------------- .../countrypropagation/oozie_app/workflow.xml | 24 ++++++++++++-- 4 files changed, 33 insertions(+), 35 deletions(-) rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/{Country.java => CountrySbs.java} (100%) delete mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/Country.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 068c673ce..829c96420 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -34,5 +34,17 @@ "paramLongName":"resultTableName", "paramDescription": "the name of the result table we are currently working on", "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json deleted file mode 100644 index ff1b79d9c..000000000 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json +++ /dev/null @@ -1,32 +0,0 @@ -[ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, - { - "paramName":"s", - "paramLongName":"sourcePath", - "paramDescription": "the path of the sequencial file to read", - "paramRequired": true - }, - { - "paramName":"wl", - "paramLongName":"whitelist", - "paramDescription": "datasource id that will be considered even if not in the allowed typology list. Split by ;", - "paramRequired": true - }, - { - "paramName":"at", - "paramLongName":"allowedtypes", - "paramDescription": "the types of the allowed datasources. Split by ;", - "paramRequired": true - }, - { - "paramName":"h", - "paramLongName":"hive_metastore_uris", - "paramDescription": "the hive metastore uris", - "paramRequired": true - } -] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index d91207f46..38aa93335 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -36,6 +36,15 @@ saveGraph writes new version of the graph after the propagation step + + outputPath + the path used to store temporary output files + + + preparedInfoPath + the path where prepared info have been stored + + @@ -65,13 +74,14 @@ --whitelist${whitelist} --allowedtypes${allowedtypes} --hive_metastore_uris${hive_metastore_uris} + --outputPath${workingDir}/country_propagation/preparedInfo - - + + @@ -98,6 +108,8 @@ --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/country_propagation/publication + --preparedInfoPath${workingDir}/country_propagation/preparedInfo @@ -125,6 +137,8 @@ --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/country_propagation/publication + --preparedInfoPath${workingDir}/country_propagation/preparedInfo @@ -152,6 +166,8 @@ --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Otherresearchproduct + --outputPath${workingDir}/country_propagation/publication + --preparedInfoPath${workingDir}/country_propagation/preparedInfo @@ -179,11 +195,13 @@ --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/country_propagation/publication + --preparedInfoPath${workingDir}/country_propagation/preparedInfo - + ${jobTracker} From 90469789b9885a0395ca0c3fc1dbd720c572c9b2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 9 Apr 2020 13:29:01 +0200 Subject: [PATCH 064/259] two new classes fro new implementation of project to result propagation --- .../PrepareResultProjectAssociation.java | 69 ++++++ ...SparkResultToProjectThroughSemRelJob3.java | 219 ++++++++++++++++++ 2 files changed, 288 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java new file mode 100644 index 000000000..d7c29a697 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java @@ -0,0 +1,69 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskResultLost; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.PropagationConstant.TRUE; +import static eu.dnetlib.dhp.PropagationConstant.createOutputDirs; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultProjectAssociation { + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception{ + + String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_projecttoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + log.info("writeUpdates: {} ", writeUpdates); + + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + log.info("saveGraph {}", saveGraph); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + }); + + + + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java new file mode 100644 index 000000000..f85eb2bed --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -0,0 +1,219 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToProjectThroughSemRelJob3 { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils.toString(SparkResultToProjectThroughSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); + parser.parseArgument(args); + + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToProjectThroughSemRelJob3.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/projecttoresult"; + boolean writeUpdates = "true".equals(parser.get("writeUpdate")); + boolean saveGraph = "true".equals(parser.get("saveGraph")); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + relation.createOrReplaceTempView("relation"); + + String query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false and relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; + + Dataset resproj_relation = spark.sql(query); + + query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + getConstraintList(" relClass = '", allowedsemrel ); + + Dataset resres_relation = spark.sql(query); + resres_relation.createOrReplaceTempView("resres_relation"); + resproj_relation.createOrReplaceTempView("resproj_relation"); + + query ="SELECT proj, collect_set(r1target) result_set " + + "FROM (" + + " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + + " FROM resres_relation r1 " + + " JOIN resproj_relation r2 " + + " ON r1.source = r2.source " + + " ) tmp " + + "GROUP BY proj "; + + Dataset toaddrelations = spark.sql(query); + + query = "select target, collect_set(source) result_list from " + + "resproj_relation " + + "group by target"; + + Dataset project_resultlist = spark.sql(query); + + //if (writeUpdaes){ + toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/toupdaterelations"); + //} + + if(saveGraph){ + JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + .leftOuterJoin(project_resultlist.toJavaRDD().mapToPair(pr -> new Tuple2<>(pr.getString(0), pr.getList(1)))) + .flatMap(c -> { + List toAddRel = new ArrayList<>(); + toAddRel.addAll(c._2()._1()); + if (c._2()._2().isPresent()) { + List originalRels = c._2()._2().get(); + for (Object o : originalRels) { + if (toAddRel.contains(o)) { + toAddRel.remove(o); + } + } + } + List relationList = new ArrayList<>(); + String projId = c._1(); + for (Object r : toAddRel) { + String rId = (String) r; + relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + + } + if(relationList.size()==0){ + return null; + } + return relationList.iterator(); + }).filter(r -> !(r==null)); + + + new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/new_relations" ); + + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .union(new_relations) + .map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/relation"); + + } + + + //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); + +// JavaPairRDD result_project = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) +// && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) +// .map(rel ->{ +// +// TypedRow tr = new TypedRow(); +// tr.setSourceId(rel.getSource()); +// tr.setTargetId(rel.getTarget()); +// return tr; +// }) +// .mapToPair(toPair()); +// +// //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result +// JavaPairRDD project_result = result_project.join(result_result) +// .map(c -> { +// String projectId = c._2()._1().getTargetId(); +// String resultId = c._2()._2().getTargetId(); +// TypedRow tr = new TypedRow(); tr.setSourceId(projectId); tr.setTargetId(resultId); +// return tr; +// }) +// .mapToPair(toPair()); +// +// //relationships from project to result. One Pair for each project => project id list of results related to the project +// JavaPairRDD project_results = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) +// .map(r -> { +// TypedRow tr = new TypedRow(); tr.setSourceId(r.getSource()); tr.setTargetId(r.getTarget()); +// return tr; +// }) +// .mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) { +// return b; +// } +// if (b == null) { +// return a; +// } +// a.addAll(b.getAccumulator()); +// return a; +// }); +// +// +// +// JavaRDD newRels = project_result.join(project_results) +// .flatMap(c -> { +// String resId = c._2()._1().getTargetId(); +// +// if (c._2()._2().getAccumulator().contains(resId)) { +// return null; +// } +// String progId = c._2()._1().getSourceId(); +// List rels = new ArrayList(); +// +// rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, +// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); +// rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, +// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); +// return rels.iterator(); +// }) +// .cache(); +// +// newRels.map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/relation_new"); +// +// newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/relation"); + + } + + + + +} From 7783b09c5bc601d78a85209b6b9ae071728cb779 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:26:23 +0200 Subject: [PATCH 065/259] new implementation for result to project propagation. Prepare some info to be used in propagation --- .../PrepareProjectResultsAssociation.java | 113 ++++++++++++++++++ .../PrepareResultProjectAssociation.java | 69 ----------- .../dhp/projecttoresult/ProjectResultSet.java | 4 + .../input_prepareassoc_parameters.json | 62 ++++++++++ ...put_prepareprojecttoresult_parameters.json | 39 ++++++ .../CountryPropagationJobTest.java | 4 + .../PrepareDataForTest.java | 4 + 7 files changed, 226 insertions(+), 69 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java new file mode 100644 index 000000000..5daaceeea --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -0,0 +1,113 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.TaskResultLost; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultProjectAssociation { + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception{ + + String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepareprojecttoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String potentialUpdatePath = parser.get("potentialUpdatePath"); + log.info("potentialUpdatePath {}: ", potentialUpdatePath); + + String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath: {} ", alreadyLinkedPath); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + createOutputDirs(potentialUpdatePath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + prepareResultProjAssoc(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); + + }); + } + + private static void prepareResultProjAssoc(SparkSession spark, String inputPath, String potentialUpdatePath, + String alreadyLinkedPath, List allowedsemrel) { + JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + relation.createOrReplaceTempView("relation"); + + String query = "SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; + + Dataset resproj_relation = spark.sql(query); + resproj_relation.createOrReplaceTempView("resproj_relation"); + + query ="SELECT projectId, collect_set(r1target) resultSet " + + "FROM (" + + " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + + " FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + getConstraintList(" relClass = '", allowedsemrel ) + ") r1" + + " JOIN resproj_relation r2 " + + " ON r1.source = r2.source " + + " ) tmp " + + "GROUP BY proj "; + + spark.sql(query).as(Encoders.bean(ProjectResultSet.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + + + query = "SELECT target, collect_set(source) result_list " + + "FROM resproj_relation " + + "GROUP BY target"; + + spark.sql(query) + .as(Encoders.bean(ProjectResultSet.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + + + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java deleted file mode 100644 index d7c29a697..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareResultProjectAssociation.java +++ /dev/null @@ -1,69 +0,0 @@ -package eu.dnetlib.dhp.projecttoresult; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.TaskResultLost; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - -import static eu.dnetlib.dhp.PropagationConstant.TRUE; -import static eu.dnetlib.dhp.PropagationConstant.createOutputDirs; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - -public class PrepareResultProjectAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public static void main(String[] args) throws Exception{ - - String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_projecttoresult_parameters.json")); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); - - parser.parseArgument(args); - - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); - - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); - - boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - log.info("writeUpdates: {} ", writeUpdates); - - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - log.info("saveGraph {}", saveGraph); - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - - runWithSparkHiveSession(conf, isSparkSessionManaged, - spark -> { - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - }); - - - - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java new file mode 100644 index 000000000..39af1f59c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.projecttoresult; + +public class ProjectResultSet { +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json new file mode 100644 index 000000000..5d66e4edb --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json @@ -0,0 +1,62 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be written. No double check if information is already present", + "paramRequired": false + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": false + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "w", + "paramLongName": "whitelist", + "paramDescription": "the datasource having a type different from the allowed ones but that we want to add anyway", + "paramRequired": true + }, + { + "paramName": "at", + "paramLongName": "allowedtypes", + "paramDescription": "the allowed datasource types for country propagation", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json new file mode 100644 index 000000000..9c790f219 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json @@ -0,0 +1,39 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + + { + "paramName":"asr", + "paramLongName":"allowedsemrels", + "paramDescription": "the types of the allowed datasources. Split by ;", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java new file mode 100644 index 000000000..3c24abaf7 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.countrypropagation; + +public class CountryPropagationJobTest { +} diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java new file mode 100644 index 000000000..4fb31bf94 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.countrypropagation; + +public class PrepareDataForTest { +} From 182247661333a7d36ed5b6257d0b4e9f7c06a305 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:28:09 +0200 Subject: [PATCH 066/259] Test for country propagation --- .../CountryPropagationJobTest.java | 159 ++++++++++++++++++ 1 file changed, 159 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 3c24abaf7..db8e3fd99 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -1,4 +1,163 @@ package eu.dnetlib.dhp.countrypropagation; +import com.google.gson.Gson; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.Country; +import eu.dnetlib.dhp.schema.oaf.Datasource; +import eu.dnetlib.dhp.schema.oaf.Software; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.*; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import scala.Array; +import scala.Tuple2; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + public class CountryPropagationJobTest { + private static final Logger log = LoggerFactory.getLogger(CountryPropagationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = CountryPropagationJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(CountryPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(CountryPropagationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(CountryPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + + + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void testCountryPropagationSoftware() throws Exception { + SparkCountryPropagationJob2.main(new String[]{ + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/countrypropagation/sample/software").getPath(), + "-hive_metastore_uris", "", + "-writeUpdate","false", + "-saveGraph","true", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath",workingDir.toString() + "/software", + "-preparedInfoPath", getClass().getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/software") + .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); + + //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + + Assertions.assertEquals(10, tmp.count()); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); + + Assertions.assertEquals(6, verificationDs.filter("size(country) > 0").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 1").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 2").count()); + Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); + + + Dataset countryExploded = verificationDs + .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) + .map(c -> c.getClassid(), Encoders.STRING()); + + Assertions.assertEquals(9, countryExploded.count()); + + Assertions.assertEquals(1, countryExploded.filter("value = 'FR'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'TR'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'IT'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'US'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'MX'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'CH'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); + + Dataset> countryExplodedWithCountryclassid = verificationDs + .flatMap(row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getClassid()))); + return prova.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); + + countryExplodedWithCountryclassid.show(false); + Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'FR' ").count()); + Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'TR' ").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'IT' or _2 = 'MX') ").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'IT' or _2 = 'US') ").count()); + Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'JP'").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'CH' or _2 = 'JP') ").count()); + + Dataset> countryExplodedWithCountryclassname = verificationDs + .flatMap(row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getClassname()))); + return prova.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + countryExplodedWithCountryclassname.show(false); + Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'France' ").count()); + Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'Turkey' ").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'Italy' or _2 = 'Mexico') ").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'Italy' or _2 = 'United States') ").count()); + Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'Japan' ").count()); + Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'Switzerland' or _2 = 'Japan') ").count()); + + Dataset> countryExplodedWithCountryProvenance = verificationDs + .flatMap(row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getDataInfo().getInferenceprovenance()))); + return prova.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Assertions.assertEquals(7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); + } + } + + From 79b8ea4fedfd56af40928b61ee1323bdfe0403de Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:29:41 +0200 Subject: [PATCH 067/259] prepared information to be used in actual country propagation. Subset of info --- .../preparedInfo/preparedInfo.json.gz | Bin 0 -> 985 bytes .../sample/software/software_10.json.gz | Bin 0 -> 6906 bytes 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..91a89da8c3d58147aebc199c322dd86d977a9d74 GIT binary patch literal 985 zcmV;~119_*iwFq6kdR&g18{O>aA9&~WJzvjZ!T(cZ*Bm!mr-k5M-YYI`zwN;`>?aS zvoq`0P*Vqz7CV8ymD!nD5OpOZ%cQBB|K3p_1(%8_S0EWl;7@b+oHOUXZcL8yuzxwI z-`VD71L>drlkaRVhu`CIb0c1`bZ#W#t}S(`xg~AW2W6af^W#SMFT3O6%jV{FqmT0Z zygmJ+kN2lpkkL?dNE~nD3EU(V>`UzXPiO4=m-t3@Z*JZEAjj?g!r`G4 z7gnUMWf1pFp40$`RPBuyf57(`9&!lE1Ts60RI4eLnou*2YHYqHyN9rQhB>rm9@q+x zlf%3;v^q1a$+kJiXg;k_i#f;0S#8!1UK!kUO?LLIiIZXNiCU70-BNh%q`kp))oYII zO4(*oQ9-bY?F6r#v=mObCc6Po5uni8$UftssgzbRQ;`96RW^1{Bexj8&+J5#E%aUD zPa+jwjPDP>oUuRb51-}IB*jjS2~(apXJaoOzUlxqr;xa;IWUKlJD0(y1P9}(hO@+` z>e3Kl(OlQ0xEUs|*0oZYDP)$bbB|Yc8wFQQ@x)XUW7xQW`+(Uw4^1wU!3xeb=7pg? z(RdiPmN=AL#%wsqL2|{#^?VIer&C8sQj~^53k`>tjJc3m?2RSn+n>*v51+Tkzy0ti zJG+cLc_K8>*=yuSY^&IfCt=Iz2$9{Zl#xEYdaS}I1a*KA=M2#iENN;*oR*-s=g`}O z?9?wjSB~Alo=0J?!s?{6&KCKRks2>vgB5J^>Yfz}th-c?XKUD%d$^o%w_O;L8DdqAlW=JCF0o9mzvg+QsN#1O6x z>k3g!J1dr~%U-<-%FKvZ2`NpQVVW}MB*o~sP+8Xzl1Kc;nL7BemL_6&C}cd1N;m9_ zKKR|4Tqu8G}qpNlVPTk7vxgqdZ<14HiX)03q%dFeDuF zc8oq1UgbzBS*#P+FjI-x^GTN4oYsn!Di)GW9cIXCQAOXSvVP;d?|(aEzxThky1%fq zC4{&wG0Pd5ti_6{Q6#{lv0H`3fv6WGA`3Y{~_a;0)FrazcWV$9wybxBDp3Q$B=V^P#RAAcvgMX!U0; zOI~5iD~TY82D%#!bmIr$$3a2`J;>8yF#d7C^Ale2C}V?hO0$X$AthyH$&zJ4!ZIdU z5-sE#md9OQQP!0hj0eUj95i3z`*~TMu$;=A19;Ir(^&8#N5XMNtBM1aLA9v(Y+jUN z9G68^ozm>XxMcI9;-V-Q@VU&I(;2%?<|>}D8IuJkITPcK_Ku`oE%P)bUXtslpN0dV zE9yD0iwQnsm7ufvU~Ia9Yq+i#26o^>qmhzp{BM!Gm;b?QYdrRtPs7MEsg*L8IrI^z41o*rMrS@ss@62FqJ7e1&Ic|t_X(pJ|QP4op!-!VwkH?4b z%~Xi__~}z1Hv)PIhqjL7kFj##$H6WUSh5;|gS_9Zpz3TmJMC9CR8h{c?(b$`&GQ(Ch1j@~7& zcWG79dnl^NiV0_%yY$6evPb8WnoicL!1Bo&_5FO3@oKt(cw53|qTSn{u!q*U0HY*3 zKvhRSKyOkHf(x{`An~?i?t7j<8o_7g2_a%_3S(3K%ez zAXF-twlZ$Sf;LjXSk^qrF;4v*7tpzRR10alm4TBVDif!QilH8*v^SU|!r=r2T2(Ok zO+}X}??}KO*w?dTwv-Mg^So-b0E|Y)W0qGd8i9jO1*1+}GO!aTEG}j)n&_8LZfvLip(Pz_bm`&p4(GJdRN_^A{7#bMJh ztZDx|X#D-^6=3i?XIvmtxh7;mu7Ik3~SAuGn$P+=Vuv4541q;Qy`P`f8>ln zpIyv~az544xL0X_Pr$VV?o$E@4CN)zI6ouc)KI|09&a_D%K(}eFa4LCCG^;aSa_$O#a+Ah5y zTTKxOi5BGLP!_{J&ZU<{rXZTK__UZH2Npw8l$h&S<$~!71J9h`{3VA8%##t3)m*6r zJXiA!*R~RZns!>idth-69!WJO@S7wxlNEy^yJ|HJ>guj&7Vx8Nzu=;RFH)%%Le2FF zI)*Z5$xyXjU)BML0|S4kb4BHp zi>cy7Q|T$4XW+b@3<)}b2|=miYdU9AIB%d(@*hO`M`jqhL-KCt1qqEIaXf3=@!^lt zDKra;&azgQ2!<&WS|$W~XHizNzN->}_ZPT5r%)x-k%qH8pVxwb=fzO)lwep~J#JMD z^-6MUjC@2lb0j{BCs-XQQ&LNLSqYNy9E(aYjtN?hFYRI}HwdRp)e zS+?dv;aE?jBfc!DuSAf?GGtc*RZ5F2D^3*)(5_0ctas`yWqetoPOaO^m!i?1 z7a+N1fx|{t%8tS)!a;93!ppw8Zde*YFN&H>z&I$&rMR(4)?+l$X!ln8>+l)|YuNSY z2@a#3sZ>UFrMv9CtcGOorGOvM%pr=TXJ&w7Fyp9Ld7^6djAMLb(d$?JF`y#5P{J-0 zxSOnS>0(A{CD&WhGI7;-Lq9n!aa_sOSGm%2o&X=gd_alPzPDx-^W=4$F^~#ofZD*Y zUt_SCyaYZjw*Sm{4&7!#K>2ybqmtLN5qVjhqLRr-HWYJ;!VZ`o zg|r!heo)MUii5!AOhPIPR#QFg_FFC6wUFhN$WN#mj7g{wjUiesuo)SllLKiXo1R>F zo3Olqw#oMuLo+Q{wiuD;ty%Bhk1v)#eJ2Y&7Tac9V{Bpf`I7jU%s^N9JliBq5HXYl zpU8r?K!iB000o5HD38_Lz~$cXU*hwH8HQ@>26j|ZF_JoluCY>M*}HJYQH$0n1G$z> zI32-Ks*zsxxyi)!=U37UPp_nDY{Sj54Hebc290gF6Kn%zwrhuh=Tpj9>=_|-EYFC& zltsSfrp{-@HZ+TAx4|~tRmHWz*3btn(q9C92)w}DuwB43yz$tI{FsKm8eg?tzzs~#GA-Mpu4hI*3nC|Ut;h_bATpTiJ!0sCZ+{W!gRh|v8v3AD zur&1HkqbmaA0Co4aJl9D3>(+1w@l8@^7AvH`^kMvMuSrBTLN@GZ-W5@nyF@^x?kM8 z#nMorhbR}hHg^8cM0bw|g4j9)rzqwutH>Kx&T4`Ch2#0#x5t+YNMpDjqG)e`;d;4G z-W!W@0`qklg@MHOQ^p~i7`d328pIgL0 zS?4R*AWTCHMLaDEaH@DJQ%<3FxSEO>kGWE|qEw_*k!`uh`#XC;SJkaeTtS}kY6I%1 zB3mn^w0%7 z$JOyzhB#L0T5YQA7H30MMhtS|36mg;eh!8vCtEDhA!5mQgj}eV7KOoSAPEse#Kq3r z-GhS$n}eVf>@RJODLr8%AQ=6IE{$3;qaXRKo{b4;1D>c7NcflJOY;rSs}0XPpjeysYKeM^7ous`tBqi=Ue2y|$U&f1^JcSCc-dL$5*`6epV5T1 zqG>TX4XTc8egFNw`cn-l-+wRRc*teRXzy zCo}{NUZ|(aEt*gQr=)(wMTy9hB2Od}5plZAu4*9js)|e0Nk?ZyUKQ<|PV?DpjzF-y zATutPc)9|gbOo+AyH|ijz)CZklAU+taDV6RJ7V`4{O>(^tYF}l*xyz;pcFa4NMrb_ z5{mO!E65FWJ4}|$)*3fl>mcrLoo>$=#g37N^h|^2LTaB9LnR z<35QIIZy+mG*Q5VOItfZz{192mb`q0wGz#MTsF;lD{h-o{g&;!`!0qpqe9>9*T9KfWD_KQZa z-u|X!!1p zVix)!91Y*q@ZE=?HX6QrE%&L>6@{gB#_y+Q`fQlD5PdQxB{O` zS!|ktXSj}Sn33njM(ifB?S{TZ-5`8?_}l>TxmGZ8@BU~q2Sy|RvZKk&@C)H{k7~K! zkr(PfZ@zY%H(x!>dzpKnA-cZ=q6?Fi^M7QxT@|-fH_-?UVYp8uNh^~16zEIE(V(mQ zhr+>ZX38`?^r1*3rs!KSs-SmAG>ZI$+P3jogUwBJ z)=6OA3F55seL-x+eUU1pZaAXwUMZ)Szh?20#`N{{l=yoGl>b0k=ueC0wFvaUBf1Cq=VWq(vhsI zB3aEKts^ZPA}zN@$6YU!(#*~u5^hhM1RX1R>>|A?RuVcva2akdO&n@imX)TVOC2+D zEfxjL^%6!YqegmFti%pOBcw@6lQ8j(z&3)^q96m`wi%TCC}SnY*!4z%b?4`5Iz|xK zmz}Hm#c+Gi7$|ARN^rajU|P@dZXP?wyUlvbaLEz$E}X}I%V@s+hGXaNsI4aUfP(U) zTLw&app-h>{oS#k3&xGzH_kPB(IwDd_ z;a-REJSir0#&u`7-vOf*1<&!^8JPS>-~)h9s3r0+0R6i#quQ?oKca#Wb0p*4dU5?L z66y69*ZZO#5=x%Y1*4t(3Y zY^Oxv8EKH%Y0A9V@Iv3R9M5zTHw|2mdi2WGf9BILvP^2FjHP&ft{sGiV_P(}9m{bZ zzoMxI`^9$;;=Ckjy!_-HQJ)O@5%~|HKvla!TcT z=5oouP3ScVbMt@w-{887{(Aj()B5dEk>Gupl{$Wz34Q$01BpX|U|1%x`FTR#YN0#` zp{S#|@#-F!uYpyeTwm;a&*A!Z=ZcoY{q~Hdg~Hnq&d*AHHrRcOB+crNc%b2ge$s+? z2!ggegRpOm6P?Z_~1kd&!Iiy1H zn2~2iCO*w*o_CvVm)97r)j`<_yxng?D_JAFgsKGBEnCRNOT73grs}$%DJVi&;H_C) zfQLEi8k-B&+~k8-LP29rSqbj8Y!04-2lVjPsK&`9^8O&a_GgYeg&DSG$|qC2bczWI zSranjDB*SvTs0TyEGoe;mK2keR#Rw$ydI}w7qa7xp->g9YbiwlRi3stYr(7mk3yhX z&jUpksB5>18AeFw(}mtie51~C#lnHB7Y_7VtzN6uYqh`HwOYA2@Q~JOwaL|MwU1q- zSFF|I-6`f}JGB|}9M6Y;0X37z4qVIhB9;aQH7(bQg2B(P)%s?V7)HVj<{Q4}BvxP; z)W?ffBGYq|=rOL<0$Xi!r3s_=0dQ#RQIdUm6PpyDZTC5 zD*xbiE47EvqiA-m%Uo2}!FAVY^@8RV$Fc2#(hIp%GoNBOS}kuv$|;M;(ZS&#$U!dW z$;ec2Jl`(xvRPiwbV%sy8B0FT;qZ>0m1#%#=8kZlXh&E(!e4_UtY%&h$r08jS3AOw zU8Gkz!mjQ19N|^aa^(3@9Gb3Y*lBE-%yHb*irgRwBI;Y_m5wktzaiwbQ!BNZA36*i z;lPLt-?eRnrH|1OcFeKuk8H=id(g6PnwyU-bGNPF3pv8i=}CzPG!#3 znD>@0@QV`9WZJ2dk}Wza%je$VRwcW>`@2p9Pv`=te*c7Ji@s0aCG-i&+=JTzm3P4^ z;0osU5!gLeh#@&XKbx!s5{}uw{yw>INdDS)gWH`~_Kbpg2R~SW%?B$u^VPsblW}mn zBgzi zlDGs5pTKY-XbXnFJP35d>gx&aEiiMj4BaL2tof|cvbaxKOwP}W^8744Ka&ldsoh0d zAonJa6R=z`xz%>+{zZWs-eL{uy;m936<)fN)HqaVNjwa2_1?-{N)F_5NJlhnEKgyq z$e4g5RI`l0Y`!EJB`bDKBIbYF$hMgW;0j(Qub(fr*G43nXU6jyX_(+)6T>iM8I&}W ziQkfaA-%63it=(j(Mx;}QN#58&w7bZV_m-zwb4s_*Si0i8iwQ1#EH#_rm@3P!(ooe zeAi8_!1jV`1=X()g_R-ahqMByEwWw#eC#5Vw?~ zE!SYK8^tLNf;6Pw;EEMM>d~0li4P{oqH$o_%n8Bvcph~e$9|j_JX>SS8hOUuLtzcy z^S!~nA6r!Y!0L}f@*VA8m(o0~T+D*`2DgO3Qea@3vLRW!yxunEj)M8KR))s?q&c10ASyG A6#xJL literal 0 HcmV?d00001 From 85766a02d8a5262fd3147f697c94356d0309f4e8 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:34:22 +0200 Subject: [PATCH 068/259] added dependency to use hive on local machine --- dhp-workflows/dhp-propagation/pom.xml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml index 0e7ebe804..7d822b9d2 100644 --- a/dhp-workflows/dhp-propagation/pom.xml +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.6-SNAPSHOT + 1.1.7-SNAPSHOT 4.0.0 @@ -31,7 +31,11 @@ dhp-schemas ${project.version} - + + org.apache.spark + spark-hive_2.11 + test + From 6897c920a293caa5ba254d80f45458cb60797d42 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:35:26 +0200 Subject: [PATCH 069/259] classes in support of new implementation of country propagation --- .../eu/dnetlib/dhp/countrypropagation/CountrySbs.java | 2 +- .../dnetlib/dhp/countrypropagation/DatasourceCountry.java | 6 +++--- .../dnetlib/dhp/countrypropagation/ResultCountrySet.java | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java index accd278d7..32c893261 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java @@ -2,7 +2,7 @@ package eu.dnetlib.dhp.countrypropagation; import java.io.Serializable; -public class Country implements Serializable { +public class CountrySbs implements Serializable { private String classid; private String classname; diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java index 3cce8f4bb..0ac8b108e 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java @@ -4,7 +4,7 @@ import java.io.Serializable; public class DatasourceCountry implements Serializable { private String dataSourceId; - private Country country; + private CountrySbs country; public String getDataSourceId() { return dataSourceId; @@ -14,11 +14,11 @@ public class DatasourceCountry implements Serializable { this.dataSourceId = dataSourceId; } - public Country getCountry() { + public CountrySbs getCountry() { return country; } - public void setCountry(Country country) { + public void setCountry(CountrySbs country) { this.country = country; } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java index 4978d9182..69f7a59e9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java @@ -1,11 +1,11 @@ package eu.dnetlib.dhp.countrypropagation; import java.io.Serializable; -import java.util.Set; +import java.util.ArrayList; public class ResultCountrySet implements Serializable { private String resultId; - private Set countrySet; + private ArrayList countrySet; public String getResultId() { return resultId; @@ -15,11 +15,11 @@ public class ResultCountrySet implements Serializable { this.resultId = resultId; } - public Set getCountrySet() { + public ArrayList getCountrySet() { return countrySet; } - public void setCountrySet(Set countrySet) { + public void setCountrySet(ArrayList countrySet) { this.countrySet = countrySet; } } From a2d833d5dda36562c5ce32c41dfc390507ff1e30 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:36:03 +0200 Subject: [PATCH 070/259] step of data preparation before actual country propagation will take palce --- .../PrepareDatasourceCountryAssociation.java | 51 ++++++++++--------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index 887ddbc69..85d3e58c4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -5,6 +5,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; @@ -14,14 +15,14 @@ import org.apache.spark.sql.SaveMode; import java.util.Arrays; import java.util.List; +import java.util.Optional; -import static eu.dnetlib.dhp.PropagationConstant.createOutputDirs; -import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; /** * For the association of the country to the datasource * The association is computed only for datasource of specific type or having whitelisted ids @@ -30,48 +31,50 @@ import org.slf4j.LoggerFactory; */ public class PrepareDatasourceCountryAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepare_dc_assoc.json")); + .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); parser.parseArgument(args); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkCountryPropagationJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareDatasourceCountryAssociation(spark, + Arrays.asList(parser.get("whitelist").split(";")), + Arrays.asList(parser.get("allowedtypes").split(";")), + inputPath, + outputPath); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - prepareDatasourceCountryAssociation(spark, - Arrays.asList(parser.get("whitelist").split(";")), - Arrays.asList(parser.get("allowedtypes").split(";")), - inputPath, - outputPath); + }); } + + private static void prepareDatasourceCountryAssociation(SparkSession spark, List whitelist, List allowedtypes, @@ -97,7 +100,7 @@ public class PrepareDatasourceCountryAssociation { relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); - String query = "SELECT source ds, named_struct('classid', country.classid, 'classname', country.classname) country " + + String query = "SELECT source dataSourceId, named_struct('classid', country.classid, 'classname', country.classname) country " + "FROM ( SELECT id " + " FROM datasource " + " WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + @@ -115,9 +118,9 @@ public class PrepareDatasourceCountryAssociation { spark.sql(query) .as(Encoders.bean(DatasourceCountry.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath + "/prepared_datasource_country"); + .toJavaRDD() + .map(c -> OBJECT_MAPPER.writeValueAsString(c)) + .saveAsTextFile(outputPath, GzipCodec.class); } From aef9b3aa904221cbddc8871e4050f3006f7df1c0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:36:59 +0200 Subject: [PATCH 071/259] new parametric implementation of country propagation. Exploits information compute before and broadcasts it to each executor --- .../SparkCountryPropagationJob2.java | 106 ++++++++++++------ 1 file changed, 70 insertions(+), 36 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 967c940b5..b4a415bd7 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -4,6 +4,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -18,6 +21,9 @@ import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + public class SparkCountryPropagationJob2 { private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob2.class); @@ -35,30 +41,54 @@ public class SparkCountryPropagationJob2 { parser.parseArgument(args); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); - final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; - final String datasourcecountrypath = outputPath + "/prepared_datasource_country"; - final String resultClassName = parser.get("resultTableName"); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".")+1); + final String datasourcecountrypath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", datasourcecountrypath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + final Boolean writeUpdates = Optional + .ofNullable(parser.get("writeUpdate")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("writeUpdate: {}", writeUpdates); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); Class resultClazz = (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkCountryPropagationJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - final boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + //createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + removeOutputDir(spark, outputPath); + execPropagation(spark, datasourcecountrypath, inputPath, outputPath, resultClazz, resultType, + writeUpdates, saveGraph); + }); + } + + private static void execPropagation(SparkSession spark, String datasourcecountrypath, + String inputPath, String outputPath, Class resultClazz, String resultType, + boolean writeUpdates, boolean saveGraph){ final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); //Load parque file with preprocessed association datasource - country @@ -74,10 +104,9 @@ public class SparkCountryPropagationJob2 { } if(saveGraph){ - updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath + "/" + resultType); + updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); } - } private static void updateResultTable(SparkSession spark, Dataset potentialUpdates, @@ -96,33 +125,32 @@ public class SparkCountryPropagationJob2 { Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); Dataset new_table = result_pair - .joinWith(potential_update_pair, result_pair.col("_1").equalTo(potential_update_pair.col("_1")), "left") - .map((MapFunction, Tuple2>, R>) value -> { + .joinWith(potentialUpdates, result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), "left_outer") + .map((MapFunction, ResultCountrySet>, R>) value -> { R r = value._1()._2(); - Optional potentialNewCountries = Optional.ofNullable(value._2()).map(Tuple2::_2); + Optional potentialNewCountries = Optional.ofNullable(value._2()); if (potentialNewCountries.isPresent()) { HashSet countries = new HashSet<>(); for (Qualifier country : r.getCountry()) { countries.add(country.getClassid()); } - for (Country country : potentialNewCountries.get().getCountrySet()) { + for (CountrySbs country : potentialNewCountries.get().getCountrySet()) { if (!countries.contains(country.getClassid())) { - r.getCountry().add(getCountry(country.getClassid(),country.getClassname())); + r.getCountry().add(getCountry(country.getClassid(), country.getClassname())); } } } return r; - }, Encoders.bean(resultClazz)); - log.info("Saving graph table to path: {}", outputPath); + //log.info("number of saved recordsa: {}", new_table.count()); new_table - .toJSON() - .write() - .option("compression", "gzip") - .text(outputPath); + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath , GzipCodec.class); + } @@ -133,6 +161,7 @@ public class SparkCountryPropagationJob2 { Dataset result = readPathEntity(spark, inputPath, resultClazz); result.createOrReplaceTempView("result"); + //log.info("number of results: {}", result.count()); createCfHbforresult(spark); return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); } @@ -147,6 +176,7 @@ public class SparkCountryPropagationJob2 { "LATERAL VIEW EXPLODE(instance) i AS inst"; Dataset cfhb = spark.sql(query); cfhb.createOrReplaceTempView("cfhb"); + log.info("cfhb_number : {}", cfhb.count()); } @@ -155,19 +185,22 @@ public class SparkCountryPropagationJob2 { Dataset datasource_country = broadcast_datasourcecountryassoc.value(); datasource_country.createOrReplaceTempView("datasource_country"); + log.info("datasource_country number : {}",datasource_country.count()); - String query = "SELECT id, collect_set(country) country "+ + String query = "SELECT id resultId, collect_set(country) countrySet "+ "FROM ( SELECT id, country " + - "FROM rels " + + "FROM datasource_country " + "JOIN cfhb " + - " ON cf = ds " + + " ON cf = dataSourceId " + "UNION ALL " + "SELECT id , country " + - "FROM rels " + + "FROM datasource_country " + "JOIN cfhb " + - " ON hb = ds ) tmp " + + " ON hb = dataSourceId ) tmp " + "GROUP BY id"; - return spark.sql(query); + Dataset potentialUpdates = spark.sql(query); + log.info("potential update number : {}", potentialUpdates.count()); + return potentialUpdates; } private static Dataset readPathEntity(SparkSession spark, String inputEntityPath, Class resultClazz) { @@ -180,14 +213,15 @@ public class SparkCountryPropagationJob2 { } private static Dataset readAssocDatasourceCountry(SparkSession spark, String relationPath) { - return spark.read() - .load(relationPath) - .as(Encoders.bean(DatasourceCountry.class)); + return spark + .read() + .textFile(relationPath) + .map(value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), Encoders.bean(DatasourceCountry.class)); } private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) - .saveAsTextFile(outputPath); + .saveAsTextFile(outputPath, GzipCodec.class); } From 1251ad4455a500cd3ea40fe90ce8a1545ff90f34 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:38:13 +0200 Subject: [PATCH 072/259] removed unuseful class --- .../eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java deleted file mode 100644 index 4fb31bf94..000000000 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/PrepareDataForTest.java +++ /dev/null @@ -1,4 +0,0 @@ -package eu.dnetlib.dhp.countrypropagation; - -public class PrepareDataForTest { -} From a562080b0b92e6e30d3410024be136aa9906c142 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:39:17 +0200 Subject: [PATCH 073/259] parameters to be used in the prepared Job and in the actual country propagation job --- .../input_countrypropagation_parameters.json | 14 ++++---- .../input_prepareassoc_parameters.json | 32 +------------------ 2 files changed, 8 insertions(+), 38 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 829c96420..8884c74c2 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -20,7 +14,7 @@ { "paramName":"wu", "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramDescription": "true if the update must be written. No double check if information is already present", "paramRequired": true }, { @@ -46,5 +40,11 @@ "paramLongName": "preparedInfoPath", "paramDescription": "the path where prepared info have been stored", "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json index 5d66e4edb..208c0ac44 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -17,35 +11,11 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be written. No double check if information is already present", - "paramRequired": false - }, - { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": false - }, - { - "paramName":"tn", - "paramLongName":"resultTableName", - "paramDescription": "the name of the result table we are currently working on", - "paramRequired": false - }, { "paramName": "out", "paramLongName": "outputPath", "paramDescription": "the path used to store temporary output files", - "paramRequired": false - }, - { - "paramName": "p", - "paramLongName": "preparedInfoPath", - "paramDescription": "the path where prepared info have been stored", - "paramRequired": false + "paramRequired": true }, { "paramName": "w", From 87f802821e217c5d59a1979ffbac68f9e01cf5c7 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:40:22 +0200 Subject: [PATCH 074/259] new workflow for country propagation: it is composed of the preparation step and in the propagation. The propagation part runs in parallel on the result types --- .../countrypropagation/oozie_app/workflow.xml | 147 +++++++++--------- 1 file changed, 70 insertions(+), 77 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 38aa93335..79789e095 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -36,14 +36,6 @@ saveGraph writes new version of the graph after the propagation step - - outputPath - the path used to store temporary output files - - - preparedInfoPath - the path where prepared info have been stored - @@ -59,17 +51,16 @@ cluster PrepareDatasourceCountryAssociation eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation - dhp-graph-propagation-${projectVersion}.jar + dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} - --driver-memory=${sparkDriverMemoryForJoining} + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - -mt yarn-cluster --sourcePath${sourcePath} --whitelist${whitelist} --allowedtypes${allowedtypes} @@ -80,8 +71,8 @@ - - + + @@ -92,18 +83,19 @@ cluster countryPropagationForPublications eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 - dhp-graph-propagation-${projectVersion}.jar + dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} - --driver-memory=${sparkDriverMemoryForJoining} + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - -mt yarn-cluster - --sourcePath${sourcePath} + --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --writeUpdate${writeUpdate} --saveGraph${saveGraph} @@ -111,7 +103,7 @@ --outputPath${workingDir}/country_propagation/publication --preparedInfoPath${workingDir}/country_propagation/preparedInfo - + @@ -121,26 +113,27 @@ cluster countryPropagationForDataset eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 - dhp-graph-propagation-${projectVersion}.jar + dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} - --driver-memory=${sparkDriverMemoryForJoining} + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - -mt yarn-cluster - --sourcePath${sourcePath} + --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/country_propagation/publication + --outputPath${workingDir}/country_propagation/dataset --preparedInfoPath${workingDir}/country_propagation/preparedInfo - + @@ -150,26 +143,27 @@ cluster countryPropagationForORP eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 - dhp-graph-propagation-${projectVersion}.jar + dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} - --driver-memory=${sparkDriverMemoryForJoining} + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - -mt yarn-cluster - --sourcePath${sourcePath} + --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --writeUpdate${writeUpdate} --saveGraph${saveGraph} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Otherresearchproduct - --outputPath${workingDir}/country_propagation/publication + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/country_propagation/otherresearchproduct --preparedInfoPath${workingDir}/country_propagation/preparedInfo - + @@ -179,15 +173,47 @@ cluster countryPropagationForSoftware eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 - dhp-graph-propagation-${projectVersion}.jar + dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCoresForJoining} - --executor-memory=${sparkExecutorMemoryForJoining} - --driver-memory=${sparkDriverMemoryForJoining} + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/country_propagation/software + --preparedInfoPath${workingDir}/country_propagation/preparedInfo + + + + + + + + yarn + cluster + countryPropagationForSoftware + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} -mt yarn-cluster --sourcePath${sourcePath} @@ -198,42 +224,9 @@ --outputPath${workingDir}/country_propagation/publication --preparedInfoPath${workingDir}/country_propagation/preparedInfo - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - CountryPropagation - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-propagation-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - -mt yarn-cluster - --sourcePath${sourcePath} - --whitelist${whitelist} - --allowedtypes${allowedtypes} - --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} - --saveGraph${saveGraph} - - + \ No newline at end of file From 8f12292daaa221e58e725da81381ac07d92ccd39 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 11 Apr 2020 16:47:34 +0200 Subject: [PATCH 075/259] changed the way to save the results on filesystem --- .../SparkCountryPropagationJob2.java | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index b4a415bd7..dcdefaf10 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -100,7 +100,7 @@ public class SparkCountryPropagationJob2 { broadcast_datasourcecountryassoc).as(Encoders.bean(ResultCountrySet.class)); if(writeUpdates){ - writeUpdates(potentialUpdates.toJavaRDD(), outputPath + "/update_" + resultType); + writeUpdates(potentialUpdates, outputPath + "/update_" + resultType); } if(saveGraph){ @@ -147,9 +147,13 @@ public class SparkCountryPropagationJob2 { log.info("Saving graph table to path: {}", outputPath); //log.info("number of saved recordsa: {}", new_table.count()); new_table - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath , GzipCodec.class); + .toJSON() + .write() + .option("compression", "gzip") + .text(outputPath); +// .toJavaRDD() +// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) +// .saveAsTextFile(outputPath , GzipCodec.class); } @@ -219,9 +223,14 @@ public class SparkCountryPropagationJob2 { .map(value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), Encoders.bean(DatasourceCountry.class)); } - private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ - potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) - .saveAsTextFile(outputPath, GzipCodec.class); + private static void writeUpdates(Dataset potentialUpdates, String outputPath){ + potentialUpdates + .toJSON() + .write() + .option("compression", "gzip") + .text(outputPath); +// map(u -> OBJECT_MAPPER.writeValueAsString(u)) +// .saveAsTextFile(outputPath, GzipCodec.class); } From 4b01dc60e665d9cf596bbf890ab0fd1b943b898c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 15:28:00 +0200 Subject: [PATCH 076/259] test unit for result to project propagation --- .../ProjectPropagationJobTest.java | 194 ++++++++++++++++++ .../sample/relation/relation_100.json.gz | Bin 0 -> 3639 bytes 2 files changed, 194 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java new file mode 100644 index 000000000..54c7721a0 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -0,0 +1,194 @@ +package eu.dnetlib.dhp.projecttoresult; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.countrypropagation.CountryPropagationJobTest; +import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2; +import eu.dnetlib.dhp.schema.oaf.Country; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Software; +import org.apache.commons.io.FileUtils; +import org.apache.neethi.Assertion; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; + +public class ProjectPropagationJobTest { + + private static final Logger log = LoggerFactory.getLogger(ProjectPropagationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = ProjectPropagationJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(ProjectPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(ProjectPropagationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + + spark = SparkSession + .builder() + .appName(ProjectPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + /** + * There are no new relations to be added. All the possible relations have already been linked with the project in the graph + * @throws Exception + */ + @Test + public void NoUpdateTest() throws Exception { + + SparkResultToProjectThroughSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), + "-hive_metastore_uris", "", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(0, tmp.count()); + + + } + + /** + * All the possible updates will produce a new relation. No relations are already linked in the grpha + * @throws Exception + */ + @Test + public void UpdateTenTest() throws Exception{ + SparkResultToProjectThroughSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), + "-hive_metastore_uris", "", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + //got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(20, tmp.count()); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + + Assertions.assertEquals(10, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(10, verificationDs.filter("relClass = 'isProducedBy'").count()); + + Assertions.assertEquals(10, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("50") && + r.getTarget().substring(0,2).equals("40") && + r.getRelClass().equals("isProducedBy")).count()); + Assertions.assertEquals(10, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("40") && + r.getTarget().substring(0,2).equals("50") && + r.getRelClass().equals("produces")).count()); + + verificationDs.createOrReplaceTempView("temporary"); + + Assertions.assertEquals(20, spark.sql("Select * from temporary where datainfo.inferenceprovenance = 'propagation'").count()); + } + + /** + * One of the relations in the possible updates is already linked to the project in the graph. + * All the others are not. There will be 9 new associations leading to 18 new relations + * @throws Exception + */ + @Test + public void UpdateMixTest() throws Exception{ + SparkResultToProjectThroughSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), + "-hive_metastore_uris", "", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + +// JavaRDD tmp = sc.textFile("/tmp/relation") +// .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + + //got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(18, tmp.count()); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + + Assertions.assertEquals(9, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(9, verificationDs.filter("relClass = 'isProducedBy'").count()); + + Assertions.assertEquals(9, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("50") && + r.getTarget().substring(0,2).equals("40") && + r.getRelClass().equals("isProducedBy")).count()); + Assertions.assertEquals(9, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("40") && + r.getTarget().substring(0,2).equals("50") && + r.getRelClass().equals("produces")).count()); + + verificationDs.createOrReplaceTempView("temporary"); + + Assertions.assertEquals(18, spark.sql("Select * from temporary where datainfo.inferenceprovenance = 'propagation'").count()); + + } +} diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..82a685ad26e1be86032ef82871fdff3058039741 GIT binary patch literal 3639 zcmX|@cQhON8^(i+qDJizMN3fxwaTs0XpOjbsjZh=yHsM-XvC^oHCiKTt5vP7EwPGF z5?jQMB8k08t@>&2J-DpY^S#Y3pZoM_ZWd z(=|BXypplZ9V^4h>SD{i{SEDu`{Un_h=s$iT2VFA@fb%?r8m5F%3-I$!H(bF=U@#U zzv5D~@mV0tVa-UOt??vr?pw^srgGJ1R6K@>QgyIkk0Da3ls{{(G0)EKkq!JX$7~+1RXIBD>k?YvID*=&eaqO!D~Hym zFZ+o3cG_)`u_{^R;4?=bhgre$HuUr~USN1jMRTMdcRn&(v6)24$~zCJA{HE@!YJ`` zhtou^!?#tR&Kpk)-A_KF@VCVKJSfg4PJwt)C}|hP7KM;ZpJz0WcMBOoIUiNKOe=5&~Wt) zL3G5*(iS~Vv0RY48PViXpahtPiMQGuA14=WR3MHXqv2GA5<6E$O=$o$l!5viV=noEfk>*=oRGz zK-86x3bW?R?$^Iw$fO%VQ(rnzzE__jO!4OGO6#t?QT`&lMNwKt(Z8W7z9Hq2Bm&W)9E0JBJf7i?d5{KmC7OdY4PHBX};ymK^#K}Yp726SdK;*MI_YU(@`D& zaYP4xyvll}aaQ{9^K`}R#w@+!6L^p^Yg8Nk=<)4Sb>0I%ReKGuef(%bKPtMJO@fM6 zZj*tU{<=QMqeOVbobIS=c}shdEmZ0cukK7n-OFj_i7mxCPehofL&GLoqMLJL?&SR% zqSRss{n4ZlkA~i6S77ds?joT^GcW{m?pvO~ti<+ipc*W9_JFxRraPJ`pMeyS9G5Bj z70z2dQp;$BATvFcIA;D~9SiRv8ikT$eZ-ERpLBkfoeQ&fFq^*_Ckg)WD%uce#W`Zd zk^aEa^=B~42+Gk9WV&yuz#U^DdFYK&wzpM9j@~kkm>j|q5UEMdGm+1YoQ#nZ5Yg$M z_k0pt3`O5@>y9^T`$;N*U#inf^oNgeAL0>|KGPI)Ml@GaDo%+(3%a8i$kP(?uoc|wF5)Cht0ULnO*qowF~HmpLu#M~0)E~BzhJwk>$$yN~Wt#8d#O$HKjehsXK~HXluw!3b zn?$6BR0yyhsK%;4cIx!+Z%K;MXx&n~H9b5P{!2`j4%LX3ZJB2kTv68(-Qg0zI5NJ?0wq(eMbYr+dmO&W%L ztaIq2yk-Md0|#aJiqroH&iZOL$@%Of2|iGt-8fVb)P4aOnTq~D8>G5k#L!-r_YJsjfVB`2ans%0+L8{-1P9enx z($c@t=TTdlLDM|Lf952KxZ7*i{gH9&q1U=PmCGTi`hGflS~IEk(^il4X4n)UWgc`pm6*lObo@q7xD@`HHlU{Cr!|@)!120%8FDUQ<8xW^VSV)* zeO?34y0BcZF~2$sz_z?Y)hJ{&TTOE@j@3irkwq|>kQ$3nN=*BRtm5vpIn7pByAaC_ z?TQ$;+d+af4Z3r%g*s0`k6=z6N_wKwe-vxV&$V48oORnE__?E!lW{8LrUrLkB&erk zm5%1gw-+RYv=4qy2+{gY;tSn`p-}XLmV&`y3l2M zzzWHc8O5(fz6u*9IqN3S)Uzb<(wi7(j1n(iCbb}ZQGRZ$7TE1_&6gp_%Z8=H@-rU| z#Y!Bt)@HBq6+&7MLPPEuDGoXF44a*EXijrm(FwQ>d)9SZ_+Ll&ROGw|W9kAG^0|=f z&=arO-jB2$77-k%=*q)=VGWuTlY*!0x|ixXU6xdtG@^6A=qLczxB2`3wR9P{Wz^6I zHCL;2@hhGWI0kcz_5#8lQjG1mB?Qpp;F6&?s(WUw-A0iQh4!XG7deAT;Ftw6pt^Gh zhBzEk_0Q$jw2e1o9wLtfZ`mnPG<(C#ctuss{+_#SDRrgs9-&bD8t#=+w{*C>1K$%y zb>Gt<;KcVIs#?E>WNf?R3nGr^M_4AAroPzhBwc)eQed_c|?7-dH5rKF}L+8^Mk1ln>4%g ze3i3}iRScK)U^d-H4i2JW!ur_k&t-1Ec-l!_9XHY*!rpyF%!FWXD4y1rn^$_nv*Ac zGM_efRZ0))puUyFyQPvo)d!Va;HOg1@=DLCw>s7wx;$Q~qheg}>=#Z1zU{i&o+;md zaT?6fw-;s6u=e;m7Aq^pnFO>Ols~K*Yf!oJ<~`=0<#efF9ci2$sDMK1Gm*l!Se5ka z4$5CGZAg_B(p9%N%E%LQDVe!%{*v1U^ipix1=lkV?t4@CR=Kplr}D8->2JANxV!x|#dOf$?!iFgv-Ba$+V!9m zl=xRKw;R4%y=U7Lt-RM0<~Nk@9AnrT7Ul4T9Z<7~U>fO6K^Mgd!j4mw@1P4*GU1BJ ziGi?@Wk}#h4QFmU)auCkJHSAWT(T zkEHmwlcV%*OVQ5Y%8wGs zu}EQz()XG|;+!Uj_V9;!`xKj7RgMh0jHvl|R*wkv3NydSYmtM@5dHrzznII_--Au+ zAEkZG$KbKf){yoXA1Yenk(JRGcMxQR&$ec2UibD$KZ9EJQkM5PHtGb5;UG@ zK(QCtJ;nwBe`_m@0Yy*jrgBDP0QslkYgd`+IfO1cQTRtfQYhiBm?qx%H zaiyBg{IC za>DVe3UMWu6(-e)2@*EEs6#yfJ*zj9PE>(sbW6YD(3Vi0j`kx#_sDD!E>iZ1eDyt} zNnyO=UaI?E&CxF4W226OJv>*z*UnewUI6lW$6suw+3aF&{bX>(_3q`!J?~y?M&-dq taWxjm&MH`kc`@W6*68G-PkwopAJ(cI Date: Tue, 14 Apr 2020 15:28:34 +0200 Subject: [PATCH 077/259] the starting relation set for testing --- .../sample/relation/relation_100.json.gz | Bin 3639 -> 3639 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz index 82a685ad26e1be86032ef82871fdff3058039741..bd4abe26f7dd34d52484ecf2319e3c251f95e1c3 100644 GIT binary patch delta 16 Xcmdlkvt5Q=zMF$%-TK~*>;`-QFBJty delta 16 Xcmdlkvt5Q=zMF%?FSTYPy8#~nDr5wW From f47ee5b78e7a9665773b356852c704235bbfe646 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 15:29:21 +0200 Subject: [PATCH 078/259] directory where to store the prepared infor before actual propagation will take place --- .../alreadyLinked/alreadyLinked_57.json.gz | Bin 0 -> 6170 bytes .../potentialUpdates/potentialUpdates_1.json.gz | Bin 0 -> 831 bytes .../potentialUpdates/potentialUpdates_1.json.gz | Bin 0 -> 357 bytes .../potentialUpdates/potentialUpdates_1.json.gz | Bin 0 -> 352 bytes 4 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked_57.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates_1.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates_1.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked_57.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked_57.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..0b687a7693d12894e3836db07b92832046bc0d6a GIT binary patch literal 6170 zcmV+#80F_5iwFoLa+O{H17U1(WnpA_OlfXwWn^D9H!f;(Z*BmMo6W9lw{eB%d5k;G zfd4GsH^?}{AP|a0GC|@*f+Qme?7VxvYW=|HiYL1bSdfjgySd4#wZ28N|MKOBAHV;% zZ9o6R`ttQl34hgp*YAG&&)2UZSZk`Mo@DuyF-kkJTGW-chLykk{g)s2Pk;W?&;Pi8 z{_^#ozSQv7@7J$Sw^F};ovo!Ur-iUuEpd#vC0fgE1UYBKFaP|%fA^P%G4Qu_oD^2c zsin3`*|y`Up|llsUdFKY`tuLp{nz;}g!uJq-evVVZHE!!Q3kVy4P|Ikjs<&X40H4x z(oS`T*7B^WPYo(v`S7sL|2GfcMR|RInblfnoQIc4b=YiUoH=XFw%!?oSx-&-DlJzV zeT5)Ww51Yrh;cv8L$a`TFQt`IcWb*Ro0?au*2S95H^z|8VYtlw4r)atX_m}v+HNcN z6duQ5Ve~UoDweu+H9H}n%Ibs?wAJv&7)qliH?KhfNZt%1Q)=5J4BI@%HA#H5pyl4oZH-WW~jE(V~Zt)l$fzG$0=Gg$I5~|kGqG;dgU@=>@`*y z(e~NG%LsO$1-!|e_KG};<@%w zOnvh&O;?w=&%#dT;q4}^fgHZmN4-hLjMh;9vKB=J0dY-`GMLTBbvUOT{*GB;dTgO3 zVg(&X%DkAzbxv~*GE%84)E+G)mT+p7L*LD{-qCLX{`EGo=8h#t4q;9v?J`QXI?cU&nF;L?S%ZqXhTb({lI6^w_%xUYd`KUC` zP@$XqfN)S|beXPw)>J(SN$E7eHs&OtbbMrT-qAF!<7Qe~#kgi>Dl1=Oh_KMX4Lexz z9X(^B5;PR&LJ)HlzG<|Xd$v84nSAymuRJ{mmI23@G;7k!5+wClJ@2sH=q(|@jQcDm z;qcNSj&v9h#kJdI3_s>RgRUTane*j1R7q?=L|i2v%e%$Rs`oxO_4qsng=^2;&>gei z$Ltx0uWJ`9ht4jK^MKNhl$ipm^sLmvRBES%Bx4q7@65wKyt=;hm#!L>DG3dg(73nc z`RvOjomgs&85YTOPRMaqx(DEsVJ|S_nV>|VN?dbiWVFC?Knatu z-kCeHNM;(+Y<-tF<4sR0{5oc( z(3$N7H}-Ox8L9-$B5Sv2Cg9D`upBM)7Yve-hR)I#0!}Pf@0B^0L25N*>TtijOso9zcLe0(}%yLTn9a4U;#sB>AZ)!0V zo@RAKb_}WvbCOoQ6|9nD<++C_f2=AVX;WcxK4o@M{F49sNV<(1k&bDsU_ml1FhZEC zEl&q<8hUo+CdhMGVd1d0NFHdj4>)p0qEQT2ZaOgfz&U%@6I}d^?8Kk%hWE65<-pTA zMrN>bIV6O_V-y(=N{$V|e`iiDAn4O0`>IP)#M}5}P6Uk>L|BREdA#@s5R45kFpPqOWY zcf|~nNv;3C%Dg!`Dkq|vt; zB2n{Vy**Q=x5E$?bt{i-U*EQg$ykl*uUYFS``7RPwEz2uKYjn>roPuWz5x*i>#FpJ z#H9y!!opjO*L+3=wM7)86vK%oR8xtu#|ORwdOEGMF~)U+bjEKegb`O%_v{~K3VG|-9`|GclCFc`KJHnZJ_zG(Ak8+5Dfng* z1r(d@lK8CaTz7Bv+~bzQ(oCU9N~I@?ul3$ny-2>dD@4;>BV0dC0@j@qhMkEc;!>qlPdaiR#ksYd}DN7>(m)K~G@+Qou0Et)qQ*wMaGG`h?j&BJT z1lBik6#KodkBj9&I0>7CtG-|vUh0)tQrLRk)0*WRq!DDQJK5Q3w{EN_VNp&=_c_Q6 z8v%s+x_}&1>dCnhwVf*Kl>rxtnQ;-Bl7fs44z1*kG~kB}eO&h)RxNn!4x4o)>8hu% zUSWlTedv9EBcdq8O7I;cu2Osp&9@J8&2!py4(JvNIFLw~rM3#34-(g^4SOPO-}eu- zGPQU-@crh`GBfcD>P~FjvCi`qYf3{ z84z4+>r9v`4jLXfV9E4a2Z8XFjt4nI{(-K5t@m0xCX7Pf_i>;cwE?S5aixKFuwB@m zMwEK*Q!UM?HSns-@^ZF?OoBQG0dD&`2N<%QI#&Ha`JoX>4{5un z%#z>?WvU8zt+DQVPGO!38XbH%t$-05k$mHSmkjWjaiNMl!v zz4*anz$K8k<9oSRJ;0s_0@+3llANfh>*zofL(}`3!pLbJ9mTgA2AU9ieukGa;FLEWzLC zL0%ukFy9`z*1>?rI@LJQoP2g%YXDnk`PnH$yyjvdrNF2wHoP76Tm)PCx}_F;PxgAA zz+^SVDf=b?&ep)qNq-WH>!`6^=K$bNLr^GGy@}|q)Henfm^{MYb$tejU5h{yp%+gi zM&o3}uG6P>eTD!*wU8%I%6YZDC51}krGNBF5>L+}1p7DV@p|**q zrf7a7-F*Wx^yNM+(0_g2Z$ZO%KKLaph!ql<+x8lc-qE>Vd+PvpkZP|I4Ru9d4{`6X zB9uYA_oI!06%}N7FXxPq5ScdU#T;1F>zw-DJ@7-KjAazj&Vj;ept~QD6JD~H|_t`!LE=1s+qMbM)r2@3l zi@ef1dmYrjPhISMMjLn`wNl8)zHy!WBc9f-q3H+>V%3qjN*C1SyCmrhotoPlW2p2| zD%d)W zk$;v41wNI69vIm3x$+@5w(OyV_0|$`xxPlTMshgoo&h>#D1~I&W2-8OB5Rq&rgU=3 zSGMrxI%KOfUfFo}0IIu;@6kxg_cJutkE#e;EU_98L#q@_T}Q6ze(r{-Lz0v%|H&xn zqLD{If2*Tszk2>4R*@5M;%d3u40B8+LB&Z}*WY^b_dR0X`IWsDNj1_2eT$I`d;(m8 z&vTab$$9f@f7CB|1oQ22D~No{37d$Y?BNUgi>x2Qd5oeB+QO9V^gIX!_kGj6g`}9< zl;IbBQaFB8&v%F+q#P%AQVj!+J00FTm?!={=@9tjbt?l90C|I4vGb)tf*q)&w z&t>=b*~<*Y%w_DIU6gc^h6(J;4@B-9XD{0EQx*>@09p21PZZC_ZO9>S^a(8QDfP5p zPst+jDH1(dD?;KMd+X+651tVF;KNAMPT%oc@XM?K_8Vi6nDc833xI>g3Tr{S&6D-a zBJ7+MmoL!0+T=WPXD~MR|82(A!KJSOwLdL~W zn~a{?LJP=WEiwR0t6u$ZcQ)(3jv+-ZR+BvsAQsvN6)ze z?Mm9cY9o-WEPo-vtP$U z=BJc+or7A!tbLGHP!PO&ln8x#1^Z>FdG)TT(W{%RDu5~9-3s5de1`^#$0@!nqPS-<~%cZ9Plz6#*tEML z!~LS6Ogt;C;`jOP8DXBb3oQm7;Jl5BAWGv@3aK&d`~G6~@adT@tWxOdNG2YHNdqy) zyz35qMnO{kJtHD%5L?75RZ8W>e&-QHM~@2$TtPy`Rg+^~_QTmZ?|a}tUl-DTI7^#) zr~2D3AtCN~=fxCCt|+}HrBUl^ACLS_M#c=z{oHi}%>*eL8WPVTkCJzqezZ-Fk$Bz9 zC3u7liutJg?oG+Q;pyY}fuVeoi4P zBu8v|=tRKbyu*I4^NIk>_;sEPs1sYt3PnjHL)Y!@;xzIQu{N1dNw%-11?eg{CM?QvD%8e;%cAWWB)J)=a&wP z?2!E(mYX=RGQtsPHr?*aeRDsEcZ~xB*bwZY8W9|_sy#toq@HF?_ZO#1RdvpkRVASNWvkr-}i9V^uA>vI|d<}B&Z5a0Bf1H_J=In zUiWgzUeHwaUvqSnpE-UVG#RUPKYMocq<)K~vtD~Sgg^1~uT#b%OuVrU%mTEWPND`Q zYUqaz47uTQy|4IuS0p2h)PM*%B+l5n?@tP$!b2Vz`;9T2X-MOIUEDx7(S^cm1+)(f z9Q!#J^5z6&@jJ!f$zCOEBIWauOk+#;vzg^~gp}#1t|PI90mxUxo^;bc$tgeD-;A5j znK^U{9?Qo=AX69k1<(0O@bldHMt=5Fwd$L#sU%{HVm5$^S=!gyf(cZqi;(KZHK!&z zQB#V-BJvLJjGJs!3I2JZ$(sN}N zaLuo?dDoH*lVcThH2oHv-!{DWUf+TANU~0uW>81Tjs-hl7~g&EiBx40lK>DuwuM#v zQUpm|lviz^+{gGZKh%Fl9xCSTk%JUkM~JB$LQzR?+yNT3lS^zeqf~JZs=&rBIC}4X z?DozWhz>m89oGGNAG}{XZ-VuWjg~xT;EFzFFRQJ_j4bz^nfwI8Z#LaCqa}uD4PRto zL0Du!$QT+;w}emDHF@_Aw97!tr4*(3rA0Ea=ioQqmCO(%o@9e2Vx=5ZQ(4*XdHIpy zvoHH^i2U+O@%xZfKwIeE@}v}i%4?kv|8$!vRav&7VI%d%im~ z7Dh~F6y}l91cExL!8}f8UG)>D`7nmI$oT;$)a2J%{I?L9i6we~c}t$xVmbq!9g`lA zwEC}VjMebp=xrwEoiV)ja~8|rG|~0D3t=eMI++##<9Q4KYTuj2Wf-C{)3?d*ndk}G zPQ2e0$LfM#70VvBQb2z(h331exKaLg-~5-m?=@~LYWr2`O^g;2exMunBGzsG9`=$K~*<`!R`dffvD0HKKHKe8K%(r$A@ zK+F8MU=7E)^Oitm%{3@F&8LCT9iWy{@?V5r=dGx;*SUw7{}g}%B2Vzul{m#JzO&xb zPyRcjN^SB_7D=E*;j>OC=lQpo<+y{lz2GJ5mi*Lf(!r6cKkJ^~JnyhZ>X%7NXT)9o z2MKEtZt>Mma?poy588TE3>)bI;OmNK;I73Z

}6f7uDl@|!~fZv)#+?RDZ>dG|m5 z*5LIU?l?~h9`Hw~KpsZOFbq3gK3nG&taAo={Qx{CXM5j-n!-DzPZ;~-xMk#B3uM@H!jz(7 svj!VvG-|wYM!DKpX_W-{&3gQX=&I3wA_-(v!UALXKT0hqc!NIx09_0wDgXcg literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..7dfd477ad4581f78a7e0bc7d65efc12dc2594e60 GIT binary patch literal 831 zcmV-F1Hk+riwFpjbCq5I18{G2Wo~q7VQf`!WMOn=b6+toYIARH0Bx1aj%GCsgx}>~ zvxqN=oqhw>ED>rsz6hZfJ?iNZNDS|ee<4=nY@^Dh>MNhW9$!A*e@p-TF&-}u9^dWv zkJtacmlrxYMae*9p-l<~+#w-*@7hI=A08k1_1oL$Px<+H`S~%&_sID6<@Jwx9Rpro zN+ErmfN_wxmdat}8c?+P=;;mqr$H1)j3gzrNDo^q&wi4(Fdc$ z@?;!x?53F--fI-w97!aHpDngUD|d|fujZ@&`u>)GetCQU$miYsnCg<8KvRzEuoS`p zRDyLqM;B1k;$VX|ajYR@Z16K>@vL#vuVKSO@j>k3h? zoBK>>htd8yaBfu4n`4}&!zyuPFXZ#*}9#F3?-pXkS7{9VI7g>h*%@A`I%r{xp8HOwhWa{;&czF5+ZgQSJ0W*$K5nb7w^glSjxVq z$0Bb&7p~zU*K?pBzE0ON1hJ>n+FMSuxv&d9VO`&o-JGr6v@%_)mXl`okQ!~o9@n`u z_I5T_cNxM`-LyocF0C}ze$~|GPElnS5}p2+aA$e;BJH*h;2Q^^Wj*1|<)8;=9)*|y zPdH*BT)U0WAuD;d_C)m#M`kZn_aV9rSKmQ=!rC}_@KLp0YyI8N`0I~0G1wpEe3O3^7Cq4e&uD8oe*!{t`eXH;}UvH;> zXWvPqNvByv7-u7|1&52v7$Zm0_G7!n+xz+c824>|+8TU%`1|$rktcw%@0Es50Km%5 z0Xo#Yw;bX!#+4I%W18G=nKN^$gp;#Mma%Gt946WbA! zT!~eYEu*m__SGJhEV@-`{V(%oKQHI_bv<8hvG2VGMUp}hjtbHXHZ%19aH?(1Dv%JdTBh!G~770m%u6jvrEtTAKhENF*Jfgorw!q5dGaB?O|*Pb+N z9(u!UgV|b1#gu{!>z;+7hyHPYXRJ%4=zT@(#rR{{V4 Di3+4$ literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates_1.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates_1.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..ab09f1f9b318583f34b86d62f863c2711f9f9c65 GIT binary patch literal 352 zcmV-m0iXUKiwFpFj+I^j18{G2Wo~q7VQf`!WMOn=b6+toYIARH0Bw=Ijub%*gx_Ui z0^8lTyXOsv2~TL{_Ky&(w5z>6fyD9d3_?VV)6!>`tA1|x$L(wS^J8rL*5Td0+2!Bt zJIDl{MG(D_kVNn4iN(+*(AM^0d*tie_4z5E+y1gKyhp~{{qnso0J!fhdnhkYn1E#^ z5YwkEku}P}j|KlRMCwG5Z8Wk-50hwzlC5DHo-(r*r(|flBLPPZBuDY=1iI{u;Sqjl zEv7j&)dXKtZR)6;0sNPF^)I(;{S@jLG&k(R0#wyw36HJ=Q+u< Date: Tue, 14 Apr 2020 15:31:26 +0200 Subject: [PATCH 079/259] code to compute the prepared information used in the actual propagation step. This step will produce who files: one with potential updates (association between projects and a list of results), the other already linked entities (association between projects and the list of results already linked to them) --- .../PrepareProjectResultsAssociation.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index 5daaceeea..39adf92a4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -9,8 +9,6 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -import org.apache.spark.TaskResultLost; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; import org.slf4j.Logger; @@ -18,20 +16,19 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; import java.util.List; -import java.util.Optional; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -public class PrepareResultProjectAssociation { +public class PrepareProjectResultsAssociation { private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception{ - String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepareprojecttoresult_parameters.json")); + String jsonConfiguration = IOUtils.toString(PrepareProjectResultsAssociation.class + .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); @@ -58,17 +55,18 @@ public class PrepareResultProjectAssociation { runWithSparkHiveSession(conf, isSparkSessionManaged, spark -> { - createOutputDirs(potentialUpdatePath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - prepareResultProjAssoc(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); + removeOutputDir(spark, potentialUpdatePath); + removeOutputDir(spark, alreadyLinkedPath); + prepareResultProjProjectResults(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); }); } - private static void prepareResultProjAssoc(SparkSession spark, String inputPath, String potentialUpdatePath, - String alreadyLinkedPath, List allowedsemrel) { + private static void prepareResultProjProjectResults(SparkSession spark, String inputPath, String potentialUpdatePath, + String alreadyLinkedPath, List allowedsemrel) { JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + Dataset relation = spark.createDataset(sc.textFile(inputPath ) + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); relation.createOrReplaceTempView("relation"); @@ -80,9 +78,9 @@ public class PrepareResultProjectAssociation { Dataset resproj_relation = spark.sql(query); resproj_relation.createOrReplaceTempView("resproj_relation"); - query ="SELECT projectId, collect_set(r1target) resultSet " + + query ="SELECT projectId, collect_set(resId) resultSet " + "FROM (" + - " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + + " SELECT r1.target resId, r2.target projectId " + " FROM (SELECT source, target " + " FROM relation " + " WHERE datainfo.deletedbyinference = false " + @@ -90,7 +88,7 @@ public class PrepareResultProjectAssociation { " JOIN resproj_relation r2 " + " ON r1.source = r2.source " + " ) tmp " + - "GROUP BY proj "; + "GROUP BY projectId "; spark.sql(query).as(Encoders.bean(ProjectResultSet.class)) .toJavaRDD() @@ -98,7 +96,7 @@ public class PrepareResultProjectAssociation { .saveAsTextFile(potentialUpdatePath, GzipCodec.class); - query = "SELECT target, collect_set(source) result_list " + + query = "SELECT target projectId, collect_set(source) resultSet " + "FROM resproj_relation " + "GROUP BY target"; From e0038bde5baf1a3903b9065fa28cad8fad1e41ac Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 15:32:12 +0200 Subject: [PATCH 080/259] Support class to serialize/deserialize the association project, set of linked results --- .../dhp/projecttoresult/ProjectResultSet.java | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java index 39af1f59c..da1be3b69 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java @@ -1,4 +1,25 @@ package eu.dnetlib.dhp.projecttoresult; -public class ProjectResultSet { +import java.io.Serializable; +import java.util.ArrayList; + +public class ProjectResultSet implements Serializable { + private String projectId; + private ArrayList resultSet; + + public String getProjectId() { + return projectId; + } + + public void setProjectId(String projectId) { + this.projectId = projectId; + } + + public ArrayList getResultSet() { + return resultSet; + } + + public void setResultSet(ArrayList resultSet) { + this.resultSet = resultSet; + } } From ceb1f299bf299ea5e8374298e2c7b50c3931a686 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:45:12 +0200 Subject: [PATCH 081/259] minor changes --- .../PrepareProjectResultsAssociation.java | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index 39adf92a4..f94d67734 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -55,8 +55,8 @@ public class PrepareProjectResultsAssociation { runWithSparkHiveSession(conf, isSparkSessionManaged, spark -> { - removeOutputDir(spark, potentialUpdatePath); - removeOutputDir(spark, alreadyLinkedPath); +// removeOutputDir(spark, potentialUpdatePath); +// removeOutputDir(spark, alreadyLinkedPath); prepareResultProjProjectResults(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); }); @@ -91,9 +91,14 @@ public class PrepareProjectResultsAssociation { "GROUP BY projectId "; spark.sql(query).as(Encoders.bean(ProjectResultSet.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(potentialUpdatePath); +// .toJavaRDD() +// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) +// .saveAsTextFile(potentialUpdatePath, GzipCodec.class); query = "SELECT target projectId, collect_set(source) resultSet " + @@ -102,9 +107,14 @@ public class PrepareProjectResultsAssociation { spark.sql(query) .as(Encoders.bean(ProjectResultSet.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(alreadyLinkedPath); +// .toJavaRDD() +// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) +// .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); } From cadab9b81dd0cd1f6eee02d621e593e50a4f1f4f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:46:07 +0200 Subject: [PATCH 082/259] new implementation for result to project propagation. Use the prepared info in propagation --- ...SparkResultToProjectThroughSemRelJob3.java | 218 ++++++++++-------- 1 file changed, 120 insertions(+), 98 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index f85eb2bed..01c6e9089 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -1,143 +1,160 @@ package eu.dnetlib.dhp.projecttoresult; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.DatasourceCountry; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import eu.dnetlib.dhp.countrypropagation.ResultCountrySet; import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.arrow.flatbuf.Bool; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.FlatMapFunction2; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.omg.CORBA.OBJ_ADAPTER; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; public class SparkResultToProjectThroughSemRelJob3 { + + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(SparkResultToProjectThroughSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils.toString(SparkResultToProjectThroughSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); + jsonConfiguration); + parser.parseArgument(args); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + final String potentialUpdatePath = parser.get("potentialUpdatePath"); + log.info("potentialUpdatePath {}: ", potentialUpdatePath); + + final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); + + final Boolean writeUpdates = Boolean.valueOf(parser.get("writeUpdate")); + log.info("writeUpdate: {}", writeUpdates); + + final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); + log.info("saveGraph: {}", saveGraph); SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToProjectThroughSemRelJob3.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/projecttoresult"; - boolean writeUpdates = "true".equals(parser.get("writeUpdate")); - boolean saveGraph = "true".equals(parser.get("saveGraph")); + runWithSparkSession(conf, isSparkSessionManaged, + spark -> { + //createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + if(isTest(parser)) { + removeOutputDir(spark, outputPath); + } + execPropagation(spark, inputPath, outputPath, alreadyLinkedPath, potentialUpdatePath, writeUpdates, saveGraph); + }); + } - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + private static void execPropagation(SparkSession spark, String inputPath, String outputPath, String alreadyLinkedPath, String potentialUpdatePath, + Boolean writeUpdate, Boolean saveGraph){ - Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - relation.createOrReplaceTempView("relation"); + Dataset toaddrelations = readAssocProjectResults(spark, potentialUpdatePath); + Dataset alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath); - String query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false and relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; + if(writeUpdate){ + toaddrelations + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(outputPath +"/potential_updates"); + //writeUpdates(toaddrelations.toJavaRDD(), outputPath + "/potential_updates"); + } + if (saveGraph){ + getNewRelations(alreadyLinked, toaddrelations) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .text(outputPath); +// JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); +// sc.textFile(inputPath) +// .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) +// .union(getNewRelations(alreadyLinked, toaddrelations) +// .toJavaRDD()) +// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) +// .saveAsTextFile(outputPath , GzipCodec.class); + } - Dataset resproj_relation = spark.sql(query); - - query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false " + getConstraintList(" relClass = '", allowedsemrel ); + } - Dataset resres_relation = spark.sql(query); - resres_relation.createOrReplaceTempView("resres_relation"); - resproj_relation.createOrReplaceTempView("resproj_relation"); + private static Dataset getNewRelations(Dataset alreadyLinked, + Dataset toaddrelations){ - query ="SELECT proj, collect_set(r1target) result_set " + - "FROM (" + - " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + - " FROM resres_relation r1 " + - " JOIN resproj_relation r2 " + - " ON r1.source = r2.source " + - " ) tmp " + - "GROUP BY proj "; - Dataset toaddrelations = spark.sql(query); - - query = "select target, collect_set(source) result_list from " + - "resproj_relation " + - "group by target"; - - Dataset project_resultlist = spark.sql(query); - - //if (writeUpdaes){ - toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/toupdaterelations"); - //} - - if(saveGraph){ - JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - .leftOuterJoin(project_resultlist.toJavaRDD().mapToPair(pr -> new Tuple2<>(pr.getString(0), pr.getList(1)))) - .flatMap(c -> { - List toAddRel = new ArrayList<>(); - toAddRel.addAll(c._2()._1()); - if (c._2()._2().isPresent()) { - List originalRels = c._2()._2().get(); - for (Object o : originalRels) { - if (toAddRel.contains(o)) { - toAddRel.remove(o); + return toaddrelations + .joinWith(alreadyLinked, toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")), "left") + .flatMap(value -> { + List new_relations = new ArrayList<>(); + ProjectResultSet potential_update = value._1(); + ProjectResultSet already_linked = value._2(); + String projId = already_linked.getProjectId(); + potential_update + .getResultSet() + .stream() + .forEach(rId -> { + if (!already_linked.getResultSet().contains(rId)){ + new_relations.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + new_relations.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); } - } - } - List relationList = new ArrayList<>(); - String projId = c._1(); - for (Object r : toAddRel) { - String rId = (String) r; - relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + }); + return new_relations.iterator(); } - if(relationList.size()==0){ - return null; - } - return relationList.iterator(); - }).filter(r -> !(r==null)); + ,Encoders.bean(Relation.class)); - new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/new_relations" ); - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .union(new_relations) - .map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/relation"); } - + private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ + potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) + .saveAsTextFile(outputPath, GzipCodec.class); + } //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); // JavaPairRDD result_project = relations @@ -211,9 +228,14 @@ public class SparkResultToProjectThroughSemRelJob3 { // newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) // .saveAsTextFile(outputPath + "/relation"); + //} + + + private static Dataset readAssocProjectResults(SparkSession spark, String potentialUpdatePath) { + return spark + .read() + .textFile(potentialUpdatePath) + .map(value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class), Encoders.bean(ProjectResultSet.class)); } - - - } From 92f19fa0a097e965ad4689baf91f2526caeb4be3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:46:57 +0200 Subject: [PATCH 083/259] parameters for the project2result preparation phase --- ...nput_prepareprojecttoresult_parameters.json | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json index 9c790f219..a70dbd6a0 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -25,15 +19,15 @@ "paramRequired": true }, { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramName":"pu", + "paramLongName":"potentialUpdatePath", + "paramDescription": "the path of the potential updates ", "paramRequired": true }, { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", + "paramName":"al", + "paramLongName":"alreadyLinkedPath", + "paramDescription": "the path of the already linked project result_set", "paramRequired": true } ] \ No newline at end of file From 61d39e659ebc8a92a1e90a370264a8dc7307742e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:47:39 +0200 Subject: [PATCH 084/259] parameters for the project2result propagation phase --- .../input_projecttoresult_parameters.json | 55 +++++++++++++++---- 1 file changed, 45 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json index 695dc176c..9d5354c64 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json @@ -1,21 +1,56 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"asr", - "paramLongName":"allowedsemrels", - "paramDescription": "the types of the allowed datasources. Split by ;", + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + }, + { + "paramName":"pu", + "paramLongName":"potentialUpdatePath", + "paramDescription": "the path of the potential updates ", + "paramRequired": true + }, + { + "paramName":"al", + "paramLongName":"alreadyLinkedPath", + "paramDescription": "the path of the already linked project result_set", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "test", + "paramLongName": "isTest", + "paramDescription": "true if it is a test running", + "paramRequired": false } ] \ No newline at end of file From ca2b40952e8942a16fdc7e2edf19b133d18a94ad Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:48:02 +0200 Subject: [PATCH 085/259] minor changes --- .../oozie_app/config-default.xml | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml index 2e0ed9aee..caf3c6050 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml @@ -2,9 +2,11 @@ jobTracker yarnRM + nameNode + hdfs://nameservice1 @@ -15,4 +17,47 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + + + \ No newline at end of file From 3f4b579e7f4ef00906a31da85c596be41c8c037e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Apr 2020 16:49:24 +0200 Subject: [PATCH 086/259] new workflow. It is composed of four steps. The first removes the directory where to store the results. The second copies the relation to the new locatio, the third id the preparation phase and then the actual propagation --- .../projecttoresult/oozie_app/workflow.xml | 108 ++++++++++++++++-- 1 file changed, 100 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 4c073f0a2..1d15391ab 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -20,14 +20,96 @@ sparkExecutorCores number of cores used by single executor + + writeUpdate + writes the information found for the update. No double check done if the information is already present + + + saveGraph + writes new version of the graph after the propagation step + - - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/projecttoresult_propagation/relation + + + + + + + + yarn + cluster + PrepareProjectResultsAssociation + eu.dnetlib.dhp.projecttoresult.PrepareProjectResultsAssociation + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/relation + --allowedsemrels${allowedsemrels} + --hive_metastore_uris${hive_metastore_uris} + --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked + + + + + + + + yarn + cluster + ProjectToResultPropagation + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob3 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/relation + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --hive_metastore_uris${hive_metastore_uris} + --outputPath${workingDir}/projecttoresult_propagation/relation + --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked + + + + + ${jobTracker} @@ -35,17 +117,27 @@ yarn-cluster cluster ProjectToResultPropagation - eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob2 dhp-propagation-${projectVersion}.jar - --executor-memory ${sparkExecutorMemory} - --executor-cores ${sparkExecutorCores} + + --num-executors=${sparkExecutorNumber} + --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.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - -mt yarn-cluster --sourcePath${sourcePath} --allowedsemrels${allowedsemrels} + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked From 27f1d3ee8f78e53e9bcd02ce37dad5cb814678e1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:21:05 +0200 Subject: [PATCH 087/259] minor refactoring --- .../SparkCountryPropagationJob2.java | 33 +- .../DatasourceOrganization.java | 4 + .../PrepareResultInstRepoAssociation.java | 4 + .../ResultOrganizationSet.java | 4 + ...rkResultToOrganizationFromIstRepoJob2.java | 449 ++++++++++++++++++ .../input_prepareresultorg_parameters.json | 44 ++ .../Result2OrganizationJobTest.java | 4 + 7 files changed, 523 insertions(+), 19 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index dcdefaf10..029be645d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -114,6 +114,7 @@ public class SparkCountryPropagationJob2 { Class resultClazz, String outputPath) { + log.info("Reading Graph table from: {}", inputPath); Dataset result = readPathEntity(spark, inputPath, resultClazz); Dataset> result_pair = result @@ -171,17 +172,17 @@ public class SparkCountryPropagationJob2 { } - private static void createCfHbforresult(SparkSession spark) { - String query; - query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + - "FROM ( SELECT id, instance " + - "FROM result " + - " WHERE datainfo.deletedbyinference = false) ds " + - "LATERAL VIEW EXPLODE(instance) i AS inst"; - Dataset cfhb = spark.sql(query); - cfhb.createOrReplaceTempView("cfhb"); - log.info("cfhb_number : {}", cfhb.count()); - } +// private static void createCfHbforresult(SparkSession spark) { +// String query; +// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + +// "FROM ( SELECT id, instance " + +// "FROM result " + +// " WHERE datainfo.deletedbyinference = false) ds " + +// "LATERAL VIEW EXPLODE(instance) i AS inst"; +// Dataset cfhb = spark.sql(query); +// cfhb.createOrReplaceTempView("cfhb"); +// //log.info("cfhb_number : {}", cfhb.count()); +// } private static Dataset countryPropagationAssoc(SparkSession spark, @@ -203,18 +204,11 @@ public class SparkCountryPropagationJob2 { " ON hb = dataSourceId ) tmp " + "GROUP BY id"; Dataset potentialUpdates = spark.sql(query); - log.info("potential update number : {}", potentialUpdates.count()); + //log.info("potential update number : {}", potentialUpdates.count()); return potentialUpdates; } - private static Dataset readPathEntity(SparkSession spark, String inputEntityPath, Class resultClazz) { - log.info("Reading Graph table from: {}", inputEntityPath); - return spark - .read() - .textFile(inputEntityPath) - .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), Encoders.bean(resultClazz)); - } private static Dataset readAssocDatasourceCountry(SparkSession spark, String relationPath) { return spark @@ -227,6 +221,7 @@ public class SparkCountryPropagationJob2 { potentialUpdates .toJSON() .write() + .mode(SaveMode.Overwrite) .option("compression", "gzip") .text(outputPath); // map(u -> OBJECT_MAPPER.writeValueAsString(u)) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java new file mode 100644 index 000000000..859d53f76 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +public class DatasourceOrganization { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java new file mode 100644 index 000000000..5a5678e10 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +public class PrepareResultInstRepoAssociation { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java new file mode 100644 index 000000000..9c51ceb02 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +public class ResultOrganizationSet { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java new file mode 100644 index 000000000..cdae25e85 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -0,0 +1,449 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.hadoop.io.Text; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.File; +import java.util.*; + +import eu.dnetlib.dhp.schema.oaf.*; +import scala.Tuple2; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToOrganizationFromIstRepoJob { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); + + String query = "SELECT source ds, target org " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + + "AND datainfo.deletedbyinference = false ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source "; + + org.apache.spark.sql.Dataset rels = spark.sql(query); + rels.createOrReplaceTempView("rels"); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); + + org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); + + + software.createOrReplaceTempView("software"); + final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); + + dataset.createOrReplaceTempView("dataset"); + final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); + + other.createOrReplaceTempView("other"); + final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); + + publication.createOrReplaceTempView("publication"); + final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); + + writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + + + query = "Select source resultId, collect_set(target) org_list " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + + "group by source"; + + JavaRDD result_orglist = spark.sql(query).toJavaRDD(); + + JavaPairRDD> toupdateunion = toupdateresultdataset.mapToPair(d -> new Tuple2<>(d.getString(0), d.getList(1))) + .union(toupdateresultother.mapToPair(o -> new Tuple2<>(o.getString(0), o.getList(1)))) + .union(toupdateresultpublication.mapToPair(p -> new Tuple2<>(p.getString(0), p.getList(1)))) + .union(toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))); + + JavaRDD new_rels = getNewRels(result_orglist.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))), + toupdateunion); + + + + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .union(new_rels) + .map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/relation"); + + + } + + private static JavaRDD getNewRels(JavaPairRDD> relationOrgs, JavaPairRDD > newRels){ + return newRels + .leftOuterJoin(relationOrgs) + .flatMap(c -> { + List toAddOrgs = new ArrayList<>(); + toAddOrgs.addAll(c._2()._1()); + if (c._2()._2().isPresent()) { + Set originalOrgs = new HashSet<>(); + originalOrgs.addAll(c._2()._2().get()); + for (Object oId : originalOrgs) { + if (toAddOrgs.contains(oId)) { + toAddOrgs.remove(oId); + } + } + } + List relationList = new ArrayList<>(); + String resId = c._1(); + for (Object org : toAddOrgs) { + relationList.add(getRelation((String)org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList.add(getRelation(resId, (String)org, RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + + } + return relationList.iterator(); + }); + + } + + + private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { + createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); + createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); + createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); + createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); + + } + + private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { + toupdaterelation.flatMap(s -> { + List relationList = new ArrayList<>(); + List orgs = s.getList(1); + String resId = s.getString(0); + for (String org : orgs) { + relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + + } + return relationList.iterator(); + }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); + } + + private static JavaRDD propagateOnResult(SparkSession spark, String table) { + String query; + query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM " + table + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + org.apache.spark.sql.Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + + return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); + + } + + private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ + String query = "SELECT id, collect_set(org) org "+ + "FROM ( SELECT id, org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON cf = ds " + + "UNION ALL " + + "SELECT id , org " + + "FROM rels " + + "JOIN " + cfhbTable + + " ON hb = ds ) tmp " + + "GROUP BY id"; + return spark.sql(query); + } + +} + +//package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +// +//import eu.dnetlib.dhp.TypedRow; +//import eu.dnetlib.dhp.application.ArgumentApplicationParser; +//import org.apache.commons.io.IOUtils; +//import org.apache.hadoop.fs.FileSystem; +//import org.apache.spark.SparkConf; +//import org.apache.spark.api.java.JavaPairRDD; +//import org.apache.spark.api.java.JavaRDD; +//import org.apache.spark.api.java.JavaSparkContext; +//import org.apache.spark.sql.Encoders; +//import org.apache.spark.sql.Row; +//import org.apache.spark.sql.SparkSession; +//import org.apache.hadoop.io.Text; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import java.io.File; +//import java.util.ArrayList; +//import java.util.Arrays; +//import java.util.List; +//import java.util.Set; +// +//import eu.dnetlib.dhp.schema.oaf.*; +//import scala.Tuple2; +// +//import static eu.dnetlib.dhp.PropagationConstant.*; +// +//public class SparkResultToOrganizationFromIstRepoJob { +// public static void main(String[] args) throws Exception { +// +// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); +// parser.parseArgument(args); +// SparkConf conf = new SparkConf(); +// conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); +// final SparkSession spark = SparkSession +// .builder() +// .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) +// .master(parser.get("master")) +// .config(conf) +// .enableHiveSupport() +// .getOrCreate(); +// +// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); +// final String inputPath = parser.get("sourcePath"); +// final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; +// +// createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); +// +// org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") +// .map(item -> new ObjectMapper().readValue(item, Datasource.class)) +// .rdd(), Encoders.bean(Datasource.class)); +// +// JavaRDD relation_rdd_all = sc.textFile(inputPath + "/relation") +// .map(item -> new ObjectMapper().readValue(item, Relation.class)); +// JavaRDD relation_rdd = relation_rdd_all.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); +// +// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); +// +// org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") +// .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); +// +// datasource.createOrReplaceTempView("datasource"); +// relation.createOrReplaceTempView("relation"); +// organization.createOrReplaceTempView("organization"); +// +// String query = "SELECT source ds, target org " + +// "FROM ( SELECT id " + +// "FROM datasource " + +// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + +// "AND datainfo.deletedbyinference = false ) d " + +// "JOIN ( SELECT source, target " + +// "FROM relation " + +// "WHERE relclass = 'provides' " + +// "AND datainfo.deletedbyinference = false ) rel " + +// "ON d.id = rel.source "; +// +// org.apache.spark.sql.Dataset rels = spark.sql(query); +// rels.createOrReplaceTempView("rels"); +// +// org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); +// +// org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); +// +// org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); +// +// org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); +// +// +// software.createOrReplaceTempView("software"); +// final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); +// +// dataset.createOrReplaceTempView("dataset"); +// final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); +// +// other.createOrReplaceTempView("other"); +// final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); +// +// publication.createOrReplaceTempView("publication"); +// final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); +// +// writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); +// +// JavaPairRDD relation_rdd_pair = relation_rdd +// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getSource()); +// tp.add(r.getTarget()); +// return tp; +// }).mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) { +// return b; +// } +// if (b == null) { +// return a; +// } +// +// a.addAll(b.getAccumulator()); +// return a; +// }).cache(); +// +// +// JavaRDD new_rels = getNewRels(relation_rdd_pair, +// toupdateresultother.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultdataset.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultpublication.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))); +// +// +// relation_rdd_all.union(new_rels).map(r -> new ObjectMapper().writeValueAsString(r)) +// .saveAsTextFile(outputPath + "/relation"); +// +// } +// +// private static JavaRDD getNewRels(JavaPairRDD relation_rdd_pair, JavaPairRDD > newRels){ +// return newRels//.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))) +// .leftOuterJoin(relation_rdd_pair) +// .flatMap(c -> { +// List toAddOrgs = c._2()._1(); +// if (c._2()._2().isPresent()) { +// Set originalOrgs = c._2()._2().get().getAccumulator(); +// for (String oId : toAddOrgs) { +// if (originalOrgs.contains(oId)) { +// toAddOrgs.remove(oId); +// } +// } +// } +// List relationList = new ArrayList<>(); +// String resId = c._1(); +// for (String org : toAddOrgs) { +// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// +// } +// return relationList.iterator(); +// }); +// +// } +// +// private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { +// createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); +// createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); +// createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); +// createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); +// +// } +// +// private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { +// toupdaterelation.flatMap(s -> { +// List relationList = new ArrayList<>(); +// List orgs = s.getList(1); +// String resId = s.getString(0); +// for (String org : orgs) { +// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// +// } +// return relationList.iterator(); +// }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); +// } +// +// private static JavaRDD propagateOnResult(SparkSession spark, String table) { +// String query; +// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + +// "FROM ( SELECT id, instance " + +// "FROM " + table + +// " WHERE datainfo.deletedbyinference = false) ds " + +// "LATERAL VIEW EXPLODE(instance) i AS inst"; +// org.apache.spark.sql.Dataset cfhb = spark.sql(query); +// cfhb.createOrReplaceTempView("cfhb"); +// +// return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); +// +// } +// +// private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ +// String query = "SELECT id, collect_set(org) org "+ +// "FROM ( SELECT id, org " + +// "FROM rels " + +// "JOIN " + cfhbTable + +// " ON cf = ds " + +// "UNION ALL " + +// "SELECT id , org " + +// "FROM rels " + +// "JOIN " + cfhbTable + +// " ON hb = ds ) tmp " + +// "GROUP BY id"; +// return spark.sql(query); +// } +// +//} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json new file mode 100644 index 000000000..8d2133075 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json @@ -0,0 +1,44 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + }, + { + "paramName":"dop", + "paramLongName":"datasourceOrganizationPath", + "paramDescription": "path where to store/find association from datasource and organization", + "paramRequired": true + }, + { + "paramName":"alp", + "paramLongName":"alreadyLinkedPath", + "paramDescription": "path where to store/find already linked results and organizations", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java new file mode 100644 index 000000000..bc2142314 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +public class Result2OrganizationJobTest { +} From 1859ce8902a9c948f4622c0aeef8d5ae2cb7f8c4 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:21:31 +0200 Subject: [PATCH 088/259] minor refactoring --- .../SparkResultToProjectThroughSemRelJob3.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index 01c6e9089..fb726e86c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -73,16 +73,15 @@ public class SparkResultToProjectThroughSemRelJob3 { runWithSparkSession(conf, isSparkSessionManaged, spark -> { - //createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); if(isTest(parser)) { removeOutputDir(spark, outputPath); } - execPropagation(spark, inputPath, outputPath, alreadyLinkedPath, potentialUpdatePath, writeUpdates, saveGraph); + execPropagation(spark, outputPath, alreadyLinkedPath, potentialUpdatePath, writeUpdates, saveGraph); }); } - private static void execPropagation(SparkSession spark, String inputPath, String outputPath, String alreadyLinkedPath, String potentialUpdatePath, + private static void execPropagation(SparkSession spark, String outputPath, String alreadyLinkedPath, String potentialUpdatePath, Boolean writeUpdate, Boolean saveGraph){ Dataset toaddrelations = readAssocProjectResults(spark, potentialUpdatePath); @@ -95,7 +94,6 @@ public class SparkResultToProjectThroughSemRelJob3 { .mode(SaveMode.Overwrite) .option("compression","gzip") .text(outputPath +"/potential_updates"); - //writeUpdates(toaddrelations.toJavaRDD(), outputPath + "/potential_updates"); } if (saveGraph){ getNewRelations(alreadyLinked, toaddrelations) @@ -104,13 +102,7 @@ public class SparkResultToProjectThroughSemRelJob3 { .mode(SaveMode.Append) .option("compression", "gzip") .text(outputPath); -// JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); -// sc.textFile(inputPath) -// .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) -// .union(getNewRelations(alreadyLinked, toaddrelations) -// .toJavaRDD()) -// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) -// .saveAsTextFile(outputPath , GzipCodec.class); + } } From 62b09be43ce5eeeb0a5f0d3582d5230a60a0dc26 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:23:06 +0200 Subject: [PATCH 089/259] added correct descritption for parameter isSparkSessionManaged --- .../countrypropagation/input_countrypropagation_parameters.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 8884c74c2..7545f30c4 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -44,7 +44,7 @@ { "paramName": "ssm", "paramLongName": "isSparkSessionManaged", - "paramDescription": "the path where prepared info have been stored", + "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false } ] \ No newline at end of file From 5a3487280df7520a21c6e6f888c6e302c7fbfea1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:24:36 +0200 Subject: [PATCH 090/259] classes to serialize/deserialize the prepared data --- .../DatasourceOrganization.java | 23 ++++++++++++++++++- .../ResultOrganizationSet.java | 23 ++++++++++++++++++- 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java index 859d53f76..e13e2a68d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java @@ -1,4 +1,25 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -public class DatasourceOrganization { +import java.io.Serializable; + +public class DatasourceOrganization implements Serializable { + + private String datasourceId; + private String organizationId; + + public String getDatasourceId() { + return datasourceId; + } + + public void setDatasourceId(String datasourceId) { + this.datasourceId = datasourceId; + } + + public String getOrganizationId() { + return organizationId; + } + + public void setOrganizationId(String organizationId) { + this.organizationId = organizationId; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java index 9c51ceb02..bad581c1d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java @@ -1,4 +1,25 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -public class ResultOrganizationSet { +import java.io.Serializable; +import java.util.ArrayList; + +public class ResultOrganizationSet implements Serializable { + private String resultId; + private ArrayList organizationSet; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public ArrayList getOrganizationSet() { + return organizationSet; + } + + public void setOrganizationSet(ArrayList organizationSet) { + this.organizationSet = organizationSet; + } } From 6a377a758274b45849ba58c32ad8075f7c2f78f0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:25:11 +0200 Subject: [PATCH 091/259] class to compute some information needed for the actual propagation --- .../PrepareResultInstRepoAssociation.java | 106 ++++++++++++++++++ 1 file changed, 106 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 5a5678e10..aa2861623 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -1,4 +1,110 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Datasource; +import eu.dnetlib.dhp.schema.oaf.Organization; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static eu.dnetlib.dhp.PropagationConstant.RELATION_RESULT_ORGANIZATION_REL_CLASS; +import static eu.dnetlib.dhp.PropagationConstant.isSparkSessionManaged; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + public class PrepareResultInstRepoAssociation { + + private static final Logger log = LoggerFactory.getLogger(PrepareResultInstRepoAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception{ + + String jsonConfiguration = IOUtils.toString(PrepareResultInstRepoAssociation.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String datasourceOrganizationPath = parser.get("datasourceOrganizationPath"); + log.info("datasourceOrganizationPath {}: ", datasourceOrganizationPath); + + final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + prepareAssociations(spark, inputPath, datasourceOrganizationPath, alreadyLinkedPath); + }); + } + + private static void prepareAssociations(SparkSession spark, String inputPath, String datasourceOrganizationPath, + String alreadyLinkedPath){ + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); + + String query = "SELECT source datasourceId, target organizationId " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + + "AND datainfo.deletedbyinference = false ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = 'provides' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source "; + + spark.sql(query) + .as(Encoders.bean(DatasourceOrganization.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(datasourceOrganizationPath); + + query = "Select source, collect_set(target) organizationSet " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + + "group by source"; + + spark.sql(query) + .as(Encoders.bean(ResultOrganizationSet.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(alreadyLinkedPath); + + + } + } From 473d17767c05b2a315756343b49bf41cfb0af25f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:25:44 +0200 Subject: [PATCH 092/259] new business logic for the actual propagation. It exploits previously computed information --- ...rkResultToOrganizationFromIstRepoJob2.java | 590 ++++++------------ 1 file changed, 189 insertions(+), 401 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index cdae25e85..7826f598b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -1,178 +1,202 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -import eu.dnetlib.dhp.TypedRow; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.DatasourceCountry; +import eu.dnetlib.dhp.countrypropagation.ResultCountrySet; +import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2; +import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.apache.hadoop.io.Text; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.File; -import java.util.*; - -import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; import scala.Tuple2; -import static eu.dnetlib.dhp.PropagationConstant.*; +import java.util.*; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class SparkResultToOrganizationFromIstRepoJob2 { + + private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); -public class SparkResultToOrganizationFromIstRepoJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); + String jsonConfiguration = IOUtils.toString(SparkCountryPropagationJob2.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String datasourceorganization = parser.get("datasourceOrganizationPath"); + log.info("datasourceOrganizationPath: {}", datasourceorganization); + + final String alreadylinked = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath: {}", alreadylinked); + + final String resultorganizationsetpath = parser.get("resultOrganizationsetPath"); + log.info("resultOrganizationsetPath: {}", resultorganizationsetpath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + final Boolean writeUpdates = Optional + .ofNullable(parser.get("writeUpdate")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("writeUpdate: {}", writeUpdates); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = (Class) Class.forName(resultClassName); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); - - datasource.createOrReplaceTempView("datasource"); - relation.createOrReplaceTempView("relation"); - organization.createOrReplaceTempView("organization"); - - String query = "SELECT source ds, target org " + - "FROM ( SELECT id " + - "FROM datasource " + - "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + - "AND datainfo.deletedbyinference = false ) d " + - "JOIN ( SELECT source, target " + - "FROM relation " + - "WHERE relclass = 'provides' " + - "AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source "; - - org.apache.spark.sql.Dataset rels = spark.sql(query); - rels.createOrReplaceTempView("rels"); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); - - org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); - - - software.createOrReplaceTempView("software"); - final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); - - dataset.createOrReplaceTempView("dataset"); - final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); - - other.createOrReplaceTempView("other"); - final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); - - publication.createOrReplaceTempView("publication"); - final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); - - writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); - - - query = "Select source resultId, collect_set(target) org_list " + - "from relation " + - "where datainfo.deletedbyinference = false " + - "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + - "group by source"; - - JavaRDD result_orglist = spark.sql(query).toJavaRDD(); - - JavaPairRDD> toupdateunion = toupdateresultdataset.mapToPair(d -> new Tuple2<>(d.getString(0), d.getList(1))) - .union(toupdateresultother.mapToPair(o -> new Tuple2<>(o.getString(0), o.getList(1)))) - .union(toupdateresultpublication.mapToPair(p -> new Tuple2<>(p.getString(0), p.getList(1)))) - .union(toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))); - - JavaRDD new_rels = getNewRels(result_orglist.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))), - toupdateunion); - - - - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .union(new_rels) - .map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/relation"); - - - } - - private static JavaRDD getNewRels(JavaPairRDD> relationOrgs, JavaPairRDD > newRels){ - return newRels - .leftOuterJoin(relationOrgs) - .flatMap(c -> { - List toAddOrgs = new ArrayList<>(); - toAddOrgs.addAll(c._2()._1()); - if (c._2()._2().isPresent()) { - Set originalOrgs = new HashSet<>(); - originalOrgs.addAll(c._2()._2().get()); - for (Object oId : originalOrgs) { - if (toAddOrgs.contains(oId)) { - toAddOrgs.remove(oId); - } - } + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if(isTest(parser)) { + removeOutputDir(spark, outputPath); } - List relationList = new ArrayList<>(); - String resId = c._1(); - for (Object org : toAddOrgs) { - relationList.add(getRelation((String)org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList.add(getRelation(resId, (String)org, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - - } - return relationList.iterator(); + execPropagation(spark, datasourceorganization, alreadylinked, inputPath, outputPath, resultClazz, resultType, + writeUpdates, saveGraph); }); - } - private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { - createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); - createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); - createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); - createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); } - private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { + private static void execPropagation(SparkSession spark, String datasourceorganization, String alreadylinked, String inputPath, + String outputPath, Class resultClazz, String resultType, + Boolean writeUpdates, Boolean saveGraph) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + org.apache.spark.sql.Dataset datasourceorganizationassoc = + readAssocDatasourceOrganization(spark, datasourceorganization); + + //broadcasting the result of the preparation step + Broadcast> broadcast_datasourceorganizationassoc = + sc.broadcast(datasourceorganizationassoc); + + org.apache.spark.sql.Dataset potentialUpdates = getPotentialRelations(spark, inputPath, resultClazz, + broadcast_datasourceorganizationassoc).as(Encoders.bean(ResultOrganizationSet.class)); + + if(writeUpdates){ + createUpdateForRelationWrite(potentialUpdates, outputPath + "/" + resultType); + } + + if(saveGraph){ + getNewRelations(spark + .read() + .textFile(alreadylinked) + .map(value -> OBJECT_MAPPER.readValue(value, ResultOrganizationSet.class), + Encoders.bean(ResultOrganizationSet.class)), potentialUpdates); + } + + + } + + private static Dataset getNewRelations(Dataset alreadyLinked, Dataset potentialUpdates) { + return potentialUpdates + .joinWith(alreadyLinked, potentialUpdates.col("resultId") + .equalTo(alreadyLinked.col("resultId")), "left_outer") + .flatMap((FlatMapFunction, Relation>) value -> { + List new_relations = new ArrayList<>(); + ResultOrganizationSet potential_update = value._1(); + Optional already_linked = Optional.ofNullable(value._2()); + List organization_list = potential_update.getOrganizationSet(); + if(already_linked.isPresent()){ + already_linked.get().getOrganizationSet() + .stream() + .forEach(rId -> { + if (organization_list.contains(rId)) { + organization_list.remove(rId); + } + }); + } + String resultId = potential_update.getResultId(); + organization_list + .stream() + .forEach(orgId -> { + new_relations.add(getRelation(orgId, resultId, RELATION_ORGANIZATION_RESULT_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + new_relations.add(getRelation(resultId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + } + ); + return new_relations.iterator(); + + } + ,Encoders.bean(Relation.class)); + + } + + + private static org.apache.spark.sql.Dataset getPotentialRelations(SparkSession spark, + String inputPath, + Class resultClazz, + Broadcast> broadcast_datasourceorganizationassoc) { + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + createCfHbforresult(spark); + + return organizationPropagationAssoc(spark, broadcast_datasourceorganizationassoc); + } + + + + + private static org.apache.spark.sql.Dataset readAssocDatasourceOrganization(SparkSession spark, + String datasourcecountryorganization) { + return spark + .read() + .textFile(datasourcecountryorganization) + .map(value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), Encoders.bean(DatasourceOrganization.class)); + } + + + + private static void createUpdateForRelationWrite(Dataset toupdaterelation, String outputPath) { toupdaterelation.flatMap(s -> { List relationList = new ArrayList<>(); - List orgs = s.getList(1); - String resId = s.getString(0); + List orgs = s.getOrganizationSet(); + String resId = s.getResultId(); for (String org : orgs) { relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, @@ -183,267 +207,31 @@ public class SparkResultToOrganizationFromIstRepoJob { } return relationList.iterator(); - }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); + }, Encoders.bean(Relation.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(outputPath) + ; } - private static JavaRDD propagateOnResult(SparkSession spark, String table) { - String query; - query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + - "FROM ( SELECT id, instance " + - "FROM " + table + - " WHERE datainfo.deletedbyinference = false) ds " + - "LATERAL VIEW EXPLODE(instance) i AS inst"; - org.apache.spark.sql.Dataset cfhb = spark.sql(query); - cfhb.createOrReplaceTempView("cfhb"); - return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); - - } - - private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ - String query = "SELECT id, collect_set(org) org "+ - "FROM ( SELECT id, org " + + private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, Broadcast> broadcast_datasourceorganizationassoc){ + org.apache.spark.sql.Dataset datasourceorganization = broadcast_datasourceorganizationassoc.value(); + datasourceorganization.createOrReplaceTempView("rels"); + String query = "SELECT id resultId, collect_set(organizationId) organizationSet "+ + "FROM ( SELECT id, organizationId " + "FROM rels " + - "JOIN " + cfhbTable + - " ON cf = ds " + + "JOIN cfhb " + + " ON cf = datasourceId " + "UNION ALL " + - "SELECT id , org " + + "SELECT id , organizationId " + "FROM rels " + - "JOIN " + cfhbTable + - " ON hb = ds ) tmp " + + "JOIN cfhb " + + " ON hb = datasourceId ) tmp " + "GROUP BY id"; - return spark.sql(query); + return spark.sql(query).as(Encoders.bean(ResultOrganizationSet.class)); } } - -//package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -// -//import eu.dnetlib.dhp.TypedRow; -//import eu.dnetlib.dhp.application.ArgumentApplicationParser; -//import org.apache.commons.io.IOUtils; -//import org.apache.hadoop.fs.FileSystem; -//import org.apache.spark.SparkConf; -//import org.apache.spark.api.java.JavaPairRDD; -//import org.apache.spark.api.java.JavaRDD; -//import org.apache.spark.api.java.JavaSparkContext; -//import org.apache.spark.sql.Encoders; -//import org.apache.spark.sql.Row; -//import org.apache.spark.sql.SparkSession; -//import org.apache.hadoop.io.Text; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import java.io.File; -//import java.util.ArrayList; -//import java.util.Arrays; -//import java.util.List; -//import java.util.Set; -// -//import eu.dnetlib.dhp.schema.oaf.*; -//import scala.Tuple2; -// -//import static eu.dnetlib.dhp.PropagationConstant.*; -// -//public class SparkResultToOrganizationFromIstRepoJob { -// public static void main(String[] args) throws Exception { -// -// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); -// parser.parseArgument(args); -// SparkConf conf = new SparkConf(); -// conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); -// final SparkSession spark = SparkSession -// .builder() -// .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) -// .master(parser.get("master")) -// .config(conf) -// .enableHiveSupport() -// .getOrCreate(); -// -// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); -// final String inputPath = parser.get("sourcePath"); -// final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; -// -// createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); -// -// org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") -// .map(item -> new ObjectMapper().readValue(item, Datasource.class)) -// .rdd(), Encoders.bean(Datasource.class)); -// -// JavaRDD relation_rdd_all = sc.textFile(inputPath + "/relation") -// .map(item -> new ObjectMapper().readValue(item, Relation.class)); -// JavaRDD relation_rdd = relation_rdd_all.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); -// -// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); -// -// org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") -// .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); -// -// datasource.createOrReplaceTempView("datasource"); -// relation.createOrReplaceTempView("relation"); -// organization.createOrReplaceTempView("organization"); -// -// String query = "SELECT source ds, target org " + -// "FROM ( SELECT id " + -// "FROM datasource " + -// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + -// "AND datainfo.deletedbyinference = false ) d " + -// "JOIN ( SELECT source, target " + -// "FROM relation " + -// "WHERE relclass = 'provides' " + -// "AND datainfo.deletedbyinference = false ) rel " + -// "ON d.id = rel.source "; -// -// org.apache.spark.sql.Dataset rels = spark.sql(query); -// rels.createOrReplaceTempView("rels"); -// -// org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); -// -// org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); -// -// org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); -// -// org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); -// -// -// software.createOrReplaceTempView("software"); -// final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); -// -// dataset.createOrReplaceTempView("dataset"); -// final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); -// -// other.createOrReplaceTempView("other"); -// final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); -// -// publication.createOrReplaceTempView("publication"); -// final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); -// -// writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); -// -// JavaPairRDD relation_rdd_pair = relation_rdd -// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getSource()); -// tp.add(r.getTarget()); -// return tp; -// }).mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) { -// return b; -// } -// if (b == null) { -// return a; -// } -// -// a.addAll(b.getAccumulator()); -// return a; -// }).cache(); -// -// -// JavaRDD new_rels = getNewRels(relation_rdd_pair, -// toupdateresultother.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultdataset.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultpublication.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))); -// -// -// relation_rdd_all.union(new_rels).map(r -> new ObjectMapper().writeValueAsString(r)) -// .saveAsTextFile(outputPath + "/relation"); -// -// } -// -// private static JavaRDD getNewRels(JavaPairRDD relation_rdd_pair, JavaPairRDD > newRels){ -// return newRels//.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))) -// .leftOuterJoin(relation_rdd_pair) -// .flatMap(c -> { -// List toAddOrgs = c._2()._1(); -// if (c._2()._2().isPresent()) { -// Set originalOrgs = c._2()._2().get().getAccumulator(); -// for (String oId : toAddOrgs) { -// if (originalOrgs.contains(oId)) { -// toAddOrgs.remove(oId); -// } -// } -// } -// List relationList = new ArrayList<>(); -// String resId = c._1(); -// for (String org : toAddOrgs) { -// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// -// } -// return relationList.iterator(); -// }); -// -// } -// -// private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { -// createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); -// createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); -// createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); -// createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); -// -// } -// -// private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { -// toupdaterelation.flatMap(s -> { -// List relationList = new ArrayList<>(); -// List orgs = s.getList(1); -// String resId = s.getString(0); -// for (String org : orgs) { -// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// -// } -// return relationList.iterator(); -// }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); -// } -// -// private static JavaRDD propagateOnResult(SparkSession spark, String table) { -// String query; -// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + -// "FROM ( SELECT id, instance " + -// "FROM " + table + -// " WHERE datainfo.deletedbyinference = false) ds " + -// "LATERAL VIEW EXPLODE(instance) i AS inst"; -// org.apache.spark.sql.Dataset cfhb = spark.sql(query); -// cfhb.createOrReplaceTempView("cfhb"); -// -// return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); -// -// } -// -// private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ -// String query = "SELECT id, collect_set(org) org "+ -// "FROM ( SELECT id, org " + -// "FROM rels " + -// "JOIN " + cfhbTable + -// " ON cf = ds " + -// "UNION ALL " + -// "SELECT id , org " + -// "FROM rels " + -// "JOIN " + cfhbTable + -// " ON hb = ds ) tmp " + -// "GROUP BY id"; -// return spark.sql(query); -// } -// -//} From 43f0590d4b0a5f8caeaee020b249588ea74c7a07 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:29:50 +0200 Subject: [PATCH 093/259] change in the testing because the business logic is changed. --- .../dhp/projecttoresult/ProjectPropagationJobTest.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index 54c7721a0..15cb42b1a 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -1,13 +1,8 @@ package eu.dnetlib.dhp.projecttoresult; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.countrypropagation.CountryPropagationJobTest; -import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2; -import eu.dnetlib.dhp.schema.oaf.Country; import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; import org.apache.commons.io.FileUtils; -import org.apache.neethi.Assertion; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -20,13 +15,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; - import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.ArrayList; -import java.util.List; public class ProjectPropagationJobTest { From 964b22d41806228542f9497d3dc4a2b9a0e2faf1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:32:01 +0200 Subject: [PATCH 094/259] modified the writing of the new relations. before: read old rels, add the new ones to them, write all the relations in new location. Now: first step of the wf copies the old relation i new location. If new relations are found, they are saved in the new location in append mode. --- ...SparkResultToProjectThroughSemRelJob2.java | 156 +++++++++--------- 1 file changed, 78 insertions(+), 78 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java index 563fcb3bc..9dbbf140b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java @@ -3,10 +3,9 @@ package eu.dnetlib.dhp.projecttoresult; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Relation; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.Text; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -17,59 +16,65 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.io.File; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.PropagationConstant.toPair; -public class SparkResultToProjectThroughSemRelJob { +public class SparkResultToProjectThroughSemRelJob2 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToProjectThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils.toString(SparkResultToProjectThroughSemRelJob2.class + .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); parser.parseArgument(args); + + parser.getObjectMap().keySet().stream().forEach(k -> System.out.println(k + " = " + parser.getObjectMap().get(k))); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() - .appName(SparkResultToProjectThroughSemRelJob.class.getSimpleName()) + .appName(SparkResultToProjectThroughSemRelJob2.class.getSimpleName()) .master(parser.get("master")) .config(conf) .enableHiveSupport() .getOrCreate(); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/projecttoresult"; + boolean writeUpdates = "true".equals(parser.get("writeUpdate")); + boolean saveGraph = "true".equals(parser.get("saveGraph")); final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - JavaRDD all_relations = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); + Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + relation.createOrReplaceTempView("relation"); - JavaRDD relations = all_relations.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); + String query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false and relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; - JavaRDD result_result = relations - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())); - - org.apache.spark.sql.Dataset resres_relation = spark.createDataset(result_result.rdd(), - Encoders.bean(Relation.class)); - - JavaRDD result_project = relations - .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) - && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())); - - org.apache.spark.sql.Dataset resproj_relation = spark.createDataset(result_project.rdd(), - Encoders.bean(Relation.class)); + Dataset resproj_relation = spark.sql(query); + + query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + getConstraintList(" relClass = '", allowedsemrel ); + Dataset resres_relation = spark.sql(query); resres_relation.createOrReplaceTempView("resres_relation"); resproj_relation.createOrReplaceTempView("resproj_relation"); - String query ="SELECT proj, collect_set(r1target) result_set " + + query ="SELECT proj, collect_set(r1target) result_set " + "FROM (" + - " SELECT r1.source as sourcer, r1.relclass as r1rel, r1.target as r1target, r2.target as proj " + + " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + " FROM resres_relation r1 " + " JOIN resproj_relation r2 " + " ON r1.source = r2.source " + @@ -77,68 +82,63 @@ public class SparkResultToProjectThroughSemRelJob { "GROUP BY proj "; Dataset toaddrelations = spark.sql(query); + + query = "select target, collect_set(source) result_list from " + + "resproj_relation " + + "group by target"; + Dataset project_resultlist = spark.sql(query); - JavaPairRDD project_resultlist = relations - .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass())) - .map(r -> { - TypedRow tp = new TypedRow(); - tp.setSourceId(r.getSource()); - tp.add(r.getTarget()); - return tp; - }).mapToPair(toPair()) - .reduceByKey((a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } + //if (writeUpdaes){ + toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/toupdaterelations"); + //} - a.addAll(b.getAccumulator()); - return a; - }).cache(); - - - JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - .leftOuterJoin(project_resultlist) - .flatMap(c -> { - List toAddRel = new ArrayList<>(); - toAddRel.addAll(c._2()._1()); - if (c._2()._2().isPresent()) { - Set originalRels = c._2()._2().get().getAccumulator(); - for (String o : originalRels) { - if (toAddRel.contains(o)) { - toAddRel.remove(o); + if(saveGraph){ + JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + .leftOuterJoin(project_resultlist.toJavaRDD().mapToPair(pr -> new Tuple2<>(pr.getString(0), pr.getList(1)))) + .flatMap(c -> { + List toAddRel = new ArrayList<>(); + toAddRel.addAll(c._2()._1()); + if (c._2()._2().isPresent()) { + List originalRels = c._2()._2().get(); + for (Object o : originalRels) { + if (toAddRel.contains(o)) { + toAddRel.remove(o); + } } } - } - List relationList = new ArrayList<>(); - String projId = c._1(); - for (Object r : toAddRel) { - String rId = (String) r; - relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + List relationList = new ArrayList<>(); + String projId = c._1(); + for (Object r : toAddRel) { + String rId = (String) r; + relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - } - return relationList.iterator(); - }).cache(); + } + if(relationList.size()==0){ + return null; + } + return relationList.iterator(); + }).filter(r -> !(r==null)); - toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/toupdaterelations"); - new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/new_relations" ); + new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/new_relations" ); - all_relations.union(new_relations) - .map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/relation"); + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .union(new_relations) + .map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/relation"); + + } //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); From 35772191275fd9c5ebac4b8a4168ff628ec14882 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 15 Apr 2020 12:45:49 +0200 Subject: [PATCH 095/259] removed unuseful classes --- .../SparkCountryPropagationJob.java | 333 ------------------ .../SparkResultToProjectThroughSemRelJob.java | 118 ------- ...SparkResultToProjectThroughSemRelJob2.java | 222 ------------ 3 files changed, 673 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java deleted file mode 100644 index cf80649b6..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java +++ /dev/null @@ -1,333 +0,0 @@ -package eu.dnetlib.dhp.countrypropagation; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.PropagationConstant; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import net.sf.saxon.expr.ContextMappingFunction; -import net.sf.saxon.expr.flwor.Tuple; -import net.sf.saxon.om.Item; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.*; -import org.apache.spark.sql.Dataset; -import org.codehaus.janino.Java; -import scala.Tuple2; - -import javax.sql.DataSource; -import java.beans.Encoder; -import java.io.File; -import java.io.IOException; -import java.util.*; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkCountryPropagationJob { - - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkCountryPropagationJob.class.getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json"))); - parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkCountryPropagationJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/countrytoresultfrominstitutionalrepositories"; - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - List whitelist = Arrays.asList(parser.get("whitelist").split(";")); - List allowedtypes = Arrays.asList(parser.get("allowedtypes").split(";")); - boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - -// datasource(spark, whitelist, outputPath, inputPath, "true".equals(parser.get("writeUpdate")), -// "true".equals(parser.get("saveGraph")), allowedtypes); -// -// } -// -// -// private static void datasource(SparkSession spark, List whitelist, String outputPath, String inputPath, -// boolean writeUpdates, boolean saveGraph, List allowedtypes){ - - String whitelisted = ""; - for (String i : whitelist){ - whitelisted += " OR id = '" + i + "'"; - } - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - - Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); - - Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); - - Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); - - Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); - - Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); - - datasource.createOrReplaceTempView("datasource"); - relation.createOrReplaceTempView("relation"); - organization.createOrReplaceTempView("organization"); -//todo add length(country.classid)>0 - String query = "SELECT source ds, target org, country.classid country " + - "FROM ( SELECT id " + - "FROM datasource " + - "WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + - getConstraintList("datasourcetype.classid = '", allowedtypes) + - // "datasourcetype.classid = 'pubsrepository::institutional' " + - // "AND (datainfo.deletedbyinference = false " + whitelisted + ") ) d " + - "JOIN ( SELECT source, target " + - "FROM relation " + - "WHERE relclass = 'provides' " + - "AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source " + - "JOIN (SELECT id, country " + - "FROM organization " + - "WHERE datainfo.deletedbyinference = false ) o " + - "ON o.id = rel.target"; - - //todo broadcast - Dataset rels = spark.sql(query); - rels.createOrReplaceTempView("rels"); - - software.createOrReplaceTempView("software"); - final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); - - dataset.createOrReplaceTempView("dataset"); - final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); - - other.createOrReplaceTempView("other"); - final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); - - publication.createOrReplaceTempView("publication"); - final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); - - if(writeUpdates){ - writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); - } - - if(saveGraph){ - createUpdateForSoftwareDataset(toupdateresultsoftware, inputPath, spark) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/software"); - - createUpdateForDatasetDataset(toupdateresultdataset,inputPath,spark) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/dataset"); - - createUpdateForOtherDataset(toupdateresultother, inputPath, spark) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); - - createUpdateForPublicationDataset(toupdateresultpublication, inputPath, spark) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); - } - - } - - private static void writeUpdates(JavaRDD software, JavaRDD dataset, JavaRDD other , JavaRDD publication, String outputPath){ - createUpdateForResultDatasetWrite(software, outputPath, "update_software"); - createUpdateForResultDatasetWrite(dataset, outputPath, "update_dataset"); - createUpdateForResultDatasetWrite(other, outputPath, "update_other"); - createUpdateForResultDatasetWrite(publication, outputPath, "update_publication"); - } - - private static JavaRDD createUpdateForOtherDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - OtherResearchProduct oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD createUpdateForPublicationDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - Publication oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - Software oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD createUpdateForDatasetDataset(JavaRDD toupdateresult, String inputPath, SparkSession spark) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - return sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)) - .mapToPair(d -> new Tuple2<>(d.getId(), d)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult)) - .map(c -> { - eu.dnetlib.dhp.schema.oaf.Dataset oaf = c._2()._1(); - List countryList = oaf.getCountry(); - if (c._2()._2().isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : countryList) { - countries.add(country.getClassid()); - } - Result r = c._2()._2().get(); - for (Country country : r.getCountry()) { - if (!countries.contains(country.getClassid())) { - countryList.add(country); - } - } - oaf.setCountry(countryList); - } - return oaf; - }); - } - - private static JavaRDD propagateOnResult(SparkSession spark, String result_type) { - String query; - query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + - "FROM ( SELECT id, instance " + - "FROM " + result_type + - " WHERE datainfo.deletedbyinference = false) ds " + - "LATERAL VIEW EXPLODE(instance) i AS inst"; - Dataset cfhb = spark.sql(query); - cfhb.createOrReplaceTempView("cfhb"); - - return countryPropagationAssoc(spark, "cfhb").toJavaRDD(); - - } - - private static Dataset countryPropagationAssoc(SparkSession spark, String cfhbTable){ - String query = "SELECT id, collect_set(country) country "+ - "FROM ( SELECT id, country " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON cf = ds " + - "UNION ALL " + - "SELECT id , country " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON hb = ds ) tmp " + - "GROUP BY id"; - return spark.sql(query); - } - - private static JavaPairRDD getStringResultJavaPairRDD(JavaRDD toupdateresult) { - return toupdateresult.map(c -> { - List countryList = new ArrayList<>(); - List tmp = c.getList(1); - for (String country : tmp) { - countryList.add(getCountry(country)); - } - Result r = new Result(); - r.setId(c.getString(0)); - r.setCountry(countryList); - return r; - }).mapToPair(r -> new Tuple2<>(r.getId(), r)); - } - - private static void createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type){ - toupdateresult.map(c -> { - List countryList = new ArrayList<>(); - List tmp = c.getList(1); - for (String country : tmp) { - countryList.add(getCountry(country)); - } - Result r = new Result(); - r.setId(c.getString(0)); - r.setCountry(countryList); - return r; - - }).map(r ->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath+"/"+type); - } - - - -} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java deleted file mode 100644 index 1798aa67a..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java +++ /dev/null @@ -1,118 +0,0 @@ -package eu.dnetlib.dhp.projecttoresult; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.PropagationConstant.toPair; - -public class SparkResultToProjectThroughSemRelJob { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToProjectThroughSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToProjectThroughSemRelJob.class.getSimpleName()) - .master(parser.get("master")) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/projecttoresult"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - - File directory = new File(outputPath); - - if (!directory.exists()) { - directory.mkdirs(); - } - - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); - - JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - - JavaPairRDD result_project = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); - - //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result - JavaPairRDD project_result = result_project.join(result_result) - .map(c -> { - String projectId = c._2()._1().getTargetId(); - String resultId = c._2()._2().getTargetId(); - return new TypedRow().setSourceId(projectId).setTargetId(resultId); - }) - .mapToPair(toPair()); - - //relationships from project to result. One Pair for each project => project id list of results related to the project - JavaPairRDD project_results = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()) - .reduceByKey((a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } - a.addAll(b.getAccumulator()); - return a; - }); - - - - JavaRDD newRels = project_result.join(project_results) - .flatMap(c -> { - String resId = c._2()._1().getTargetId(); - - if (c._2()._2().getAccumulator().contains(resId)) { - return null; - } - String progId = c._2()._1().getSourceId(); - List rels = new ArrayList(); - - rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - return rels.iterator(); - }) - .cache(); - - newRels.map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation_new"); - - newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/relation"); - - } - - - - -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java deleted file mode 100644 index 9dbbf140b..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob2.java +++ /dev/null @@ -1,222 +0,0 @@ -package eu.dnetlib.dhp.projecttoresult; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkResultToProjectThroughSemRelJob2 { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils.toString(SparkResultToProjectThroughSemRelJob2.class - .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json"))); - parser.parseArgument(args); - - parser.getObjectMap().keySet().stream().forEach(k -> System.out.println(k + " = " + parser.getObjectMap().get(k))); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToProjectThroughSemRelJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/projecttoresult"; - boolean writeUpdates = "true".equals(parser.get("writeUpdate")); - boolean saveGraph = "true".equals(parser.get("saveGraph")); - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - relation.createOrReplaceTempView("relation"); - - String query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false and relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; - - Dataset resproj_relation = spark.sql(query); - - query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false " + getConstraintList(" relClass = '", allowedsemrel ); - - Dataset resres_relation = spark.sql(query); - resres_relation.createOrReplaceTempView("resres_relation"); - resproj_relation.createOrReplaceTempView("resproj_relation"); - - query ="SELECT proj, collect_set(r1target) result_set " + - "FROM (" + - " SELECT r1.source as source, r1.target as r1target, r2.target as proj " + - " FROM resres_relation r1 " + - " JOIN resproj_relation r2 " + - " ON r1.source = r2.source " + - " ) tmp " + - "GROUP BY proj "; - - Dataset toaddrelations = spark.sql(query); - - query = "select target, collect_set(source) result_list from " + - "resproj_relation " + - "group by target"; - - Dataset project_resultlist = spark.sql(query); - - //if (writeUpdaes){ - toaddrelations.toJavaRDD().map(r->new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/toupdaterelations"); - //} - - if(saveGraph){ - JavaRDD new_relations = toaddrelations.toJavaRDD().mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - .leftOuterJoin(project_resultlist.toJavaRDD().mapToPair(pr -> new Tuple2<>(pr.getString(0), pr.getList(1)))) - .flatMap(c -> { - List toAddRel = new ArrayList<>(); - toAddRel.addAll(c._2()._1()); - if (c._2()._2().isPresent()) { - List originalRels = c._2()._2().get(); - for (Object o : originalRels) { - if (toAddRel.contains(o)) { - toAddRel.remove(o); - } - } - } - List relationList = new ArrayList<>(); - String projId = c._1(); - for (Object r : toAddRel) { - String rId = (String) r; - relationList.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - relationList.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - - } - if(relationList.size()==0){ - return null; - } - return relationList.iterator(); - }).filter(r -> !(r==null)); - - - new_relations.map(r-> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/new_relations" ); - - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .union(new_relations) - .map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/relation"); - - } - - - //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - -// JavaPairRDD result_project = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) -// && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) -// .map(rel ->{ -// -// TypedRow tr = new TypedRow(); -// tr.setSourceId(rel.getSource()); -// tr.setTargetId(rel.getTarget()); -// return tr; -// }) -// .mapToPair(toPair()); -// -// //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result -// JavaPairRDD project_result = result_project.join(result_result) -// .map(c -> { -// String projectId = c._2()._1().getTargetId(); -// String resultId = c._2()._2().getTargetId(); -// TypedRow tr = new TypedRow(); tr.setSourceId(projectId); tr.setTargetId(resultId); -// return tr; -// }) -// .mapToPair(toPair()); -// -// //relationships from project to result. One Pair for each project => project id list of results related to the project -// JavaPairRDD project_results = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) -// .map(r -> { -// TypedRow tr = new TypedRow(); tr.setSourceId(r.getSource()); tr.setTargetId(r.getTarget()); -// return tr; -// }) -// .mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) { -// return b; -// } -// if (b == null) { -// return a; -// } -// a.addAll(b.getAccumulator()); -// return a; -// }); -// -// -// -// JavaRDD newRels = project_result.join(project_results) -// .flatMap(c -> { -// String resId = c._2()._1().getTargetId(); -// -// if (c._2()._2().getAccumulator().contains(resId)) { -// return null; -// } -// String progId = c._2()._1().getSourceId(); -// List rels = new ArrayList(); -// -// rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, -// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); -// rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, -// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); -// return rels.iterator(); -// }) -// .cache(); -// -// newRels.map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/relation_new"); -// -// newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/relation"); - - } - - - - -} From 79b978ec57d4bf2ee25c90b4a94ac5792251448b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:48:41 +0200 Subject: [PATCH 096/259] refactoring --- ...SparkResultToProjectThroughSemRelJob3.java | 101 ------------------ .../sample/relation/relation_100.json.gz | Bin 3639 -> 0 bytes .../alreadyLinked/alreadyLinked_20.json | 20 ++++ .../datasourceOrganization_28.json | 28 +++++ .../alreadyLinked/alreadyLinked_20.json | 20 ++++ .../datasourceOrganization_28.json | 28 +++++ 6 files changed, 96 insertions(+), 101 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index fb726e86c..2125f351f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -1,34 +1,16 @@ package eu.dnetlib.dhp.projecttoresult; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.DatasourceCountry; import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; -import eu.dnetlib.dhp.countrypropagation.ResultCountrySet; import eu.dnetlib.dhp.schema.oaf.Relation; -import org.apache.arrow.flatbuf.Bool; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -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.FlatMapFunction2; -import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; -import org.omg.CORBA.OBJ_ADAPTER; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; - import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; - import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; @@ -50,9 +32,6 @@ public class SparkResultToProjectThroughSemRelJob3 { Boolean isSparkSessionManaged = isSparkSessionManaged(parser); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); log.info("outputPath {}: ", outputPath); @@ -143,86 +122,6 @@ public class SparkResultToProjectThroughSemRelJob3 { } - private static void writeUpdates(JavaRDD potentialUpdates, String outputPath){ - potentialUpdates.map(u -> OBJECT_MAPPER.writeValueAsString(u)) - .saveAsTextFile(outputPath, GzipCodec.class); - } - //JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - -// JavaPairRDD result_project = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_RESULT_PROJECT_REL_CLASS.equals(r.getRelClass()) -// && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) -// .map(rel ->{ -// -// TypedRow tr = new TypedRow(); -// tr.setSourceId(rel.getSource()); -// tr.setTargetId(rel.getTarget()); -// return tr; -// }) -// .mapToPair(toPair()); -// -// //relationships from project to result. One pair for each relationship for results having allowed semantics relation with another result -// JavaPairRDD project_result = result_project.join(result_result) -// .map(c -> { -// String projectId = c._2()._1().getTargetId(); -// String resultId = c._2()._2().getTargetId(); -// TypedRow tr = new TypedRow(); tr.setSourceId(projectId); tr.setTargetId(resultId); -// return tr; -// }) -// .mapToPair(toPair()); -// -// //relationships from project to result. One Pair for each project => project id list of results related to the project -// JavaPairRDD project_results = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_PROJECT_RESULT_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTPROJECT_REL_TYPE.equals(r.getRelType())) -// .map(r -> { -// TypedRow tr = new TypedRow(); tr.setSourceId(r.getSource()); tr.setTargetId(r.getTarget()); -// return tr; -// }) -// .mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) { -// return b; -// } -// if (b == null) { -// return a; -// } -// a.addAll(b.getAccumulator()); -// return a; -// }); -// -// -// -// JavaRDD newRels = project_result.join(project_results) -// .flatMap(c -> { -// String resId = c._2()._1().getTargetId(); -// -// if (c._2()._2().getAccumulator().contains(resId)) { -// return null; -// } -// String progId = c._2()._1().getSourceId(); -// List rels = new ArrayList(); -// -// rels.add(getRelation(progId, resId, RELATION_PROJECT_RESULT_REL_CLASS, -// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); -// rels.add(getRelation(resId, progId, RELATION_RESULT_PROJECT_REL_CLASS, -// RELATION_RESULTPROJECT_REL_TYPE, RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); -// return rels.iterator(); -// }) -// .cache(); -// -// newRels.map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/relation_new"); -// -// newRels.union(relations).map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/relation"); - - //} - - private static Dataset readAssocProjectResults(SparkSession spark, String potentialUpdatePath) { return spark .read() diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/sample/relation/relation_100.json.gz deleted file mode 100644 index bd4abe26f7dd34d52484ecf2319e3c251f95e1c3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3639 zcmX|@cQhON8^(=`qDJizMN3fxwaTs0XpOjbsjZh=yHsLSXvC^oHCnT_TGiUx5~~O$ zAy&kWB8k08t@>&2J-I>=mj`-6 zeSN4lX6=0KRa2?vDQDc_d5d(u_jwM|2QXBVF$0~G!2&V~@0~4+_-$@APuo1JKiWcH zpRUFE<(H0S?pPa5)|6Q7?QiI$+#mmbL@XM9)rPE{jz>EIt9;;XQ;s{0j`ji$z6WdY z_!ZaUjn9JFj%&t(?M)|%bKhc4HdU%WBjeG`l82Q-mVt@2rGjb(Ork92TwfaewV z5uJ3Hos(gc#W&149`rU%v`sp;h6W%f9r{Uu$TQ*QpVK9C>s5iiD#wesz|X;>N$?Q& zzw9ICJLtAW$ExL2U}sLgjZ#$Wpv?^|fj-QMSFc&1xn8EMMiboE-3c7mXt9 z5A3e0Jk(kvp=R;K5;B11ESE=)5nw4Yme7sU7U4Ix!-cfg6}a2Ne;`QYS%|FbS_4og#xY$%LFa_X%gC1tT*~dfwYQ)_KVs{ zTM0JvMW^S51nl#9`CfV8h2gOqo11~jZCOh(SG+yjzP;ZhJ5didwW-z{){LW85?4?nzi)W%cq~eR^fXCh~0~HsEaCP%lq00S`-NUaLL(VaTVKbO!T2^BoDd}vdknP`V)VQ@#RNr&6^YUX_K9&( zfi;w~6lX1#J+6PfkV!9swxMjWVy_`ZgyO^9oz_!zqvAz)tCEbYQb1#Kd}GQZDX#Zk zRs0V+#o}*7FM&2?vmV}BnKaU3bk(^iApWa>rbKPoU*6)WPn3W$a02?S@_S9|`Vu6I z=MpSmirb~iw#!irN#Kj5-^&B}C|7_D(&8oV(tgXG13QHf){tHBv0TkCifCwnmy-tm z0c3OwyCgN8 z{3as}!*v6oXQ{}D1^rR?@|Mmbd#Lmu-aVO2dY98I5?f1jpNO(dhlWkGMz`d}+{ynn zMCrv&hNDSgUQPYY?x4IMJ;lOJ<{&Wo+^-^mMVbBGU=2wA>;cO_Oiwg(0V63QIWAM| zE1a)pq>jlLL1um`dCc;|CKldHG!7-l`idVvKk51`Hy7sMXg+^4P73tlRkRVnnrp;+fNwl2-3+PXtr;q$P;5Jb?Adsaj;X%8ogx_F*$@GAX1ZDW+I;(JDX%pfW@YN z-t$dtH4=Nrqc`57<1eKMda1!6IS@X^bBIS!`pr@-m{8nFsW@du(bsMt#Q)y*A}<)Z zUQ{v_6ExlR6AP)q8mvok3RbxqBvXUp#Px*jzt&w|zF)Kinxt#9{^)uaScou}j4D!= zQr{RUpx;b%Q$};aoR!gjk76oq=zlX8i$kP(?TNZ|cK||qhRt~}nceuDwF~HmAt1LZ z;p^%oTNfV@A(@79pOR&NCXJv%ogFCE1Xy|AZ_0u^7Hq3rK&;Y9Wxte4!xN3DP>{^l zo6)NFsUDQBj|8NaF1u8;y0N`_(~>q7!*Y8xr)<#sgTVHe)Gi$2zlC}duy_=t)$(I6 z>~z}^tb&y#WrvhfuY)~CFiq<A;;3v02I501+ zO(If5Dh1gN)M7OrJ9h;Pv?fJqwrwrKw=C%2vB##erN4W@a~Ra$Ovrrh0!*z@UURt7 z2T_3~4i_8J1<01=9u&KILCdRx5#!&RNz`MbME|3SU~S0r;*+VDa%)5alR zn_Pw{@7ciBAgnBZN%|kK?62mNT+cp|;DZf0O+$sj9T$+1sTuyWvHG9|4aHcNZ0G9| z5wi7#U*3r9ZS~|(1d)>cg-SQgtR4Q58Nb#%G+H929^{q_GKtTp?O^T2QvX(TDocDY zE&UrqK8=+*B+V=QXKs?Hhl5uA9~rkEdar9xyB?Bi?x%C4wUFvQZS~4*hD}k$49f6t zLw{gWshcIbSy{gUPjeL4F2r(2 zxgo~wcd~$5hCR8MBHgFJM^NWZWqmQ3KT5O|=Gw0k&U$PS0z6U4$vD*tGsC+t5;Rh> z%SQ7RItmj)I+%;&>Zj< zT4!-)MhR$>uR=#jE_w;H4XjCg45lU-qr{7sX&n$>T#y&54RXI+`(+6HvT^CK;>=f5 zsR~DZ(3F2hip!dLhRMl2w4lAM7XiwTa@^cCTL&_*qaY2i~2y-N*Tu1jjnn$dY*bQP)Aw*?0NwRBmyRn*W2 zbvNsDi7Q?YII(%fdx2pODJJ$jl7gslQ0dScwLSB;9^=S|!h2Jpi(D`gC}x37RnxTt zMI4T)1?2H)*~Oc)43S4*TlUHnt-kPbJ~1`(zvpgSNndHYM<|lGhI^&lBNOi7$p3^% z!|ya0F!B9|n)a_Dnb__40g2-U5mrfNsV_FWh`NO@%^cgOECX(O=ZReE<%=|o8tJHn zfIl@`Eali}*}c_~*1XWK`p4yhJYX3b%b_TQpGS>BQov)@<+gz@GwY5$pi`m#7(jut z3(5b8h{BA4<#{si}*zvX6&&zhc% z4|2-L=*e?7WG{L!1h5!r_P#N|1=Ww08z=EmLtIeA-R-X_X4nAx2ZK$|(ub_-)`L@! z5?{UDZ}@5Voo!RJ^IuO`+)%l5jAn0Kl*bo!Ld+vzv@)4Ou1XVx9p`F4tSdw+;fm>r zp@^|nNYF=37an`$>d5+I>^Ny?+4sT(hY3`xE0E^UeSvszd4DO|hk~lL+(+{>R$MwV z)oScKEmYuqH(vo+e32s+G7wv%UHO$eYjoz~P`W2R?!fj7p#RgXoypnO)r{sKOif3h zr1ZD6+7hX#>l5$UVW3%TYMnUw9R(w$|I(w+^iHjumB%KHe01I?Jrq?wJ&HX2@MPgaN2LE4vao4NA2b(l7 zO81(d(Q}=>G3_xvRIJo9JEK4DAlMk6W5e9C?&Fz$2H8NCS7bd+zO}b6wci~)xY)r; zUimkSbt{xnqru9PnJp=?zNKj>=U0;+q!Z=#_Ud?A5w>BZ zGfqHFm^-<=D5+Llh_K;J6Y5FTyLvO}L=A98zw|2(Z3)-w>O2zih|Ce?Cgq$c)Z8S9fafXt+55@f3(R`n`4_uc4u`n=02x$yy=OUc&!^9ZNd?;^ sq0S24Sq15`EQUP97@u79DWJgm!$ysh{AH${t?is4V(Av%=bVb_e}?W^$N&HU diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json new file mode 100644 index 000000000..b585baf88 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json @@ -0,0 +1,20 @@ +{"resultId":"50|acm_________::3133635707788d2180bcef09e01a903c","organizationSet":["20|dedup_wf_001::5ab59ffa94c31a140d4a56c594ea5865"]} +{"resultId":"50|core________::0308a76f6f8bc4db75a817d53a7e76a4","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} +{"resultId":"50|core________::04c8f896aef9e54867f2bf4236e9c810","organizationSet":["20|snsf________::1496b1b4fc4d5509b16f2c217be480dc","20|dedup_wf_001::06731b587a9ea654103a6b0ebcb234ff","20|nih_________::c5722b087a5e707a50aa8f9f2ebf785d","20|snsf________::71d0a944b61b1a94068595f840005a2f","20|nih_________::dd3428794aef214a3bc2cad6dd548ba6","20|rcuk________::45aac2108e54b6503d1e611aa5872c03","20|nih_________::e1d47fdb7bba9eaeed82a95c578d6e90","20|dedup_wf_001::e3b52200d2fd4ff883478f5bef312efe","20|snsf________::c5c565d3422a7eb22886f3a4c93c32ea","20|nih_________::91154321f75ba26021efa18f7eeaa541","20|wt__________::38013971ca0c021fd65abce2171b82eb","20|wt__________::a6114989a56a1dfae6cbb201d14823f0","20|snsf________::b7af2f99e1e06750a4664ae401802734","20|wt__________::757c54e33d4e925c8c17edf032cdfacc","20|wt__________::1d57a87af1bbc2b7e211305fc747c9ad","20|wt__________::7cbb8c06f702b8871948acd370df892f","20|dedup_wf_001::53a8606f32787c4b3c663fd90ee97b17","20|wt__________::8473a929b793e56d2299a1a5aa08f617","20|nih_________::5e0fc2ef31bc189207f250db818fea0e","20|nih_________::1cd08fd26ef03fd2f51e9aeb34ed9486","20|nih_________::1c270e0dd2552b4e3cf295cdb7db1cc9","20|wt__________::04abd842647bdbc751b1eebe2f142013","20|nsf_________::3eaa8be3f16b0f0d7563b9117cd1f660","20|dedup_wf_001::c1b81dadf1e4cbf23a61833ff9ae8a31","20|nih_________::3716e1c14ab7ca14161278c9bbb8bdbb","20|dedup_wf_001::b7b403a764ea4e3acb12d999675aa73c","20|nih_________::300049f12fa0f5bc37db3a5636869743","20|wt__________::ed1e2be693353d370066fddbf862f23a","20|nsf_________::72a3747a18c56f3701494a0c0eadc5c9","20|rcuk________::e8877abcab4bc187339a242aa5bc2e09","20|microsoft___::119a535bfd240d7560fe4efec416bcd2","20|wt__________::be4e939abf9617557a35862e255493da","20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7","20|snsf________::1f4e34433767faf965f33849bb0f7fb1","20|dedup_wf_001::9f1647eae28911113d1dcbe49c42275b","20|opendoar____::6835febcf408fe892504bdfd2ebe669c","20|dedup_wf_001::528bd21573101493c6c7d120b17a67e9"]} +{"resultId":"50|core________::061dc912098a6a52e85a824161bd32a2","organizationSet":["20|dedup_wf_001::67f983a98e2c0cc0121e2db46d2bd00a","20|wt__________::59b5d99b2dde58df1655e0e5bb236c0a","20|wt__________::e84b06dbc1b26f413791c1304ca8d6a3","20|dedup_wf_001::7b118dab509f49b4fbd6dccfdbc479af","20|wt__________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::43255cf9c16732bc4ec1d5f580f44928","20|gsrt________::455b984b47945e1fd04e92c9c0eeca04","20|dedup_wf_001::e712c08f721e8f167f93888f590314ea","20|rcuk________::8b0aee2a7026dc92d4c05683ae45c894","20|dedup_wf_001::3c19a02cea18f8eeb2034b6acc544b7e","20|wt__________::26a9d64d00b8e1005cb6bbad2b7364cf","20|rcuk________::e57ab9be7acd7b93ba34aafab1a5b96c","20|nih_________::ca09277064edbd89f71e1602d98b4dd8","20|gsrt________::7c309ee758e6c6e0dce43c67b9343e82","20|nsf_________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::c57387345f51a40ad2284089b120be3f"]} +{"resultId":"50|core________::088190cf9dc9632e8d9ba5f5f1de1303","organizationSet":["20|dedup_wf_001::286621caef868bbdc52918699e3cdc79","20|dedup_wf_001::84707a912d45d666fef35c5cd77fc203","20|dedup_wf_001::30fd03f4977438b0471a1d4db6317e71","20|dedup_wf_001::83d9dc4a0eba8737819b87ce8e737a49","20|dedup_wf_001::aeb9e738d873acebdca52c9ccd9559bd","20|dedup_wf_001::427a8bfe68648f9e30cb47434144da68","20|nih_________::ffa0ea174845f1548e520a047cf53165"]} +{"resultId":"50|core________::117d295998199f498fa561e9c26e7ae3","organizationSet":["20|dedup_wf_001::d9f82e62c3114dc7f42b9da9b7f6fc64"]} +{"resultId":"50|core________::168a8f2e4ffe9b0e7c6bc100f34f2de5","organizationSet":["20|wt__________::63fe73f079c5ff4e925c6cfc1758a98b","20|dedup_wf_001::b3c2f5e700cee15ad9752ab961df5930"]} +{"resultId":"50|core________::16a3b520030d82ad16a30992b124e69d","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} +{"resultId":"50|core________::172e3da668d18f41ea4ccdf7f2f39e53","organizationSet":["20|nih_________::126cbf4b13249e65098ddb4835f47456","20|aka_________::506e3d2f7507a66584b8b3430ade20cb","20|dedup_wf_001::4746df4ff8bbb7e991ad343ccff8bbc7","20|wt__________::aff5133ca9cf0b810cc331d498bac9b0","20|wt__________::0c3bf892603817e5eff6e4f08a530ea2"]} +{"resultId":"50|core________::19f2fc91fe1db2ad62db598aa9aa8ab3","organizationSet":["20|dedup_wf_001::dbbd988f8d57a9d11286caefdf35acaa"]} +{"resultId":"50|core________::1dceb5a29cd42728e410474fe0fda191","organizationSet":["20|wt__________::b1ef2f643c948a2ef49005f9145ed556","20|dedup_wf_001::866fa622e3c0ab6227cd462f40cdcac8","20|rcuk________::63ecf5736189d299fc3e043e14428b8d","20|nsf_________::fcf880eab7315e0a5f3937c5a16c04b0","20|dedup_wf_001::65862ec7d57f700a130dee916bea66de"]} +{"resultId":"50|core________::2580c0b59b7457f571acdc829d1765a3","organizationSet":["20|doajarticles::0f6e2c32a27c307b06edf7862c591973","20|opendoar____::4f10fb61c457cf124e5917391baaa3c2"]} +{"resultId":"50|core________::2624b8248a9febdad9bc456d358b30ed","organizationSet":["20|dedup_wf_001::fb4eba3cea53264bddd59a4ade9973b3","20|rcuk________::b00968d2100a4b62447841aef5bdff62"]} +{"resultId":"50|core________::26820a282ef54882f7a5be74767fc02b","organizationSet":["20|rcuk________::8ad6d06f3b4d09dc67142c158c7cf5b9","20|rcuk________::01ad471b66687b1213ceb08b5d7aa6c2"]} +{"resultId":"50|core________::2a8de3e0bbcab49066aa9de4bbb89bfa","organizationSet":["20|dedup_wf_001::2ea78875d19c8cea63f7e958e5204136","20|corda_______::6821a8e260b8b97f5fb5e80168329d5b","20|dedup_wf_001::9d0ba437d73b19f55b53c578ac970ea2"]} +{"resultId":"50|core________::2c7d139419d2895d3bf0112b50108f75","organizationSet":["20|dedup_wf_001::96ada508ea5d85a1e516bf9799413906","20|dedup_wf_001::d0ea749da6988bcdb2f30d77c64e2f1e","20|wt__________::f1ba5bd552edf15db494dc3020f27470","20|nih_________::ceeae4f78a5666daf4c45acdbbedde99","20|wt__________::84ef588eeeb4ef77e45ccfbbf3aef69c","20|wt__________::8eef7e1370ea81c2aa3dbc239b2bf5d8"]} +{"resultId":"50|core________::2cf1f6282498fa37aeaf678f8c6e5843","organizationSet":["20|snsf________::73999c828ca67fd2d006100a8369c1eb"]} +{"resultId":"50|core________::2dffff00500c2354b506814f6a1ec148","organizationSet":["20|wt__________::c6d89e908582fddf3e4c658a458807c3","20|wt__________::e7b2c9f3d3f3f1503092bf1ba2b163db","20|gsrt________::ab510bb43d6c654ed3d37b9c5ed5c971","20|dedup_wf_001::179d0313fa7d5fb2bef5f312ecdd16fe","20|gsrt________::cbffb510b01e81cc055fe61105c86154","20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e","20|dedup_wf_001::646f14555ea42b260499239a7231b285","20|wt__________::5d462d78d512c1184dd384ef2dc35b7e","20|nih_________::a32a254b024265db2e24a66291c7c1e0","20|dedup_wf_001::5490ec18da9721e2c8d974fb73c62467","20|dedup_wf_001::3bc91ed90f44d0908258e132659bc754"]} +{"resultId":"50|core________::3031a50bf5c80865af4841ab42aaf57e","organizationSet":["20|nih_________::1b46e3665d8be2b524c285a27ca952b8","20|nsf_________::71450a4b98015592ee3f525a51584608","20|snsf________::fc921725875adb56f2275579b31f805c","20|aka_________::fa5b7357f86c71ea15734282054f1183","20|wt__________::18fdb5b42b22fdcc45e323eb4d20c91b","20|wt__________::71450a4b98015592ee3f525a51584608","20|dedup_wf_001::8aaf46d4e4919dc55b8a5cac7a15399f"]} +{"resultId":"50|core________::31116372ae189ee456fc06dfa0f6cf7a","organizationSet":["20|aka_________::c5b9aa0a905f89c51221f9f4fda22b20","20|aka_________::d9d3242062a7a3c483a7926fdba17bb6","20|nih_________::ede5c9e31cfb37a397d6cfe1940d045e","20|wt__________::8adcc12ffee195ae46679e8cf332a364","20|wt__________::5e954c57b0ac7aaf3fc16deeaf442389","20|snsf________::ddd964d550bfc6e1ce18f83655ba6901","20|rcuk________::a705d2ee7bf0bd225264b4a5794795ce","20|nih_________::8adcc12ffee195ae46679e8cf332a364","20|microsoft___::53732c6c7bb9daf5953fdb61fc0cd5bd"]} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json new file mode 100644 index 000000000..efc3af110 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json @@ -0,0 +1,28 @@ +{"datasourceId":"10|opendoar____::1068c6e4c8051cfd4e9ea8072e3189e2","organizationId":"20|opendoar____::4429502fa1936b0941f4647b69b844c8"} +{"datasourceId":"10|opendoar____::211ed78fe91938b90f84a51944b08d5a","organizationId":"20|opendoar____::124266ebc4ece2934eb80edfda3f2091"} +{"datasourceId":"10|opendoar____::21c5bba1dd6aed9ab48c2b34c1a0adde","organizationId":"20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2"} +{"datasourceId":"10|opendoar____::376c6b9ff3bedbbea56751a84fffc10c","organizationId":"20|opendoar____::c6ac1e6c39e6f3a36e274f8d3f9f7fce"} +{"datasourceId":"10|opendoar____::49c0fa7f96aa0a5fb95c62909d5190a6","organizationId":"20|dedup_wf_001::760a7568cbfdc750a5f92009a64bfc68"} +{"datasourceId":"10|opendoar____::4fa177df22864518b2d7818d4db5db2d","organizationId":"20|dedup_wf_001::b6b1fdfd98370bc44a5ecb215051fc61"} +{"datasourceId":"10|opendoar____::58521e4e2bd3d4b988cbd17d7365df3c","organizationId":"20|dedup_wf_001::09f3a5978add91bbfec7530e47c23fb6"} +{"datasourceId":"10|opendoar____::5dec707028b05bcbd3a1db5640f842c5","organizationId":"20|dedup_wf_001::7a34852b4b552a70789ccf070da2e558"} +{"datasourceId":"10|opendoar____::671d8d05d795091118b1344f715307c4","organizationId":"20|opendoar____::84e005eec7a780eafb0f5e54d438d56e"} +{"datasourceId":"10|opendoar____::79cc30c73507cfc25d20fe7f7bcfd91b","organizationId":"20|opendoar____::7230dc85992df9a4fa7026d44c9f5a24"} +{"datasourceId":"10|opendoar____::79e3eb7e992b7f766bdd77cc502ff082","organizationId":"20|dedup_wf_001::1e48cc6aee415348f0bdb3b7d3180091"} +{"datasourceId":"10|opendoar____::7af6266cc52234b5aa339b16695f7fc4","organizationId":"20|dedup_wf_001::cbfb7a433608c965dc59146c60e38dbf"} +{"datasourceId":"10|opendoar____::81e74d678581a3bb7a720b019f4f1a93","organizationId":"20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7"} +{"datasourceId":"10|opendoar____::8744cf92c88433f8cb04a02e6db69a0d","organizationId":"20|opendoar____::f800da58d6ad6d35fc9cf57b982bbd1a"} +{"datasourceId":"10|opendoar____::8c01a75941549a705cf7275e41b21f0d","organizationId":"20|dedup_wf_001::ae993d8ee228ff161c2a8724f11c2403"} +{"datasourceId":"10|opendoar____::9cb67ffb59554ab1dabb65bcb370ddd9","organizationId":"20|opendoar____::2404720b8a764a4f576240a51a6cdd13"} +{"datasourceId":"10|opendoar____::9fc3d7152ba9336a670e36d0ed79bc43","organizationId":"20|dedup_wf_001::dc3a6f93b2cc529cd247a255b5127525"} +{"datasourceId":"10|opendoar____::a655fbe4b8d7439994aa37ddad80de56","organizationId":"20|opendoar____::bba529dfcc369815510999ccadff78d0"} +{"datasourceId":"10|opendoar____::aebf7782a3d445f43cf30ee2c0d84dee","organizationId":"20|opendoar____::931aa8950244dea6b3fe9f7798b916ea"} +{"datasourceId":"10|opendoar____::c4819d06b0ca810d38506453cfaae9d8","organizationId":"20|dedup_wf_001::bee19d6bbb94675a7a4aaf2b66f94ae3"} +{"datasourceId":"10|opendoar____::d88518acbcc3d08d1f18da62f9bb26ec","organizationId":"20|dedup_wf_001::f1914dbbcc2861e30d7e6b0e605013fd"} +{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|dedup_wf_001::b19557d6f918479582295814b1dfb38b"} +{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|opendoar____::c9c886345622486e072060cb814b9ebf"} +{"datasourceId":"10|opendoar____::e9fb2eda3d9c55a0d89c98d6c54b5b3e","organizationId":"20|opendoar____::64cf1b2094ff19786a573980568762b4"} +{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::3f339760507dd23f9be59be2cff8204f"} +{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::76548cf94fcd27f67759cc6148ee1e7f"} +{"datasourceId":"10|opendoar____::f35a2bc72dfdc2aae569a0c7370bd7f5","organizationId":"20|dedup_wf_001::2476836f27326ee53e586a82cbaedb1a"} +{"datasourceId":"10|opendoar____::f91e24dfe80012e2a7984afa4480a6d6","organizationId":"20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e"} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json new file mode 100644 index 000000000..b585baf88 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json @@ -0,0 +1,20 @@ +{"resultId":"50|acm_________::3133635707788d2180bcef09e01a903c","organizationSet":["20|dedup_wf_001::5ab59ffa94c31a140d4a56c594ea5865"]} +{"resultId":"50|core________::0308a76f6f8bc4db75a817d53a7e76a4","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} +{"resultId":"50|core________::04c8f896aef9e54867f2bf4236e9c810","organizationSet":["20|snsf________::1496b1b4fc4d5509b16f2c217be480dc","20|dedup_wf_001::06731b587a9ea654103a6b0ebcb234ff","20|nih_________::c5722b087a5e707a50aa8f9f2ebf785d","20|snsf________::71d0a944b61b1a94068595f840005a2f","20|nih_________::dd3428794aef214a3bc2cad6dd548ba6","20|rcuk________::45aac2108e54b6503d1e611aa5872c03","20|nih_________::e1d47fdb7bba9eaeed82a95c578d6e90","20|dedup_wf_001::e3b52200d2fd4ff883478f5bef312efe","20|snsf________::c5c565d3422a7eb22886f3a4c93c32ea","20|nih_________::91154321f75ba26021efa18f7eeaa541","20|wt__________::38013971ca0c021fd65abce2171b82eb","20|wt__________::a6114989a56a1dfae6cbb201d14823f0","20|snsf________::b7af2f99e1e06750a4664ae401802734","20|wt__________::757c54e33d4e925c8c17edf032cdfacc","20|wt__________::1d57a87af1bbc2b7e211305fc747c9ad","20|wt__________::7cbb8c06f702b8871948acd370df892f","20|dedup_wf_001::53a8606f32787c4b3c663fd90ee97b17","20|wt__________::8473a929b793e56d2299a1a5aa08f617","20|nih_________::5e0fc2ef31bc189207f250db818fea0e","20|nih_________::1cd08fd26ef03fd2f51e9aeb34ed9486","20|nih_________::1c270e0dd2552b4e3cf295cdb7db1cc9","20|wt__________::04abd842647bdbc751b1eebe2f142013","20|nsf_________::3eaa8be3f16b0f0d7563b9117cd1f660","20|dedup_wf_001::c1b81dadf1e4cbf23a61833ff9ae8a31","20|nih_________::3716e1c14ab7ca14161278c9bbb8bdbb","20|dedup_wf_001::b7b403a764ea4e3acb12d999675aa73c","20|nih_________::300049f12fa0f5bc37db3a5636869743","20|wt__________::ed1e2be693353d370066fddbf862f23a","20|nsf_________::72a3747a18c56f3701494a0c0eadc5c9","20|rcuk________::e8877abcab4bc187339a242aa5bc2e09","20|microsoft___::119a535bfd240d7560fe4efec416bcd2","20|wt__________::be4e939abf9617557a35862e255493da","20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7","20|snsf________::1f4e34433767faf965f33849bb0f7fb1","20|dedup_wf_001::9f1647eae28911113d1dcbe49c42275b","20|opendoar____::6835febcf408fe892504bdfd2ebe669c","20|dedup_wf_001::528bd21573101493c6c7d120b17a67e9"]} +{"resultId":"50|core________::061dc912098a6a52e85a824161bd32a2","organizationSet":["20|dedup_wf_001::67f983a98e2c0cc0121e2db46d2bd00a","20|wt__________::59b5d99b2dde58df1655e0e5bb236c0a","20|wt__________::e84b06dbc1b26f413791c1304ca8d6a3","20|dedup_wf_001::7b118dab509f49b4fbd6dccfdbc479af","20|wt__________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::43255cf9c16732bc4ec1d5f580f44928","20|gsrt________::455b984b47945e1fd04e92c9c0eeca04","20|dedup_wf_001::e712c08f721e8f167f93888f590314ea","20|rcuk________::8b0aee2a7026dc92d4c05683ae45c894","20|dedup_wf_001::3c19a02cea18f8eeb2034b6acc544b7e","20|wt__________::26a9d64d00b8e1005cb6bbad2b7364cf","20|rcuk________::e57ab9be7acd7b93ba34aafab1a5b96c","20|nih_________::ca09277064edbd89f71e1602d98b4dd8","20|gsrt________::7c309ee758e6c6e0dce43c67b9343e82","20|nsf_________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::c57387345f51a40ad2284089b120be3f"]} +{"resultId":"50|core________::088190cf9dc9632e8d9ba5f5f1de1303","organizationSet":["20|dedup_wf_001::286621caef868bbdc52918699e3cdc79","20|dedup_wf_001::84707a912d45d666fef35c5cd77fc203","20|dedup_wf_001::30fd03f4977438b0471a1d4db6317e71","20|dedup_wf_001::83d9dc4a0eba8737819b87ce8e737a49","20|dedup_wf_001::aeb9e738d873acebdca52c9ccd9559bd","20|dedup_wf_001::427a8bfe68648f9e30cb47434144da68","20|nih_________::ffa0ea174845f1548e520a047cf53165"]} +{"resultId":"50|core________::117d295998199f498fa561e9c26e7ae3","organizationSet":["20|dedup_wf_001::d9f82e62c3114dc7f42b9da9b7f6fc64"]} +{"resultId":"50|core________::168a8f2e4ffe9b0e7c6bc100f34f2de5","organizationSet":["20|wt__________::63fe73f079c5ff4e925c6cfc1758a98b","20|dedup_wf_001::b3c2f5e700cee15ad9752ab961df5930"]} +{"resultId":"50|core________::16a3b520030d82ad16a30992b124e69d","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} +{"resultId":"50|core________::172e3da668d18f41ea4ccdf7f2f39e53","organizationSet":["20|nih_________::126cbf4b13249e65098ddb4835f47456","20|aka_________::506e3d2f7507a66584b8b3430ade20cb","20|dedup_wf_001::4746df4ff8bbb7e991ad343ccff8bbc7","20|wt__________::aff5133ca9cf0b810cc331d498bac9b0","20|wt__________::0c3bf892603817e5eff6e4f08a530ea2"]} +{"resultId":"50|core________::19f2fc91fe1db2ad62db598aa9aa8ab3","organizationSet":["20|dedup_wf_001::dbbd988f8d57a9d11286caefdf35acaa"]} +{"resultId":"50|core________::1dceb5a29cd42728e410474fe0fda191","organizationSet":["20|wt__________::b1ef2f643c948a2ef49005f9145ed556","20|dedup_wf_001::866fa622e3c0ab6227cd462f40cdcac8","20|rcuk________::63ecf5736189d299fc3e043e14428b8d","20|nsf_________::fcf880eab7315e0a5f3937c5a16c04b0","20|dedup_wf_001::65862ec7d57f700a130dee916bea66de"]} +{"resultId":"50|core________::2580c0b59b7457f571acdc829d1765a3","organizationSet":["20|doajarticles::0f6e2c32a27c307b06edf7862c591973","20|opendoar____::4f10fb61c457cf124e5917391baaa3c2"]} +{"resultId":"50|core________::2624b8248a9febdad9bc456d358b30ed","organizationSet":["20|dedup_wf_001::fb4eba3cea53264bddd59a4ade9973b3","20|rcuk________::b00968d2100a4b62447841aef5bdff62"]} +{"resultId":"50|core________::26820a282ef54882f7a5be74767fc02b","organizationSet":["20|rcuk________::8ad6d06f3b4d09dc67142c158c7cf5b9","20|rcuk________::01ad471b66687b1213ceb08b5d7aa6c2"]} +{"resultId":"50|core________::2a8de3e0bbcab49066aa9de4bbb89bfa","organizationSet":["20|dedup_wf_001::2ea78875d19c8cea63f7e958e5204136","20|corda_______::6821a8e260b8b97f5fb5e80168329d5b","20|dedup_wf_001::9d0ba437d73b19f55b53c578ac970ea2"]} +{"resultId":"50|core________::2c7d139419d2895d3bf0112b50108f75","organizationSet":["20|dedup_wf_001::96ada508ea5d85a1e516bf9799413906","20|dedup_wf_001::d0ea749da6988bcdb2f30d77c64e2f1e","20|wt__________::f1ba5bd552edf15db494dc3020f27470","20|nih_________::ceeae4f78a5666daf4c45acdbbedde99","20|wt__________::84ef588eeeb4ef77e45ccfbbf3aef69c","20|wt__________::8eef7e1370ea81c2aa3dbc239b2bf5d8"]} +{"resultId":"50|core________::2cf1f6282498fa37aeaf678f8c6e5843","organizationSet":["20|snsf________::73999c828ca67fd2d006100a8369c1eb"]} +{"resultId":"50|core________::2dffff00500c2354b506814f6a1ec148","organizationSet":["20|wt__________::c6d89e908582fddf3e4c658a458807c3","20|wt__________::e7b2c9f3d3f3f1503092bf1ba2b163db","20|gsrt________::ab510bb43d6c654ed3d37b9c5ed5c971","20|dedup_wf_001::179d0313fa7d5fb2bef5f312ecdd16fe","20|gsrt________::cbffb510b01e81cc055fe61105c86154","20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e","20|dedup_wf_001::646f14555ea42b260499239a7231b285","20|wt__________::5d462d78d512c1184dd384ef2dc35b7e","20|nih_________::a32a254b024265db2e24a66291c7c1e0","20|dedup_wf_001::5490ec18da9721e2c8d974fb73c62467","20|dedup_wf_001::3bc91ed90f44d0908258e132659bc754"]} +{"resultId":"50|core________::3031a50bf5c80865af4841ab42aaf57e","organizationSet":["20|nih_________::1b46e3665d8be2b524c285a27ca952b8","20|nsf_________::71450a4b98015592ee3f525a51584608","20|snsf________::fc921725875adb56f2275579b31f805c","20|aka_________::fa5b7357f86c71ea15734282054f1183","20|wt__________::18fdb5b42b22fdcc45e323eb4d20c91b","20|wt__________::71450a4b98015592ee3f525a51584608","20|dedup_wf_001::8aaf46d4e4919dc55b8a5cac7a15399f"]} +{"resultId":"50|core________::31116372ae189ee456fc06dfa0f6cf7a","organizationSet":["20|aka_________::c5b9aa0a905f89c51221f9f4fda22b20","20|aka_________::d9d3242062a7a3c483a7926fdba17bb6","20|nih_________::ede5c9e31cfb37a397d6cfe1940d045e","20|wt__________::8adcc12ffee195ae46679e8cf332a364","20|wt__________::5e954c57b0ac7aaf3fc16deeaf442389","20|snsf________::ddd964d550bfc6e1ce18f83655ba6901","20|rcuk________::a705d2ee7bf0bd225264b4a5794795ce","20|nih_________::8adcc12ffee195ae46679e8cf332a364","20|microsoft___::53732c6c7bb9daf5953fdb61fc0cd5bd"]} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json new file mode 100644 index 000000000..efc3af110 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json @@ -0,0 +1,28 @@ +{"datasourceId":"10|opendoar____::1068c6e4c8051cfd4e9ea8072e3189e2","organizationId":"20|opendoar____::4429502fa1936b0941f4647b69b844c8"} +{"datasourceId":"10|opendoar____::211ed78fe91938b90f84a51944b08d5a","organizationId":"20|opendoar____::124266ebc4ece2934eb80edfda3f2091"} +{"datasourceId":"10|opendoar____::21c5bba1dd6aed9ab48c2b34c1a0adde","organizationId":"20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2"} +{"datasourceId":"10|opendoar____::376c6b9ff3bedbbea56751a84fffc10c","organizationId":"20|opendoar____::c6ac1e6c39e6f3a36e274f8d3f9f7fce"} +{"datasourceId":"10|opendoar____::49c0fa7f96aa0a5fb95c62909d5190a6","organizationId":"20|dedup_wf_001::760a7568cbfdc750a5f92009a64bfc68"} +{"datasourceId":"10|opendoar____::4fa177df22864518b2d7818d4db5db2d","organizationId":"20|dedup_wf_001::b6b1fdfd98370bc44a5ecb215051fc61"} +{"datasourceId":"10|opendoar____::58521e4e2bd3d4b988cbd17d7365df3c","organizationId":"20|dedup_wf_001::09f3a5978add91bbfec7530e47c23fb6"} +{"datasourceId":"10|opendoar____::5dec707028b05bcbd3a1db5640f842c5","organizationId":"20|dedup_wf_001::7a34852b4b552a70789ccf070da2e558"} +{"datasourceId":"10|opendoar____::671d8d05d795091118b1344f715307c4","organizationId":"20|opendoar____::84e005eec7a780eafb0f5e54d438d56e"} +{"datasourceId":"10|opendoar____::79cc30c73507cfc25d20fe7f7bcfd91b","organizationId":"20|opendoar____::7230dc85992df9a4fa7026d44c9f5a24"} +{"datasourceId":"10|opendoar____::79e3eb7e992b7f766bdd77cc502ff082","organizationId":"20|dedup_wf_001::1e48cc6aee415348f0bdb3b7d3180091"} +{"datasourceId":"10|opendoar____::7af6266cc52234b5aa339b16695f7fc4","organizationId":"20|dedup_wf_001::cbfb7a433608c965dc59146c60e38dbf"} +{"datasourceId":"10|opendoar____::81e74d678581a3bb7a720b019f4f1a93","organizationId":"20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7"} +{"datasourceId":"10|opendoar____::8744cf92c88433f8cb04a02e6db69a0d","organizationId":"20|opendoar____::f800da58d6ad6d35fc9cf57b982bbd1a"} +{"datasourceId":"10|opendoar____::8c01a75941549a705cf7275e41b21f0d","organizationId":"20|dedup_wf_001::ae993d8ee228ff161c2a8724f11c2403"} +{"datasourceId":"10|opendoar____::9cb67ffb59554ab1dabb65bcb370ddd9","organizationId":"20|opendoar____::2404720b8a764a4f576240a51a6cdd13"} +{"datasourceId":"10|opendoar____::9fc3d7152ba9336a670e36d0ed79bc43","organizationId":"20|dedup_wf_001::dc3a6f93b2cc529cd247a255b5127525"} +{"datasourceId":"10|opendoar____::a655fbe4b8d7439994aa37ddad80de56","organizationId":"20|opendoar____::bba529dfcc369815510999ccadff78d0"} +{"datasourceId":"10|opendoar____::aebf7782a3d445f43cf30ee2c0d84dee","organizationId":"20|opendoar____::931aa8950244dea6b3fe9f7798b916ea"} +{"datasourceId":"10|opendoar____::c4819d06b0ca810d38506453cfaae9d8","organizationId":"20|dedup_wf_001::bee19d6bbb94675a7a4aaf2b66f94ae3"} +{"datasourceId":"10|opendoar____::d88518acbcc3d08d1f18da62f9bb26ec","organizationId":"20|dedup_wf_001::f1914dbbcc2861e30d7e6b0e605013fd"} +{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|dedup_wf_001::b19557d6f918479582295814b1dfb38b"} +{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|opendoar____::c9c886345622486e072060cb814b9ebf"} +{"datasourceId":"10|opendoar____::e9fb2eda3d9c55a0d89c98d6c54b5b3e","organizationId":"20|opendoar____::64cf1b2094ff19786a573980568762b4"} +{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::3f339760507dd23f9be59be2cff8204f"} +{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::76548cf94fcd27f67759cc6148ee1e7f"} +{"datasourceId":"10|opendoar____::f35a2bc72dfdc2aae569a0c7370bd7f5","organizationId":"20|dedup_wf_001::2476836f27326ee53e586a82cbaedb1a"} +{"datasourceId":"10|opendoar____::f91e24dfe80012e2a7984afa4480a6d6","organizationId":"20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e"} \ No newline at end of file From a8100baed6c7d2e2ecd22ef250d8465dec76f21b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:50:09 +0200 Subject: [PATCH 097/259] changed the way to save the results to aviod NegativeArray... error --- .../PrepareResultInstRepoAssociation.java | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index aa2861623..bb4dfe429 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -6,8 +6,10 @@ import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.Organization; import eu.dnetlib.dhp.schema.oaf.Relation; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -50,12 +52,34 @@ public class PrepareResultInstRepoAssociation { runWithSparkHiveSession(conf, isSparkSessionManaged, spark -> { - prepareAssociations(spark, inputPath, datasourceOrganizationPath, alreadyLinkedPath); + readNeededResources(spark, inputPath); + prepareDatasourceOrganizationAssociations(spark, datasourceOrganizationPath, alreadyLinkedPath); + prepareAlreadyLinkedAssociation(spark, alreadyLinkedPath); }); } - private static void prepareAssociations(SparkSession spark, String inputPath, String datasourceOrganizationPath, - String alreadyLinkedPath){ + private static void prepareAlreadyLinkedAssociation(SparkSession spark, String alreadyLinkedPath) { + String query = "Select source resultId, collect_set(target) organizationSet " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + + "group by source"; + + + spark.sql(query) + .as(Encoders.bean(ResultOrganizationSet.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); +// .as(Encoders.bean(ResultOrganizationSet.class)) +// .toJSON() +// .write() +// .mode(SaveMode.Overwrite) +// .option("compression","gzip") +// .text(alreadyLinkedPath); + } + + private static void readNeededResources(SparkSession spark, String inputPath) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") @@ -70,6 +94,11 @@ public class PrepareResultInstRepoAssociation { datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); + } + + private static void prepareDatasourceOrganizationAssociations(SparkSession spark, String datasourceOrganizationPath, + String alreadyLinkedPath){ + String query = "SELECT source datasourceId, target organizationId " + "FROM ( SELECT id " + @@ -90,19 +119,6 @@ public class PrepareResultInstRepoAssociation { .option("compression","gzip") .text(datasourceOrganizationPath); - query = "Select source, collect_set(target) organizationSet " + - "from relation " + - "where datainfo.deletedbyinference = false " + - "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + - "group by source"; - - spark.sql(query) - .as(Encoders.bean(ResultOrganizationSet.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(alreadyLinkedPath); } From c28333d43f7528a039450acd2898570c60f74481 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:52:50 +0200 Subject: [PATCH 098/259] minor --- .../PrepareResultInstRepoAssociation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index bb4dfe429..9fbe7a814 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -64,7 +64,7 @@ public class PrepareResultInstRepoAssociation { "where datainfo.deletedbyinference = false " + "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + "group by source"; - + spark.sql(query) .as(Encoders.bean(ResultOrganizationSet.class)) From 53f418098b19a25e8a57f74ece7530baa03af605 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:53:48 +0200 Subject: [PATCH 099/259] added the isTest checkpoint --- ...rkResultToOrganizationFromIstRepoJob2.java | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index 7826f598b..c4efaefba 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -8,6 +8,7 @@ import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -58,9 +59,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { final String alreadylinked = parser.get("alreadyLinkedPath"); log.info("alreadyLinkedPath: {}", alreadylinked); - final String resultorganizationsetpath = parser.get("resultOrganizationsetPath"); - log.info("resultOrganizationsetPath: {}", resultorganizationsetpath); - final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); @@ -93,9 +91,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { writeUpdates, saveGraph); }); - - - } private static void execPropagation(SparkSession spark, String datasourceorganization, String alreadylinked, String inputPath, @@ -122,17 +117,20 @@ public class SparkResultToOrganizationFromIstRepoJob2 { .read() .textFile(alreadylinked) .map(value -> OBJECT_MAPPER.readValue(value, ResultOrganizationSet.class), - Encoders.bean(ResultOrganizationSet.class)), potentialUpdates); + Encoders.bean(ResultOrganizationSet.class)), potentialUpdates) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath , GzipCodec.class); } } private static Dataset getNewRelations(Dataset alreadyLinked, Dataset potentialUpdates) { - return potentialUpdates - .joinWith(alreadyLinked, potentialUpdates.col("resultId") - .equalTo(alreadyLinked.col("resultId")), "left_outer") - .flatMap((FlatMapFunction, Relation>) value -> { + + return potentialUpdates + .joinWith(alreadyLinked, potentialUpdates.col("resultId") + .equalTo(alreadyLinked.col("resultId")), "left_outer").flatMap((FlatMapFunction, Relation>) value -> { List new_relations = new ArrayList<>(); ResultOrganizationSet potential_update = value._1(); Optional already_linked = Optional.ofNullable(value._2()); From 7bd49abbef2acfb26bcb6aacde1d924816b9c4c0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:59:09 +0200 Subject: [PATCH 100/259] commit to delete --- ...arkResultToOrganizationFromIstRepoJob.java | 295 +++++++++++++++++- 1 file changed, 291 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java index 8d47bdd90..cdae25e85 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -14,9 +14,7 @@ import org.apache.spark.sql.SparkSession; import org.apache.hadoop.io.Text; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; +import java.util.*; import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; @@ -50,7 +48,7 @@ public class SparkResultToOrganizationFromIstRepoJob { org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") + org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); datasource.createOrReplaceTempView("datasource"); @@ -102,8 +100,66 @@ public class SparkResultToOrganizationFromIstRepoJob { writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); + + query = "Select source resultId, collect_set(target) org_list " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + + "group by source"; + + JavaRDD result_orglist = spark.sql(query).toJavaRDD(); + + JavaPairRDD> toupdateunion = toupdateresultdataset.mapToPair(d -> new Tuple2<>(d.getString(0), d.getList(1))) + .union(toupdateresultother.mapToPair(o -> new Tuple2<>(o.getString(0), o.getList(1)))) + .union(toupdateresultpublication.mapToPair(p -> new Tuple2<>(p.getString(0), p.getList(1)))) + .union(toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))); + + JavaRDD new_rels = getNewRels(result_orglist.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))), + toupdateunion); + + + + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .union(new_rels) + .map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/relation"); + + } + private static JavaRDD getNewRels(JavaPairRDD> relationOrgs, JavaPairRDD > newRels){ + return newRels + .leftOuterJoin(relationOrgs) + .flatMap(c -> { + List toAddOrgs = new ArrayList<>(); + toAddOrgs.addAll(c._2()._1()); + if (c._2()._2().isPresent()) { + Set originalOrgs = new HashSet<>(); + originalOrgs.addAll(c._2()._2().get()); + for (Object oId : originalOrgs) { + if (toAddOrgs.contains(oId)) { + toAddOrgs.remove(oId); + } + } + } + List relationList = new ArrayList<>(); + String resId = c._1(); + for (Object org : toAddOrgs) { + relationList.add(getRelation((String)org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList.add(getRelation(resId, (String)org, RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + + } + return relationList.iterator(); + }); + + } + + private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); @@ -160,3 +216,234 @@ public class SparkResultToOrganizationFromIstRepoJob { } } + +//package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +// +//import eu.dnetlib.dhp.TypedRow; +//import eu.dnetlib.dhp.application.ArgumentApplicationParser; +//import org.apache.commons.io.IOUtils; +//import org.apache.hadoop.fs.FileSystem; +//import org.apache.spark.SparkConf; +//import org.apache.spark.api.java.JavaPairRDD; +//import org.apache.spark.api.java.JavaRDD; +//import org.apache.spark.api.java.JavaSparkContext; +//import org.apache.spark.sql.Encoders; +//import org.apache.spark.sql.Row; +//import org.apache.spark.sql.SparkSession; +//import org.apache.hadoop.io.Text; +//import com.fasterxml.jackson.databind.ObjectMapper; +//import java.io.File; +//import java.util.ArrayList; +//import java.util.Arrays; +//import java.util.List; +//import java.util.Set; +// +//import eu.dnetlib.dhp.schema.oaf.*; +//import scala.Tuple2; +// +//import static eu.dnetlib.dhp.PropagationConstant.*; +// +//public class SparkResultToOrganizationFromIstRepoJob { +// public static void main(String[] args) throws Exception { +// +// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); +// parser.parseArgument(args); +// SparkConf conf = new SparkConf(); +// conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); +// final SparkSession spark = SparkSession +// .builder() +// .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) +// .master(parser.get("master")) +// .config(conf) +// .enableHiveSupport() +// .getOrCreate(); +// +// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); +// final String inputPath = parser.get("sourcePath"); +// final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; +// +// createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); +// +// org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") +// .map(item -> new ObjectMapper().readValue(item, Datasource.class)) +// .rdd(), Encoders.bean(Datasource.class)); +// +// JavaRDD relation_rdd_all = sc.textFile(inputPath + "/relation") +// .map(item -> new ObjectMapper().readValue(item, Relation.class)); +// JavaRDD relation_rdd = relation_rdd_all.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); +// +// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); +// +// org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") +// .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); +// +// datasource.createOrReplaceTempView("datasource"); +// relation.createOrReplaceTempView("relation"); +// organization.createOrReplaceTempView("organization"); +// +// String query = "SELECT source ds, target org " + +// "FROM ( SELECT id " + +// "FROM datasource " + +// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + +// "AND datainfo.deletedbyinference = false ) d " + +// "JOIN ( SELECT source, target " + +// "FROM relation " + +// "WHERE relclass = 'provides' " + +// "AND datainfo.deletedbyinference = false ) rel " + +// "ON d.id = rel.source "; +// +// org.apache.spark.sql.Dataset rels = spark.sql(query); +// rels.createOrReplaceTempView("rels"); +// +// org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); +// +// org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); +// +// org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); +// +// org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") +// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), +// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); +// +// +// software.createOrReplaceTempView("software"); +// final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); +// +// dataset.createOrReplaceTempView("dataset"); +// final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); +// +// other.createOrReplaceTempView("other"); +// final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); +// +// publication.createOrReplaceTempView("publication"); +// final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); +// +// writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); +// +// JavaPairRDD relation_rdd_pair = relation_rdd +// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getSource()); +// tp.add(r.getTarget()); +// return tp; +// }).mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) { +// return b; +// } +// if (b == null) { +// return a; +// } +// +// a.addAll(b.getAccumulator()); +// return a; +// }).cache(); +// +// +// JavaRDD new_rels = getNewRels(relation_rdd_pair, +// toupdateresultother.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultdataset.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) +// .union(getNewRels(relation_rdd_pair, +// toupdateresultpublication.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))); +// +// +// relation_rdd_all.union(new_rels).map(r -> new ObjectMapper().writeValueAsString(r)) +// .saveAsTextFile(outputPath + "/relation"); +// +// } +// +// private static JavaRDD getNewRels(JavaPairRDD relation_rdd_pair, JavaPairRDD > newRels){ +// return newRels//.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))) +// .leftOuterJoin(relation_rdd_pair) +// .flatMap(c -> { +// List toAddOrgs = c._2()._1(); +// if (c._2()._2().isPresent()) { +// Set originalOrgs = c._2()._2().get().getAccumulator(); +// for (String oId : toAddOrgs) { +// if (originalOrgs.contains(oId)) { +// toAddOrgs.remove(oId); +// } +// } +// } +// List relationList = new ArrayList<>(); +// String resId = c._1(); +// for (String org : toAddOrgs) { +// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// +// } +// return relationList.iterator(); +// }); +// +// } +// +// private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { +// createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); +// createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); +// createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); +// createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); +// +// } +// +// private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { +// toupdaterelation.flatMap(s -> { +// List relationList = new ArrayList<>(); +// List orgs = s.getList(1); +// String resId = s.getString(0); +// for (String org : orgs) { +// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, +// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, +// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); +// +// } +// return relationList.iterator(); +// }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); +// } +// +// private static JavaRDD propagateOnResult(SparkSession spark, String table) { +// String query; +// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + +// "FROM ( SELECT id, instance " + +// "FROM " + table + +// " WHERE datainfo.deletedbyinference = false) ds " + +// "LATERAL VIEW EXPLODE(instance) i AS inst"; +// org.apache.spark.sql.Dataset cfhb = spark.sql(query); +// cfhb.createOrReplaceTempView("cfhb"); +// +// return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); +// +// } +// +// private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ +// String query = "SELECT id, collect_set(org) org "+ +// "FROM ( SELECT id, org " + +// "FROM rels " + +// "JOIN " + cfhbTable + +// " ON cf = ds " + +// "UNION ALL " + +// "SELECT id , org " + +// "FROM rels " + +// "JOIN " + cfhbTable + +// " ON hb = ds ) tmp " + +// "GROUP BY id"; +// return spark.sql(query); +// } +// +//} From eaf19ce01b7d385f5afc8bf73e90dd9981407d29 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 10:59:33 +0200 Subject: [PATCH 101/259] removed unuseful class --- ...arkResultToOrganizationFromIstRepoJob.java | 449 ------------------ 1 file changed, 449 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java deleted file mode 100644 index cdae25e85..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java +++ /dev/null @@ -1,449 +0,0 @@ -package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; - -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.apache.hadoop.io.Text; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.File; -import java.util.*; - -import eu.dnetlib.dhp.schema.oaf.*; -import scala.Tuple2; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkResultToOrganizationFromIstRepoJob { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); - parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); - - datasource.createOrReplaceTempView("datasource"); - relation.createOrReplaceTempView("relation"); - organization.createOrReplaceTempView("organization"); - - String query = "SELECT source ds, target org " + - "FROM ( SELECT id " + - "FROM datasource " + - "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + - "AND datainfo.deletedbyinference = false ) d " + - "JOIN ( SELECT source, target " + - "FROM relation " + - "WHERE relclass = 'provides' " + - "AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source "; - - org.apache.spark.sql.Dataset rels = spark.sql(query); - rels.createOrReplaceTempView("rels"); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); - - org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); - - - software.createOrReplaceTempView("software"); - final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); - - dataset.createOrReplaceTempView("dataset"); - final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); - - other.createOrReplaceTempView("other"); - final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); - - publication.createOrReplaceTempView("publication"); - final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); - - writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); - - - query = "Select source resultId, collect_set(target) org_list " + - "from relation " + - "where datainfo.deletedbyinference = false " + - "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + - "group by source"; - - JavaRDD result_orglist = spark.sql(query).toJavaRDD(); - - JavaPairRDD> toupdateunion = toupdateresultdataset.mapToPair(d -> new Tuple2<>(d.getString(0), d.getList(1))) - .union(toupdateresultother.mapToPair(o -> new Tuple2<>(o.getString(0), o.getList(1)))) - .union(toupdateresultpublication.mapToPair(p -> new Tuple2<>(p.getString(0), p.getList(1)))) - .union(toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))); - - JavaRDD new_rels = getNewRels(result_orglist.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))), - toupdateunion); - - - - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .union(new_rels) - .map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/relation"); - - - } - - private static JavaRDD getNewRels(JavaPairRDD> relationOrgs, JavaPairRDD > newRels){ - return newRels - .leftOuterJoin(relationOrgs) - .flatMap(c -> { - List toAddOrgs = new ArrayList<>(); - toAddOrgs.addAll(c._2()._1()); - if (c._2()._2().isPresent()) { - Set originalOrgs = new HashSet<>(); - originalOrgs.addAll(c._2()._2().get()); - for (Object oId : originalOrgs) { - if (toAddOrgs.contains(oId)) { - toAddOrgs.remove(oId); - } - } - } - List relationList = new ArrayList<>(); - String resId = c._1(); - for (Object org : toAddOrgs) { - relationList.add(getRelation((String)org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList.add(getRelation(resId, (String)org, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - - } - return relationList.iterator(); - }); - - } - - - private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { - createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); - createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); - createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); - createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); - - } - - private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { - toupdaterelation.flatMap(s -> { - List relationList = new ArrayList<>(); - List orgs = s.getList(1); - String resId = s.getString(0); - for (String org : orgs) { - relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - - } - return relationList.iterator(); - }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); - } - - private static JavaRDD propagateOnResult(SparkSession spark, String table) { - String query; - query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + - "FROM ( SELECT id, instance " + - "FROM " + table + - " WHERE datainfo.deletedbyinference = false) ds " + - "LATERAL VIEW EXPLODE(instance) i AS inst"; - org.apache.spark.sql.Dataset cfhb = spark.sql(query); - cfhb.createOrReplaceTempView("cfhb"); - - return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); - - } - - private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ - String query = "SELECT id, collect_set(org) org "+ - "FROM ( SELECT id, org " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON cf = ds " + - "UNION ALL " + - "SELECT id , org " + - "FROM rels " + - "JOIN " + cfhbTable + - " ON hb = ds ) tmp " + - "GROUP BY id"; - return spark.sql(query); - } - -} - -//package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -// -//import eu.dnetlib.dhp.TypedRow; -//import eu.dnetlib.dhp.application.ArgumentApplicationParser; -//import org.apache.commons.io.IOUtils; -//import org.apache.hadoop.fs.FileSystem; -//import org.apache.spark.SparkConf; -//import org.apache.spark.api.java.JavaPairRDD; -//import org.apache.spark.api.java.JavaRDD; -//import org.apache.spark.api.java.JavaSparkContext; -//import org.apache.spark.sql.Encoders; -//import org.apache.spark.sql.Row; -//import org.apache.spark.sql.SparkSession; -//import org.apache.hadoop.io.Text; -//import com.fasterxml.jackson.databind.ObjectMapper; -//import java.io.File; -//import java.util.ArrayList; -//import java.util.Arrays; -//import java.util.List; -//import java.util.Set; -// -//import eu.dnetlib.dhp.schema.oaf.*; -//import scala.Tuple2; -// -//import static eu.dnetlib.dhp.PropagationConstant.*; -// -//public class SparkResultToOrganizationFromIstRepoJob { -// public static void main(String[] args) throws Exception { -// -// final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToOrganizationFromIstRepoJob.class.getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json"))); -// parser.parseArgument(args); -// SparkConf conf = new SparkConf(); -// conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); -// final SparkSession spark = SparkSession -// .builder() -// .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) -// .master(parser.get("master")) -// .config(conf) -// .enableHiveSupport() -// .getOrCreate(); -// -// final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); -// final String inputPath = parser.get("sourcePath"); -// final String outputPath = "/tmp/provision/propagation/resulttoorganizationfrominstitutionalrepositories"; -// -// createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); -// -// org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") -// .map(item -> new ObjectMapper().readValue(item, Datasource.class)) -// .rdd(), Encoders.bean(Datasource.class)); -// -// JavaRDD relation_rdd_all = sc.textFile(inputPath + "/relation") -// .map(item -> new ObjectMapper().readValue(item, Relation.class)); -// JavaRDD relation_rdd = relation_rdd_all.filter(r -> !r.getDataInfo().getDeletedbyinference()).cache(); -// -// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); -// -// org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organziation") -// .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); -// -// datasource.createOrReplaceTempView("datasource"); -// relation.createOrReplaceTempView("relation"); -// organization.createOrReplaceTempView("organization"); -// -// String query = "SELECT source ds, target org " + -// "FROM ( SELECT id " + -// "FROM datasource " + -// "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + -// "AND datainfo.deletedbyinference = false ) d " + -// "JOIN ( SELECT source, target " + -// "FROM relation " + -// "WHERE relclass = 'provides' " + -// "AND datainfo.deletedbyinference = false ) rel " + -// "ON d.id = rel.source "; -// -// org.apache.spark.sql.Dataset rels = spark.sql(query); -// rels.createOrReplaceTempView("rels"); -// -// org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); -// -// org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); -// -// org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); -// -// org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") -// .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), -// Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); -// -// -// software.createOrReplaceTempView("software"); -// final JavaRDD toupdateresultsoftware = propagateOnResult(spark, "software"); -// -// dataset.createOrReplaceTempView("dataset"); -// final JavaRDD toupdateresultdataset = propagateOnResult(spark, "dataset"); -// -// other.createOrReplaceTempView("other"); -// final JavaRDD toupdateresultother = propagateOnResult(spark, "other"); -// -// publication.createOrReplaceTempView("publication"); -// final JavaRDD toupdateresultpublication = propagateOnResult(spark, "publication"); -// -// writeUpdates(toupdateresultsoftware, toupdateresultdataset, toupdateresultother, toupdateresultpublication, outputPath); -// -// JavaPairRDD relation_rdd_pair = relation_rdd -// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getSource()); -// tp.add(r.getTarget()); -// return tp; -// }).mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) { -// return b; -// } -// if (b == null) { -// return a; -// } -// -// a.addAll(b.getAccumulator()); -// return a; -// }).cache(); -// -// -// JavaRDD new_rels = getNewRels(relation_rdd_pair, -// toupdateresultother.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1)))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultsoftware.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultdataset.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))) -// .union(getNewRels(relation_rdd_pair, -// toupdateresultpublication.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))))); -// -// -// relation_rdd_all.union(new_rels).map(r -> new ObjectMapper().writeValueAsString(r)) -// .saveAsTextFile(outputPath + "/relation"); -// -// } -// -// private static JavaRDD getNewRels(JavaPairRDD relation_rdd_pair, JavaPairRDD > newRels){ -// return newRels//.mapToPair(s -> new Tuple2<>(s.getString(0), s.getList(1))) -// .leftOuterJoin(relation_rdd_pair) -// .flatMap(c -> { -// List toAddOrgs = c._2()._1(); -// if (c._2()._2().isPresent()) { -// Set originalOrgs = c._2()._2().get().getAccumulator(); -// for (String oId : toAddOrgs) { -// if (originalOrgs.contains(oId)) { -// toAddOrgs.remove(oId); -// } -// } -// } -// List relationList = new ArrayList<>(); -// String resId = c._1(); -// for (String org : toAddOrgs) { -// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// -// } -// return relationList.iterator(); -// }); -// -// } -// -// private static void writeUpdates(JavaRDD toupdateresultsoftware, JavaRDD toupdateresultdataset, JavaRDD toupdateresultother, JavaRDD toupdateresultpublication, String outputPath) { -// createUpdateForRelationWrite(toupdateresultsoftware, outputPath, "update_software"); -// createUpdateForRelationWrite(toupdateresultdataset, outputPath, "update_dataset"); -// createUpdateForRelationWrite(toupdateresultother, outputPath, "update_other"); -// createUpdateForRelationWrite(toupdateresultpublication, outputPath, "update_publication"); -// -// } -// -// private static void createUpdateForRelationWrite(JavaRDD toupdaterelation, String outputPath, String update_type) { -// toupdaterelation.flatMap(s -> { -// List relationList = new ArrayList<>(); -// List orgs = s.getList(1); -// String resId = s.getString(0); -// for (String org : orgs) { -// relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, -// RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, -// PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); -// -// } -// return relationList.iterator(); -// }).map(s -> new ObjectMapper().writeValueAsString(s)).saveAsTextFile(outputPath + "/" + update_type); -// } -// -// private static JavaRDD propagateOnResult(SparkSession spark, String table) { -// String query; -// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + -// "FROM ( SELECT id, instance " + -// "FROM " + table + -// " WHERE datainfo.deletedbyinference = false) ds " + -// "LATERAL VIEW EXPLODE(instance) i AS inst"; -// org.apache.spark.sql.Dataset cfhb = spark.sql(query); -// cfhb.createOrReplaceTempView("cfhb"); -// -// return organizationPropagationAssoc(spark, "cfhb").toJavaRDD(); -// -// } -// -// private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, String cfhbTable){ -// String query = "SELECT id, collect_set(org) org "+ -// "FROM ( SELECT id, org " + -// "FROM rels " + -// "JOIN " + cfhbTable + -// " ON cf = ds " + -// "UNION ALL " + -// "SELECT id , org " + -// "FROM rels " + -// "JOIN " + cfhbTable + -// " ON hb = ds ) tmp " + -// "GROUP BY id"; -// return spark.sql(query); -// } -// -//} From 5e1bd676801a2222df5cfd1dc4d526f1404ebf34 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 11:02:01 +0200 Subject: [PATCH 102/259] removed unuseful parameter --- .../input_projecttoresult_parameters.json | 6 ---- .../projecttoresult/oozie_app/workflow.xml | 36 +------------------ 2 files changed, 1 insertion(+), 41 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json index 9d5354c64..af52c4e5b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"s", - "paramLongName":"sourcePath", - "paramDescription": "the path of the sequencial file to read", - "paramRequired": true - }, { "paramName":"h", "paramLongName":"hive_metastore_uris", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 1d15391ab..d8df18c58 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -31,7 +31,6 @@ - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -98,7 +97,7 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${sourcePath}/relation + --writeUpdate${writeUpdate} --saveGraph${saveGraph} --hive_metastore_uris${hive_metastore_uris} @@ -110,38 +109,5 @@ - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - ProjectToResultPropagation - eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob2 - dhp-propagation-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --sourcePath${sourcePath} - --allowedsemrels${allowedsemrels} - --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} - --saveGraph${saveGraph} - --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates - --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked - - - - - \ No newline at end of file From a5e5c81a2cd47e56eaf070c95c4f1830be433a2f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 11:03:41 +0200 Subject: [PATCH 103/259] input parameters and workflow definition for propagation of result to organization from institutional repositories --- .../input_prepareresultorg_parameters.json | 12 - ...sulaffiliationfrominstrepo_parameters.json | 54 +++- .../oozie_app/config-default.xml | 36 +++ .../oozie_app/workflow.xml | 246 +++++++++++++++--- 4 files changed, 297 insertions(+), 51 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json index 8d2133075..c74496350 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json @@ -11,18 +11,6 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", - "paramRequired": true - }, - { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true - }, { "paramName":"dop", "paramLongName":"datasourceOrganizationPath", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json index 215f69fcc..9da25874a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -16,5 +10,53 @@ "paramLongName":"hive_metastore_uris", "paramDescription": "the hive metastore uris", "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + }, + { + "paramName":"dop", + "paramLongName":"datasourceOrganizationPath", + "paramDescription": "path where to store/find association from datasource and organization", + "paramRequired": true + }, + { + "paramName":"alp", + "paramLongName":"alreadyLinkedPath", + "paramDescription": "path where to store/find already linked results and organizations", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "test", + "paramLongName": "isTest", + "paramDescription": "true if it is a test running", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml index ea3a4d922..2744ea92b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml @@ -19,4 +19,40 @@ hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 08a07f497..19e7e6507 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -5,47 +5,227 @@ the source path - sparkDriverMemory - memory for driver process + writeUpdate + writes the information found for the update. No double check done if the information is already present - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor + saveGraph + writes new version of the graph after the propagation step - - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - AffiliationPropagation - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${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" - - -mt yarn-cluster - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - - - - + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/resulttoorganization_propagation/relation + + + + + + + + yarn + cluster + PrepareResultOrganizationAssociation + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.PrepareResultInstRepoAssociation + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + + + + + + + + + + + + + + + yarn + cluster + resultToOrganizationFromInstRepoPropagationForPublications + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/publication + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/resulttoorganization_propagation/relation + --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + + + + + + + + yarn + cluster + resultToOrganizationFromInstRepoPropagationForDataset + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/dataset + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/resulttoorganization_propagation/relation + --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + + + + + + + + yarn + cluster + resultToOrganizationFromInstRepoPropagationForORP + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/otherresearchproduct + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/resulttoorganization_propagation/relation + --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + + + + + + + + yarn + cluster + resultToOrganizationFromInstRepoPropagationForSoftware + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/resulttoorganization_propagation/relation + --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - \ No newline at end of file From b078710924b30265c289493f17d59d91ea027a01 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 11:04:39 +0200 Subject: [PATCH 104/259] modification to the test due to the removal of unused parameters --- .../dhp/projecttoresult/ProjectPropagationJobTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index 15cb42b1a..521e12c11 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -72,7 +72,7 @@ public class ProjectPropagationJobTest { SparkResultToProjectThroughSemRelJob3.main(new String[]{ "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), +// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), "-hive_metastore_uris", "", "-writeUpdate", "false", "-saveGraph", "true", @@ -100,7 +100,7 @@ public class ProjectPropagationJobTest { SparkResultToProjectThroughSemRelJob3.main(new String[]{ "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), +// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), "-hive_metastore_uris", "", "-writeUpdate", "false", "-saveGraph", "true", @@ -144,7 +144,7 @@ public class ProjectPropagationJobTest { SparkResultToProjectThroughSemRelJob3.main(new String[]{ "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/sample/relation").getPath(), +// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), "-hive_metastore_uris", "", "-writeUpdate", "false", "-saveGraph", "true", From a97e915c24f4943af5fa1323d30cc10778384d90 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 11:05:21 +0200 Subject: [PATCH 105/259] test unit for propagation of result to organization from institutional repository --- .../Result2OrganizationJobTest.java | 169 ++++++++++++++++++ 1 file changed, 169 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index bc2142314..61366b6d7 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -1,4 +1,173 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + public class Result2OrganizationJobTest { + + private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + + spark = SparkSession + .builder() + .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) + .config(conf) + .getOrCreate(); + + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + /** + * No modifications done to the sample sets, so that no possible updates are created + * @throws Exception + */ + @Test + public void NoUpdateTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix").getPath(), + "-hive_metastore_uris", "", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(0, tmp.count()); + + + } + + /** + * Testing set with modified association between datasource and organization. Copied some hostedby collectedfrom + * from the software sample set. No intersection with the already linked (all the possible new relations, will became + * new relations) + * @throws Exception + */ + @Test + public void UpdateNoMixTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix").getPath(), + "-hive_metastore_uris", "", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(20, tmp.count()); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Assertions.assertEquals(8, verificationDs.filter("target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2'").count()); + Assertions.assertEquals(1, verificationDs.filter("target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091'").count()); + Assertions.assertEquals(1, verificationDs.filter("target = '20|opendoar____::4429502fa1936b0941f4647b69b844c8'").count()); + + Assertions.assertEquals(2, verificationDs.filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and " + + "(target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091' " + + "or target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2')").count()); + } + + @Test + public void UpdateMixTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix").getPath(), + "-hive_metastore_uris", "", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", "false", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization").getPath(), + "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + + Assertions.assertEquals(8, verificationDs.count()); + + Assertions.assertEquals(2, verificationDs.filter("source = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6'").count()); + Assertions.assertEquals(1, verificationDs.filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218'").count()); + Assertions.assertEquals(1, verificationDs.filter("source = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523'").count()); + + Assertions.assertEquals(1, verificationDs.filter("source = '20|wt__________::a72760363ca885e6bef165804770e00c'").count()); + + Assertions.assertEquals(4, verificationDs.filter("relclass = 'hasAuthorInstitution' and substring(source, 1,2) = '50'").count()); + Assertions.assertEquals(4, verificationDs.filter("relclass = 'isAuthorInstitutionOf' and substring(source, 1,2) = '20'").count()); + + Assertions.assertEquals(4, verificationDs.filter("relclass = 'hasAuthorInstitution' and " + + "substring(source, 1,2) = '50' and substring(target, 1, 2) = '20'").count()); + Assertions.assertEquals(4, verificationDs.filter("relclass = 'isAuthorInstitutionOf' and " + + "substring(source, 1,2) = '20' and substring(target, 1, 2) = '50'").count()); + } + + } From 08227cfcbd889ec57b8df766032702656c5319d5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 11:06:10 +0200 Subject: [PATCH 106/259] resources needed for running the test on propagation of result to organization from institutional repositories --- .../alreadyLinked/alreadyLinked_20.json.gz | Bin 0 -> 3160 bytes .../datasourceOrganization_28.json.gz | Bin 0 -> 1269 bytes .../noupdate_updatenomix/software_10.json.gz | Bin 0 -> 6906 bytes .../sample/updatemix/software_10.json.gz | Bin 0 -> 7002 bytes .../alreadyLinked/alreadyLinked_20.json | 20 ------------- .../alreadyLinked/alreadyLinked_20.json.gz | Bin 0 -> 3175 bytes .../datasourceOrganization_28.json | 28 ------------------ .../datasourceOrganization_28.json.gz | Bin 0 -> 1285 bytes .../alreadyLinked/alreadyLinked_20.json | 20 ------------- .../alreadyLinked/alreadyLinked_20.json.gz | Bin 0 -> 3160 bytes .../datasourceOrganization_28.json | 28 ------------------ .../datasourceOrganization_28.json.gz | Bin 0 -> 1259 bytes 12 files changed, 96 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..ee822e372a4a590bd303308b2cc6fd849bf1bbfd GIT binary patch literal 3160 zcmV-e45#xSiwFo+9+zGK17U1(WnpA_OlfXwWn^D6FfM9yZ*BnHnaPeFwh=}5`3$YI zAgeg)53Lbd;910 z@zv|Uypr(CT=T~tzWHf?6GD9bI-BRdxAinMT1;Uk%Vp%AmgQFRtH1v14?n*+$awdD zeK|?MQcK`U!&=L$_}a1L7E-NYg>de} znzwKM_PaHC)E!zMu~xU#cbA~Lt1Pk2Z|P~x^9zsGe7ClOrMXOMG+nZr%IsN0+XEL) zqwUs3j6;SX;=>@Ob^Q!gGpZHkt2=pM_KA+h>OEzEH{d?;0; z=T@yRD>=oWRy-^pE}Hhf6=~!u;(;}JffmcbtnI#K`K~R`r!~*dR^tpnB6*1poKh$) z_qrh#CJ6n%2CzZHsc9Rk$u5GdHx)ygz@oCL-;A;y}dyT`wV8Nb+nH&os zL~k_dw#Fz@>a8=X7U#BAhP#9~V``#1-0tJmw(Qln8+apr2EZAS+IUb#uskoTymj&Ae1565pKlwE^(Yc=-NY)q|T zGHIXj_JX<8T49MwZFMBqQHt)_!&<$1thaqys%l-j*RCs$0$JYvP!E;FJO*b0*n2u5QaUt;yBOeT^KZn^STYr|4q~um7!=GbuEX`Vge4I=jRIy4&H!b^QF+MhaG0kMIGSQG2<@fIzIy_ z#%XiMm4%R>c$u zGtThBW(^aV&de^Ga7DlI+XBm#X*N#>JXN)uB!LSUjTL@f^ZlFg{@sUn`{NT1B4e%D zfld0GeqXrf9G39n2Bx7|x+jj@WnE_P98y(C!b)9yrwW8Rup4z6G(6EBk&W(LQ z#OQLpaY)>BB_-9$0%NnhRa>H`Vb|^PwhyfVo)EP(PzD#wI0pyr17?LMpVqwlVZEL2 z?ETX|rD@)%+D?qoV(4;6KB)-UE~Q_!F$=Jgm{G{n8t7ou86`jkR_f{x2z%jVoh7|l zu`H`$W#j8(f*V{8-87y71cX*&Wv268%5?Y6IHQ-qx6@t{W&;vu+TI>NR(W-*)YO}lXaXZch*!97olvax1u@)5H z++l3#5rOFF>>RbGaI4H#MS4J5B4A)xrx(RK z)hk0bo!kyBS&nhi%7qhVk$W~(a*J1O*RSV>_lSh@*7ZI~Jd|o>K0_f*oow$8x zzrS!OY%PR1#=lUWgkyylp|2&(%yqW#P;;MbiND2d^k_s!MOx3ljB6U9d^swu=wqW1 z5woBQSpXu#Cx@%!AL=I((ZmL!k}FY zYN?~OyGKmK%%3~_f^i{pD+0psR?Z&?KMCuf0%BmJ-K4qbg7c7WLLh|J_s|s(OUM)F z+2&0$_h1NU3yL4{bLf_KB~n|&)Fr=Wt@6pa<6Uzl4hKWzCXp8NzX0Q0|GlunK#xFE z9fViT9B~t)30APc(1_#U@V@tQ>c%Hc-tu~)qr_q1c! zUgBWmu@5)#vsHU^GlK*lIDElS;JV>b5&)UA;~t6X1R4k7UnjcfRoo-|_=;genQguv?q*PmqU^X~xnXPE0MH3-Yj?7$(@jea$lp_eV0&i*}Q8_*1PX7LDz`Nl5#*@cm-V z;wSOrr}8ZhVc3b3q9j`1l;x3T?8uBU`NtvRdzbA}(HG(g3jB4_dhv3ty@NYxgV4XH zkiN>9=cb%IE`Q)4%=+F)2zE#`C*V+8OuNw85mS3mmPR3ptsYv21}3Vw(tz46QQ82Z zUK4$FJp9(FTj?-f4}*8wj?F6+^p@nv@{y5EItg-_2zcJJ9B+#FF$vee5i7%^q&P!m zHZ7vzpI*Yp#8`fQ4u8H6E^2?;Dv-3}ATsBD-s*|9tq=@uYLJhp&Wc4KrN`Jvqk7OK zz&8JunqSCYDjVs}8N(+IBP3j|c8se=mzJ+D6c6q5d@}3^8{G|(q%cg_*@V853L)9J zuF{-gMKI8a9p}M824NJ#g1QgXqSR|3biCG@{XBd?=Aog^3<`Vp^I-cxZXx87{sVI~ zT4)IPPDhsfydxs+>rp4{30sSuoy`{{Cq#GX3UN0)xwo5tL3hp<1>WIeVx=r1dzNe& zElMS{+Z!R?QZVls2{4>x+2@rn*X|m311&elO(dj6_IxM6D zKHd*V7_b4aEfs~cCLSHbas8g*q5Hh>ic}eIpH{;4%z@58rkKRaDIuE&o-jzPjBgcN zSjPpO5?BLi!Ejgui}2M3J}=Th`OSiL+%0hotV`S^YTU9=gm5eD?33Ulp@>!75V^-H z;epDL>D{kiL@cuK=u0D{4-advPz!0Hr!WImzVg4(fhZ(vel8<9rPRLK*vgn`xqi8n zaRmv*_Hc@6P~M36TX-Kwt*zg{g^law)AyxL=wETH^B$*#t{|l$-kg!ukSDFK$h%LG yeEeWCAem~<P^y)YL5 literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..074789c0bfd75851161e26f9dc3bb875dcbada25 GIT binary patch literal 1269 zcmVV>{GB_@3b8l_{wO31S z9X1Ty_ZVJxp$|WjZ_rZ&q(~Oo#6g;(K-1j4^SX0@hicIpFyLq4Q6q9VGy3)t_SJrS z`~AzwU*hHUrMJJ{KIf-+vo9a;^ZMGGtRqQhHSVW^a;L4PBWGVb$IH(zZ(n}3PaprZ zuOHt&{ZnN8ScGV7ZlkR4Dc&~qA|Z9zz6hmXzW?;?Uy+R7GjvsMR9U`TEi&$#$lD6G zDFB4D!8jvFI8)g`?)P z>AXkMHfF+G7nrrXdnGzETaunJ%jFJ9;9E`GSl-xyVuHQSLXRaHhwr9ilW-#^2U`v0 z);_0DHEbmF-E>5ItY{nxMcBP#K7A6Ec3esiT`_i1w#JD&nPPpdycH)?(1a4m9b*;f zXFEC>=hhh+Dmk(1hX81iM3WT8BQu zKAAR%#;0^BW8hGxdWz(IPYg7hu`;2LRbBcSwsfGY_>0=SLb8veo#nil$-IZT%R^Y8 zmI0l*>IqJx$55B45LSm(@Gn@p$%*KfNb1ahcnk+ArkE&LCIL^^4c)%uNt|g?d-`}V zf5UxAZ7@4`^mp(T5)+2)CHV@ax%Vb)k49Lm7Rk>X@O=@)3IK?LG|66_xi?fg$D&H1 zn|DakOLYx{>kt>sQ!tDir^TYkXZhrZ#?EfG9e5pnm8C1!)R7($W@Mn;x*Jc+fC1?)IF8_0{j5D831ZTBI0 z=f%|_WXM_PAY1f=^lRd7DUDm76_-dHrt8o`NJ1Y4nR~c$A`2sJNZs|MWAil7fxZY< zx&{*V>V#D+M>_sLk^C@XXz7wkEMw4;#M7WjJN^K(^HS}>EFXyw9{VsuR1qyU#PTpq z$mbnn1u_jK#Pb5WhQK7oIyc@evY;$}=f%|!Tr*-!-y8C{rOrBr@ONsk-evn)uuXli zLLZgDv1I&?Q?*8;yPo_fNK_cH0DDppH5j16K9CYR!2sZ}mq;d}j2}hB3Alwp%wT*- zPKd^#b(bC?sq!Gz(20FZp2?Wd8prX5@=FH<-E$Bu9E|NFkwP~ffn`JsZ;ZIpSPqiM fn*cA9aS%W$pi7iBU;~U8_VWF2Sebqd7Y_gcV{~X^ literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..d97498cddccaf2d434be327cf50902755c443988 GIT binary patch literal 6906 zcmV19NX?ba!ELWnVEcE^2dcZUF6FUvu2Jk$=BWft#va6WGA`3Y{~_a;0)FrazcWV$9wybxBDp3Q$B=V^P#RAAcvgMX!U0; zOI~5iD~TY82D%#!bmIr$$3a2`J;>8yF#d7C^Ale2C}V?hO0$X$AthyH$&zJ4!ZIdU z5-sE#md9OQQP!0hj0eUj95i3z`*~TMu$;=A19;Ir(^&8#N5XMNtBM1aLA9v(Y+jUN z9G68^ozm>XxMcI9;-V-Q@VU&I(;2%?<|>}D8IuJkITPcK_Ku`oE%P)bUXtslpN0dV zE9yD0iwQnsm7ufvU~Ia9Yq+i#26o^>qmhzp{BM!Gm;b?QYdrRtPs7MEsg*L8IrI^z41o*rMrS@ss@62FqJ7e1&Ic|t_X(pJ|QP4op!-!VwkH?4b z%~Xi__~}z1Hv)PIhqjL7kFj##$H6WUSh5;|gS_9Zpz3TmJMC9CR8h{c?(b$`&GQ(Ch1j@~7& zcWG79dnl^NiV0_%yY$6evPb8WnoicL!1Bo&_5FO3@oKt(cw53|qTSn{u!q*U0HY*3 zKvhRSKyOkHf(x{`An~?i?t7j<8o_7g2_a%_3S(3K%ez zAXF-twlZ$Sf;LjXSk^qrF;4v*7tpzRR10alm4TBVDif!QilH8*v^SU|!r=r2T2(Ok zO+}X}??}KO*w?dTwv-Mg^So-b0E|Y)W0qGd8i9jO1*1+}GO!aTEG}j)n&_8LZfvLip(Pz_bm`&p4(GJdRN_^A{7#bMJh ztZDx|X#D-^6=3i?XIvmtxh7;mu7Ik3~SAuGn$P+=Vuv4541q;Qy`P`f8>ln zpIyv~az544xL0X_Pr$VV?o$E@4CN)zI6ouc)KI|09&a_D%K(}eFa4LCCG^;aSa_$O#a+Ah5y zTTKxOi5BGLP!_{J&ZU<{rXZTK__UZH2Npw8l$h&S<$~!71J9h`{3VA8%##t3)m*6r zJXiA!*R~RZns!>idth-69!WJO@S7wxlNEy^yJ|HJ>guj&7Vx8Nzu=;RFH)%%Le2FF zI)*Z5$xyXjU)BML0|S4kb4BHp zi>cy7Q|T$4XW+b@3<)}b2|=miYdU9AIB%d(@*hO`M`jqhL-KCt1qqEIaXf3=@!^lt zDKra;&azgQ2!<&WS|$W~XHizNzN->}_ZPT5r%)x-k%qH8pVxwb=fzO)lwep~J#JMD z^-6MUjC@2lb0j{BCs-XQQ&LNLSqYNy9E(aYjtN?hFYRI}HwdRp)e zS+?dv;aE?jBfc!DuSAf?GGtc*RZ5F2D^3*)(5_0ctas`yWqetoPOaO^m!i?1 z7a+N1fx|{t%8tS)!a;93!ppw8Zde*YFN&H>z&I$&rMR(4)?+l$X!ln8>+l)|YuNSY z2@a#3sZ>UFrMv9CtcGOorGOvM%pr=TXJ&w7Fyp9Ld7^6djAMLb(d$?JF`y#5P{J-0 zxSOnS>0(A{CD&WhGI7;-Lq9n!aa_sOSGm%2o&X=gd_alPzPDx-^W=4$F^~#ofZD*Y zUt_SCyaYZjw*Sm{4&7!#K>2ybqmtLN5qVjhqLRr-HWYJ;!VZ`o zg|r!heo)MUii5!AOhPIPR#QFg_FFC6wUFhN$WN#mj7g{wjUiesuo)SllLKiXo1R>F zo3Olqw#oMuLo+Q{wiuD;ty%Bhk1v)#eJ2Y&7Tac9V{Bpf`I7jU%s^N9JliBq5HXYl zpU8r?K!iB000o5HD38_Lz~$cXU*hwH8HQ@>26j|ZF_JoluCY>M*}HJYQH$0n1G$z> zI32-Ks*zsxxyi)!=U37UPp_nDY{Sj54Hebc290gF6Kn%zwrhuh=Tpj9>=_|-EYFC& zltsSfrp{-@HZ+TAx4|~tRmHWz*3btn(q9C92)w}DuwB43yz$tI{FsKm8eg?tzzs~#GA-Mpu4hI*3nC|Ut;h_bATpTiJ!0sCZ+{W!gRh|v8v3AD zur&1HkqbmaA0Co4aJl9D3>(+1w@l8@^7AvH`^kMvMuSrBTLN@GZ-W5@nyF@^x?kM8 z#nMorhbR}hHg^8cM0bw|g4j9)rzqwutH>Kx&T4`Ch2#0#x5t+YNMpDjqG)e`;d;4G z-W!W@0`qklg@MHOQ^p~i7`d328pIgL0 zS?4R*AWTCHMLaDEaH@DJQ%<3FxSEO>kGWE|qEw_*k!`uh`#XC;SJkaeTtS}kY6I%1 zB3mn^w0%7 z$JOyzhB#L0T5YQA7H30MMhtS|36mg;eh!8vCtEDhA!5mQgj}eV7KOoSAPEse#Kq3r z-GhS$n}eVf>@RJODLr8%AQ=6IE{$3;qaXRKo{b4;1D>c7NcflJOY;rSs}0XPpjeysYKeM^7ous`tBqi=Ue2y|$U&f1^JcSCc-dL$5*`6epV5T1 zqG>TX4XTc8egFNw`cn-l-+wRRc*teRXzy zCo}{NUZ|(aEt*gQr=)(wMTy9hB2Od}5plZAu4*9js)|e0Nk?ZyUKQ<|PV?DpjzF-y zATutPc)9|gbOo+AyH|ijz)CZklAU+taDV6RJ7V`4{O>(^tYF}l*xyz;pcFa4NMrb_ z5{mO!E65FWJ4}|$)*3fl>mcrLoo>$=#g37N^h|^2LTaB9LnR z<35QIIZy+mG*Q5VOItfZz{192mb`q0wGz#MTsF;lD{h-o{g&;!`!0qpqe9>9*T9KfWD_KQZa z-u|X!!1p zVix)!91Y*q@ZE=?HX6QrE%&L>6@{gB#_y+Q`fQlD5PdQxB{O` zS!|ktXSj}Sn33njM(ifB?S{TZ-5`8?_}l>TxmGZ8@BU~q2Sy|RvZKk&@C)H{k7~K! zkr(PfZ@zY%H(x!>dzpKnA-cZ=q6?Fi^M7QxT@|-fH_-?UVYp8uNh^~16zEIE(V(mQ zhr+>ZX38`?^r1*3rs!KSs-SmAG>ZI$+P3jogUwBJ z)=6OA3F55seL-x+eUU1pZaAXwUMZ)Szh?20#`N{{l=yoGl>b0k=ueC0wFvaUBf1Cq=VWq(vhsI zB3aEKts^ZPA}zN@$6YU!(#*~u5^hhM1RX1R>>|A?RuVcva2akdO&n@imX)TVOC2+D zEfxjL^%6!YqegmFti%pOBcw@6lQ8j(z&3)^q96m`wi%TCC}SnY*!4z%b?4`5Iz|xK zmz}Hm#c+Gi7$|ARN^rajU|P@dZXP?wyUlvbaLEz$E}X}I%V@s+hGXaNsI4aUfP(U) zTLw&app-h>{oS#k3&xGzH_kPB(IwDd_ z;a-REJSir0#&u`7-vOf*1<&!^8JPS>-~)h9s3r0+0R6i#quQ?oKca#Wb0p*4dU5?L z66y69*ZZO#5=x%Y1*4t(3Y zY^Oxv8EKH%Y0A9V@Iv3R9M5zTHw|2mdi2WGf9BILvP^2FjHP&ft{sGiV_P(}9m{bZ zzoMxI`^9$;;=Ckjy!_-HQJ)O@5%~|HKvla!TcT z=5oouP3ScVbMt@w-{887{(Aj()B5dEk>Gupl{$Wz34Q$01BpX|U|1%x`FTR#YN0#` zp{S#|@#-F!uYpyeTwm;a&*A!Z=ZcoY{q~Hdg~Hnq&d*AHHrRcOB+crNc%b2ge$s+? z2!ggegRpOm6P?Z_~1kd&!Iiy1H zn2~2iCO*w*o_CvVm)97r)j`<_yxng?D_JAFgsKGBEnCRNOT73grs}$%DJVi&;H_C) zfQLEi8k-B&+~k8-LP29rSqbj8Y!04-2lVjPsK&`9^8O&a_GgYeg&DSG$|qC2bczWI zSranjDB*SvTs0TyEGoe;mK2keR#Rw$ydI}w7qa7xp->g9YbiwlRi3stYr(7mk3yhX z&jUpksB5>18AeFw(}mtie51~C#lnHB7Y_7VtzN6uYqh`HwOYA2@Q~JOwaL|MwU1q- zSFF|I-6`f}JGB|}9M6Y;0X37z4qVIhB9;aQH7(bQg2B(P)%s?V7)HVj<{Q4}BvxP; z)W?ffBGYq|=rOL<0$Xi!r3s_=0dQ#RQIdUm6PpyDZTC5 zD*xbiE47EvqiA-m%Uo2}!FAVY^@8RV$Fc2#(hIp%GoNBOS}kuv$|;M;(ZS&#$U!dW z$;ec2Jl`(xvRPiwbV%sy8B0FT;qZ>0m1#%#=8kZlXh&E(!e4_UtY%&h$r08jS3AOw zU8Gkz!mjQ19N|^aa^(3@9Gb3Y*lBE-%yHb*irgRwBI;Y_m5wktzaiwbQ!BNZA36*i z;lPLt-?eRnrH|1OcFeKuk8H=id(g6PnwyU-bGNPF3pv8i=}CzPG!#3 znD>@0@QV`9WZJ2dk}Wza%je$VRwcW>`@2p9Pv`=te*c7Ji@s0aCG-i&+=JTzm3P4^ z;0osU5!gLeh#@&XKbx!s5{}uw{yw>INdDS)gWH`~_Kbpg2R~SW%?B$u^VPsblW}mn zBgzi zlDGs5pTKY-XbXnFJP35d>gx&aEiiMj4BaL2tof|cvbaxKOwP}W^8744Ka&ldsoh0d zAonJa6R=z`xz%>+{zZWs-eL{uy;m936<)fN)HqaVNjwa2_1?-{N)F_5NJlhnEKgyq z$e4g5RI`l0Y`!EJB`bDKBIbYF$hMgW;0j(Qub(fr*G43nXU6jyX_(+)6T>iM8I&}W ziQkfaA-%63it=(j(Mx;}QN#58&w7bZV_m-zwb4s_*Si0i8iwQ1#EH#_rm@3P!(ooe zeAi8_!1jV`1=X()g_R-ahqMByEwWw#eC#5Vw?~ zE!SYK8^tLNf;6Pw;EEMM>d~0li4P{oqH$o_%n8Bvcph~e$9|j_JX>SS8hOUuLtzcy z^S!~nA6r!Y!0L}f@*VA8m(o0~T+D*`2DgO3Qea@3vLRW!yxunEj)M8KR))s?q&c1063a? A_W%F@ literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..c0b27d2aa3889cb1214a9d6d1169998382ae68cf GIT binary patch literal 7002 zcmV-g8>QqQiwFoU6PR8A19NX?ba!ELWnVEcE^2dcZUF6FTXWpFm44q}f$genMi z_e)PUTe2iyCYG&|on&V-DHlk9-5j&Y4hc$b&(1^M_NPQo`4M`}kJ@tpvdM0>MwgK+ zc?8vrEP@~oaL&O2&cO%Z#{r{)zRHtgF#d6n=ErH3Mj0QB6Pi_g2q_6KOU{-FjAvYM z7A@o(p2uBYQP!0hj0eUj95i3z`*~R$^PI|@19;Ir(^#ZMj)dckR#ggA2GycUXY-;I zV#$=$0eT^RVs>d0iVmPIi2yVWUk^VpK)0*%efeLw09)!YMG}g@q%1G{WKf^ zT~W`0U0kFyUI{vz55}e&xQ6R`VPFS7G#V+n#{U-VgZzQl)_Cl3pN5fTQY+y+aa_;0 zgV1nnizc>XISy2!DAP%r)9e)#@Rz~r=Z9HpbdoK$8UL0)IsPf+3< z+wzyc%2&Kd8Uj*Y1IsYOZNuEQocE?TcHFVy|IeV!DN34GNm0&dlN;XfPmrq0$07@w z_O;^2rulokipzA~D#etR$H3=3D78=1es2b2+Zo&L$Z{2( z-%N#=kDopTawDL}QfTW){z}THi9jV~JQrz_@)CJP)A4$?vEx{-?=_8UgpPFywoP-} zwB8%WSpJ9gm~laok#9hksyUSk-&En`%D5V4wkVK=F&r%_HT@J~+MRrWnoCCGyRWGHg%hmjl2EzCVYTC#Xg$p#2B+b%B zJW}4n6>2Y`hZS{Bhva}~i=5GpT9?ofYqKw@S(i`)4Jdh4XF@FIe69O~{+!T~uXpq= zdA&=kl0HCDMOI8wzPU@E&n0_wKB?(stqMG!tWn?3Cs|reHxO@2*i5v0`xExiIu~G+ zWCy6~=m+Rc>OrIeEiOpB?U=iu`ht^5N$1n_Z#5@e5U^>SobZUe%0b(6$xoVrLd?+K z)ibg`OJAo1R1>@S-)fpQdfji^3^au(daF%bd9mI~)ahLy|6w&lzo~0kej4g=I@k_rnz9!(n*`!w}hP$rZF^Iv_`Yl78E&JM#CIvwqKR)Q1Lp#UX5Ku87e-DXgMih zz*K@zsbJd5xDgB5NC9J6^CZVO^>)QL+DcH)@F#cT%cmV-`{ zy?PogwsZE>hVe(M%{A0!MP&UtzdhW4vsSaWFyfOP`N4*2sFFU+3UooXDhp zU|FVbJIsg;&!*hJ(lm59sLKo|#LX<~+$639)h^l7W|D?>GObiUgeqr6+?#@~&5(cL z1X)|;yRgV*feN4IlNcsI6HaG1XKD@O`rEiJvH6SkeJO1uXWb-|6>$xOh17ySN=<^) zrM*;koAIM%gJhm1HJEYu_D$1w+|FlaoYnIWC&>rHFv00H zIJnD}V+Lm8fxYmf!1H(%M@bS<*Rn71nO(z-ESEW!kpxB%2b{5(g^}s;(DW12e0)B0 zICdSNIr2RJu6{-;Rdlh|d0@JUs1LbL4bG{fk^?FmN@jMo>)k{N;QRYs-@24qrJmmo6zsD)| zL`r6y!hD&NA|YbR$ui~%-(wnZWZLVmL=^C!pc!eq^nz?PMKBUA$jhNDhJTt%FN{n< zG-dI5F+mP2hNLJl*OAHv(-jV$JHh!&4ilIsBOc`3ZZ5E8D>Zs0NKEr(9e_A6@P|5AR8CScRh(!lJ*D#uoV(*8K?ji$ zlq$Zab1sE*2!)dWAj)?#!_Xa)_d730Xbg$tS=){ef1FIASx|JAwYo$wjLB%p2=vaP ztYm#xB?9lyaeGdoN~j|ZXVZLM3j!V)N5NBqVRQAkRdLiS$*nQ+5#7*{_#~cSb)Za1 zElJBtkSxuyD28!P&~ki{*Ru%P2fUN*N4KsbTk<<-CeSj2C&RUzJua_gbs#4;Mruh< zpi+igkf)j!b;d|u2$V{Oku78fRh+J7s2otuUcc&T!8c^tnsbF?J&lg|vZTHeK_1JH zT?tevDYC3MQ7k~aD#fziske|{$i`MxIu^*ROOBy#o%@&-`ELcP9`u+@E=@om;^2ei zF*vbL#spYP9H=)jiTdA=|C(ep8g0S5xX9{RUOkbimV5($+3*W8$jpKKZ4<9a!wg}Q z=ED;?lD>1*_5)CrTpA1P@U>W~)K;rDkSMwQ4ffaw`$n2eb5)e-XRtoN)&;PTyx=)6 z!P+$@4m1)ZGl3@NO4G)YXF#dPke+3HS)op?+sl`t(VrI}xn+UFMpnv>!YGo0-gJbQ zeRbWiG=g3fHJN~MP?k$^W0S1MXrj^Xt@hX96%5v}>(3J$MmtlfjOt2v*?Up0^e70dv&fnmSKU^00Ld|Yh*nWs5)n+XBs zPqQ>C(|R@{FN+gYG8yTHVop$4LiTVJeZ`NZf}xM{Uu3V4Hbc;lidj%`5V)L4NM*rl zs;Aw4t7W?uvb+-c2~~qJ2{ocIM5_fhBO`QjAT4Cm*}1oj=LNJ)zOOi%X~DC_h&*r2 zdiQ>OzWnKXS?H12HrpCw3%k#k#K&X?x|+_jP0|DrLrL(7ENBZvh|>yCK*){qSj`Pw z?hXGXKA)RmsJ3okMuif^yrn!@%KJXZA3iafoDbBvYdoDo=xM+wiUR6 z=~<>_Th#T;$mc=igsv5tK@>y=cfCgpk?`#=0+H~q1d({S@Cc3azbeZA&fpN*&T0t6 zS1%9^fp|#Lz-5=SQ*2zb?lL($&CgDS?kD#x84XUkZwb)(ybT5rXr`Kt>V9$W7E42h z9->_28rj)DneHAB1iAGZoT518Sw#+cIjaTkD~{*y-W^>mAdTgEh@!m)mh0s{IXrs% z;%Mi99PS?-{P8{6d9z3Q$-Afl;Mel)=r2+E^sj`FKexz%vd&j9 zLYRhFig;QS;8dlFOgVwt;i@WPJ?2W;ic*nMMYiQ0?eFXXT~)U>aRq^vRvXYq71>$= zA0hfeq;Z8PByUJe#~GWJ?Qffw@~IHIOg~7(tlE za&}kb<5nA6t~VP&VZEGP?T~{&txl^c62&SU(=vs=0}MpeLc83x zf;4OVWYDCfq!SF^R}w9`_EI8H3Lyb^U}zO&vg^0B`q&}wdd17mN|!JRVET+Q-ioHh z<}~O!vi1G<`|3|Mq@S4kGi8QwEw+p>&Z;v_Vz1 z94A`}74`(7LhZ{w>Z~kIpzY5w5S}em$3Z9-0=p2(RhIQ!VU}8aqC{(8FIy28!l2L) zG`OLjDz~UY380et5f>#QP>P&MCL;25nO)UD=2aD!=#!4lh`cGBfrJt=}) z(L&lCm`_Z?IOP8Ia4SS(UWZRXWLG8V3V1WRY0AleB8VsK`33 z!^a$fW>ECuwk)&3Qe%#fsRC|uX_$~%$khLoO;GC5C`mJJtI5Ooiw&6lK`xdc)6Q1F zgbP^LjYbmDb>X;K-POR^tLM**kBz+A!4bqOM@%&7xCj5$x!e zN3f$WKZ2@ZCFt@3PppeA6|#%UqW+Zt~DIe9w=Qz%yg&UJ3GcE%>glk1+en zyOzf1{<=hZ2|m{{ZQD4H&;1bE?s)8Y)SxU3J;(IT*k%ke`*z|wmLG-Gj0czCb19Ea zGw=-8u?;iwyx53c7Ta#P}7J9|;5Sw_L&bhP?O zE9CHt!BuacT%%Xw5r_x0O!@Y=Ma6sMRug*!;gcmb`&!gq0QXZmB+o%vc$vl@J8J!; zj+XjhNzJ(ywRbRkK=I(=9+R{h^iKCsj~$&h>ynbImkxf}9iE3Y{s`ZZgS|sh6j@Y! zB6^$MS2WsQ_sDdPM&x60O!j!3%DB>@nqlEyC{ThYq*Vi;hB(%WB+a?ol{yVsoeAC? z%3Ea1bJbyv4vwkd+c@Z7Ol3@xG{;L~n}dmmEhtp*vWa*>$4TQwKLxSAv?RUh8KKz( z3J}v-r4D(nidk=05~3g_mUY<*`F>+lj=02x$H;9#UIQUKB3;8Kucd?9G18H&%OY9L zAgv=U8zLy^Uv#eK z7sKs66fV!zBmMyKo-=EvNbRVTzrtrPrTe_tl{8LGaza`a>^Dui(@KEptbiaEWeb$Koj)h70WL+y7uYK-FsjN#n{Ws%_{H~YQQEWe2(hRN_ z`w|6@Kk^J!G~;>RSJCFV7yW87FKLB3`gH@~Y@C64%Tl3nfcJ_(<8@5SMWMxTzhWq{ zO-$|HCaP;_i98I*azqzrrxnZ`axIa2mB`s?tVT0(e0DlHI~BT*-1`~B>WD}!g?k;s zb5=~~EY+Rieg}+N6lsp<&cNh90v`Z;LM@Sp0qEa_8P$Fz(gP|uF-J1qtryq7B9UHw zalJ3!G2Y{GMS@Dw9d;Jut` zX}uDBUJbMmZ*O&4U-70_r}aJyz0cy;y87QNH$9})e{FL0>i=UG=_RZGcHrCIMLQ(| z&qxAhCkgjr!wY@Oay-*vZW6d2_2{Lm|Jk&|L6G8ko-42P9_AEF?yw?P+UZshsvw|~Z6xM>C5kYNobKt7EKxa`2j=W zE0O8BEP9MjKaD)RR zGJMyz4W2wkN7ymPwm-5R_wGTvkwb@z9j2AhV&Fbq3?hP<=u?*cYdDeVZX<6K-EGB2CMR|6boSn)B&eZNAEs%Q? z$T3(hnA~bRb^oHk4R5iA^xmrs>IyI2VKojFS`rTfT)nq4my!dy9MTa@8_QD|D>5eF z2-Q3zFqHnMHz0l0#f$?NBf?X?j}=9#5=jWkT~u!&(9vJ6U^$;5BT zzL4J64@G&gp6DgMhp1ut{%5_!cbz3Z0zVq>s+alhbD57cKUcX6ni_`VQRc*EM3dOz ziQ#a^+QnhsDb9^UtjYlHSji?}bimms(NmTP$I*MJac^kQ%XX`$ok4 z#Jyr8v?13EhgzgRE7Hpr4uc@DHy~(DBg9_nyAe;QXHg$aOlY|VcikvXXb>bJ^#+$L z98!%t8Gv#^IaUA<`-V1GwEo~r|pf|j6@644pJDzq}hmUp)d1<{BG~d@kn&5d+S9&-7wRB25raHo?chf%vwb9YVO`8bL1>^!2yJ28Q0Y(%} s79=rq6Wemp&39eXyzQtW`ldJ8W1u66Uro>ZIwFbx5B;XWsN?_v0D(G=k^lez literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json deleted file mode 100644 index b585baf88..000000000 --- a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json +++ /dev/null @@ -1,20 +0,0 @@ -{"resultId":"50|acm_________::3133635707788d2180bcef09e01a903c","organizationSet":["20|dedup_wf_001::5ab59ffa94c31a140d4a56c594ea5865"]} -{"resultId":"50|core________::0308a76f6f8bc4db75a817d53a7e76a4","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} -{"resultId":"50|core________::04c8f896aef9e54867f2bf4236e9c810","organizationSet":["20|snsf________::1496b1b4fc4d5509b16f2c217be480dc","20|dedup_wf_001::06731b587a9ea654103a6b0ebcb234ff","20|nih_________::c5722b087a5e707a50aa8f9f2ebf785d","20|snsf________::71d0a944b61b1a94068595f840005a2f","20|nih_________::dd3428794aef214a3bc2cad6dd548ba6","20|rcuk________::45aac2108e54b6503d1e611aa5872c03","20|nih_________::e1d47fdb7bba9eaeed82a95c578d6e90","20|dedup_wf_001::e3b52200d2fd4ff883478f5bef312efe","20|snsf________::c5c565d3422a7eb22886f3a4c93c32ea","20|nih_________::91154321f75ba26021efa18f7eeaa541","20|wt__________::38013971ca0c021fd65abce2171b82eb","20|wt__________::a6114989a56a1dfae6cbb201d14823f0","20|snsf________::b7af2f99e1e06750a4664ae401802734","20|wt__________::757c54e33d4e925c8c17edf032cdfacc","20|wt__________::1d57a87af1bbc2b7e211305fc747c9ad","20|wt__________::7cbb8c06f702b8871948acd370df892f","20|dedup_wf_001::53a8606f32787c4b3c663fd90ee97b17","20|wt__________::8473a929b793e56d2299a1a5aa08f617","20|nih_________::5e0fc2ef31bc189207f250db818fea0e","20|nih_________::1cd08fd26ef03fd2f51e9aeb34ed9486","20|nih_________::1c270e0dd2552b4e3cf295cdb7db1cc9","20|wt__________::04abd842647bdbc751b1eebe2f142013","20|nsf_________::3eaa8be3f16b0f0d7563b9117cd1f660","20|dedup_wf_001::c1b81dadf1e4cbf23a61833ff9ae8a31","20|nih_________::3716e1c14ab7ca14161278c9bbb8bdbb","20|dedup_wf_001::b7b403a764ea4e3acb12d999675aa73c","20|nih_________::300049f12fa0f5bc37db3a5636869743","20|wt__________::ed1e2be693353d370066fddbf862f23a","20|nsf_________::72a3747a18c56f3701494a0c0eadc5c9","20|rcuk________::e8877abcab4bc187339a242aa5bc2e09","20|microsoft___::119a535bfd240d7560fe4efec416bcd2","20|wt__________::be4e939abf9617557a35862e255493da","20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7","20|snsf________::1f4e34433767faf965f33849bb0f7fb1","20|dedup_wf_001::9f1647eae28911113d1dcbe49c42275b","20|opendoar____::6835febcf408fe892504bdfd2ebe669c","20|dedup_wf_001::528bd21573101493c6c7d120b17a67e9"]} -{"resultId":"50|core________::061dc912098a6a52e85a824161bd32a2","organizationSet":["20|dedup_wf_001::67f983a98e2c0cc0121e2db46d2bd00a","20|wt__________::59b5d99b2dde58df1655e0e5bb236c0a","20|wt__________::e84b06dbc1b26f413791c1304ca8d6a3","20|dedup_wf_001::7b118dab509f49b4fbd6dccfdbc479af","20|wt__________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::43255cf9c16732bc4ec1d5f580f44928","20|gsrt________::455b984b47945e1fd04e92c9c0eeca04","20|dedup_wf_001::e712c08f721e8f167f93888f590314ea","20|rcuk________::8b0aee2a7026dc92d4c05683ae45c894","20|dedup_wf_001::3c19a02cea18f8eeb2034b6acc544b7e","20|wt__________::26a9d64d00b8e1005cb6bbad2b7364cf","20|rcuk________::e57ab9be7acd7b93ba34aafab1a5b96c","20|nih_________::ca09277064edbd89f71e1602d98b4dd8","20|gsrt________::7c309ee758e6c6e0dce43c67b9343e82","20|nsf_________::53c2c2895613ff9b759f64c24b6cb17c","20|nih_________::c57387345f51a40ad2284089b120be3f"]} -{"resultId":"50|core________::088190cf9dc9632e8d9ba5f5f1de1303","organizationSet":["20|dedup_wf_001::286621caef868bbdc52918699e3cdc79","20|dedup_wf_001::84707a912d45d666fef35c5cd77fc203","20|dedup_wf_001::30fd03f4977438b0471a1d4db6317e71","20|dedup_wf_001::83d9dc4a0eba8737819b87ce8e737a49","20|dedup_wf_001::aeb9e738d873acebdca52c9ccd9559bd","20|dedup_wf_001::427a8bfe68648f9e30cb47434144da68","20|nih_________::ffa0ea174845f1548e520a047cf53165"]} -{"resultId":"50|core________::117d295998199f498fa561e9c26e7ae3","organizationSet":["20|dedup_wf_001::d9f82e62c3114dc7f42b9da9b7f6fc64"]} -{"resultId":"50|core________::168a8f2e4ffe9b0e7c6bc100f34f2de5","organizationSet":["20|wt__________::63fe73f079c5ff4e925c6cfc1758a98b","20|dedup_wf_001::b3c2f5e700cee15ad9752ab961df5930"]} -{"resultId":"50|core________::16a3b520030d82ad16a30992b124e69d","organizationSet":["20|wt__________::a72760363ca885e6bef165804770e00c","20|nih_________::5c7f089c177ba49f92033f72e2aff724","20|dedup_wf_001::c88bf88e0a4dea271a3e2f832d952238"]} -{"resultId":"50|core________::172e3da668d18f41ea4ccdf7f2f39e53","organizationSet":["20|nih_________::126cbf4b13249e65098ddb4835f47456","20|aka_________::506e3d2f7507a66584b8b3430ade20cb","20|dedup_wf_001::4746df4ff8bbb7e991ad343ccff8bbc7","20|wt__________::aff5133ca9cf0b810cc331d498bac9b0","20|wt__________::0c3bf892603817e5eff6e4f08a530ea2"]} -{"resultId":"50|core________::19f2fc91fe1db2ad62db598aa9aa8ab3","organizationSet":["20|dedup_wf_001::dbbd988f8d57a9d11286caefdf35acaa"]} -{"resultId":"50|core________::1dceb5a29cd42728e410474fe0fda191","organizationSet":["20|wt__________::b1ef2f643c948a2ef49005f9145ed556","20|dedup_wf_001::866fa622e3c0ab6227cd462f40cdcac8","20|rcuk________::63ecf5736189d299fc3e043e14428b8d","20|nsf_________::fcf880eab7315e0a5f3937c5a16c04b0","20|dedup_wf_001::65862ec7d57f700a130dee916bea66de"]} -{"resultId":"50|core________::2580c0b59b7457f571acdc829d1765a3","organizationSet":["20|doajarticles::0f6e2c32a27c307b06edf7862c591973","20|opendoar____::4f10fb61c457cf124e5917391baaa3c2"]} -{"resultId":"50|core________::2624b8248a9febdad9bc456d358b30ed","organizationSet":["20|dedup_wf_001::fb4eba3cea53264bddd59a4ade9973b3","20|rcuk________::b00968d2100a4b62447841aef5bdff62"]} -{"resultId":"50|core________::26820a282ef54882f7a5be74767fc02b","organizationSet":["20|rcuk________::8ad6d06f3b4d09dc67142c158c7cf5b9","20|rcuk________::01ad471b66687b1213ceb08b5d7aa6c2"]} -{"resultId":"50|core________::2a8de3e0bbcab49066aa9de4bbb89bfa","organizationSet":["20|dedup_wf_001::2ea78875d19c8cea63f7e958e5204136","20|corda_______::6821a8e260b8b97f5fb5e80168329d5b","20|dedup_wf_001::9d0ba437d73b19f55b53c578ac970ea2"]} -{"resultId":"50|core________::2c7d139419d2895d3bf0112b50108f75","organizationSet":["20|dedup_wf_001::96ada508ea5d85a1e516bf9799413906","20|dedup_wf_001::d0ea749da6988bcdb2f30d77c64e2f1e","20|wt__________::f1ba5bd552edf15db494dc3020f27470","20|nih_________::ceeae4f78a5666daf4c45acdbbedde99","20|wt__________::84ef588eeeb4ef77e45ccfbbf3aef69c","20|wt__________::8eef7e1370ea81c2aa3dbc239b2bf5d8"]} -{"resultId":"50|core________::2cf1f6282498fa37aeaf678f8c6e5843","organizationSet":["20|snsf________::73999c828ca67fd2d006100a8369c1eb"]} -{"resultId":"50|core________::2dffff00500c2354b506814f6a1ec148","organizationSet":["20|wt__________::c6d89e908582fddf3e4c658a458807c3","20|wt__________::e7b2c9f3d3f3f1503092bf1ba2b163db","20|gsrt________::ab510bb43d6c654ed3d37b9c5ed5c971","20|dedup_wf_001::179d0313fa7d5fb2bef5f312ecdd16fe","20|gsrt________::cbffb510b01e81cc055fe61105c86154","20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e","20|dedup_wf_001::646f14555ea42b260499239a7231b285","20|wt__________::5d462d78d512c1184dd384ef2dc35b7e","20|nih_________::a32a254b024265db2e24a66291c7c1e0","20|dedup_wf_001::5490ec18da9721e2c8d974fb73c62467","20|dedup_wf_001::3bc91ed90f44d0908258e132659bc754"]} -{"resultId":"50|core________::3031a50bf5c80865af4841ab42aaf57e","organizationSet":["20|nih_________::1b46e3665d8be2b524c285a27ca952b8","20|nsf_________::71450a4b98015592ee3f525a51584608","20|snsf________::fc921725875adb56f2275579b31f805c","20|aka_________::fa5b7357f86c71ea15734282054f1183","20|wt__________::18fdb5b42b22fdcc45e323eb4d20c91b","20|wt__________::71450a4b98015592ee3f525a51584608","20|dedup_wf_001::8aaf46d4e4919dc55b8a5cac7a15399f"]} -{"resultId":"50|core________::31116372ae189ee456fc06dfa0f6cf7a","organizationSet":["20|aka_________::c5b9aa0a905f89c51221f9f4fda22b20","20|aka_________::d9d3242062a7a3c483a7926fdba17bb6","20|nih_________::ede5c9e31cfb37a397d6cfe1940d045e","20|wt__________::8adcc12ffee195ae46679e8cf332a364","20|wt__________::5e954c57b0ac7aaf3fc16deeaf442389","20|snsf________::ddd964d550bfc6e1ce18f83655ba6901","20|rcuk________::a705d2ee7bf0bd225264b4a5794795ce","20|nih_________::8adcc12ffee195ae46679e8cf332a364","20|microsoft___::53732c6c7bb9daf5953fdb61fc0cd5bd"]} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..d9b92debaa9c5abd597b9a2c8fb836b3e3ba3608 GIT binary patch literal 3175 zcmV-t44CsDiwFqp5|~~917U1(WnpA_OlfXwWn^D6FfM9yZ*BnHnah$LHW7yRJcidS z)Kcr_H()jC@#HrI5YBggdSw{M<4{%L%8|Bvzc z{nLlv+vhiL|L`Wh{+K*w?(U=ON-{*l){)D~U2Wu6@|!>Y`#1mmuLqf>&ebKCkbDI( z)$DQR*4i93O8e$P{{H#hPyf7q%kNe`qNFui%T}hjl-z>W8Z3mx$FSzZ`#=9;Or1%((C$5kl)OfZ!}&v>*StLL;#}UEg`t}YRvF2nw-k5JF{XCA@Y8*d-kRw zgv`Bf`|Qo_TkSf`qv%L;#Wi->J1bBq&uc!eAHMr}jpi}9a%h&>D~GhjipT(ZHLj3u z`VJ4RW1euK{ z-PR0>q`h@Ub;$N%Wz89bY|$2zhueK-9Ui@R69Yz|Kue)l8l;6-liv2Jxvs31(pKv+ zug0j^_K@OY*R^hIWam0So67_NvvLzjA^WOYS0DSj565pS6t5x9)~a-EW9?MK_SV1R z?GS3F3W zOE^rZ9&^~U(N`P6Zfj(1K5)lk)QUeni}hhMsco)S?rY*c<5m*ECJ!P$a zyHC(Dchgw3&OKMnjH}s%&_sd5*ZKJw1_y6Iu}w?R>kJ-t%_U8MTh~TO%5{DgP?SBk zL+>a|gmt7rwa%s5NOHSRs!~iAOfc(0S)>S^T7A-L4m)rAFw_dA)q?s^fw88*%e{lw zF>1PsK!TIteu+K;BfS#!N(@FSt@NttRyTuZ;!X;m0+9d%r$DeR}fG&o~gqI^_v$ z^hL29>KkiHsS zt~U-5rz@%{RTd03%brqGdK$d?l-oYE26#f$v4JwUV9Q<{xG$I$o_t>O^u2x9PvhhB zKBcAXsG2KdWDH#nIyV&o+oklYHs%PdqGS~Ev<5m@bxRCT!6?=GP1x7dNBhMz6;9S! z(%UF_C~JANcz`=M>K-4oRFtMSXr>twwy;E~K-^QIo4G?6w+7}uGeDnlZ_8~T>jUpt zhSgJ9_chjZhQip(jKx(_?TlLrii6r^q+5|7Mrj+*7rN6n^@%{Qg`6&}efafk2_dI> zxyYILSuu=7S46Ry&w3q7L@?*&STh_;K|`1u!Wa-N+#aS+Xh z5QZZk1jfp@k@1 zzwVP3QyJp2qvj2Y#c6VXBrxNWs@>`q%jml(EuYN<?p6D(>9_^_GK8=-?}9>L0x*Sirs4>~b-+>pw~5)RNDoL;0tSY4 z&yuiC^~#VfZElB_*u*%c%7rJE#+vaLa?NcGW}@@I+h)O7#)Tj(W@Qm$Ad`JtqY+~^ z-4I440ayx)Vbi3dx<_fZ1`-VsIH}s`A?6mR9s|Qzm**t+eE2mky_3-SgjA2w9)*ch zIW`hrL;swdhkqy!HgI>x4VOqv6{TaYarR9RL%Dnv{dFWsX#}5W1OV%Vrn=Bm=91Lr z5!!t)sA4iHyocIXc7OK8WjW3`H%R~PYZx6r@hdDd$z%7LBZH3MKvgL`a`asdv>i_n4j_S7wZd+uMk>IgZxPDxR_5ixti z=jrhmKtKQJ#K(8zmlqC&ts^0h@h_A|ajftn^ktEmxy}|IYWvBS_*>jYmqBz)NbC78 z>zYO=UzS8G`f@ZvFbgR`7Jz8rlf%{VuQX32qKOSc$>v@_bZB_S^CI*>VTt~Z&2fVC z1W5&Nl5sqvlMBXtlu&W-2n=F);$Gh(Fzlh%4M&bO#sddIbY>p0uMIOrOs9ZD#e(`C z(&a%&lX$pLq1Gof64bP#wcZ62vGdmszhYd-96|zNc&o_|gdfHFYd{RfYBy;v$>2Ps zn-I9kjlPlv#1itvdA5C%OkWHEZ9(w`KZkCmo`}>#Fm*bwS*v_;?s(T+iNj$aa+^qt z`8UA0?biz%8R!*g>IC7HGYf8FwMdt0$>YCt_-XuQeEj@={nb7JHIN0}VtkKVsCZ4H zf8}s1cj-0VjJvu7pG6iMFF)MG@7C01jxiv?2M%8_6u54I29$p?gd_x^0Lgh$h zgF>)vM!>?Dt`O%p5C6)co@cZ3w1IyMSc6iD^GkADM|S5mY6b`gvoY~04VNE#UtYr3##nxR4u84NmoJviR)M4~i^!b)ywwwHTOkKNA~Gq!wvp?GMYmyqtgGdy&kS6-1Sz$+GhvmkM1$ zN`u^-(WoJBT3wO%xns=759$DtT6V>{GB_@3b8l_{wUHhe=>ziR1g_s_tVt)Vovid(qpg0WWc1Au-ZRBdE)TLjVnUfW&pc0*YofC@MalBOlg z-Nq`^&$e|k?pu3sj#fuYW7nh8S4woumfLu(kp(v52(%#ysI6ijbgAS_;PjwK;Qk_! zsk-+AnbmAuO$Uo+dp~GGXeC*-$e5uyyR}=gsh5C{xb+wlb9SF|MHgB8K*6)=yjOzL z8|pNtG}{ZSzIk;5sM^7JqOOz}$WAG0Y~5#|ppF);ucGR{=9Q9JQeD>)B_+|4MgYS> zm2HQ%-YHQ`6CB!Rc&5S`ZsctoN%)tyXFPddq%ebYHaJE~POiOnv4ROBzy!ZjvRelq zVV_J3i^jWjC}ZFtQ$1Dkz9)_~n`!5Ok5ygz9&S3&Rs5nhuavAEXlDg4X7cFi5%K^F zXqn*DRZnm?dJNTEg>ZCO1^;sRm+Z`bsigKyz+(iKVv37m%Ov6Hy1?zbJ;^;?YRwpr z&0la|QXkC09sM1Amrn#qBo2k8(p+m1PLF1KTFb-F9PoV+#0m(=0-9u5XYLJ^_V%n& zaPv;diBh%Ua2-NMo4a5bXY7_nd7kByZyGzhyFFqX;To5ib&v;iS*$MSZeurDU)hlP zwp(vGjT=lEf)FQHKFe(1c=0(XA`!80h}ppwM@S06$O!rLN=cd_3-}InHk)NHL;{yl zD`LpG^Wy3N8FJR%$QJzo{d(|hM;f<2D=(DitTI0Mj5LUN68k1SWA-yt*_5N{}V)yto>I>&YB5)&d@v)VA#if4c_jUACVE z+tkNa=%W&L>@dD#SJ`NE*OUK5i3%eY;GCeN0|uzD52S=nFaS90rILv#6I&5+5^mvu z8H^9f3DG#Uo~6gHUq1Z(0hC}>(xyz1z!=8{gBCVe0Epf2s&Yqr#Ocm4*TAYH{Eg9d vQnuY>`q3v!K7agN|0wCSknRI5p=5+mLbd;910 z@zv|Uypr(CT=T~tzWHf?6GD9bI-BRdxAinMT1;Uk%Vp%AmgQFRtH1v14?n*+$awdD zeK|?MQcK`U!&=L$_}a1L7E-NYg>de} znzwKM_PaHC)E!zMu~xU#cbA~Lt1Pk2Z|P~x^9zsGe7ClOrMXOMG+nZr%IsN0+XEL) zqwUs3j6;SX;=>@Ob^Q!gGpZHkt2=pM_KA+h>OEzEH{d?;0; z=T@yRD>=oWRy-^pE}Hhf6=~!u;(;}JffmcbtnI#K`K~R`r!~*dR^tpnB6*1poKh$) z_qrh#CJ6n%2CzZHsc9Rk$u5GdHx)ygz@oCL-;A;y}dyT`wV8Nb+nH&os zL~k_dw#Fz@>a8=X7U#BAhP#9~V``#1-0tJmw(Qln8+apr2EZAS+IUb#uskoTymj&Ae1565pKlwE^(Yc=-NY)q|T zGHIXj_JX<8T49MwZFMBqQHt)_!&<$1thaqys%l-j*RCs$0$JYvP!E;FJO*b0*n2u5QaUt;yBOeT^KZn^STYr|4q~um7!=GbuEX`Vge4I=jRIy4&H!b^QF+MhaG0kMIGSQG2<@fIzIy_ z#%XiMm4%R>c$u zGtThBW(^aV&de^Ga7DlI+XBm#X*N#>JXN)uB!LSUjTL@f^ZlFg{@sUn`{NT1B4e%D zfld0GeqXrf9G39n2Bx7|x+jj@WnE_P98y(C!b)9yrwW8Rup4z6G(6EBk&W(LQ z#OQLpaY)>BB_-9$0%NnhRa>H`Vb|^PwhyfVo)EP(PzD#wI0pyr17?LMpVqwlVZEL2 z?ETX|rD@)%+D?qoV(4;6KB)-UE~Q_!F$=Jgm{G{n8t7ou86`jkR_f{x2z%jVoh7|l zu`H`$W#j8(f*V{8-87y71cX*&Wv268%5?Y6IHQ-qx6@t{W&;vu+TI>NR(W-*)YO}lXaXZch*!97olvax1u@)5H z++l3#5rOFF>>RbGaI4H#MS4J5B4A)xrx(RK z)hk0bo!kyBS&nhi%7qhVk$W~(a*J1O*RSV>_lSh@*7ZI~Jd|o>K0_f*oow$8x zzrS!OY%PR1#=lUWgkyylp|2&(%yqW#P;;MbiND2d^k_s!MOx3ljB6U9d^swu=wqW1 z5woBQSpXu#Cx@%!AL=I((ZmL!k}FY zYN?~OyGKmK%%3~_f^i{pD+0psR?Z&?KMCuf0%BmJ-K4qbg7c7WLLh|J_s|s(OUM)F z+2&0$_h1NU3yL4{bLf_KB~n|&)Fr=Wt@6pa<6Uzl4hKWzCXp8NzX0Q0|GlunK#xFE z9fViT9B~t)30APc(1_#U@V@tQ>c%Hc-tu~)qr_q1c! zUgBWmu@5)#vsHU^GlK*lIDElS;JV>b5&)UA;~t6X1R4k7UnjcfRoo-|_=;genQguv?q*PmqU^X~xnXPE0MH3-Yj?7$(@jea$lp_eV0&i*}Q8_*1PX7LDz`Nl5#*@cm-V z;wSOrr}8ZhVc3b3q9j`1l;x3T?8uBU`NtvRdzbA}(HG(g3jB4_dhv3ty@NYxgV4XH zkiN>9=cb%IE`Q)4%=+F)2zE#`C*V+8OuNw85mS3mmPR3ptsYv21}3Vw(tz46QQ82Z zUK4$FJp9(FTj?-f4}*8wj?F6+^p@nv@{y5EItg-_2zcJJ9B+#FF$vee5i7%^q&P!m zHZ7vzpI*Yp#8`fQ4u8H6E^2?;Dv-3}ATsBD-s*|9tq=@uYLJhp&Wc4KrN`Jvqk7OK zz&8JunqSCYDjVs}8N(+IBP3j|c8se=mzJ+D6c6q5d@}3^8{G|(q%cg_*@V853L)9J zuF{-gMKI8a9p}M824NJ#g1QgXqSR|3biCG@{XBd?=Aog^3<`Vp^I-cxZXx87{sVI~ zT4)IPPDhsfydxs+>rp4{30sSuoy`{{Cq#GX3UN0)xwo5tL3hp<1>WIeVx=r1dzNe& zElMS{+Z!R?QZVls2{4>x+2@rn*X|m311&elO(dj6_IxM6D zKHd*V7_b4aEfs~cCLSHbas8g*q5Hh>ic}eIpH{;4%z@58rkKRaDIuE&o-jzPjBgcN zSjPpO5?BLi!Ejgui}2M3J}=Th`OSiL+%0hotV`S^YTU9=gm5eD?33Ulp@>!75V^-H z;epDL>D{kiL@cuK=u0D{4-advPz!0Hr!WImzVg4(fhZ(vel8<9rPRLK*vgn`xqi8n zaRmv*_Hc@6P~M36TX-Kwt*zg{g^law)AyxL=wETH^B$*#t{|l$-kg!ukSDFK$h%LG yeEeWCAem~<P^y)YL5 literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json deleted file mode 100644 index efc3af110..000000000 --- a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json +++ /dev/null @@ -1,28 +0,0 @@ -{"datasourceId":"10|opendoar____::1068c6e4c8051cfd4e9ea8072e3189e2","organizationId":"20|opendoar____::4429502fa1936b0941f4647b69b844c8"} -{"datasourceId":"10|opendoar____::211ed78fe91938b90f84a51944b08d5a","organizationId":"20|opendoar____::124266ebc4ece2934eb80edfda3f2091"} -{"datasourceId":"10|opendoar____::21c5bba1dd6aed9ab48c2b34c1a0adde","organizationId":"20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2"} -{"datasourceId":"10|opendoar____::376c6b9ff3bedbbea56751a84fffc10c","organizationId":"20|opendoar____::c6ac1e6c39e6f3a36e274f8d3f9f7fce"} -{"datasourceId":"10|opendoar____::49c0fa7f96aa0a5fb95c62909d5190a6","organizationId":"20|dedup_wf_001::760a7568cbfdc750a5f92009a64bfc68"} -{"datasourceId":"10|opendoar____::4fa177df22864518b2d7818d4db5db2d","organizationId":"20|dedup_wf_001::b6b1fdfd98370bc44a5ecb215051fc61"} -{"datasourceId":"10|opendoar____::58521e4e2bd3d4b988cbd17d7365df3c","organizationId":"20|dedup_wf_001::09f3a5978add91bbfec7530e47c23fb6"} -{"datasourceId":"10|opendoar____::5dec707028b05bcbd3a1db5640f842c5","organizationId":"20|dedup_wf_001::7a34852b4b552a70789ccf070da2e558"} -{"datasourceId":"10|opendoar____::671d8d05d795091118b1344f715307c4","organizationId":"20|opendoar____::84e005eec7a780eafb0f5e54d438d56e"} -{"datasourceId":"10|opendoar____::79cc30c73507cfc25d20fe7f7bcfd91b","organizationId":"20|opendoar____::7230dc85992df9a4fa7026d44c9f5a24"} -{"datasourceId":"10|opendoar____::79e3eb7e992b7f766bdd77cc502ff082","organizationId":"20|dedup_wf_001::1e48cc6aee415348f0bdb3b7d3180091"} -{"datasourceId":"10|opendoar____::7af6266cc52234b5aa339b16695f7fc4","organizationId":"20|dedup_wf_001::cbfb7a433608c965dc59146c60e38dbf"} -{"datasourceId":"10|opendoar____::81e74d678581a3bb7a720b019f4f1a93","organizationId":"20|dedup_wf_001::3f6f17996747467f6047dfe019c8e4c7"} -{"datasourceId":"10|opendoar____::8744cf92c88433f8cb04a02e6db69a0d","organizationId":"20|opendoar____::f800da58d6ad6d35fc9cf57b982bbd1a"} -{"datasourceId":"10|opendoar____::8c01a75941549a705cf7275e41b21f0d","organizationId":"20|dedup_wf_001::ae993d8ee228ff161c2a8724f11c2403"} -{"datasourceId":"10|opendoar____::9cb67ffb59554ab1dabb65bcb370ddd9","organizationId":"20|opendoar____::2404720b8a764a4f576240a51a6cdd13"} -{"datasourceId":"10|opendoar____::9fc3d7152ba9336a670e36d0ed79bc43","organizationId":"20|dedup_wf_001::dc3a6f93b2cc529cd247a255b5127525"} -{"datasourceId":"10|opendoar____::a655fbe4b8d7439994aa37ddad80de56","organizationId":"20|opendoar____::bba529dfcc369815510999ccadff78d0"} -{"datasourceId":"10|opendoar____::aebf7782a3d445f43cf30ee2c0d84dee","organizationId":"20|opendoar____::931aa8950244dea6b3fe9f7798b916ea"} -{"datasourceId":"10|opendoar____::c4819d06b0ca810d38506453cfaae9d8","organizationId":"20|dedup_wf_001::bee19d6bbb94675a7a4aaf2b66f94ae3"} -{"datasourceId":"10|opendoar____::d88518acbcc3d08d1f18da62f9bb26ec","organizationId":"20|dedup_wf_001::f1914dbbcc2861e30d7e6b0e605013fd"} -{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|dedup_wf_001::b19557d6f918479582295814b1dfb38b"} -{"datasourceId":"10|opendoar____::d8bf84be3800d12f74d8b05e9b89836f","organizationId":"20|opendoar____::c9c886345622486e072060cb814b9ebf"} -{"datasourceId":"10|opendoar____::e9fb2eda3d9c55a0d89c98d6c54b5b3e","organizationId":"20|opendoar____::64cf1b2094ff19786a573980568762b4"} -{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::3f339760507dd23f9be59be2cff8204f"} -{"datasourceId":"10|opendoar____::f095cedd23b99f1696fc8caecbcf257e","organizationId":"20|dedup_wf_001::76548cf94fcd27f67759cc6148ee1e7f"} -{"datasourceId":"10|opendoar____::f35a2bc72dfdc2aae569a0c7370bd7f5","organizationId":"20|dedup_wf_001::2476836f27326ee53e586a82cbaedb1a"} -{"datasourceId":"10|opendoar____::f91e24dfe80012e2a7984afa4480a6d6","organizationId":"20|opendoar____::5d462d78d512c1184dd384ef2dc35b7e"} \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..323d66d6e7daf06e4e943c8be9412908e35147bf GIT binary patch literal 1259 zcmVV>{GB_@3b8l_{wO31S zEHn_@bBwH6_|qRdH{cXc{c$5UFX#b6;=$diVVA%NFCbE+Or&X-yQ{0pUtiLGwx8a9 z`4Z>5e0hE8?f17I&*yxzFQ4%9`f6wL+)LE#oRGGShP046u7itSzI}Q7^1Xfj;WzvF zhqupv6&W9k5RJ`kl=VHu+ooP5q%PYRQLL9gzWMs^NLnMYr08xHbgI#zew4Y!nzF8t z^a1#C9N&*K_N3#hovgH3qwW4rNIq2HHJ%)mCx7|z)35cZwf_3bUDobug2o+V92!BJui$_0nhUZN3y>PMYnmUEE=CkR%N76QC!dsVR z?(SX(otf>B#+c=Dha~Z>rfn>5>_9QCqa8x67LDM$>DVOP$jQN0L%Fph3RTm_VZNJ= zXwQ|6Q?c^9cg&|xqSB5_Y1|3MqSG2faMTq0Y2~f>FeS~EEO(4mpr0*tGS00SoTJsz zQjdCc`U;7T*)kUIJ+dHvSb;Vqp<27x23;aK6MA`2#BsieXR1a#G_l&Ho`7g8;-Myl zR?@2G>C-i5U+sx>{T+#wKmc*{7!z|gn{#^?+5C`#XVZC)1l#LsG^ex`8&nV8ogh^0 zU_4$|NEW1iDH`2p;!xd zD(rA0YvV}5psYR9llMgmGf3l=uqesN6+4O*nlK!iV0TEOb?77fk!gcy9Hm1U1BWuz zQzY+uVxZZ~bq@5gs!NYyO9#4&zo^YCBs(1KEa%Nk<~_|_9>M~(Oz6~APjDJNhPq6J zusW=Qe;NEsBC}s2shA1z7!FcQF;TEg5~i&ix_!r!jA>GP`gkyZ!+l9@Fgthjcko>R z5kw6?Ija%SCz46_P!>*?QzM!gox3)~MeP=_c3tPSAT-;#F0EE1_^I0hS z;Kk>lh`9U460<>D3?az{BO?URDg@b`&_C$nC0UjW#oX)DeFwIw4_4@-5;%4kzay&F zXmr<;{{)E&BNkv!3Ze!BRM-bnLMIpi9QG2)M3nJRM4W_M7{m<5hvbB4oLYD35t1qo zQVpHhx5M)=CbUMl;puYefS~&s1Pcda`#4CUdy$n9Exa-EPGdPq9&ZA?OhzDpQbL#T VU>vXkMhtuT<4?4@bk6t=002F&Zpi=u literal 0 HcmV?d00001 From fd5d792e353c5121a0c3fedbfba2ff9b5167de45 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:53:34 +0200 Subject: [PATCH 107/259] refactoring --- .../SparkCountryPropagationJob2.java | 6 +- .../PrepareResultOrcidAssociationStep1.java | 88 ++++ .../PrepareResultOrcidAssociationStep2.java | 119 ++++++ .../ResultOrcidList.java | 26 ++ .../ResultWithOrcid.java | 4 - .../SparkOrcidToResultFromSemRelJob3.java | 398 ++++++++++++++++++ .../oozie_app/config-default.xml | 18 - .../oozie_app/workflow.xml | 55 --- .../input_orcidtoresult_parameters.json | 12 + ...input_prepareorcidtoresult_parameters.json | 32 ++ ...nput_prepareorcidtoresult_parameters2.json | 38 ++ .../oozie_app/config-default.xml | 54 +++ .../oozie_app/workflow.xml | 71 ++++ 13 files changed, 841 insertions(+), 80 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml delete mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/{orcidtoresultfromremrel => orcidtoresultfromsemrel}/input_orcidtoresult_parameters.json (55%) create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 029be645d..745a99db8 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -121,9 +121,9 @@ public class SparkCountryPropagationJob2 { .map(r -> new Tuple2<>(r.getId(), r), Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); - Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), - pu), - Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); +// Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), +// pu), +// Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); Dataset new_table = result_pair .joinWith(potentialUpdates, result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), "left_outer") diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java new file mode 100644 index 000000000..c8a7b90ce --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -0,0 +1,88 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultOrcidAssociation { + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociation.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo(spark, inputPath, outputPath, resultClazz, resultType); + }); + } + + private static void prepareInfo(SparkSession spark, String inputPath, + String outputPath, Class resultClazz, + String resultType) { + + //read the relation table and the table related to the result it is using + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + relation.createOrReplaceTempView("relation"); + + log.info("Reading Graph table from: {}", inputPath + "/" + resultType); + Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + + + + + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java new file mode 100644 index 000000000..4af652ef0 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -0,0 +1,119 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultOrcidAssociationStep1 { + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo(spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); + }); + } + + private static void prepareInfo(SparkSession spark, String inputPath, + String outputPath, Class resultClazz, + String resultType, + List allowedsemrel) { + + //read the relation table and the table related to the result it is using + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + relation.createOrReplaceTempView("relation"); + + log.info("Reading Graph table from: {}", inputPath + "/" + resultType); + Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + + result.createOrReplaceTempView("result"); + + getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath); + + } + + private static void getPossibleResultOrcidAssociation(SparkSession spark, List allowedsemrel, String outputPath){ + String query = " select target resultId, author authorList" + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from result " + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false " + + getConstraintList(" relclass = '" ,allowedsemrel) + ") rel_rel " + + " on source = id"; + + spark.sql(query) + .as(Encoders.bean(ResultOrcidList.class)) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression","gzip") + .text(outputPath) + ; + } + + +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java new file mode 100644 index 000000000..e90795b1d --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java @@ -0,0 +1,26 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +public class ResultWithOrcid implements Serializable { + String id; + List authorList = new ArrayList<>(); + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public List getAuthorList() { + return authorList; + } + + public void setAuthorList(List authorList) { + this.authorList = authorList; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java deleted file mode 100644 index 49fbea567..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultWithOrcid.java +++ /dev/null @@ -1,4 +0,0 @@ -package eu.dnetlib.dhp.orcidtoresultfromsemrel; - -public class ResultWithOrcid { -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java new file mode 100644 index 000000000..ddeee391c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -0,0 +1,398 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.*; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkOrcidToResultFromSemRelJob2 { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob2.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkOrcidToResultFromSemRelJob2.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/orcidtoresult"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); + + org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); + + + relation.createOrReplaceTempView("relation"); + String query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + getConstraintList(" relclass = '" , allowedsemrel); + + org.apache.spark.sql.Dataset result_result = spark.sql(query); + + publication.createOrReplaceTempView("publication"); + org.apache.spark.sql.Dataset pubs_with_orcid = getResultWithOrcid("publication", spark) + .as(Encoders.bean(ResultWithOrcid.class)); + + dataset.createOrReplaceTempView("dataset"); + org.apache.spark.sql.Dataset dats_with_orcid = getResultWithOrcid("dataset", spark) + .as(Encoders.bean(ResultWithOrcid.class)); + + other.createOrReplaceTempView("orp"); + org.apache.spark.sql.Dataset orp_with_orcid = getResultWithOrcid("orp", spark) + .as(Encoders.bean(ResultWithOrcid.class)); + + dataset.createOrReplaceTempView("software"); + org.apache.spark.sql.Dataset software_with_orcid = getResultWithOrcid("software", spark) + .as(Encoders.bean(ResultWithOrcid.class)); + //get the results having at least one author pid we are interested in + + //target of the relation from at least one source with orcid. + //the set of authors contains all those that have orcid and are related to target + //from any source with allowed semantic relationship + JavaPairRDD> target_authorlist_from_pubs = getTargetAutoritativeAuthorList(pubs_with_orcid); + + JavaPairRDD> target_authorlist_from_dats = getTargetAutoritativeAuthorList(dats_with_orcid); + + JavaPairRDD> target_authorlist_from_orp = getTargetAutoritativeAuthorList(orp_with_orcid); + + JavaPairRDD> target_authorlist_from_sw = getTargetAutoritativeAuthorList(software_with_orcid); + + if(writeUpdate){ + target_authorlist_from_dats.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_dats"); + target_authorlist_from_pubs.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_pubs"); + target_authorlist_from_orp.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_orp"); + target_authorlist_from_sw.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_sw"); + } + + if(saveGraph){ + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)) + .mapToPair(p -> new Tuple2<>(p.getId(),p)) + .leftOuterJoin(target_authorlist_from_pubs) + .map(c -> { + Result r = c._2()._1(); + if(!c._2()._2().isPresent()){ + return r; + } + List toenrich_author = r.getAuthor(); + List autoritativeAuthors = c._2()._2().get(); + for(eu.dnetlib.dhp.schema.oaf.Author author: toenrich_author){ + if (!containsAllowedPid(author)){ + enrichAuthor(author, autoritativeAuthors); + } + } + return r; + }); + + } + + } + + private static void enrichAuthor(eu.dnetlib.dhp.schema.oaf.Author a, List au){ + for (AutoritativeAuthor aa: au){ + if(enrichAuthor(aa, a)){ + return; + } + } + + } + +// private static JavaPairRDD> getTargetAutoritativeAuthorList(org.apache.spark.sql.Dataset result_result, org.apache.spark.sql.Dataset pubs_with_orcid) { +// return pubs_with_orcid +// .toJavaRDD() +// .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) +// .join(result_result.toJavaRDD().mapToPair(rel -> new Tuple2<>(rel.getString(0), rel.getString(1)))) +// .mapToPair(c -> new Tuple2<>(c._2._2(), c._2()._1())) +// .reduceByKey((a, b) -> { +// if(a == null){ +// return b; +// } +// if(b==null){ +// return a; +// } +// +// Set authSet = new HashSet<>(); +// a.stream().forEach(au -> authSet.add(au.getOrcid())); +// +// b.stream().forEach(au -> { +// if (!authSet.contains(au.getOrcid())) { +// a.add(au); +// } +// } +// ); +// return a; +// }); +// } +private static JavaPairRDD> getTargetAutoritativeAuthorList( org.apache.spark.sql.Dataset pubs_with_orcid) { + return pubs_with_orcid + .toJavaRDD() + .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) + .reduceByKey((a, b) -> { + if(a == null){ + return b; + } + if(b==null){ + return a; + } + Set authSet = new HashSet<>(); + a.stream().forEach(au -> authSet.add(au.getOrcid())); + + b.stream().forEach(au -> { + if (!authSet.contains(au.getOrcid())) { + a.add(au); + } + } + ); + return a; + }); +} + + private static org.apache.spark.sql.Dataset getResultWithOrcid(String table, SparkSession spark){ + String query = " select target, author " + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from " + table + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false and (relclass = 'isSupplementedBy' or relclass = 'isSupplementTo') rel_rel " + + " on source = id"; + + return spark.sql(query); + } + + + private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { + boolean toaddpid = false; + + if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { + if (StringUtils.isNoneEmpty(author.getSurname())) { + if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { + + //have the same surname. Check the name + if (StringUtils.isNoneEmpty(autoritative_author.getName())) { + if (StringUtils.isNoneEmpty(author.getName())) { + if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { + toaddpid = true; + } + //they could be differently written (i.e. only the initials of the name in one of the two + if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { + toaddpid = true; + } + } + } + } + } + } + if (toaddpid){ + StructuredProperty pid = new StructuredProperty(); + String aa_pid = autoritative_author.getOrcid(); + pid.setValue(aa_pid); + pid.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); + pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + if(author.getPid() == null){ + author.setPid(Arrays.asList(pid)); + }else{ + author.getPid().add(pid); + } + + } + return toaddpid; + + } + + +// private static List enrichAuthors(List autoritative_authors, List to_enrich_authors, boolean filter){ +//// List autoritative_authors = p._2()._2().get().getAuthors(); +//// List to_enrich_authors = r.getAuthor(); +// +// return to_enrich_authors +// .stream() +// .map(a -> { +// if (filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } +// +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if (lst.size() == 0) { +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// +// }).collect(Collectors.toList()); +// } +// +// private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, +// String outputPath, String type) { +// +// results.join(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// +// List autoritative_authors = p._2()._2().getAuthors(); +// List to_enrich_authors = r.getAuthor(); +// +// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); +//// .stream() +//// .map(a -> { +//// if(filter) { +//// if (containsAllowedPid(a)) { +//// return a; +//// } +//// } +//// +//// List lst = autoritative_authors.stream() +//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +//// if(lst.size() == 0){ +//// return a; +//// } +//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +//// +//// }).collect(Collectors.toList())); +// +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/" + type + "_update"); +// } + + +// private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, +// String outputPath, String type) { +// results.leftOuterJoin(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// if (p._2()._2().isPresent()){ +// List autoritative_authors = p._2()._2().get().getAuthors(); +// List to_enrich_authors = r.getAuthor(); +// +// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); +//// .stream() +//// .map(a -> { +//// if(filter) { +//// if (containsAllowedPid(a)) { +//// return a; +//// } +//// } +//// +//// List lst = autoritative_authors.stream() +//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +//// if(lst.size() == 0){ +//// return a; +//// } +//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +//// +//// }).collect(Collectors.toList())); +// } +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath+"/"+type); +// } + + + + private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a) { + for (StructuredProperty pid : a.getPid()) { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return true; + } + } + return false; + } + +} + + +/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ + ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); + boolean updated = false; + + for (FieldTypeProtos.Author a: author_list){ + FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); + if(author != null){ + updated = true; + metadataBuilder.addAuthor(author); + }else{ + metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); + } + } + if(updated) + return metadataBuilder; + return null; + } + private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ + if(containsOrcid(a.getPidList())) + return null; + for(FieldTypeProtos.Author autoritative_author : author_list) { + if (equals(autoritative_author, a)) { + if(!containsOrcid(a.getPidList())) + return update(a, autoritative_author); + } + } + return null; + + } + + private boolean containsOrcid(List pidList){ + if(pidList == null) + return false; + return pidList + .stream() + .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) + .collect(Collectors.toList()).size() > 0; + } + */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml deleted file mode 100644 index 2e0ed9aee..000000000 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/config-default.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml deleted file mode 100644 index 15065b35a..000000000 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/oozie_app/workflow.xml +++ /dev/null @@ -1,55 +0,0 @@ - - - - sourcePath - the source path - - - allowedsemrels - the semantic relationships allowed for propagation - - - 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 - AffiliatioPropagation - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${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" - - -mt yarn-cluster - --sourcePath${sourcePath} - --allowedsemrels${allowedsemrels} - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json similarity index 55% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json index ffb314cdf..ea2f0b96c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json @@ -16,5 +16,17 @@ "paramLongName":"allowedsemrels", "paramDescription": "the allowed sematinc relations for propagation", "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json new file mode 100644 index 000000000..ea2f0b96c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName":"mt", + "paramLongName":"master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"as", + "paramLongName":"allowedsemrels", + "paramDescription": "the allowed sematinc relations for propagation", + "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json new file mode 100644 index 000000000..08648d61a --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json @@ -0,0 +1,38 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"as", + "paramLongName":"allowedsemrels", + "paramDescription": "the allowed sematinc relations for propagation", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml new file mode 100644 index 000000000..56d0ac43c --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml @@ -0,0 +1,54 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml new file mode 100644 index 000000000..b4fca5e58 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -0,0 +1,71 @@ + + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + + + + + + + + + + + + + writeUpdate + writes the information found for the update. No double check done if the information is already present + + + saveGraph + writes new version of the graph after the propagation step + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + OrcidToResultFromSemRelPropagation + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob + dhp-propagation-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + -mt yarn-cluster + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + --writeUpdate${writeUpdate} + --saveGraph${saveGraph} + + + + + + + \ No newline at end of file From f077f22f73c58fbb006c5187f2c5c8cad621e200 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:54:16 +0200 Subject: [PATCH 108/259] minor --- .../countrypropagation/input_prepareassoc_parameters.json | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json index 208c0ac44..95d4c1c60 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json @@ -28,5 +28,11 @@ "paramLongName": "allowedtypes", "paramDescription": "the allowed datasource types for country propagation", "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false } ] \ No newline at end of file From d6cd700a3236b4a33bbf71c4bf83caca18844c48 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:55:25 +0200 Subject: [PATCH 109/259] new implementation that exploits prepared information (the list of possible updates: resultId - possible list of orcid to be added --- .../SparkOrcidToResultFromSemRelJob3.java | 410 +++++------------- 1 file changed, 97 insertions(+), 313 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index ddeee391c..f9dfa2970 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -9,131 +9,115 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class SparkOrcidToResultFromSemRelJob3 { + private static final Logger log = LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); -public class SparkOrcidToResultFromSemRelJob2 { public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob2.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json"))); parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String possibleUpdates = parser.get("possibleUpdatesPath"); + log.info("possibleUpdatesPath: {}", possibleUpdates); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = (Class) Class.forName(resultClassName); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkOrcidToResultFromSemRelJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/orcidtoresult"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); - - org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); - - - relation.createOrReplaceTempView("relation"); - String query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false " + getConstraintList(" relclass = '" , allowedsemrel); - - org.apache.spark.sql.Dataset result_result = spark.sql(query); - - publication.createOrReplaceTempView("publication"); - org.apache.spark.sql.Dataset pubs_with_orcid = getResultWithOrcid("publication", spark) - .as(Encoders.bean(ResultWithOrcid.class)); - - dataset.createOrReplaceTempView("dataset"); - org.apache.spark.sql.Dataset dats_with_orcid = getResultWithOrcid("dataset", spark) - .as(Encoders.bean(ResultWithOrcid.class)); - - other.createOrReplaceTempView("orp"); - org.apache.spark.sql.Dataset orp_with_orcid = getResultWithOrcid("orp", spark) - .as(Encoders.bean(ResultWithOrcid.class)); - - dataset.createOrReplaceTempView("software"); - org.apache.spark.sql.Dataset software_with_orcid = getResultWithOrcid("software", spark) - .as(Encoders.bean(ResultWithOrcid.class)); - //get the results having at least one author pid we are interested in - - //target of the relation from at least one source with orcid. - //the set of authors contains all those that have orcid and are related to target - //from any source with allowed semantic relationship - JavaPairRDD> target_authorlist_from_pubs = getTargetAutoritativeAuthorList(pubs_with_orcid); - - JavaPairRDD> target_authorlist_from_dats = getTargetAutoritativeAuthorList(dats_with_orcid); - - JavaPairRDD> target_authorlist_from_orp = getTargetAutoritativeAuthorList(orp_with_orcid); - - JavaPairRDD> target_authorlist_from_sw = getTargetAutoritativeAuthorList(software_with_orcid); - - if(writeUpdate){ - target_authorlist_from_dats.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_dats"); - target_authorlist_from_pubs.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_pubs"); - target_authorlist_from_orp.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_orp"); - target_authorlist_from_sw.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_sw"); - } - - if(saveGraph){ - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)) - .mapToPair(p -> new Tuple2<>(p.getId(),p)) - .leftOuterJoin(target_authorlist_from_pubs) - .map(c -> { - Result r = c._2()._1(); - if(!c._2()._2().isPresent()){ - return r; - } - List toenrich_author = r.getAuthor(); - List autoritativeAuthors = c._2()._2().get(); - for(eu.dnetlib.dhp.schema.oaf.Author author: toenrich_author){ - if (!containsAllowedPid(author)){ - enrichAuthor(author, autoritativeAuthors); - } - } - return r; - }); - - } + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if(isTest(parser)) { + removeOutputDir(spark, outputPath); + } + if(saveGraph) + execPropagation(spark, possibleUpdates, inputPath, outputPath, resultClazz); + }); } - - private static void enrichAuthor(eu.dnetlib.dhp.schema.oaf.Author a, List au){ + + private static void execPropagation(SparkSession spark, String possibleUpdatesPath, String inputPath, + String outputPath, Class resultClazz ) { + + //read possible updates (resultId and list of possible orcid to add + Dataset possible_updates = readAssocResultOrcidList(spark, possibleUpdatesPath); + //read the result we have been considering + Dataset result = readPathEntity(spark, inputPath, resultClazz); + //make join result left_outer with possible updates + + result.joinWith(possible_updates, result.col("id").equalTo(possible_updates.col("resultId")), + "left_outer") + .map(value -> { + R ret = value._1(); + Optional rol = Optional.ofNullable(value._2()); + if(rol.isPresent()) { + List toenrich_author = ret.getAuthor(); + List autoritativeAuthors = rol.get().getAuthorList(); + for(Author author: toenrich_author){ + if (!containsAllowedPid(author)){ + enrichAuthor(author, autoritativeAuthors); + } + } + } + + return ret; + }, Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(outputPath); + + + } + + private static Dataset readAssocResultOrcidList(SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map(value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), Encoders.bean(ResultOrcidList.class)); + } + + private static void enrichAuthor(Author a, List au){ for (AutoritativeAuthor aa: au){ if(enrichAuthor(aa, a)){ return; @@ -142,77 +126,9 @@ public class SparkOrcidToResultFromSemRelJob2 { } -// private static JavaPairRDD> getTargetAutoritativeAuthorList(org.apache.spark.sql.Dataset result_result, org.apache.spark.sql.Dataset pubs_with_orcid) { -// return pubs_with_orcid -// .toJavaRDD() -// .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) -// .join(result_result.toJavaRDD().mapToPair(rel -> new Tuple2<>(rel.getString(0), rel.getString(1)))) -// .mapToPair(c -> new Tuple2<>(c._2._2(), c._2()._1())) -// .reduceByKey((a, b) -> { -// if(a == null){ -// return b; -// } -// if(b==null){ -// return a; -// } -// -// Set authSet = new HashSet<>(); -// a.stream().forEach(au -> authSet.add(au.getOrcid())); -// -// b.stream().forEach(au -> { -// if (!authSet.contains(au.getOrcid())) { -// a.add(au); -// } -// } -// ); -// return a; -// }); -// } -private static JavaPairRDD> getTargetAutoritativeAuthorList( org.apache.spark.sql.Dataset pubs_with_orcid) { - return pubs_with_orcid - .toJavaRDD() - .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) - .reduceByKey((a, b) -> { - if(a == null){ - return b; - } - if(b==null){ - return a; - } - Set authSet = new HashSet<>(); - a.stream().forEach(au -> authSet.add(au.getOrcid())); - - b.stream().forEach(au -> { - if (!authSet.contains(au.getOrcid())) { - a.add(au); - } - } - ); - return a; - }); -} - - private static org.apache.spark.sql.Dataset getResultWithOrcid(String table, SparkSession spark){ - String query = " select target, author " + - " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + - " from ( " + - " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + - " from " + table + - " lateral view explode (author) a as MyT " + - " lateral view explode (MyT.pid) p as MyP " + - " where MyP.qualifier.classid = 'ORCID') tmp " + - " group by id) r_t " + - " join (" + - " select source, target " + - " from relation " + - " where datainfo.deletedbyinference = false and (relclass = 'isSupplementedBy' or relclass = 'isSupplementTo') rel_rel " + - " on source = id"; - - return spark.sql(query); - } - private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { + private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, Author author) { boolean toaddpid = false; if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { @@ -252,100 +168,9 @@ private static JavaPairRDD> getTargetAutoritati } -// private static List enrichAuthors(List autoritative_authors, List to_enrich_authors, boolean filter){ -//// List autoritative_authors = p._2()._2().get().getAuthors(); -//// List to_enrich_authors = r.getAuthor(); -// -// return to_enrich_authors -// .stream() -// .map(a -> { -// if (filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } -// -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if (lst.size() == 0) { -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -// -// }).collect(Collectors.toList()); -// } -// -// private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, -// String outputPath, String type) { -// -// results.join(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// -// List autoritative_authors = p._2()._2().getAuthors(); -// List to_enrich_authors = r.getAuthor(); -// -// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); -//// .stream() -//// .map(a -> { -//// if(filter) { -//// if (containsAllowedPid(a)) { -//// return a; -//// } -//// } -//// -//// List lst = autoritative_authors.stream() -//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -//// if(lst.size() == 0){ -//// return a; -//// } -//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -//// -//// }).collect(Collectors.toList())); -// -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/" + type + "_update"); -// } -// private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, -// String outputPath, String type) { -// results.leftOuterJoin(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// if (p._2()._2().isPresent()){ -// List autoritative_authors = p._2()._2().get().getAuthors(); -// List to_enrich_authors = r.getAuthor(); -// -// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); -//// .stream() -//// .map(a -> { -//// if(filter) { -//// if (containsAllowedPid(a)) { -//// return a; -//// } -//// } -//// -//// List lst = autoritative_authors.stream() -//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -//// if(lst.size() == 0){ -//// return a; -//// } -//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -//// -//// }).collect(Collectors.toList())); -// } -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath+"/"+type); -// } - - - - private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a) { + private static boolean containsAllowedPid(Author a) { for (StructuredProperty pid : a.getPid()) { if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { return true; @@ -355,44 +180,3 @@ private static JavaPairRDD> getTargetAutoritati } } - - -/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ - ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); - boolean updated = false; - - for (FieldTypeProtos.Author a: author_list){ - FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); - if(author != null){ - updated = true; - metadataBuilder.addAuthor(author); - }else{ - metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); - } - } - if(updated) - return metadataBuilder; - return null; - } - private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ - if(containsOrcid(a.getPidList())) - return null; - for(FieldTypeProtos.Author autoritative_author : author_list) { - if (equals(autoritative_author, a)) { - if(!containsOrcid(a.getPidList())) - return update(a, autoritative_author); - } - } - return null; - - } - - private boolean containsOrcid(List pidList){ - if(pidList == null) - return false; - return pidList - .stream() - .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) - .collect(Collectors.toList()).size() > 0; - } - */ \ No newline at end of file From ac3ad25b364b002d0fe20a5e269b519bf66abc18 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:56:33 +0200 Subject: [PATCH 110/259] to (de)serialize needed information of the author to determine if the orcid can be passed (name, surname, fullname (?), orcid) --- .../AutoritativeAuthor.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java index 7e496c7cf..048e8ae46 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java @@ -1,4 +1,40 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; public class AutoritativeAuthor { + String name; + String surname; + String fullname; + String orcid; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getSurname() { + return surname; + } + + public void setSurname(String surname) { + this.surname = surname; + } + + public String getFullname() { + return fullname; + } + + public void setFullname(String fullname) { + this.fullname = fullname; + } + + public String getOrcid() { + return orcid; + } + + public void setOrcid(String orcid) { + this.orcid = orcid; + } } From 243013cea3979e61f0a5ad020389b75d6f2b035c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:57:29 +0200 Subject: [PATCH 111/259] to (de)serialize the association from the resultId and the list of autoritative authors with orcid to possibly propagate --- .../dhp/orcidtoresultfromsemrel/ResultOrcidList.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java index e90795b1d..9e2bc6e31 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java @@ -4,16 +4,16 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; -public class ResultWithOrcid implements Serializable { - String id; +public class ResultOrcidList implements Serializable { + String resultId; List authorList = new ArrayList<>(); - public String getId() { - return id; + public String getResultId() { + return resultId; } - public void setId(String id) { - this.id = id; + public void setResultId(String resultId) { + this.resultId = resultId; } public List getAuthorList() { From 548ba915ac644d75b0604485927705ab045a68a6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 15:58:42 +0200 Subject: [PATCH 112/259] first phase of data preparation. For each result type (parallel) it produces the possible updates --- .../PrepareResultOrcidAssociationStep1.java | 43 ++++++++++++++++--- 1 file changed, 37 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index c8a7b90ce..4af652ef0 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -10,19 +10,19 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Arrays; import java.util.List; -import java.util.Optional; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -public class PrepareResultOrcidAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociation.class); +public class PrepareResultOrcidAssociationStep1 { + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -64,13 +64,14 @@ public class PrepareResultOrcidAssociation { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - prepareInfo(spark, inputPath, outputPath, resultClazz, resultType); + prepareInfo(spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); }); } private static void prepareInfo(SparkSession spark, String inputPath, String outputPath, Class resultClazz, - String resultType) { + String resultType, + List allowedsemrel) { //read the relation table and the table related to the result it is using final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); @@ -81,8 +82,38 @@ public class PrepareResultOrcidAssociation { log.info("Reading Graph table from: {}", inputPath + "/" + resultType); Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + result.createOrReplaceTempView("result"); - + getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath); } + + private static void getPossibleResultOrcidAssociation(SparkSession spark, List allowedsemrel, String outputPath){ + String query = " select target resultId, author authorList" + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from result " + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false " + + getConstraintList(" relclass = '" ,allowedsemrel) + ") rel_rel " + + " on source = id"; + + spark.sql(query) + .as(Encoders.bean(ResultOrcidList.class)) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression","gzip") + .text(outputPath) + ; + } + + } From 0e5399bf74e717984c2bc12d5d3d2ef4b4517a23 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:08:51 +0200 Subject: [PATCH 113/259] seconf phase of data preparation. Groups all the possible updates by id --- .../PrepareResultOrcidAssociationStep2.java | 114 +++++++----------- 1 file changed, 43 insertions(+), 71 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index 4af652ef0..660d0b87a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -1,34 +1,27 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import java.util.Arrays; -import java.util.List; - +import scala.Tuple2; +import java.util.HashSet; +import java.util.Set; import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -public class PrepareResultOrcidAssociationStep1 { - private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); +public class PrepareResultOrcidAssociationStep2 { + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep2.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); + .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult2_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); @@ -44,76 +37,55 @@ public class PrepareResultOrcidAssociationStep1 { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); - - - Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkSession(conf, isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - prepareInfo(spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); + mergeInfo(spark, inputPath, outputPath); }); } - private static void prepareInfo(SparkSession spark, String inputPath, - String outputPath, Class resultClazz, - String resultType, - List allowedsemrel) { + private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - //read the relation table and the table related to the result it is using - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - relation.createOrReplaceTempView("relation"); + Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath); - log.info("Reading Graph table from: {}", inputPath + "/" + resultType); - Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); - - result.createOrReplaceTempView("result"); - - getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath); + resultOrcidAssoc + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) + .reduceByKey((a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set orcid_set = new HashSet<>(); + a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); + b.getAuthorList().stream().forEach(aa -> { + if (!orcid_set.contains(aa.getOrcid())) { + a.getAuthorList().add(aa); + orcid_set.add(aa.getOrcid()); + } + }); + return a; + }) + .map(c -> c._2()) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); } - private static void getPossibleResultOrcidAssociation(SparkSession spark, List allowedsemrel, String outputPath){ - String query = " select target resultId, author authorList" + - " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + - " from ( " + - " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + - " from result " + - " lateral view explode (author) a as MyT " + - " lateral view explode (MyT.pid) p as MyP " + - " where MyP.qualifier.classid = 'ORCID') tmp " + - " group by id) r_t " + - " join (" + - " select source, target " + - " from relation " + - " where datainfo.deletedbyinference = false " + - getConstraintList(" relclass = '" ,allowedsemrel) + ") rel_rel " + - " on source = id"; - - spark.sql(query) - .as(Encoders.bean(ResultOrcidList.class)) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression","gzip") - .text(outputPath) - ; + private static Dataset readAssocResultOrcidList(SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map(value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), Encoders.bean(ResultOrcidList.class)); } + + } From c33a5933815b2a8fb6599f9ca65a266e58ba2da3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:09:47 +0200 Subject: [PATCH 114/259] renamed --- .../java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java deleted file mode 100644 index 18332bc8f..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/Author.java +++ /dev/null @@ -1,4 +0,0 @@ -package eu.dnetlib.dhp.orcidtoresultfromsemrel; - -public class Author { -} From 5e72a51f11bf98d4dfc20fefe95b592c00850879 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:11:20 +0200 Subject: [PATCH 115/259] - --- .../SparkOrcidToResultFromSemRelJob.java | 160 +++++-- .../SparkOrcidToResultFromSemRelJob2.java | 447 +++++++++++------- 2 files changed, 375 insertions(+), 232 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java index adc92aa75..cddca14ed 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java @@ -4,31 +4,26 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Author; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.fs.FileSystem; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.io.File; import java.util.Arrays; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.Stream; import static eu.dnetlib.dhp.PropagationConstant.*; public class SparkOrcidToResultFromSemRelJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcid" + - "toresult_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json"))); parser.parseArgument(args); final SparkSession spark = SparkSession .builder() @@ -42,26 +37,24 @@ public class SparkOrcidToResultFromSemRelJob { final String outputPath = "/tmp/provision/propagation/orcidtoresult"; final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - File directory = new File(outputPath); + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - if (!directory.exists()) { - directory.mkdirs(); - } - - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + JavaRDD relations = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).cache(); JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)); + JavaRDD publications = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD datasets = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD software = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); //get the results having at least one author pid we are interested in JavaPairRDD resultswithorcid = publications.map(p -> getTypedRow(p)) @@ -87,15 +80,25 @@ public class SparkOrcidToResultFromSemRelJob { JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); - updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); - updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); - updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + if(writeUpdate){ + writeResult(pubs, to_add_orcid_to_result, outputPath, "publication"); + writeResult(dss, to_add_orcid_to_result, outputPath, "dataset"); + writeResult(sfw, to_add_orcid_to_result, outputPath, "software"); + writeResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + } + + if (saveGraph){ + updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); + updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); + updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); + updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + } + } - private static Author enrichAutor(Author autoritative_author, Author author) { + public static eu.dnetlib.dhp.schema.oaf.Author enrichAutor(eu.dnetlib.dhp.schema.oaf.Author autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { boolean toaddpid = false; if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { @@ -137,32 +140,91 @@ public class SparkOrcidToResultFromSemRelJob { } + private static List enrichAutors(List autoritative_authors, + List to_enrich_authors, boolean filter){ +// List autoritative_authors = p._2()._2().get().getAuthors(); +// List to_enrich_authors = r.getAuthor(); - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + return to_enrich_authors + .stream() + .map(a -> { + if (filter) { + if (containsAllowedPid(a)) { + return a; + } + } + + List lst = autoritative_authors.stream() + .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); + if (lst.size() == 0) { + return a; + } + return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people + + }).collect(Collectors.toList()); + } + + private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, + String outputPath, String type) { + + results.join(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + + List autoritative_authors = p._2()._2().getAuthors(); + List to_enrich_authors = r.getAuthor(); + + r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); +// .stream() +// .map(a -> { +// if(filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } +// +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if(lst.size() == 0){ +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// +// }).collect(Collectors.toList())); + + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type + "_update"); + } + + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, + String outputPath, String type) { results.leftOuterJoin(toupdateresult) .map(p -> { Result r = p._2()._1(); if (p._2()._2().isPresent()){ - List autoritative_authors = p._2()._2().get().getAuthors(); - List to_enrich_authors = r.getAuthor(); - //.stream().filter(a -> !containsAllowedPid(a)) - //.collect(Collectors.toList()); + List autoritative_authors = p._2()._2().get().getAuthors(); + List to_enrich_authors = r.getAuthor(); - r.setAuthor(to_enrich_authors - .stream() - .map(a -> { - if (containsAllowedPid(a)) { - return a; - } - - List lst = autoritative_authors.stream() - .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); - if(lst.size() == 0){ - return a; - } - return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people - - }).collect(Collectors.toList())); + r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); +// .stream() +// .map(a -> { +// if(filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } +// +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if(lst.size() == 0){ +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// +// }).collect(Collectors.toList())); } return r; }) @@ -195,7 +257,7 @@ public class SparkOrcidToResultFromSemRelJob { } - private static boolean containsAllowedPid(Author a){ + private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a){ return (a.getPid().stream().map(pid -> { diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java index 73b8895e1..1b58badfa 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java @@ -1,40 +1,35 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import java.util.*; import static eu.dnetlib.dhp.PropagationConstant.*; -public class SparkOrcidToResultFromSemRelJob { +public class SparkOrcidToResultFromSemRelJob2 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromremrel/input_orcidtoresult_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob2.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json"))); parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() - .appName(SparkOrcidToResultFromSemRelJob.class.getSimpleName()) + .appName(SparkOrcidToResultFromSemRelJob2.class.getSimpleName()) .master(parser.get("master")) + .config(conf) .enableHiveSupport() .getOrCreate(); @@ -48,63 +43,176 @@ public class SparkOrcidToResultFromSemRelJob { createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - JavaRDD relations = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).cache(); + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); + org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - JavaRDD publications = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD datasets = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD software = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); + org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); + + org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), + Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); + + + relation.createOrReplaceTempView("relation"); + String query = "Select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + getConstraintList(" relclass = '" , allowedsemrel); + + org.apache.spark.sql.Dataset result_result = spark.sql(query); + + publication.createOrReplaceTempView("publication"); + org.apache.spark.sql.Dataset pubs_with_orcid = getResultWithOrcid("publication", spark) + .as(Encoders.bean(ResultOrcidList.class)); + + dataset.createOrReplaceTempView("dataset"); + org.apache.spark.sql.Dataset dats_with_orcid = getResultWithOrcid("dataset", spark) + .as(Encoders.bean(ResultOrcidList.class)); + + other.createOrReplaceTempView("orp"); + org.apache.spark.sql.Dataset orp_with_orcid = getResultWithOrcid("orp", spark) + .as(Encoders.bean(ResultOrcidList.class)); + + dataset.createOrReplaceTempView("software"); + org.apache.spark.sql.Dataset software_with_orcid = getResultWithOrcid("software", spark) + .as(Encoders.bean(ResultOrcidList.class)); //get the results having at least one author pid we are interested in - JavaPairRDD resultswithorcid = publications.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())) - .union(software.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())) - .union(other.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())); + //target of the relation from at least one source with orcid. + //the set of authors contains all those that have orcid and are related to target + //from any source with allowed semantic relationship + JavaPairRDD> target_authorlist_from_pubs = getTargetAutoritativeAuthorList(pubs_with_orcid); - JavaPairRDD to_add_orcid_to_result = resultswithorcid.join(result_result) - .map(p -> p._2()._1().setSourceId(p._2()._2().getTargetId())) //associate the pid of the result (target) which should get the orcid to the typed row containing the authors with the orcid from the result(source) - .mapToPair(toPair()); + JavaPairRDD> target_authorlist_from_dats = getTargetAutoritativeAuthorList(dats_with_orcid); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD> target_authorlist_from_orp = getTargetAutoritativeAuthorList(orp_with_orcid); + + JavaPairRDD> target_authorlist_from_sw = getTargetAutoritativeAuthorList(software_with_orcid); if(writeUpdate){ - writeResult(pubs, to_add_orcid_to_result, outputPath, "publication"); - writeResult(dss, to_add_orcid_to_result, outputPath, "dataset"); - writeResult(sfw, to_add_orcid_to_result, outputPath, "software"); - writeResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + target_authorlist_from_dats.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_dats"); + target_authorlist_from_pubs.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_pubs"); + target_authorlist_from_orp.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_orp"); + target_authorlist_from_sw.map(r -> new ObjectMapper().writeValueAsString(r)) + .saveAsTextFile(outputPath + "/" + "update_sw"); + } + + if(saveGraph){ + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)) + .mapToPair(p -> new Tuple2<>(p.getId(),p)) + .leftOuterJoin(target_authorlist_from_pubs) + .map(c -> { + Result r = c._2()._1(); + if(!c._2()._2().isPresent()){ + return r; + } + List toenrich_author = r.getAuthor(); + List autoritativeAuthors = c._2()._2().get(); + for(eu.dnetlib.dhp.schema.oaf.Author author: toenrich_author){ + if (!containsAllowedPid(author)){ + enrichAuthor(author, autoritativeAuthors); + } + } + return r; + }); + } - if (saveGraph){ - updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); - updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); - updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); - updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); + } + + private static void enrichAuthor(eu.dnetlib.dhp.schema.oaf.Author a, List au){ + for (AutoritativeAuthor aa: au){ + if(enrichAuthor(aa, a)){ + return; + } } - } +// private static JavaPairRDD> getTargetAutoritativeAuthorList(org.apache.spark.sql.Dataset result_result, org.apache.spark.sql.Dataset pubs_with_orcid) { +// return pubs_with_orcid +// .toJavaRDD() +// .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) +// .join(result_result.toJavaRDD().mapToPair(rel -> new Tuple2<>(rel.getString(0), rel.getString(1)))) +// .mapToPair(c -> new Tuple2<>(c._2._2(), c._2()._1())) +// .reduceByKey((a, b) -> { +// if(a == null){ +// return b; +// } +// if(b==null){ +// return a; +// } +// +// Set authSet = new HashSet<>(); +// a.stream().forEach(au -> authSet.add(au.getOrcid())); +// +// b.stream().forEach(au -> { +// if (!authSet.contains(au.getOrcid())) { +// a.add(au); +// } +// } +// ); +// return a; +// }); +// } +private static JavaPairRDD> getTargetAutoritativeAuthorList( org.apache.spark.sql.Dataset pubs_with_orcid) { + return pubs_with_orcid + .toJavaRDD() + .mapToPair(p -> new Tuple2<>(p.getResultId(), p.getAuthorList())) + .reduceByKey((a, b) -> { + if(a == null){ + return b; + } + if(b==null){ + return a; + } + Set authSet = new HashSet<>(); + a.stream().forEach(au -> authSet.add(au.getOrcid())); - private static Author enrichAutor(Author autoritative_author, Author author) { + b.stream().forEach(au -> { + if (!authSet.contains(au.getOrcid())) { + a.add(au); + } + } + ); + return a; + }); +} + + private static org.apache.spark.sql.Dataset getResultWithOrcid(String table, SparkSession spark){ + String query = " select target, author " + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from " + table + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false and (relclass = 'isSupplementedBy' or relclass = 'isSupplementTo') rel_rel " + + " on source = id"; + + return spark.sql(query); + } + + + private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { boolean toaddpid = false; if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { @@ -128,149 +236,122 @@ public class SparkOrcidToResultFromSemRelJob { } if (toaddpid){ StructuredProperty pid = new StructuredProperty(); - for(StructuredProperty sp : autoritative_author.getPid()){ - if (PROPAGATION_AUTHOR_PID.equals(sp.getQualifier().getClassid())){ - pid.setValue(sp.getValue()); - pid.setQualifier(getQualifier(sp.getQualifier().getClassid(),sp.getQualifier().getClassname() )); - pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - if(author.getPid() == null){ - author.setPid(Arrays.asList(pid)); - }else{ - author.getPid().add(pid); - } - } + String aa_pid = autoritative_author.getOrcid(); + pid.setValue(aa_pid); + pid.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); + pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + if(author.getPid() == null){ + author.setPid(Arrays.asList(pid)); + }else{ + author.getPid().add(pid); } - return author; + } - return null; + return toaddpid; + } - private static List enrichAutors(List autoritative_authors, List to_enrich_authors, boolean filter){ -// List autoritative_authors = p._2()._2().get().getAuthors(); -// List to_enrich_authors = r.getAuthor(); - - return to_enrich_authors - .stream() - .map(a -> { - if (filter) { - if (containsAllowedPid(a)) { - return a; - } - } - - List lst = autoritative_authors.stream() - .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); - if (lst.size() == 0) { - return a; - } - return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people - - }).collect(Collectors.toList()); - } - - private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, - String outputPath, String type) { - - results.join(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - - List autoritative_authors = p._2()._2().getAuthors(); - List to_enrich_authors = r.getAuthor(); - - r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); -// .stream() -// .map(a -> { -// if(filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } +// private static List enrichAuthors(List autoritative_authors, List to_enrich_authors, boolean filter){ +//// List autoritative_authors = p._2()._2().get().getAuthors(); +//// List to_enrich_authors = r.getAuthor(); // -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if(lst.size() == 0){ -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// return to_enrich_authors +// .stream() +// .map(a -> { +// if (filter) { +// if (containsAllowedPid(a)) { +// return a; +// } +// } // -// }).collect(Collectors.toList())); - - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type + "_update"); - } - - - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, - String outputPath, String type) { - results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - List autoritative_authors = p._2()._2().get().getAuthors(); - List to_enrich_authors = r.getAuthor(); - - r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); -// .stream() -// .map(a -> { -// if(filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } +// List lst = autoritative_authors.stream() +// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +// if (lst.size() == 0) { +// return a; +// } +// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people // -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if(lst.size() == 0){ -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +// }).collect(Collectors.toList()); +// } // -// }).collect(Collectors.toList())); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - - private static TypedRow getTypedRow(Result p) { - TypedRow tp = new TypedRow(); - tp.setSourceId(p.getId()); - List authorList = p.getAuthor() - .stream() - .map(a -> { - if (a.getPid().stream().map(pid -> { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { - return a; - } - return null; - }).filter(aut -> !(aut == null)).collect(Collectors.toList()).size() > 0){ - return a; - } - return null; - }).filter(a -> !(a == null)).collect(Collectors.toList()); - tp.setAuthors(authorList); - if(authorList.size() > 0){ - return tp; - } - return null; +// private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, +// String outputPath, String type) { +// +// results.join(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// +// List autoritative_authors = p._2()._2().getAuthors(); +// List to_enrich_authors = r.getAuthor(); +// +// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); +//// .stream() +//// .map(a -> { +//// if(filter) { +//// if (containsAllowedPid(a)) { +//// return a; +//// } +//// } +//// +//// List lst = autoritative_authors.stream() +//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +//// if(lst.size() == 0){ +//// return a; +//// } +//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +//// +//// }).collect(Collectors.toList())); +// +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath + "/" + type + "_update"); +// } - } +// private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, +// String outputPath, String type) { +// results.leftOuterJoin(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// if (p._2()._2().isPresent()){ +// List autoritative_authors = p._2()._2().get().getAuthors(); +// List to_enrich_authors = r.getAuthor(); +// +// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); +//// .stream() +//// .map(a -> { +//// if(filter) { +//// if (containsAllowedPid(a)) { +//// return a; +//// } +//// } +//// +//// List lst = autoritative_authors.stream() +//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); +//// if(lst.size() == 0){ +//// return a; +//// } +//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people +//// +//// }).collect(Collectors.toList())); +// } +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath+"/"+type); +// } - private static boolean containsAllowedPid(Author a){ + - - return (a.getPid().stream().map(pid -> { + private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a) { + for (StructuredProperty pid : a.getPid()) { if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { return true; } - return false; - }).filter(aut -> (aut == true)).collect(Collectors.toList()).size()) > 0; + } + return false; } } From 4d89f3dfed10860f26a548820914f3e2c3fad000 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:11:44 +0200 Subject: [PATCH 116/259] removed unuseful classes --- .../SparkOrcidToResultFromSemRelJob.java | 312 -------------- .../SparkOrcidToResultFromSemRelJob2.java | 398 ------------------ 2 files changed, 710 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java deleted file mode 100644 index cddca14ed..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java +++ /dev/null @@ -1,312 +0,0 @@ -package eu.dnetlib.dhp.orcidtoresultfromsemrel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.Author; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkOrcidToResultFromSemRelJob { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = SparkSession - .builder() - .appName(SparkOrcidToResultFromSemRelJob.class.getSimpleName()) - .master(parser.get("master")) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/orcidtoresult"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD relations = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).cache(); - - JavaPairRDD result_result = getResultResultSemRel(allowedsemrel, relations); - - JavaRDD publications = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD datasets = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD software = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - JavaRDD other = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); - - //get the results having at least one author pid we are interested in - JavaPairRDD resultswithorcid = publications.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())) - .union(software.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())) - .union(other.map(p -> getTypedRow(p)) - .filter(p -> !(p == null)) - .mapToPair(toPair())); - - - JavaPairRDD to_add_orcid_to_result = resultswithorcid.join(result_result) - .map(p -> p._2()._1().setSourceId(p._2()._2().getTargetId())) //associate the pid of the result (target) which should get the orcid to the typed row containing the authors with the orcid from the result(source) - .mapToPair(toPair()); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - if(writeUpdate){ - writeResult(pubs, to_add_orcid_to_result, outputPath, "publication"); - writeResult(dss, to_add_orcid_to_result, outputPath, "dataset"); - writeResult(sfw, to_add_orcid_to_result, outputPath, "software"); - writeResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); - } - - if (saveGraph){ - updateResult(pubs, to_add_orcid_to_result, outputPath, "publication"); - updateResult(dss, to_add_orcid_to_result, outputPath, "dataset"); - updateResult(sfw, to_add_orcid_to_result, outputPath, "software"); - updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct"); - } - - - } - - - public static eu.dnetlib.dhp.schema.oaf.Author enrichAutor(eu.dnetlib.dhp.schema.oaf.Author autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { - boolean toaddpid = false; - - if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { - if (StringUtils.isNoneEmpty(author.getSurname())) { - if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { - - //have the same surname. Check the name - if (StringUtils.isNoneEmpty(autoritative_author.getName())) { - if (StringUtils.isNoneEmpty(author.getName())) { - if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { - toaddpid = true; - } - //they could be differently written (i.e. only the initials of the name in one of the two - if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { - toaddpid = true; - } - } - } - } - } - } - if (toaddpid){ - StructuredProperty pid = new StructuredProperty(); - for(StructuredProperty sp : autoritative_author.getPid()){ - if (PROPAGATION_AUTHOR_PID.equals(sp.getQualifier().getClassid())){ - pid.setValue(sp.getValue()); - pid.setQualifier(getQualifier(sp.getQualifier().getClassid(),sp.getQualifier().getClassname() )); - pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - if(author.getPid() == null){ - author.setPid(Arrays.asList(pid)); - }else{ - author.getPid().add(pid); - } - } - } - return author; - } - return null; - } - - - private static List enrichAutors(List autoritative_authors, - List to_enrich_authors, boolean filter){ -// List autoritative_authors = p._2()._2().get().getAuthors(); -// List to_enrich_authors = r.getAuthor(); - - return to_enrich_authors - .stream() - .map(a -> { - if (filter) { - if (containsAllowedPid(a)) { - return a; - } - } - - List lst = autoritative_authors.stream() - .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); - if (lst.size() == 0) { - return a; - } - return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people - - }).collect(Collectors.toList()); - } - - private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, - String outputPath, String type) { - - results.join(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - - List autoritative_authors = p._2()._2().getAuthors(); - List to_enrich_authors = r.getAuthor(); - - r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); -// .stream() -// .map(a -> { -// if(filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } -// -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if(lst.size() == 0){ -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -// -// }).collect(Collectors.toList())); - - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type + "_update"); - } - - - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, - String outputPath, String type) { - results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - List autoritative_authors = p._2()._2().get().getAuthors(); - List to_enrich_authors = r.getAuthor(); - - r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); -// .stream() -// .map(a -> { -// if(filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } -// -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if(lst.size() == 0){ -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -// -// }).collect(Collectors.toList())); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - - private static TypedRow getTypedRow(Result p) { - TypedRow tp = new TypedRow(); - tp.setSourceId(p.getId()); - List authorList = p.getAuthor() - .stream() - .map(a -> { - if (a.getPid().stream().map(pid -> { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { - return a; - } - return null; - }).filter(aut -> !(aut == null)).collect(Collectors.toList()).size() > 0){ - return a; - } - return null; - }).filter(a -> !(a == null)).collect(Collectors.toList()); - tp.setAuthors(authorList); - if(authorList.size() > 0){ - return tp; - } - return null; - - - } - - private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a){ - - - return (a.getPid().stream().map(pid -> { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { - return true; - } - return false; - }).filter(aut -> (aut == true)).collect(Collectors.toList()).size()) > 0; - } - -} - - -/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ - ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); - boolean updated = false; - - for (FieldTypeProtos.Author a: author_list){ - FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); - if(author != null){ - updated = true; - metadataBuilder.addAuthor(author); - }else{ - metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); - } - } - if(updated) - return metadataBuilder; - return null; - } - private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ - if(containsOrcid(a.getPidList())) - return null; - for(FieldTypeProtos.Author autoritative_author : author_list) { - if (equals(autoritative_author, a)) { - if(!containsOrcid(a.getPidList())) - return update(a, autoritative_author); - } - } - return null; - - } - - private boolean containsOrcid(List pidList){ - if(pidList == null) - return false; - return pidList - .stream() - .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) - .collect(Collectors.toList()).size() > 0; - } - */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java deleted file mode 100644 index 1b58badfa..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob2.java +++ /dev/null @@ -1,398 +0,0 @@ -package eu.dnetlib.dhp.orcidtoresultfromsemrel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -import java.util.*; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkOrcidToResultFromSemRelJob2 { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkOrcidToResultFromSemRelJob2.class.getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json"))); - parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkOrcidToResultFromSemRelJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/orcidtoresult"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - boolean writeUpdate = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Dataset.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Software.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Software.class)); - - org.apache.spark.sql.Dataset publication = spark.createDataset(sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)).rdd(), - Encoders.bean(eu.dnetlib.dhp.schema.oaf.Publication.class)); - - - relation.createOrReplaceTempView("relation"); - String query = "Select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false " + getConstraintList(" relclass = '" , allowedsemrel); - - org.apache.spark.sql.Dataset result_result = spark.sql(query); - - publication.createOrReplaceTempView("publication"); - org.apache.spark.sql.Dataset pubs_with_orcid = getResultWithOrcid("publication", spark) - .as(Encoders.bean(ResultOrcidList.class)); - - dataset.createOrReplaceTempView("dataset"); - org.apache.spark.sql.Dataset dats_with_orcid = getResultWithOrcid("dataset", spark) - .as(Encoders.bean(ResultOrcidList.class)); - - other.createOrReplaceTempView("orp"); - org.apache.spark.sql.Dataset orp_with_orcid = getResultWithOrcid("orp", spark) - .as(Encoders.bean(ResultOrcidList.class)); - - dataset.createOrReplaceTempView("software"); - org.apache.spark.sql.Dataset software_with_orcid = getResultWithOrcid("software", spark) - .as(Encoders.bean(ResultOrcidList.class)); - //get the results having at least one author pid we are interested in - - //target of the relation from at least one source with orcid. - //the set of authors contains all those that have orcid and are related to target - //from any source with allowed semantic relationship - JavaPairRDD> target_authorlist_from_pubs = getTargetAutoritativeAuthorList(pubs_with_orcid); - - JavaPairRDD> target_authorlist_from_dats = getTargetAutoritativeAuthorList(dats_with_orcid); - - JavaPairRDD> target_authorlist_from_orp = getTargetAutoritativeAuthorList(orp_with_orcid); - - JavaPairRDD> target_authorlist_from_sw = getTargetAutoritativeAuthorList(software_with_orcid); - - if(writeUpdate){ - target_authorlist_from_dats.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_dats"); - target_authorlist_from_pubs.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_pubs"); - target_authorlist_from_orp.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_orp"); - target_authorlist_from_sw.map(r -> new ObjectMapper().writeValueAsString(r)) - .saveAsTextFile(outputPath + "/" + "update_sw"); - } - - if(saveGraph){ - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, eu.dnetlib.dhp.schema.oaf.Publication.class)) - .mapToPair(p -> new Tuple2<>(p.getId(),p)) - .leftOuterJoin(target_authorlist_from_pubs) - .map(c -> { - Result r = c._2()._1(); - if(!c._2()._2().isPresent()){ - return r; - } - List toenrich_author = r.getAuthor(); - List autoritativeAuthors = c._2()._2().get(); - for(eu.dnetlib.dhp.schema.oaf.Author author: toenrich_author){ - if (!containsAllowedPid(author)){ - enrichAuthor(author, autoritativeAuthors); - } - } - return r; - }); - - } - - } - - private static void enrichAuthor(eu.dnetlib.dhp.schema.oaf.Author a, List au){ - for (AutoritativeAuthor aa: au){ - if(enrichAuthor(aa, a)){ - return; - } - } - - } - -// private static JavaPairRDD> getTargetAutoritativeAuthorList(org.apache.spark.sql.Dataset result_result, org.apache.spark.sql.Dataset pubs_with_orcid) { -// return pubs_with_orcid -// .toJavaRDD() -// .mapToPair(p -> new Tuple2<>(p.getId(), p.getAuthorList())) -// .join(result_result.toJavaRDD().mapToPair(rel -> new Tuple2<>(rel.getString(0), rel.getString(1)))) -// .mapToPair(c -> new Tuple2<>(c._2._2(), c._2()._1())) -// .reduceByKey((a, b) -> { -// if(a == null){ -// return b; -// } -// if(b==null){ -// return a; -// } -// -// Set authSet = new HashSet<>(); -// a.stream().forEach(au -> authSet.add(au.getOrcid())); -// -// b.stream().forEach(au -> { -// if (!authSet.contains(au.getOrcid())) { -// a.add(au); -// } -// } -// ); -// return a; -// }); -// } -private static JavaPairRDD> getTargetAutoritativeAuthorList( org.apache.spark.sql.Dataset pubs_with_orcid) { - return pubs_with_orcid - .toJavaRDD() - .mapToPair(p -> new Tuple2<>(p.getResultId(), p.getAuthorList())) - .reduceByKey((a, b) -> { - if(a == null){ - return b; - } - if(b==null){ - return a; - } - Set authSet = new HashSet<>(); - a.stream().forEach(au -> authSet.add(au.getOrcid())); - - b.stream().forEach(au -> { - if (!authSet.contains(au.getOrcid())) { - a.add(au); - } - } - ); - return a; - }); -} - - private static org.apache.spark.sql.Dataset getResultWithOrcid(String table, SparkSession spark){ - String query = " select target, author " + - " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + - " from ( " + - " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + - " from " + table + - " lateral view explode (author) a as MyT " + - " lateral view explode (MyT.pid) p as MyP " + - " where MyP.qualifier.classid = 'ORCID') tmp " + - " group by id) r_t " + - " join (" + - " select source, target " + - " from relation " + - " where datainfo.deletedbyinference = false and (relclass = 'isSupplementedBy' or relclass = 'isSupplementTo') rel_rel " + - " on source = id"; - - return spark.sql(query); - } - - - private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, eu.dnetlib.dhp.schema.oaf.Author author) { - boolean toaddpid = false; - - if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { - if (StringUtils.isNoneEmpty(author.getSurname())) { - if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { - - //have the same surname. Check the name - if (StringUtils.isNoneEmpty(autoritative_author.getName())) { - if (StringUtils.isNoneEmpty(author.getName())) { - if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { - toaddpid = true; - } - //they could be differently written (i.e. only the initials of the name in one of the two - if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { - toaddpid = true; - } - } - } - } - } - } - if (toaddpid){ - StructuredProperty pid = new StructuredProperty(); - String aa_pid = autoritative_author.getOrcid(); - pid.setValue(aa_pid); - pid.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); - pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - if(author.getPid() == null){ - author.setPid(Arrays.asList(pid)); - }else{ - author.getPid().add(pid); - } - - } - return toaddpid; - - } - - -// private static List enrichAuthors(List autoritative_authors, List to_enrich_authors, boolean filter){ -//// List autoritative_authors = p._2()._2().get().getAuthors(); -//// List to_enrich_authors = r.getAuthor(); -// -// return to_enrich_authors -// .stream() -// .map(a -> { -// if (filter) { -// if (containsAllowedPid(a)) { -// return a; -// } -// } -// -// List lst = autoritative_authors.stream() -// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -// if (lst.size() == 0) { -// return a; -// } -// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -// -// }).collect(Collectors.toList()); -// } -// -// private static void writeResult(JavaPairRDD results, JavaPairRDD toupdateresult, -// String outputPath, String type) { -// -// results.join(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// -// List autoritative_authors = p._2()._2().getAuthors(); -// List to_enrich_authors = r.getAuthor(); -// -// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, false)); -//// .stream() -//// .map(a -> { -//// if(filter) { -//// if (containsAllowedPid(a)) { -//// return a; -//// } -//// } -//// -//// List lst = autoritative_authors.stream() -//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -//// if(lst.size() == 0){ -//// return a; -//// } -//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -//// -//// }).collect(Collectors.toList())); -// -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath + "/" + type + "_update"); -// } - - -// private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, -// String outputPath, String type) { -// results.leftOuterJoin(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// if (p._2()._2().isPresent()){ -// List autoritative_authors = p._2()._2().get().getAuthors(); -// List to_enrich_authors = r.getAuthor(); -// -// r.setAuthor(enrichAutors(autoritative_authors, to_enrich_authors, true)); -//// .stream() -//// .map(a -> { -//// if(filter) { -//// if (containsAllowedPid(a)) { -//// return a; -//// } -//// } -//// -//// List lst = autoritative_authors.stream() -//// .map(aa -> enrichAuthor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList()); -//// if(lst.size() == 0){ -//// return a; -//// } -//// return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people -//// -//// }).collect(Collectors.toList())); -// } -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath+"/"+type); -// } - - - - private static boolean containsAllowedPid(eu.dnetlib.dhp.schema.oaf.Author a) { - for (StructuredProperty pid : a.getPid()) { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { - return true; - } - } - return false; - } - -} - - -/*private ResultProtos.Result.Metadata.Builder searchMatch(List author_list){ - ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder(); - boolean updated = false; - - for (FieldTypeProtos.Author a: author_list){ - FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors); - if(author != null){ - updated = true; - metadataBuilder.addAuthor(author); - }else{ - metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a)); - } - } - if(updated) - return metadataBuilder; - return null; - } - private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List author_list){ - if(containsOrcid(a.getPidList())) - return null; - for(FieldTypeProtos.Author autoritative_author : author_list) { - if (equals(autoritative_author, a)) { - if(!containsOrcid(a.getPidList())) - return update(a, autoritative_author); - } - } - return null; - - } - - private boolean containsOrcid(List pidList){ - if(pidList == null) - return false; - return pidList - .stream() - .filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID)) - .collect(Collectors.toList()).size() > 0; - } - */ \ No newline at end of file From 951b13ac466e1552d92d16c47945868798e9609b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:13:10 +0200 Subject: [PATCH 117/259] input parameters and workflow for new implementation of propagation of orcid to result from semrel and preparation phases --- .../input_orcidtoresult_parameters.json | 42 +-- ...input_prepareorcidtoresult_parameters.json | 30 +- ...nput_prepareorcidtoresult_parameters2.json | 18 -- .../oozie_app/workflow.xml | 275 +++++++++++++++++- 4 files changed, 304 insertions(+), 61 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json index ea2f0b96c..41d7905c2 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json @@ -1,32 +1,38 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"as", - "paramLongName":"allowedsemrels", - "paramDescription": "the allowed sematinc relations for propagation", - "paramRequired": true - }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", - "paramRequired": true - }, { "paramName":"sg", "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json index ea2f0b96c..08648d61a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -18,15 +12,27 @@ "paramRequired": true }, { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", "paramRequired": true }, { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json index 08648d61a..1a67134a6 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json @@ -5,18 +5,6 @@ "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"as", - "paramLongName":"allowedsemrels", - "paramDescription": "the allowed sematinc relations for propagation", - "paramRequired": true - }, - { - "paramName":"h", - "paramLongName":"hive_metastore_uris", - "paramDescription": "the hive metastore uris", - "paramRequired": true - }, { "paramName": "out", "paramLongName": "outputPath", @@ -28,11 +16,5 @@ "paramLongName": "isSparkSessionManaged", "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false - }, - { - "paramName":"tn", - "paramLongName":"resultTableName", - "paramDescription": "the name of the result table we are currently working on", - "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index b4fca5e58..a3c043bf0 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -8,18 +8,6 @@ allowedsemrels the semantic relationships allowed for propagation - - - - - - - - - - - - writeUpdate writes the information found for the update. No double check done if the information is already present @@ -30,12 +18,133 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} + + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} + + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} + + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} + + + + + + + ${jobTracker} @@ -67,5 +176,145 @@ + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/targetOrcidAssoc + --outputPath${workingDir}/preparedInfo/mergedOrcidAssoc + + + + + + + + + + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/publication + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/dataset + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/otherresearchproduct + + + + + + + yarn + cluster + ORCIDpropagationFromSemRel + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/software + + + + + \ No newline at end of file From d60fd3604639fba02db147872d67ffdcddb8ece2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 16:14:15 +0200 Subject: [PATCH 118/259] changed the save method --- .../SparkResultToOrganizationFromIstRepoJob2.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index c4efaefba..82c69e927 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -35,7 +35,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkCountryPropagationJob2.class + String jsonConfiguration = IOUtils.toString(SparkResultToOrganizationFromIstRepoJob2.class .getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -118,9 +118,14 @@ public class SparkResultToOrganizationFromIstRepoJob2 { .textFile(alreadylinked) .map(value -> OBJECT_MAPPER.readValue(value, ResultOrganizationSet.class), Encoders.bean(ResultOrganizationSet.class)), potentialUpdates) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath , GzipCodec.class); + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .text(outputPath); +// .toJavaRDD() +// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) +// .saveAsTextFile(outputPath , GzipCodec.class); } From 6afbd542ca2464471e5b5470d9cb7a844b893901 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 18:40:14 +0200 Subject: [PATCH 119/259] changed the save mode to avoid NegativeArraySize... error. Needed to modify also the preparationstep2 --- .../PrepareResultOrcidAssociationStep1.java | 22 +- .../PrepareResultOrcidAssociationStep2.java | 7 +- .../PrepareResultCommunitySet.java | 4 + .../ResultCommunityList.java | 4 + .../ResultOrganizations.java | 4 + ...ResultToCommunityFromOrganizationJob2.java | 390 ++++++++++++++++++ .../OrcidPropagationJobTest.java | 4 + 7 files changed, 424 insertions(+), 11 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index 4af652ef0..8f4ecb649 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -6,6 +6,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; @@ -47,7 +48,7 @@ public class PrepareResultOrcidAssociationStep1 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrel").split(";")); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); @@ -76,7 +77,7 @@ public class PrepareResultOrcidAssociationStep1 { //read the relation table and the table related to the result it is using final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); relation.createOrReplaceTempView("relation"); log.info("Reading Graph table from: {}", inputPath + "/" + resultType); @@ -84,7 +85,7 @@ public class PrepareResultOrcidAssociationStep1 { result.createOrReplaceTempView("result"); - getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath); + getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath + "/" + resultType); } @@ -107,12 +108,15 @@ public class PrepareResultOrcidAssociationStep1 { spark.sql(query) .as(Encoders.bean(ResultOrcidList.class)) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression","gzip") - .text(outputPath) - ; + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); +// .toJSON() +// .write() +// .mode(SaveMode.Append) +// .option("compression","gzip") +// .text(outputPath) +// ; } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index 660d0b87a..4ed911c42 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -21,7 +21,7 @@ public class PrepareResultOrcidAssociationStep2 { public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult2_parameters.json")); + .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); @@ -50,7 +50,10 @@ public class PrepareResultOrcidAssociationStep2 { private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath); + Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath + "/publication") + .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) + .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) + .union(readAssocResultOrcidList(spark, inputPath + "/software")); resultOrcidAssoc .toJavaRDD() diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java new file mode 100644 index 000000000..255ef7350 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +public class PrepareInfo { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java new file mode 100644 index 000000000..82a93662d --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +public class ResultCommunityList { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java new file mode 100644 index 000000000..b3ddcc3a4 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +public class ResultOrganizations { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java new file mode 100644 index 000000000..de14946cc --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -0,0 +1,390 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToCommunityFromOrganization { + + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityFromOrganization.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityFromOrganization.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/communitytoresult"; + final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); + boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + + System.out.println(new Gson().toJson(organizationMap)); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + JavaRDD relations_rdd_all = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + JavaRDD publications_rdd_all = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD dataset_rdd_all = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD orp_rdd_all = sc.textFile(inputPath + "/otheresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + JavaRDD software_rdd_all = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relations_rdd_all.rdd(), Encoders.bean(Relation.class)); + + relation.createOrReplaceTempView("relation"); +// String query = "SELECT source, target" + +// " FROM relation " + +// " WHERE datainfo.deletedbyinference = false " + +// " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "'"; +// +// org.apache.spark.sql.Dataset result_organization = spark.sql(query); +// result_organization.createOrReplaceTempView("result_organization"); +// +// query = "SELECT source, target " + +// "FROM relation " + +// "WHERE datainfo.deletedbyinference = false " + +// "AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "'" ; + + String query = "SELECT result_organization.source, result_organization.target, org_set " + + "FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + + "LEFT JOIN (SELECT source, collect_set(target) org_set " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + + " GROUP BY source) organization_organization " + + "ON result_organization.target = organization_organization.source "; + + org.apache.spark.sql.Dataset result_organizationset = spark.sql(query); + + JavaPairRDD result_communitySet = result_organizationset.toJavaRDD().map(r -> { + String rId = r.getString(0); + List orgs = r.getList(2); + String oTarget = r.getString(1); + TypedRow tp = new TypedRow(); + if (organizationMap.containsKey(oTarget)) { + tp.addAll(organizationMap.get(oTarget)); + } + try{ + for (String oId : orgs) { + if (organizationMap.containsKey(oId)) { + tp.addAll(organizationMap.get(oId)); + } + } + } + catch(Exception e){ + //System.out.println("organizationTargetID: " + oTarget); + } + + if(tp.getAccumulator() == null ){ + return null; + } + tp.setSourceId(rId); + + + return tp; + }) + .filter(tr -> !(tr==null)) + .mapToPair(toPair()).cache(); + + if(writeUpdates){ + result_communitySet.map(c->new ObjectMapper().writeValueAsString(c)).saveAsTextFile(outputPath +"/result_communityset"); + } + + if(saveGraph){ + updatePublicationResult(publications_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/publication"); + + updateDatasetResult(dataset_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/dataset"); + + updateORPResult(orp_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/otherresearchproduct"); + + updateSoftwareResult(software_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/software"); + } + + + //relations between organziations and results +// JavaPairRDD organization_result = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) +// && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getTarget()); +// tp.setTargetId(r.getSource() ); +// return tp; +// }) +// .mapToPair(toPair()); + + //relations between representative organization and merged Id. One relation per merged organization +// JavaPairRDD organization_organization = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) +// && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getSource()); +// tp.setTargetId(r.getTarget()); +// return tp; +// }) +// .mapToPair(toPair()); + + //get the original id of the organizations to be checked against the id associated to the communities +// JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) +// .map(c -> { +// if (!c._2()._2().isPresent()) +// return c._2()._1(); +// return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); +// }) +// .mapToPair(toPair()); + + //associates to each result connected to an organization the list of communities related to that organization +// JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { +// List communityList = organizationMap.get(o._1()); +// if (communityList.size() == 0) +// return null; +// TypedRow tp = o._2(); +// tp.setAccumulator(new HashSet<>(communityList)); +// tp.setSourceId(tp.getTargetId()); +// return tp; +// }) +// .filter(r -> !(r == null)) +// .mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) +// return b; +// if (b == null) +// return a; +// a.addAll(b.getAccumulator()); +// return a; +// }); + + +// JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); +// JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); +// JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); +// JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); +// +// JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// +// updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + + } + + private static JavaRDD updatePublicationResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Publication)r); + + } + private static JavaRDD updateDatasetResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Dataset) r); + + } + private static JavaRDD updateORPResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (OtherResearchProduct)r); + + } + private static JavaRDD updateSoftwareResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Software) r); + + } + private static JavaRDD updateResultDataset(JavaRDD result, JavaPairRDD result_communitySet){ + return result.mapToPair(p -> new Tuple2<>(p.getId(),p)).leftOuterJoin(result_communitySet) + .map(c -> { + Result r = c._2()._1(); + if(c._2()._2().isPresent()){ + Set communitySet = c._2()._2().get().getAccumulator(); + List contextList = r.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); + for(String cId:communitySet){ + if(!contextList.contains(cId)){ + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); + r.getContext().add(newContext); + } + } + } + return r; + }); + + } + // private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { +// results.leftOuterJoin(toupdateresult) +// .map(p -> { +// Result r = p._2()._1(); +// if (p._2()._2().isPresent()){ +// Set communityList = p._2()._2().get().getAccumulator(); +// for(Context c: r.getContext()){ +// if (communityList.contains(c.getId())){ +// //verify if the datainfo for this context contains propagation +// if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ +// c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME)); +// //community id already in the context of the result. Remove it from the set that has to be added +// communityList.remove(c.getId()); +// } +// } +// } +// List cc = r.getContext(); +// for(String cId: communityList){ +// Context context = new Context(); +// context.setId(cId); +// context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); +// cc.add(context); +// } +// r.setContext(cc); +// } +// return r; +// }) +// .map(p -> new ObjectMapper().writeValueAsString(p)) +// .saveAsTextFile(outputPath+"/"+type); +// } + + +} + + +/* +package eu.dnetlib.data.mapreduce.hbase.propagation.communitythroughorganization; + +import com.google.gson.Gson; +import eu.dnetlib.data.mapreduce.hbase.propagation.Value; +import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; +import eu.dnetlib.data.proto.OafProtos; +import eu.dnetlib.data.proto.TypeProtos; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.mapreduce.TableMapper; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; + +import java.io.IOException; +import java.util.Set; + +import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.*; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; +import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; + +public class PropagationCommunityThroughOrganizationMapper extends TableMapper { + private Text valueOut; + private ImmutableBytesWritable keyOut; + private OrganizationMap organizationMap; + + //seleziono il tipo della entry: + //Result: + //se non e' deleted by inference ed ha organizzazioni a cui e' associato, + // //emetto id della relazione ed id del risultato per ogni organizzazione a cui e' associato + //ORGANIZATION: + //se non e' deleted by inference e non e' deduplicata emetto l'id della organizzazione + //se non e' deleted by inference ed e' deduplicata, emetto id della organizzazione ed id del deduplicato e lista delle organizzazioni mergiate + @Override + protected void setup(final Context context) throws IOException, InterruptedException { + super.setup(context); + valueOut = new Text(); + keyOut = new ImmutableBytesWritable(); + organizationMap = new Gson().fromJson(context.getConfiguration().get("organizationtoresult.community.map"), OrganizationMap.class); + System.out.println("got organizationtoresult map: " + new Gson().toJson(organizationMap)) ; + } + + @Override + protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { + final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); + final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference + if (entity != null) { + switch (type) { + case organization: + DedupedList communityList = getCommunityList(Bytes.toString(keyIn.get()), + getRelationTarget(value, DEDUP_RELATION_ORGANIZATION + REL_DEDUP_REPRESENTATIVE_RESULT, context, COUNTER_PROPAGATION), context); + if (communityList.size() > 0){ + valueOut.set(Value.newInstance( + new Gson().toJson( + communityList, //search for organizationtoresult it merges + DedupedList.class), + ORGANIZATION_COMMUNITY_TRUST, + Type.fromorganization).toJson()); + context.write(keyIn, valueOut); + context.getCounter(COUNTER_PROPAGATION, "emit for organizationtoresult ").increment(1); + }else{ + context.getCounter(COUNTER_PROPAGATION, "community list size = 0 ").increment(1); + } + + break; + case result: + Set result_organization = getRelationTarget(value, RELATION_ORGANIZATION + REL_RESULT_ORGANIZATION, context, COUNTER_PROPAGATION); + for(String org: result_organization) + emit(org, Bytes.toString(keyIn.get()), context); + break; + } + } + } + + private DedupedList getCommunityList(String organizationId, Set relationTarget, Context context) { + DedupedList communityList = new DedupedList(); + relationTarget.stream().forEach(org -> communityList.addAll(organizationMap.get(StringUtils.substringAfter(org, "|")))); + communityList.addAll(organizationMap.get(StringUtils.substringAfter(organizationId,"|"))); + communityList.stream().forEach(c->context.getCounter(COUNTER_PROPAGATION,"found organizationtoresult for " + c).increment(1)); + return communityList; + } + + private void emit(String org, String resId, Context context) throws IOException, InterruptedException { + keyOut.set(Bytes.toBytes(org)); + valueOut.set(Value.newInstance(resId, ORGANIZATION_COMMUNITY_TRUST, Type.fromresult).toJson()); + context.write(keyOut,valueOut); + context.getCounter(COUNTER_PROPAGATION, "emit for result").increment(1); + } + +} + */ \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java new file mode 100644 index 000000000..a01b6d2dd --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.orcidtoresultfromsemrel; + +public class OrcidPropagationJobTest { +} From a9120164aa8377d11ff6136d2be8d3c7bb6bf6cc Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 18:42:04 +0200 Subject: [PATCH 120/259] added hive parameter and a step of reset of the working dir in the workflow --- .../oozie_app/config-default.xml | 4 ++ .../oozie_app/workflow.xml | 65 ++++++------------- 2 files changed, 24 insertions(+), 45 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml index 56d0ac43c..8d2c34105 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml @@ -19,6 +19,10 @@ spark2YarnHistoryServerAddress http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + spark2EventLogDir /user/spark/spark2ApplicationHistory diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index a3c043bf0..d20b839ee 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -17,13 +17,21 @@ writes new version of the graph after the propagation step + - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + + + + + + @@ -35,7 +43,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-PreparePhase1-Publications eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 dhp-propagation-${projectVersion}.jar @@ -63,7 +71,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-PreparePhase1-Dataset eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 dhp-propagation-${projectVersion}.jar @@ -91,7 +99,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-PreparePhase1-ORP eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 dhp-propagation-${projectVersion}.jar @@ -119,7 +127,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-PreparePhase1-Software eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 dhp-propagation-${projectVersion}.jar @@ -139,50 +147,17 @@ --outputPath${workingDir}/preparedInfo/targetOrcidAssoc --allowedsemrels${allowedsemrels} - + - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - OrcidToResultFromSemRelPropagation - eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob - dhp-propagation-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - -mt yarn-cluster - --sourcePath${sourcePath} - --allowedsemrels${allowedsemrels} - --writeUpdate${writeUpdate} - --saveGraph${saveGraph} - - - - - yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-PreparePhase2 eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep2 dhp-propagation-${projectVersion}.jar @@ -215,7 +190,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-Publication eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 dhp-propagation-${projectVersion}.jar @@ -241,7 +216,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-Dataset eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 dhp-propagation-${projectVersion}.jar @@ -267,7 +242,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-ORP eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 dhp-propagation-${projectVersion}.jar @@ -293,7 +268,7 @@ yarn cluster - ORCIDpropagationFromSemRel + ORCIDPropagation-Software eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 dhp-propagation-${projectVersion}.jar From 3fd9d6b02f5718dddaea51bb57aaa26de56dc99d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 18:43:55 +0200 Subject: [PATCH 121/259] preparation phase for the propagation of community to result from organization --- .../PrepareResultCommunitySet.java | 118 +++++++++++++++++- 1 file changed, 117 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index 255ef7350..a8a5451d9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -1,4 +1,120 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; -public class PrepareInfo { +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultCommunitySet { + + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySet.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); + log.info("organizationMap: {}", new Gson().toJson(organizationMap)); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo(spark, inputPath, outputPath, organizationMap); + }); + } + + private static void prepareInfo(SparkSession spark, String inputPath, String outputPath, OrganizationMap organizationMap) { + Dataset relation = readRelations(spark, inputPath); + relation.createOrReplaceTempView("relation"); + + String query = "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " + + "FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + + "LEFT JOIN (SELECT source, collect_set(target) org_set " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + + " GROUP BY source) organization_organization " + + "ON result_organization.target = organization_organization.source "; + + org.apache.spark.sql.Dataset result_organizationset = spark.sql(query) + .as(Encoders.bean(ResultOrganizations.class)); + + result_organizationset + .map(value -> { + String rId = value.getResultId(); + List orgs = value.getMerges(); + String oTarget = value.getOrgId(); + Set communitySet = new HashSet<>(); + if (organizationMap.containsKey(oTarget)) { + communitySet.addAll(organizationMap.get(oTarget)); + } + try{ + for (String oId : orgs) { + if (organizationMap.containsKey(oId)) { + communitySet.addAll(organizationMap.get(oId)); + } + } + }catch(Exception e){ + + } + if (communitySet.size() > 0){ + ResultCommunityList rcl = new ResultCommunityList(); + rcl.setResultId(rId); + ArrayList communityList = new ArrayList<>(); + communityList.addAll(communitySet); + rcl.setCommunityList(communityList); + return rcl; + } + return null; + }, Encoders.bean(ResultCommunityList.class)) + .filter(r -> r!= null) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } + + + } From fff1e5ec393b210e7c8987a81de6b74a0a7d0f0d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 18:44:43 +0200 Subject: [PATCH 122/259] classes to (de)serialize the data provided in the preparation step --- .../ResultCommunityList.java | 23 ++++++++++++- .../ResultOrganizations.java | 32 ++++++++++++++++++- 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java index 82a93662d..50d9a6d7a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java @@ -1,4 +1,25 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; -public class ResultCommunityList { +import java.io.Serializable; +import java.util.ArrayList; + +public class ResultCommunityList implements Serializable { + private String resultId; + private ArrayList communityList; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public ArrayList getCommunityList() { + return communityList; + } + + public void setCommunityList(ArrayList communityList) { + this.communityList = communityList; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java index b3ddcc3a4..53a5fccdf 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java @@ -1,4 +1,34 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; -public class ResultOrganizations { +import java.io.Serializable; +import java.util.ArrayList; + +public class ResultOrganizations implements Serializable { + private String resultId; + private String orgId; + private ArrayList merges; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public String getOrgId() { + return orgId; + } + + public void setOrgId(String orgId) { + this.orgId = orgId; + } + + public ArrayList getMerges() { + return merges; + } + + public void setMerges(ArrayList merges) { + this.merges = merges; + } } From 5d772e52636419de08131520dfc5280b2e2c0007 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 16 Apr 2020 18:45:22 +0200 Subject: [PATCH 123/259] new implementation of propagation of community to result from organization that exploits the prepared info --- ...ResultToCommunityFromOrganizationJob2.java | 414 ++++-------------- 1 file changed, 76 insertions(+), 338 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index de14946cc..3116161e9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -4,6 +4,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.ResultOrcidList; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; @@ -13,378 +15,114 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; import java.util.*; import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class SparkResultToCommunityFromOrganizationJob2 { + + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); -public class SparkResultToCommunityFromOrganization { public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(SparkResultToCommunityFromOrganizationJob2.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityFromOrganization.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json"))); parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String possibleupdatespath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", possibleupdatespath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = (Class) Class.forName(resultClassName); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityFromOrganization.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/communitytoresult"; - final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); - boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - System.out.println(new Gson().toJson(organizationMap)); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD relations_rdd_all = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); - JavaRDD publications_rdd_all = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD dataset_rdd_all = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD orp_rdd_all = sc.textFile(inputPath + "/otheresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); - JavaRDD software_rdd_all = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(relations_rdd_all.rdd(), Encoders.bean(Relation.class)); - - relation.createOrReplaceTempView("relation"); -// String query = "SELECT source, target" + -// " FROM relation " + -// " WHERE datainfo.deletedbyinference = false " + -// " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "'"; -// -// org.apache.spark.sql.Dataset result_organization = spark.sql(query); -// result_organization.createOrReplaceTempView("result_organization"); -// -// query = "SELECT source, target " + -// "FROM relation " + -// "WHERE datainfo.deletedbyinference = false " + -// "AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "'" ; - - String query = "SELECT result_organization.source, result_organization.target, org_set " + - "FROM (SELECT source, target " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + - "LEFT JOIN (SELECT source, collect_set(target) org_set " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + - " GROUP BY source) organization_organization " + - "ON result_organization.target = organization_organization.source "; - - org.apache.spark.sql.Dataset result_organizationset = spark.sql(query); - - JavaPairRDD result_communitySet = result_organizationset.toJavaRDD().map(r -> { - String rId = r.getString(0); - List orgs = r.getList(2); - String oTarget = r.getString(1); - TypedRow tp = new TypedRow(); - if (organizationMap.containsKey(oTarget)) { - tp.addAll(organizationMap.get(oTarget)); - } - try{ - for (String oId : orgs) { - if (organizationMap.containsKey(oId)) { - tp.addAll(organizationMap.get(oId)); + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); } - } - } - catch(Exception e){ - //System.out.println("organizationTargetID: " + oTarget); - } - - if(tp.getAccumulator() == null ){ - return null; - } - tp.setSourceId(rId); - - - return tp; - }) - .filter(tr -> !(tr==null)) - .mapToPair(toPair()).cache(); - - if(writeUpdates){ - result_communitySet.map(c->new ObjectMapper().writeValueAsString(c)).saveAsTextFile(outputPath +"/result_communityset"); - } - - if(saveGraph){ - updatePublicationResult(publications_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); - - updateDatasetResult(dataset_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/dataset"); - - updateORPResult(orp_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); - - updateSoftwareResult(software_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/software"); - } - - - //relations between organziations and results -// JavaPairRDD organization_result = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) -// && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getTarget()); -// tp.setTargetId(r.getSource() ); -// return tp; -// }) -// .mapToPair(toPair()); - - //relations between representative organization and merged Id. One relation per merged organization -// JavaPairRDD organization_organization = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) -// && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getSource()); -// tp.setTargetId(r.getTarget()); -// return tp; -// }) -// .mapToPair(toPair()); - - //get the original id of the organizations to be checked against the id associated to the communities -// JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) -// .map(c -> { -// if (!c._2()._2().isPresent()) -// return c._2()._1(); -// return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); -// }) -// .mapToPair(toPair()); - - //associates to each result connected to an organization the list of communities related to that organization -// JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { -// List communityList = organizationMap.get(o._1()); -// if (communityList.size() == 0) -// return null; -// TypedRow tp = o._2(); -// tp.setAccumulator(new HashSet<>(communityList)); -// tp.setSourceId(tp.getTargetId()); -// return tp; -// }) -// .filter(r -> !(r == null)) -// .mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) -// return b; -// if (b == null) -// return a; -// a.addAll(b.getAccumulator()); -// return a; -// }); - - -// JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); -// JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); -// JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); -// JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); -// -// JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// -// updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); + execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); + }); } - private static JavaRDD updatePublicationResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Publication)r); + private static void execPropagation(SparkSession spark, String inputPath, String outputPath, + Class resultClazz, String possibleUpdatesPath) { + org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList(spark, possibleUpdatesPath); + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - } - private static JavaRDD updateDatasetResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Dataset) r); - } - private static JavaRDD updateORPResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (OtherResearchProduct)r); - - } - private static JavaRDD updateSoftwareResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Software) r); - - } - private static JavaRDD updateResultDataset(JavaRDD result, JavaPairRDD result_communitySet){ - return result.mapToPair(p -> new Tuple2<>(p.getId(),p)).leftOuterJoin(result_communitySet) - .map(c -> { - Result r = c._2()._1(); - if(c._2()._2().isPresent()){ - Set communitySet = c._2()._2().get().getAccumulator(); - List contextList = r.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); + result + .joinWith(possibleUpdates, result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map(value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if(rcl.isPresent()){ + ArrayList communitySet = rcl.get().getCommunityList(); + List contextList = ret.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); for(String cId:communitySet){ if(!contextList.contains(cId)){ Context newContext = new Context(); newContext.setId(cId); newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - r.getContext().add(newContext); + ret.getContext().add(newContext); } } } - return r; - }); + return ret; + }, Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(outputPath); } - // private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { -// results.leftOuterJoin(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// if (p._2()._2().isPresent()){ -// Set communityList = p._2()._2().get().getAccumulator(); -// for(Context c: r.getContext()){ -// if (communityList.contains(c.getId())){ -// //verify if the datainfo for this context contains propagation -// if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ -// c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME)); -// //community id already in the context of the result. Remove it from the set that has to be added -// communityList.remove(c.getId()); -// } -// } -// } -// List cc = r.getContext(); -// for(String cId: communityList){ -// Context context = new Context(); -// context.setId(cId); -// context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); -// cc.add(context); -// } -// r.setContext(cc); -// } -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath+"/"+type); -// } - -} - - -/* -package eu.dnetlib.data.mapreduce.hbase.propagation.communitythroughorganization; - -import com.google.gson.Gson; -import eu.dnetlib.data.mapreduce.hbase.propagation.Value; -import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; -import eu.dnetlib.data.proto.OafProtos; -import eu.dnetlib.data.proto.TypeProtos; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.mapreduce.TableMapper; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.Text; - -import java.io.IOException; -import java.util.Set; - -import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.*; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; - -public class PropagationCommunityThroughOrganizationMapper extends TableMapper { - private Text valueOut; - private ImmutableBytesWritable keyOut; - private OrganizationMap organizationMap; - - //seleziono il tipo della entry: - //Result: - //se non e' deleted by inference ed ha organizzazioni a cui e' associato, - // //emetto id della relazione ed id del risultato per ogni organizzazione a cui e' associato - //ORGANIZATION: - //se non e' deleted by inference e non e' deduplicata emetto l'id della organizzazione - //se non e' deleted by inference ed e' deduplicata, emetto id della organizzazione ed id del deduplicato e lista delle organizzazioni mergiate - @Override - protected void setup(final Context context) throws IOException, InterruptedException { - super.setup(context); - valueOut = new Text(); - keyOut = new ImmutableBytesWritable(); - organizationMap = new Gson().fromJson(context.getConfiguration().get("organizationtoresult.community.map"), OrganizationMap.class); - System.out.println("got organizationtoresult map: " + new Gson().toJson(organizationMap)) ; - } - - @Override - protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { - final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); - final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference - if (entity != null) { - switch (type) { - case organization: - DedupedList communityList = getCommunityList(Bytes.toString(keyIn.get()), - getRelationTarget(value, DEDUP_RELATION_ORGANIZATION + REL_DEDUP_REPRESENTATIVE_RESULT, context, COUNTER_PROPAGATION), context); - if (communityList.size() > 0){ - valueOut.set(Value.newInstance( - new Gson().toJson( - communityList, //search for organizationtoresult it merges - DedupedList.class), - ORGANIZATION_COMMUNITY_TRUST, - Type.fromorganization).toJson()); - context.write(keyIn, valueOut); - context.getCounter(COUNTER_PROPAGATION, "emit for organizationtoresult ").increment(1); - }else{ - context.getCounter(COUNTER_PROPAGATION, "community list size = 0 ").increment(1); - } - - break; - case result: - Set result_organization = getRelationTarget(value, RELATION_ORGANIZATION + REL_RESULT_ORGANIZATION, context, COUNTER_PROPAGATION); - for(String org: result_organization) - emit(org, Bytes.toString(keyIn.get()), context); - break; - } - } - } - - private DedupedList getCommunityList(String organizationId, Set relationTarget, Context context) { - DedupedList communityList = new DedupedList(); - relationTarget.stream().forEach(org -> communityList.addAll(organizationMap.get(StringUtils.substringAfter(org, "|")))); - communityList.addAll(organizationMap.get(StringUtils.substringAfter(organizationId,"|"))); - communityList.stream().forEach(c->context.getCounter(COUNTER_PROPAGATION,"found organizationtoresult for " + c).increment(1)); - return communityList; - } - - private void emit(String org, String resId, Context context) throws IOException, InterruptedException { - keyOut.set(Bytes.toBytes(org)); - valueOut.set(Value.newInstance(resId, ORGANIZATION_COMMUNITY_TRUST, Type.fromresult).toJson()); - context.write(keyOut,valueOut); - context.getCounter(COUNTER_PROPAGATION, "emit for result").increment(1); + private static org.apache.spark.sql.Dataset readResultCommunityList(SparkSession spark, String possibleUpdatesPath) { + return spark + .read() + .textFile(possibleUpdatesPath) + .map(value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), Encoders.bean(ResultCommunityList.class)); } } - */ \ No newline at end of file From c4987dd12abe6c9f56925bfdc0a58afa981febd2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:49:08 +0200 Subject: [PATCH 124/259] minor --- .../countrypropagation/PrepareDatasourceCountryAssociation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index 85d3e58c4..2de536222 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -107,7 +107,7 @@ public class PrepareDatasourceCountryAssociation { getConstraintList("datasourcetype.classid = '", allowedtypes) + ") d " + "JOIN ( SELECT source, target " + " FROM relation " + - " WHERE relclass = 'provides' " + + " WHERE relclass = '" + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + "' " + " AND datainfo.deletedbyinference = false ) rel " + "ON d.id = rel.source " + "JOIN (SELECT id, country " + From b46b080ddcb41cb2a2d05d8a74f7ef87ff018a6b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:50:54 +0200 Subject: [PATCH 125/259] use mergeFrom method call to add the country(ies) instead of modify the result directly. --- .../countrypropagation/SparkCountryPropagationJob2.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 745a99db8..643373a1c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -135,12 +135,16 @@ public class SparkCountryPropagationJob2 { for (Qualifier country : r.getCountry()) { countries.add(country.getClassid()); } - + Result res = new Result(); + res.setId(r.getId()); + List countryList = new ArrayList<>(); for (CountrySbs country : potentialNewCountries.get().getCountrySet()) { if (!countries.contains(country.getClassid())) { - r.getCountry().add(getCountry(country.getClassid(), country.getClassname())); + countryList.add(getCountry(country.getClassid(), country.getClassname())); } } + res.setCountry(countryList); + r.mergeFrom(res); } return r; }, Encoders.bean(resultClazz)); From 390e250faf362cb4e244c534411a3d83b993797d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:52:02 +0200 Subject: [PATCH 126/259] use the addPid method of the Author class to add a new pid --- .../SparkOrcidToResultFromSemRelJob3.java | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index f9dfa2970..62f18ec2c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -5,15 +5,11 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; import java.util.*; @@ -151,16 +147,11 @@ public class SparkOrcidToResultFromSemRelJob3 { } } if (toaddpid){ - StructuredProperty pid = new StructuredProperty(); - String aa_pid = autoritative_author.getOrcid(); - pid.setValue(aa_pid); - pid.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); - pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - if(author.getPid() == null){ - author.setPid(Arrays.asList(pid)); - }else{ - author.getPid().add(pid); - } + StructuredProperty p = new StructuredProperty(); + p.setValue(autoritative_author.getOrcid()); + p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); + p.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + author.addPid(p); } return toaddpid; From eacd140a9865c09d266ac8fd691848a2d53a358b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:52:30 +0200 Subject: [PATCH 127/259] added missing parameter(s) --- .../input_orcidtoresult_parameters.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json index 41d7905c2..50869aa0e 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json @@ -34,5 +34,17 @@ "paramLongName":"resultTableName", "paramDescription": "the name of the result table we are currently working on", "paramRequired": true + }, + { + "paramName":"pu", + "paramLongName":"possibleUpdatesPath", + "paramDescription": "the path the the association resultId orcid author list can be found", + "paramRequired": true + }, + { + "paramName":"test", + "paramLongName":"isTest", + "paramDescription": "true if it is executing a test", + "paramRequired": false } ] \ No newline at end of file From 8c079c7a49ce4c27b62445e2dedae3f25a461492 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:53:03 +0200 Subject: [PATCH 128/259] unit test for orcid to result propagation from semrel --- .../OrcidPropagationJobTest.java | 191 ++++++++++++++++++ 1 file changed, 191 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index a01b6d2dd..b666846ef 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -1,4 +1,195 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.Result2OrganizationJobTest; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + public class OrcidPropagationJobTest { + + private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(OrcidPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(OrcidPropagationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + + spark = SparkSession + .builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + + @Test + public void noUpdateTest()throws Exception{ + SparkOrcidToResultFromSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate").getPath(), + "-hive_metastore_uris", "", + "-saveGraph","true", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath",workingDir.toString() + "/dataset", + "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + + Assertions.assertEquals(10, tmp.count()); + + org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + + Assertions.assertEquals(0, spark.sql(query).count()); + } + + @Test + public void oneUpdateTest() throws Exception{ + SparkOrcidToResultFromSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate").getPath(), + "-hive_metastore_uris", "", + "-saveGraph","true", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath",workingDir.toString() + "/dataset", + "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + + Assertions.assertEquals(10, tmp.count()); + + org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + + org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); + + Assertions.assertEquals(1, propagatedAuthors.count()); + + Assertions.assertEquals(1, propagatedAuthors.filter("id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " + + "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'").count()); + + Assertions.assertEquals(1, propagatedAuthors.filter("pid = '0000-0002-8825-3517'").count()); + + } + + @Test + public void twoUpdatesTest() throws Exception{ + SparkOrcidToResultFromSemRelJob3.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates").getPath(), + "-hive_metastore_uris", "", + "-saveGraph","true", + "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath",workingDir.toString() + "/dataset", + "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + + org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + + org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); + + Assertions.assertEquals(2, propagatedAuthors.count()); + + Assertions.assertEquals(1, propagatedAuthors.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions.assertEquals(1, propagatedAuthors.filter("name = 'Ruediger' and surname = 'Beckhaus'").count()); + + query = "select id, MyT.name name, MyT.surname surname, MyP.value pid ,MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP "; + + org.apache.spark.sql.Dataset authorsExplodedPids = spark.sql(query); + + Assertions.assertEquals(2, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions.assertEquals(1, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann' and pidType = 'MAG Identifier'").count()); + + } + } From 264c82f21eba5475f9d602590b5e6bfc5a66d441 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 16:54:46 +0200 Subject: [PATCH 129/259] minor --- .../PrepareResultInstRepoAssociation.java | 5 ++--- .../mergedOrcidAssoc/mergedOrcid_17.json.gz | Bin 0 -> 1230 bytes .../sample/noupdate/dataset_10.json.gz | Bin 0 -> 6787 bytes .../sample/oneupdate/dataset_10.json.gz | Bin 0 -> 6895 bytes .../sample/twoupdates/dataset_10.json.gz | Bin 0 -> 6913 bytes 5 files changed, 2 insertions(+), 3 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 9fbe7a814..51c46e2be 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -16,8 +16,7 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static eu.dnetlib.dhp.PropagationConstant.RELATION_RESULT_ORGANIZATION_REL_CLASS; -import static eu.dnetlib.dhp.PropagationConstant.isSparkSessionManaged; +import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; public class PrepareResultInstRepoAssociation { @@ -107,7 +106,7 @@ public class PrepareResultInstRepoAssociation { "AND datainfo.deletedbyinference = false ) d " + "JOIN ( SELECT source, target " + "FROM relation " + - "WHERE relclass = 'provides' " + + "WHERE relclass = '" + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + "' " + "AND datainfo.deletedbyinference = false ) rel " + "ON d.id = rel.source "; diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..118291ec57c823e413f3780d434396ad1de6cc42 GIT binary patch literal 1230 zcmV;<1Tp&`iwFp%aG72J18rq;XJuqha${*^Uokf>YIARH0F77MZrer>eeYK+^lTP; zzxXW|1a(rUMN+gyiz1l4MXYJjAjP&xnt$)mk!&tSrZOPQo;lh*I(v3z_-WInerWbr zadWv*^xv4`aQE)-{Ekv~c`39H8GuPAgOyH6148Nu+FFQ-%gx0GhW)nfzOVcJ=JNMX zn;qOHT_v#1JI|=fG*f(<<|y;m>*(Q&+HlHQYj1l0M?dF*oPSD{Hx`gktCL z6`oP8gA$fGk+XH^2urhe%qC+jUFG}Rwl1hX+oqYkh#-gPDMzk_tC%oq^ZDXo&0kWv z=G*%{CQBlKd&K@!37(eA{1Q(T3lrHkblWCju&t21Y3G0Li z$TKM=3}-%Q8KqJ@M!dlydgj+F^02F$|0`OIPhtr(N+oN2&Q<)bz_J8K-}(vMd=geW|9R<7R>{9 z=qXKMC9fFeXH_0xS|}Bg#NU}x5I%y^mI7c66?v8BOML6ebxHPVy9=plprm_*l1X}@ zTM38F#PMIJ9(!Gfo3??Td{=JtWE_6_cGeXqXfi=8R1!88Xlj;gUa62ykd>F*8^$;l z-b*GjWn3rja89W2a(cN8XN8z9Yh86TW~g0udKHK249RCCtXO4uB-VX zA-P>F1&xXP(A0mWx|><LF=a26j$_M?jK@{( zZ`yH+dR|)iMZIg_!))R@0Lepg;dX0 sehs&EX^E$4hl6I{4zs5H7{4ILPb~~X(&7_|QgS5yAM7AL^q39+0N|-#V*mgE literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..778f722624bf37dbe53d470c6510cc464d1b68fb GIT binary patch literal 6787 zcmV-}8hqs+iwFo+t(jf`17u-zVRL14UokK)YIARH0PS5*kK8tvzt5*23K(DpsIHb& zrTTYssymrBY{#%Y17ybwgOXTP>_}pTk~*nm4Qy_E-w&`buooYE82FZJUGpjSlk9sW zRZ>+~|I2ib6q||TbXSq$9HlIuR&760ZD-pu z=80F7di%E9fp+s^1?plsPgs`IWYw9v{@8bXe>@ydoC$OqTk@U%h*(st-h4lQ<2deg z+8+ktoKkl%8HD4(#2<`ke=v;ba6G0F^9CSY8lS-f{@xMM8}(0rGM-GQju%CK zlM2Kn(eo5yVi7XXS#?Unyr`{ zD^ZenJjZj6oId<{<@TqJKlO$^XX5@_r!1XInr3q$6MDGDaO(Ga?oicWKR{`X#owVh zL3v$wKNW}?LfAqeD zKU?*2(JR99um^(>J_cFTm){l87NQ%mG&cr{f=&oqlqDUe7rO zG2TSsvGOyt$d^K@u}0@}9&@99)ax7uqk;QzkuJLA83#-mv}``z{$38(-GO}N==}P{ z?CfOEM9a@AOVp33Tb=R_l}B;lpCkCsJ(~D#|HvB*#~n03u%vInSwh`H*5rple0?Xs zlAHhC{7=lfQE?}@rAiOo|Mar%S? z(uVwqQk(h4T{fl2UdFW6Rr_%Tm2 zGm-$Vn1|rec1D#61&5N+9NZXq!~`T$kzi9gI}X-IX?T=I$Ag}8Y_Yw=;M)OhC%FCm z^x2+dU$0XO=+cv zyML%42(c8ILLLRUSN2c0Q^**3NydJcQQTY7G>TadUsy~)(~AAmzwW0SqW*U9Ib_mchCB~Pi$@rBWRSN>X& z$8>4->(5ZUQ=_o0Dkj_xb*;1)OEbHfW`^$NLcPO(39tFEm?3NUCMGa2!KpF3FWcm% z-6?#V3xo>W<`=@oEgHq+1PH|S30nh zTR1QqU}z)cw+}Gf*%Z1Ygog)`Mk=%bi;-(X3cc5<{`Z7Qeu0&6odR;NVTpsk5P%Yv zw0xmC^W=5s2_>MG!u*a49+c8Pd!g_4;4@xM3rT@j|y;>h-g^GL@VO~gBF|GAy z8CimTfOzm6n!WCH!8lpUbXaAMuTZK3p-AC%Tw=fOC`6CEWJEFy)@(kKB1IvJh+`j( z>MsOYB9~zsmjO+{8v)5`N{|9SMwqdo5Z0#9iI{=fQEg1Zg1MAN;#NThKP-!crics( zUG=;y1|8%%s0N+@YfMH2s4DB(4V-%n_4r55@W^wlPTLXcv0;i&*>G%_q6J|J+>L5; zn8JEw8>X;fihB%GOq|K+$}q)zJQ)lJ;edvN5I&kSCmgwhem`(WK2>MGuMJb&RK1;H zis4Jy^G>>X?6%&Mol5-I zpH;1IdQhcatqO!37Kr`&bIG~0p%EJz`E+NN`abX$99C_j*kc_HbIcQPzc5G6)o`(A zIvNGzx!D-aF7J$5V44KR8@MhkvS7l8w+?+WO-(DO=S+q?~?PAevjRLSNY46iw6yMl@sRl??2=@82Xd#_WYYmhPE zoA;^{IfV~EPue&rD%f;q^A1YoMoy9eT#*jG>2TU_7||fX{bOji7Vj~DHpJ}=&$7x} z52sP$Vmsb3?YM4YDqQs^oE@V%9O4 ziR>C+Et#T?CUJW*#SYB7WD0n7h3C&%20!Wm|5xSN;X4N9ao-E93TzCI@RI9~5hG<{ zNzHj&)!y|)321*+z1ZRB&{y_d!DAjsuIUdH-snLlR-4~TYDPrU^2l^_c+w%zHdV!| zLTa(5WelNqR#a3BtX-s(Bs67EPjM|WwkRZ?sD6sSfx+Q!osxG-YWq&*uGTRI5ssby z@m4jx-c1=`ZAjO$Abv+W{THzy{uz&D(X^xoi>)O+w1xEGTGGQOHn1$|;WNhnM$*Gz zeC4@qkM-yMLC7YK?>TeI`o2R)%ykFjXcU1OI@gmPZmixu(nEjHo4DScNDqUa=Z+tW z^iavuJn6wC%Q#4S=uZdzspmFIdiWc7AQzL{hJ_|*Y zP<+8sJa5abgR&F`3(qYm3}?GR;Ui+^ccNra37pp?Weci*MhuIrJbE6Q6+`jrV$wVM`O>3<(lQUzLVWFOwm3hQz zX>>gm-_H&Q_{YaFJqm*GC{ywM*QsJ-oNU<`5!c%x%DkV@KuD@~g0P=gdN4BvZrGDa z)u1t{EW*u4ed+Y(9mzQE@2Lb2mo-`G57Vvha3PRovA6nDj0-H76^Rn7RGv;P-+5N3 zd#?8zLb%d|$`Cu2=29n~4=I?Q>Dzp-?^Wgs)uIT_nabIYBIhvkCMpRGv1MU`KpOk)3C0Z74D=Y?7{5cY#&qi8A$1&QN*E{fu(P*UU^&CiqsaV3tn z9J&mOsbjOvgpUdJ*$*p>d~Hon0rU;_YfF2fsHzM=Pl`1G14y*zoswK?#!N_|^DquH zVAmVQR?51@X==DP%DT+b6%AqyZH7yD)CI$QC14+>KIlF*D4>ireWBQ^f<+pCskO14 z3Q({f-dCVaL~WdkntdqekeOkD$ih(9T)M8YDf93pMX*%QBuUu0kX=(te^1fGK=)>% zqS|hZv&flDI6NF1sQ34EN2&ig8U~8CKR#0MGn>`D%0%dP@FEfvjD(e9OsHW^<@MVk-i{c!9#;lOu-$Qeg8 zguk3o2p>k_ymJE;?WXFjq@sT&*aMSsgisnt>kJo=#D%z}2 zTP8D_iY~n17P5BJdHHKaC99S0Sl-yZ^2TN&DR2+eCi!SQ==qe&Z^5sp3leEpXyIrS z4yKY}X97k`Y>63-kB!i#U1^xQm*CNGi$V)fNFx>rX`NEDP@J8SCw!q2cbu?*;>{*! zvqz^l-fUvAd3bDwt~=z}b85pqG(l0Af!j}%!~5#Khb6uppGz;rm$ z9j2hq&HDteHX^4n(R+4^>Vw;ZS&n@Ioa4>=tP2o8AF0%A6m072AD}{ndJq6#BvQqP zf##t{=bMjUx|*CMS&?cUi*A=o15(bC?e>V`tAsR-6e9>mWMTTeCM(ca0{si9;)7`_ zUS7*@q~w;0CP#881lEs1W2#Jo$ND~~o`LNU=4vCfWUZAJNjASI&f1nE1Y;`EG(iw4 zWu`%lky2!tplC~QQdBX4pm7wXBpQVxQ5-_UFER9xu)Zgx`F8+rk33fwc^M^%2!2+A zp@)I;Ca2O|R75r~os1d=QYbeca=ac(HKE4{Xs^t3mP@wn35v|J6<8Y2i~1<-kt;NX zqvKk;#|%>XJ}#YU4M}#H=HNibQ>JgdQgk$-(@7=U+*4q+2~W`gG%w8TA)`hf8HuYD zTq;Lf$OCvS`WqFsy^#Hqd8@reyw-&+b6GSR z^;gw03Ik$M5n4>z!(zFV_vj46Y!_wp7hd8p%?3+#o|)R9s#4TM)yhBxBRcFqIbwSJGV~J@@A*Q ziIb{aH9F4X_Y6*EGq zw>}y%6^f=@%i2) z?IM{V)sHpFcrffJum@RDdU@3W3L^c$OZvc);)6^+p_=gc(}m6{vzNljG-z_Hl&q#* zZFUK0oFOAcaSv`zMrF(sNeUeEE2U^4(C%3Z56z3*Y|ZQ-_5~No=6$4biOF9?wo?0g zu7`!chDi94bZ1Fl052`J)N-=fEaBAx|XW^R4`D!EBNZr=+ZP4&FAP6 z8ZpI~h;(`uCJYd$y6!fElgjo(4POM@i#GTji!k6B%h35*SgMn}W&{NYioyeyTyW(MFq`}94x#wGi_LpY z@RDL7siCJS(o{;fiNfl%vMmV_>^)=B;!q_{^TZ*OyX_Kj$njCw zK4w(1#34%@vYCl~hm*u^WVpViWU*)x3p*@Y0|lz9JmP7QXqIIMFDn*|c7So!nPQ7d zv(5_lv)~WI^{d6AEfWWZO}U6RAF>=~b&E@rxO_>!Wh7=-S=hNC&o{qdpiRm12$MBY z(kIUZ(%I;h8Nl|;yPnTWvs`xhHkv}Uq$#O(iuFy63B^m$j4m|6DBI=*N<^73%)H6K z2U?=vo};JLDZz3kL!}Oz63nIqvnj!BO0dstg0=Zjeow?c%QO3w4af4#TER0Lwwh;V zJ+kGQS)SQF^2|KPaj)W;MWcz^haZQdfQ7@T&jQbN1`%VyfDI-?w{tzu?8fTtZFjuWmRnQAQL$P&#g(d?E)vsY?lRd4=e zrrwjyuEaieHpI|ioYVr*5+H28q_$vTXtEZBCP8>rW!bPmv?B_wHmB9*%u769A2M=6 z*92FJHuU`a!uIx|=hK*w!?@S7yKIwABqGaoUJ1-NVR9m{0|^01i~?X-4%0IPLIq;# z*>6}GEV)%^m8#HKrs6iR$V>vc4Yptbn-JlC<0klC=vbZFxQUIM*tp4O5I4c?v3AEz ztVgzS6B{?V$GC}i<;@<^#EJUUoeu{z@`i)JX90`GwC@Mr*y#^$y4mB#>aC2MOx(Mj z&mZ+hgV6)VO^iHE$4y3)gK-l)pg(-&`qM#w>N+-VV&f+Efd21rmZ)z>g5l$~#c3g7 ztlAw(dluP(VRCMZSaHnlm}nLTb;i|R@zFF6GhV>jh~oJKrsfJ^4^cKeStKHqnBLt2 z(I^n7@QS40Kq7r)(ZbQ}xKJQuWLYHWt}Gh%K)yxkwp550G(lLN6g(q;qcYsQlOc=o zV7`i2*IpF~#p8?W>|dgl+3-O5>&|a6SiEbcI%B`Y z+mD!3v%5a0tncGN#T~A%D`OwFYcsMob@kzXJXDRPRxIY&#aKB-Gh>+py5Z%ec8|Qm z`%^aW@kXEO8l6b!{*<{23@xNfUect1Z=OQI&4={p3n~L9g{6z!X*k^GP?zgkom#p` lE$U(EB9<=lsnSI-D%ftii1o;pF7oyd{|_dcr49N|0RY)$9x4C; literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..f4630a7a468a03347b3edb7142f133ff4ef538dd GIT binary patch literal 6895 zcmVM-!l*Sz{E_DME^ zq<%8%-%~wOchhdSGnJwkj4v3Z7$1W__9B|o({v$v^B;RWz2F%SV%D24Xq>S=Tv;$F zSyWw!Sj=)31snAXOT+r!Tozg0oA;c-tY=E%?`tV8SW4BMJ@}#iPQ#pw6e|v6nq?en z>1CUYCuAeeNW?W=aQ{O#&aPzP+Z-b(uCdT<}6&XgsBHdDa+?|-A`2AS#^(j z;thAbdDHJfy?L<)b+Mc$EX!%K?#*3)>N~za9ZzS@3>u9!`ObetEGpKozhAs|9Ctn+ zj)QPPsXLmD!s%${k0x|D8b@?IozjSTBM?GJzT_#5PvHT7>xt;K`e!(s&gOH+iy}Yr zLeKNunHxC4EQm&n;V_)C+1U4Hy?+O-h&W~;0JD%Hfk*t<`<87$7O`e&B&fuvfgtca zsO*Bq1vd7Wt+&6RuE;Uep3|7HgRhRB9>FcORis56zkwRo*fw}>S=e<3BX{Z_K7V%f z;=ps<>4EEwhEVjog2sHoncM|75_}ImedP*7JbImP*6apos$bvqt?Ay+bP+u?Td^=! zq9pHmj^`XWL-_O39nKwp?u`e|%>B1sd3P>pnk|G(==CMWbALE+$EpPT0q)jV{0*uT zls9$vqp|N}mW7h9OVzEYynv2$47J@Z4~=uCkOZ6nt$=3itLM+4%qq{y0dzvR46>*nzbl|FL^on-ZVVI!onKdO?l}YBaRx3*b?A>hqZQNbd&=M) z_rL>TeN=@rK|vFiQ?$x20FrD?mEE9>^TWgMzyE%Ky^+dr#i7{)A(w|=J~=(~2F@Xf z@j40*m7k$Sz7kUPHM&^vm>cz@UKh|A4ct$QblE3gaKMy7&E{(Lz3i^L9r?!I`Q`JY z)8jo8Ew9yFkzIUpw?)OCf z9CSk>7E6AEr@DHV8bi`$L6>_1uq-|VYuOXv#hRu(TU{eS`IPG`u77=l<3f?<;7^o> zvLZN#?!V*4k2p9wPBRiQ5>s*{k|O0H4f*z$^k!$iC+e<@`%+m~eScTouaVjC6^(e7 z2n$1-FkEm6x{qjNVQ3YG?PbJlS7;Rj_6G>xdmp-miMGAv=XYKMld;9y~B6NYm$ zQL-qsKTpax7KL_Eu(PC=A@FC;kemVx@Y=t%J z`qb)Lbk1{BO=A)2BqltKI5;mv1Arl+fWKIlOPaGKzvX=+p*$t}iMx&3`C-2Sh}IF$ z+3VdKn$qvxVj=!LUGB7rDgj?eQvz~8DW0*27<_M;Ggx>#2d`R5kJ$xH^GHQ*aA`Se zMs^xK`vn8Y0olJ()1_nbO}VZ~;O|`Wpx7{y6$Glz=s6?hD#P|NgFzFDq~se>ke5Zk z2CveuQhvc?h6{4!h$cK1@VWSQj4RbJne)-b^bCc&>gXO#l+GEXzp zlK`%mhv3n6dX)(Uhmz48+!%Po1SC_DU~_kN7;FyG@F0y2M+4{3V*7y3w*%TvaQp1! zi#^G{T*I*PL>0J34E?0niHL59vF_aPrUw3MaM^Jbzo%LEr#w!sYdiXmP{qMtmsWax z^RK%Rgjk78A&&w)D*LD3D`bqkBxAp;C?2e68pUh?@*G~FO5oUHhISM(7GcRC>0Mze zJ-VoNRD1kUO$s!3o-V@`jAo1`$SNouhx+ezf>M7V*&^-9&tx z`GGSWhE6z~jornVx?vPXBbUx*bT%G(=-zH6KKju#ocQi&>@8+NFpa#x8PEK2vjlwugM$a8uDYJ*2oxR}<}-tg<5 z5ThN{ss~%%-aUDH=&vu-qR6M)%M50RWCOp%1Uwy#63i}ix48;>0>45h2SBiHej#ii zs8u|kv)n?^E(F#ubt(XNN{&Rj#Ki>iMZv&95!FXF>Um(7sRb>OaBKC-}GylLDA9#+{X*XhLHtzEw8;*_pbRh17%X?jp`&f@`<32X-^N4Yu znKPTzai6`w&0;zmjYr{#hNBQJEtnHd+|h6txD%hM&Gfg%eeNpW&A89_p>ZF7^e)3b z<4%Nqrp~pg6EN&E!qf>vXFhrtsS_G8id$bt`{cnZ>E^Ljxo10t_-{X{Qr`@qNV8ZS z+qhmJ_S;V-iLnYbt?R4{J5iX@Mr~-shDP4Ik)^o~T!F*tMHIVFre)H80`3>a$c5@I zc9Tr2V4RtS(d_EVs0F4?V7!s*!Xyg@e0b~7Tm96qa&pdO$U|=XO;=)Db*B+2Qw#JD zn<0>+qbHvp*KB(m_xM}q{9kD*;!e8d3S7?(3V zOVjU20IkO6u@_S;ymsr_$BwNHVRj;fiA&X84q@6c(S|T>2=g&Rn4at36vA{T-V9@p zQ+GBCW`5*G{&+m~UAmY=!_fEd2w~n?yn`W3e=zok4-8>W#_ukMS;^CW4AZ?ehUw1c z!>Ns7+8CydVSX$_%WjAUY2Y>_(>f4;;gJNbStMFV!uqAofBd)~T7#yzFDS79lvF=o^Y8_$_ z;m{c#R;n2c?n@b$Fih{j~M?u zNe`pxjr+DeHe3uxA)7hA=PW22`VO5i*BwoxNd#)>-A;PAvv~JN55v)5=6VkzJ&XpP zJAGHAhgzQYNe?Dj#(P?o}A;kgHe;dD1Bd_c_jPLvEPf%Ar>Y)SQJ#IVT9t>>Yck=5Jm zZ=WSw5Px0RVOfrt^dn&=hRZZ6V!*uIATnJ=Y?9KiZ;{tEYibRQ3Do>}D z?|o6IgTwbLLb%d|$`Bit=29n~4=I?QY1?A2?RDk~Rig;bnabIYBIhvjCMpRd#+=T?tu&zo`!r)<3wc`$!NeNC=8#Cj1RggA}A=AAlcg#mrAsSC3FKA zB7=pMsohJeNgp%Ai#oqXnZ*9P0+52A&I>hZAnXUlM$uFh3KGZrTolDmp`^$;nxDB7 z#Fg0Fa%eIrrio3J2_F;cvme$N`KnA$0rU;FtD?P7R89xWgfnw2$pJ@BndkgvTsW1-zl0H z=-v!eRNIYl7CDm%hlgVY&Gx?PDfOSBVW4RH{UZfGv;8G5>rkuPAO)CC7#EqQ^YIiH zr8BM44cYz*3WHBd4f~d(NnTJ+pH}dyx2uP0{Ja*)wTM*rs!m^k&p|cWMG9FB%HH6t zG+7$H(G>J(_}s*aprx?_*=SnZMOq029b$>_LV=oQB})BOkQ<++yazOl-NrsM14Sre zVlE5L;zq#*auvn;f{#fit!` z?HU#BpA3>&G%XeFW5?D~(Kv%IlK<&7PQq`);$o8+VQpy%gQehYp*U6M$< zLJLQ$a4?k&I}^}bVoS_uy>Emz?Mlnky#%+0TNFBgLK?A1Nb8i^h2r#-JmX81xZ{`w z6i+reJ$iC-=gB4(n|F`R*mXw(cSx#9m4%}lj4}0gN6Z`D!SCxpmI&5GtnW_*YuZ4+ z%3ebs!t~mZlWfg!YOdG^P_lk~Dp!&=#LVGem^r)-cq`5lavoT7C>;=FBb=m~%OT(n zd?PlC6P2u~BKc?*4vR^~mnp{sdUPldEP>T;49I+h+gCKeV|ElX&CnkeTSw=#K_W(x z6%X-rjt>_KnmsdGRNbwR2909fcDBfXSrlmOHgE%t-;cGUNn2@ zfZU)d>>an-J!X*7=W*#wYe=%oGzSM7o-%#vm7=2wolYvLa!-NPCOkz0(7Z6ShKw3{ zVkE9paH$+^ArIhbb@nKe$E{Xszo~I!_3I}2P#bJXLj;e*WUgg2cjJ2O_8$U4R5l8g zBDIT0`YvcP_+IeRnqro!!}@5ftnHZQ70c`Nce@)KwtGZ2&=fD26{X;(8Tel-c4+viSI2UC-8!RhAyLS82S#wYojp> z#$)V-x3PilEZ#{rkUw@G#0K&Q6L-|YK=F)DKmr$&PWf8TV`U@8??Wrhj15qX)$gb|Yw7_)0P4O(TSzTHM;yE+1tqU$O z&SlYP)nC=qC=7^2MQAbU4vXbV-lJ0tvt5+YUwDbbG#f0{d1e}&s!mZ8)iVPXkfe7_ z*DUL$i?-3F^hX#t&smt3euFBa3K$~T9r6?+k(OB^Zm>RN!8{ikB~J@{iQXaS?3_A9 z%afh1Pn=Zes?m8?pm8AFd9hTq8CnndC3RxIpJ3jX&-uQdFi8<+*wKOTQcF_t5c2KY zP%$HvdYiowQ=w?eMc&Z2&LyN65YWhS-KnB4s1zt`xZm&^c)H?&A+OXYO;MH3CKD-b zBq&O6(k_w-QvFzyj0eMx0$Y$3rI%L?pdiu@yrd7TC_c#K6KV*LpDuJxnY|QFrb3fr zrDQejYO_y3;|v)oihFQ(GAd)9NK)XKUnxZcfqG92wr?YiOHBSE zvb9>*b3MHIOBmG99d7n(Kf}|%8i0OIBgvzF+0r9+3=pG@K+{r{p9(t4cLiTx*Sj3rls9m)uBF8V6=? zid6iPl1Be3**{lD2@dAlN7~f(*R<+OpV2^a5c_!%aFF$!EfiSn#(dJ%=KEv zoI%QRBh_O7$4n^lNM9SYvLw}8L7Y$#M)0<*IA^(P8GOM!K#Bx|rk5&Cmg3UT*rdpU zjt&oFAydhq1bQC)6$}xnTvf`i1uiX@B(RL=>X=IZ^{S^bG+ph2sObokoEJh1tmsZC z7&7`+lq4*~1+Jug$E!PRJ%Q@iofr9gx^2ku!J7WANoXOLrhTUWe1lB|=mO}3tL>#Z z84vofP>iGng;tj>ib;;=HyYNHB4aIt&M{^x#TF?YzUu8%tSCHS$pu&L0JFKj>Jf_1 zyV$`lkiBI+f!}AT15O z+4g6EE4$)D`v?XNpi`x~OF+XC81?+eSSL8br63_~Zys;c2m9m^I6W z_#QovkZ*NrnL{5XnwB~A;bLo$zsta7Peb-1`1SHxy92W(JsplUREp^-2mgJ zJ;fH4cAXWjXTcwa^H+;SR|XCYn{p9tFS8s*b&E@zxO_^#Wh7%WLc+%=hOt?!sSF2DJr9hUAd@ zlNkHtSg6Akp~!Bxe^P)u0L%)V+Ut`icuSd-wD~J0^LkMPjbj;AvaoYO&bGf|piRm1 z1d}yUG9+IJq_fc*Gl1=xcQc=tX1VP0Z8U{yMN?9*6zh{36N-nR8C__CQC8&zN<^73 z%)H6K1ufC<&(YKAlwdiNp;CuU31(A**_2>5CD=zcz}kE$?-Q}l^2|PD!?8TGPVmge zo#vTYk8F8nmS^^eJTuR6+?#l2(PZWh;p2D`uy7m=S>Uu|zXVG`lC!?4{~i z^_xFAQt!!TRbn4I8)9fNPHKVZ2oN@3Qdh7rG+7Hmn;^Wbvus!(x)Fs|i_>ax<^}Gr z4;eY88-g=M8+v}fu)Tli`6MRfdfXe?U8^H0omfS0}N>ykqQ*j%ZWF~>Ef-QK1O^EQgaT9zmbgWKo+{DIBY~17{h@0T@ zShwRQ)+5`viH(~)V%)^L@nnx^=0rp4F2*AodE-&wvw%fYI`ji?>I_GBo$PUE@lM7~ zX70oG=T8Qc(c~S*O^iHk$4w@)YvU%kL4W+x_2;AE+;wc+#KukR2L1PNlxQwTg5l$` z#YrKdui6z!yBFDlVRCMZSaHm4pJ*2bwa3*S@zFL8N4$W!5ykxpOpO)7ZlY{?ut-EG zF}=G5qE#SH;1x-~fkb-CqJ^W~exX3f$f`)tU0F2jhJ1_CeW?)7X@amkD|kk}rZU|A zEJGIK#(WjAZoDcIirW{}-p9N?8s?raFqS>hMRT}6d*Vo>O&08~)x^eX`UJxSxXGUD z9nZK^_KeDyCWOeWL+p1@gz>@Qi_KVB?5Xb@F6+8v*CvFSH0h3uz1!= zy~loqrynt?Mt8kWS)a#)io3qNE{tuc=4NDL%IeMixTzZNTCeSLj8c`2R7qN7a50x&0QNeE0MXX1*bdfiI_{tC)xK% zs-&v^pJ|U|n~CF8l|=FJ{T?4Fz8@cd?8UU8XW2sb=0EmCb|G>RCA>FZ&?M)5_+`PB z;_>c>m?ym8akSA_c*g2`3svStZ{G6;vz{%9*Vjs3@Qms^dvKxtPFW#jh844f=DC1c zdif?7=~}8{&Xmma?`iVwT=BKcMIqG&itAf5n)18coU;{Axqfh*@nT-r{YclH@9wcr zyymVqZ~8r`w=$L0F<_lwt_=g;TE zal{ss`lH#1O-HkEG@--MIHu$2l*T+5fe=!OrO0S<1`qgaPsXqHKf~E{HlKSz9EWkh zf*|l`e&j{7C>||_LpJ5JaTv^c{|;J_Nx~TbvrsaHNBr3PmTy27x#n3cslum$An-h> z?1CmGHujXSw?Ct<$SKrb(1h@#Kc9Sg0=Lvwk(EjE25MMi+u*rXVc#2!{Aqan{K?6S zqrmg0M}9CGLecL^nuvwqY9H8GiUah_FP9(^@#|u<=GQ>8yZWZ@O!sc4i|C=*%7wKO zEqO2S0`JHh!k?G^aPEckU_9_<{=fCAy9-6Ld?8gzuP!m3hr@wC)+P84aJRwYuTY(& zqN%$dt$m;JoGG!cRJWq)0y@$u)ONExG%2`75^w^vBAW9*KYs#cRznpP^;3l1leAx>$%rSoNb`7tk3k++UX2vQNGefGLNX?XSDO;<`GS+)83)>C?5QC1poO*v(O(N1*7q_hvo;C^es3`C|jzA{LqPSZsZqg z`@h@&NqC>U5UI=>4c&b=4c*ldH0$_6(O>@y%NrqhUM2;4j|0yHD$03r=$&@I9`;1@ z9CSl6kxOxnr@DNX9z(KaNtXu#@H{yNYdH|$#hPX!UtJ+U^^~hCuCKnqaiPo#@F!YB zc^REU_uq5lCjuOupgD;-NhmpyX_*N|nb`iEUGK~fMBSJ1P%7uD@9wJmH8LBqqOr(R z>0oFRh6|xU_c4tf46VYjeHU}r6uh84_K-O$h#92^X7!fMm4Gj#838$?a~SPrdrn|@OPm^RBkxQO9EBr^qiAwm0|lXhd~pHWaJxJl9y$~ z2d}a}XX1jZ92exs2~9;J$(J(Uz9lC-K{w%0yk|_1GH6p##px>Cs`!@YJDuL$@%7ea z1fFh%Ud+VGcp6c)lst<~U$GH1St`csXjj31DC{>KZZ)5o%yjoCY&5XK<|&T_eL^F0 zL@q_^9YD9bE~9)C1~hOmxK-(>9I8FSATM-^JXd15D(aiE8YcL_75KI1e5XMCS!B8G zNdQ+O7vqjOiuu*D$O+(gm&%!#wGAD&uQnta~@Sse!*7T=qQ8?`hWkxk%Eh+K#@1={Wf7%1W

cDb)|hIk83Jf4nl&vZH|aBzYwe* zAI0BxpY?1Hm1Vvn3$J3?7&cyYrTWzt$jUUT$KfR|$FLpA0&Wsp+9DYFSw3GXB%)UkM1+7qB7}|Q<@2$)D7KzzllX>3g24C z3${u{T%ZQ<6pUxyo?eH|CO6u0yKXuA zc16p)Po9_XX13JQ*lUen4m3R4)3AMnqbc5zrijDSgteETkk$uQZ#m4tv}he46M zb^SIMeILWsLLyXHx44il?$s(D&w1e>Xcq$KmpT=1JR>JETjJUY`KsjLpos1x7-~r1 zf0Y9}y@LaD0je%Se)j;?y-lG{7(6^uG}fUhn2g*Qk{Z0q%)h5xi3==*^AwPK15+IQ zLI6s5B@#=s=YqWIJ)s2DQrho0;X!#W4XRlCW0;R^|3L{Q@alN6eXHk3d8s4kB`Zn= zGp3FCEhq4zpg=r$4$WWn`e2+q<0dS(z*lhH046iZxWabbQ-}e1!HMD+?A!iK$_#}l zV}Wh7s=ttAgZ&Bqw$E1C>t^OX~8`<@khg9YjXM$LntE4eioht>2s1?tz4_>UWQu6SBW`>h?UQ@2q}#{t z%01aDB!2z7F7?d-iZqMW(T%GG;=lgA;%x2$TkAS}vlnrxY}AEDTxjHj8(Es`z&mhw zy@=wr1+~0KAqDpfW8^}27q=&%4q8(M zZ(A5Lvc?OKK+hgXw4zfIugz= zbp`}@n{|YWmiPbStXs)MxNWDM=n~w~C;QP^^J=ncOu8^k z%4E9cg|K9cMknhNC`!I8b<8>;Co;bQSVyMlqDkDHOtA;^KA8euUE%o)p2J5I;Qyl9 zJABWeJg$3zRe_D+7G84mHe#eqtf<|OtLwX&C;|1a>lZux9oovjD@7tAB@F$6!W%uv z#2e##Ma_t5R&AM%uO4&=)J^s6^-X%RW@KbgIxlM~29_=}N>Z9}D5tp=IbW6vcT_*c zZ(wk^TBqfmlg7GJwW@WDL4;#(c)U~1U~pRoVHeVMEQk+Cr~e`r#6RP)9GZ^w;IMV1 zhpvzwd`Eit$U2rIJ$%CW-%5HIO|RXz9q{2|IAVO}g@L!Ad>DFk!hL@^1_lNY=ApMA=~ZPh*U*T3UUj?v&44Q||U;oI(ka6%V+`z~|Asdi(x3%&c=fzWD} zhy`HUErdA0SN*%?{A&Hm?qgq{a2Z0W1Q~T7_l(R zA=`6rx_P`V_v3OtV($%wS4cZQlFE4SP{aNrBpF&SXL;Lc%D%(J=?a$LEG!h6}mN43|obg{5=@7$Sp(SE=1As!1O+!^=9qMwP_=yAqIsOXsDYG!XuSW}_G? z3I&OiLoSNur%+NB0?p6f3F0d3Z3Q$L6w}0JmkA#e>hmAg82Q?no(AX}Y}bzVLQz#Y zfS#5c0tS#8&pRiDGK?8Uq4CfU3}81K#!kwn#%X4`H`==F)D;b419h?$JnDjDz7ntx zTOM?uSrkylhQ3hjb;)CczcSj`4FxFJ0M9EhCZZQkWy3zSbI5JCKxCn-8!lbn+LV3x ziXvFLWs0QyT&lh;Wv){+G0?posHm|U>nsYcQUMRg3YzVG)zj*KhK7Nn9rlk5{LJ>} zq^d)&Zi5tHIxH!3L+2A2E=uP{r5m#S1r!FKv>FaA$B?|BoH?!FRc~JpyYcf%B-b+5 z-D`LH0(_2klU<~c)u0><&RUa|;ag2XkA}}poCsPPFOiLAv|VO9fuKVy5nd=z)2w8r zza8Y(XKC*N4dXYl&+I_KWTM@z)xbmgUd+xz$)jb1TL#}@2rD?N&#a*2h0dR2W-xU+ zoR6_pE@u3IRJ6&F?;Uw#r_-)c(f-LGnM2c2(LQx-9Tlx3R5Y*CR5UjPIx3o@qCF@T zZ8p7@inbUnhHM&mY!rG?>`i0J;4g2&;K!IPdbd!~ZY$nND%#W=-Tz`^sc9(1#|cl4Bsym?WEL8CMPtLeR#nw71A>7wd!k4C2QQ>bG)$!<&B-lw8S+~ zm*k`MpyyMnz6HOYElF%#p@XAUIJi!RolEE~i6dsT-Zz*_yV5dsFU76l4uuY&ki|Tf z$~mQWp*TAuPsCCu?l|QU#gk3WP98nJ^<)!=&HKk@?E9mEKP0`sai?e5Icu|VdwBB;H^1J z$a&z+p>#lyjqo@#Tn-6$;2W`BoY={lX_Aj&;qZjyVwnj%pvQy)!4i1=#(>O6xP3(< zJZ48T(=7c_vvo{P8zf?sc}Xz^j^;z$Jg8J@XcEmCV3gNfH7xw(&e}s3fEIKY^*@mb zU!u02DRDvZKq8E7#vE7gggM9P|Nis8PqyzAD-E6D1eA}}JdR7Ic{Ah>TGl_}jYO&? z%_{Z~Qa&`eQw{Tc(f6DoDj2Sr>Q=;JeU{QCcrt_zRmjp z0kn}S!$!fXzWxCU#Ha@e@WnFId>E)6T6Dhs8BEuZlN2vA!(%b^3S~hmc)D92(R>v~ zvsg2NphuRr%^R`?eWlR8h-yBVq2d*d{6<=CrD<}cfSbVjF=$M+Nr=Rp2Q{#;U4^+` z2(4Ial|_>6FPgKq;|M{YN)1gAM9R3W5M!hiStcmj3Y-+xOdzNnMX88JOs1McX!#|U z9>SVyLWX|_;10+$eUO(`l8oWA77RTM+;4L#Eo4n(1JlW=Wgs!NeOKV|Sh@<+M?ia3 zp0h&nT}x18p0B~uL{T<->403LDeN6L+C65FGUsubOlwH8%MAwy8lG`;>XoLWNs~@0 z-Q}JFt4&3Q24Hw$b`2Rd^2kbDr{K~#+L(yoX?6A}o5!tIYQL#*YxV0U`Op|_MMDIS z!&PBqw0Glr?DiiLLDV)1l_IrE#O5w&GWcHb(uQJ|>BIVHqOI+e7CV;L*>83?E^POJ zY@jKILW7~xsbd3ul4v?M(8r6dV*_=94TM48F0+B$5a`%Ijt%s1Y@mr3PJ`>%K$9?7 z#9kCc5oJE7K4YQB@wd^KN8>T}!kgGYw-)at8z>z6_hJKugNZ+Wk8B_-Py1}3Y40XB zP&kh(y@rzPW^?8m+n-Qwt=XU59F8kT{>Vo z;HE^Lx~wj(0`Z*N*w%$m80T_mwCbaxb#P!TT2PElj(>2F>>7s3PDg6-!&I``6%5P9jQ~^T-yF;ErEVC+0#5LB3 zESTpqr{v2LU!wQOIeVv0(duNUs}m>HxoS+F6=)m?cU~?vZHCrEenp))>?fG_<*7K- z6DBFb3_B(eUTH~b9zwBw%QQ1WtGC%3F%^oTToes`n_NPg0RfGy)}40r1)TzA4fk7K zgUD7QvgDQeq$#S?+2k^#jRZy6b=pNTL1rFnlks5KQDO`7vhwn}0Te{~K~(gC6~zbH zd_oQ3@zaILDRYp*$yR7{tW>O~eQov$Xq+P>MRN~sPex_U6G;jJ^DAX&AW-jF1`jRD z!YGo}Ga7oCYWWLtxdVYX6e+`2gy2H(3?a%P^uLht`X{ZmD)!IiQ9^+E_K`NV{UzJ=r7vit1c?2-j0DJf z&KH_20#gNOIxSR{66R_xW6mILxv}msfMYH-d8BU)T3eFttsoB5gb}L;1w%&PijtHwT;NKpcf7j8))T0H-FcC}XWE7wAFSzbhJ?n1 zvhB0|=NoJ)Ko>zLTy5XklkuPrOU+1HP-u14qJ$K9exqeQX)@MAnjB-cQf!gd;j7+W z#hStcmRt($4ltYhs~(~Fyo>EyL-0~^si>u=Y0^|icZtH9w6Yxu5$rwZ%I1degQzJk zV`3sZsQ$|mT%FEwC6Shf-fa65z?EO}p?w5{M$oA;(!XNi-dE=;OuKF^4+B9GZ5TIpl^w#~gCZ zp@(D+O+xQl=Foygd`P`vF!jdcA&X-kE#e`IW`Q^KCzx2~M&{6M#kOA=9CDe7KEOd@KQi20QgT?diG}MHoq+sj!Jw z-*S@i@7)x)RT#pV?+v`5%HE@M(=;=ZXg-KbMA#WJl(cyjlX<-; zqsFm}I$7AcB+s_LV4zLQ^9YkQQ8FZ7Nu;yUYcqfyn0GUumtnc=^KA@;YDF_ruN0e; z8dHjgpjllQf>FN93zW(#VVHfBfgg-Szdc7!r&EFzTrsT$ z-YDig8u8I=?DuZwncZ5vdptAD2sORmd3BS)B=Fy7hN@{N&Z`@bEV*}|Xy%QM{NSZG znTNA^FmXgPM>M-5(d?z}S@oMgInnRQc2(jKI~QVTF;41$=m-!lUs6}Furyf*LYpAG ztg~!5Ai5ES){E16apnc?uVFSgzwXFv(0KwF|c34K5+V!^TbU zz0h+yb#W6HH*s;3PatlB%VXV+n>df`;wCO`@_=!Z;M$Wt;+Yo@slOPHXdH}3QOF}6 zPw6m>f~hwg-FC9at;IVTH<|hO+n+xfOh%LU7&o!i8@%&v@^;0FEiOFx{ChI8L@ zaT6CeaU1kMz)_;P90`Vx%NCDI1%1`6NV>hq4h)-fTPB)g?)pT#Fz7w5?ud`JaX1kr z%#A4SPvCm2kZu!Y%Y#KKnZoq$4v1EPcnq&d<_#n^TNWK0?e+@|LQYm?itfsx;Wp$u zlLtzQ8#4L>JBB?(B&Z znKfCk`&N@!tLYOA6W}I$VRk&@PT6NvB{bD^Sz}0ea*8>bg{JFLyNE|gN|#FBhbR%z zbBPZr;GGROl)viz7K6pJR_Ze{8 z@vb#b1U50zPSK87c7ravJk=hMmw0~4_AQ?1Qy-%fOVgezSAnI4^vMgFmT=`M+_-(0 z9eqYs#Fcb(k$ZKAyKL(6eWz1L7imO299_iGMLt%#2u20FO&4(<+0jMb{O Date: Fri, 17 Apr 2020 17:13:18 +0200 Subject: [PATCH 130/259] use mergeFrom method to add the new community contexts --- .../SparkResultToCommunityFromOrganizationJob2.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index 3116161e9..ff94edbf9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -98,15 +98,20 @@ public class SparkResultToCommunityFromOrganizationJob2 { if(rcl.isPresent()){ ArrayList communitySet = rcl.get().getCommunityList(); List contextList = ret.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); + Result res = new Result(); + res.setId(ret.getId()); + List propagatedContexts = new ArrayList<>(); for(String cId:communitySet){ if(!contextList.contains(cId)){ Context newContext = new Context(); newContext.setId(cId); newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - ret.getContext().add(newContext); + propagatedContexts.add(newContext); } } + res.setContext(propagatedContexts); + ret.mergeFrom(res); } return ret; }, Encoders.bean(resultClazz)) From 7d9fd7502038dd028958f6536d4b2a49c530776f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 17:13:48 +0200 Subject: [PATCH 131/259] add method addPid --- .../eu/dnetlib/dhp/schema/oaf/Author.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java index 93ab60b05..edce29221 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java @@ -1,8 +1,8 @@ package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; -import java.util.List; -import java.util.Objects; +import java.util.*; +import java.util.stream.Collectors; public class Author implements Serializable { @@ -83,4 +83,20 @@ public class Author implements Serializable { public int hashCode() { return Objects.hash(fullname, name, surname, rank, pid, affiliation); } + + public void addPid(StructuredProperty pid) { + + if (pid == null) + return; + + if(this.pid == null){ + this.pid = Arrays.asList(pid); + }else{ + this.pid.add(pid); + } + + + } + + } From 00c2ca3ee522be59c2247770bcdcf140e7f5b690 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 17:14:25 +0200 Subject: [PATCH 132/259] - --- ...parkResultToCommunityFromOrganization.java | 269 ++++++++++++++---- 1 file changed, 208 insertions(+), 61 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java index 1f634a693..de14946cc 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java @@ -4,17 +4,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.io.File; import java.util.*; import java.util.stream.Collectors; @@ -26,10 +27,13 @@ public class SparkResultToCommunityFromOrganization { final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityFromOrganization.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json"))); parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() .appName(SparkResultToCommunityFromOrganization.class.getSimpleName()) .master(parser.get("master")) + .config(conf) .enableHiveSupport() .getOrCreate(); @@ -37,83 +41,226 @@ public class SparkResultToCommunityFromOrganization { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/communitytoresult"; final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); + boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); + boolean saveGraph = TRUE.equals(parser.get("saveGraph")); + + System.out.println(new Gson().toJson(organizationMap)); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + JavaRDD relations_rdd_all = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + JavaRDD publications_rdd_all = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD dataset_rdd_all = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD orp_rdd_all = sc.textFile(inputPath + "/otheresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + JavaRDD software_rdd_all = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relations_rdd_all.rdd(), Encoders.bean(Relation.class)); + + relation.createOrReplaceTempView("relation"); +// String query = "SELECT source, target" + +// " FROM relation " + +// " WHERE datainfo.deletedbyinference = false " + +// " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "'"; +// +// org.apache.spark.sql.Dataset result_organization = spark.sql(query); +// result_organization.createOrReplaceTempView("result_organization"); +// +// query = "SELECT source, target " + +// "FROM relation " + +// "WHERE datainfo.deletedbyinference = false " + +// "AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "'" ; + + String query = "SELECT result_organization.source, result_organization.target, org_set " + + "FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + + "LEFT JOIN (SELECT source, collect_set(target) org_set " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + + " GROUP BY source) organization_organization " + + "ON result_organization.target = organization_organization.source "; + + org.apache.spark.sql.Dataset result_organizationset = spark.sql(query); + + JavaPairRDD result_communitySet = result_organizationset.toJavaRDD().map(r -> { + String rId = r.getString(0); + List orgs = r.getList(2); + String oTarget = r.getString(1); + TypedRow tp = new TypedRow(); + if (organizationMap.containsKey(oTarget)) { + tp.addAll(organizationMap.get(oTarget)); + } + try{ + for (String oId : orgs) { + if (organizationMap.containsKey(oId)) { + tp.addAll(organizationMap.get(oId)); + } + } + } + catch(Exception e){ + //System.out.println("organizationTargetID: " + oTarget); + } + + if(tp.getAccumulator() == null ){ + return null; + } + tp.setSourceId(rId); - File directory = new File(outputPath); + return tp; + }) + .filter(tr -> !(tr==null)) + .mapToPair(toPair()).cache(); - if (!directory.exists()) { - directory.mkdirs(); + if(writeUpdates){ + result_communitySet.map(c->new ObjectMapper().writeValueAsString(c)).saveAsTextFile(outputPath +"/result_communityset"); } - JavaRDD relations = sc.sequenceFile(inputPath + "/relation", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache(); + if(saveGraph){ + updatePublicationResult(publications_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/publication"); + + updateDatasetResult(dataset_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/dataset"); + + updateORPResult(orp_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/otherresearchproduct"); + + updateSoftwareResult(software_rdd_all, result_communitySet) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/software"); + } //relations between organziations and results - JavaPairRDD organization_result = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) - .map(r -> new TypedRow().setSourceId(r.getTarget()).setTargetId(r.getSource() )) - .mapToPair(toPair()); +// JavaPairRDD organization_result = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) +// && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getTarget()); +// tp.setTargetId(r.getSource() ); +// return tp; +// }) +// .mapToPair(toPair()); //relations between representative organization and merged Id. One relation per merged organization - JavaPairRDD organization_organization = relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) - .map(r -> new TypedRow().setSourceId(r.getSource()).setTargetId(r.getTarget())) - .mapToPair(toPair()); +// JavaPairRDD organization_organization = relations +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) +// && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) +// .map(r -> { +// TypedRow tp = new TypedRow(); +// tp.setSourceId(r.getSource()); +// tp.setTargetId(r.getTarget()); +// return tp; +// }) +// .mapToPair(toPair()); //get the original id of the organizations to be checked against the id associated to the communities - JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) - .map(c -> { - if (!c._2()._2().isPresent()) - return c._2()._1(); - return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); - }) - .mapToPair(toPair()); +// JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) +// .map(c -> { +// if (!c._2()._2().isPresent()) +// return c._2()._1(); +// return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); +// }) +// .mapToPair(toPair()); //associates to each result connected to an organization the list of communities related to that organization - JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { - List communityList = organizationMap.get(o._1()); - if (communityList.size() == 0) - return null; - TypedRow tp = o._2(); - tp.setAccumulator(new HashSet<>(communityList)).setSourceId(tp.getTargetId()); - return tp; - }) - .filter(r -> !(r == null)) - .mapToPair(toPair()) - .reduceByKey((a, b) -> { - if (a == null) - return b; - if (b == null) - return a; - a.addAll(b.getAccumulator()); - return a; - }); +// JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { +// List communityList = organizationMap.get(o._1()); +// if (communityList.size() == 0) +// return null; +// TypedRow tp = o._2(); +// tp.setAccumulator(new HashSet<>(communityList)); +// tp.setSourceId(tp.getTargetId()); +// return tp; +// }) +// .filter(r -> !(r == null)) +// .mapToPair(toPair()) +// .reduceByKey((a, b) -> { +// if (a == null) +// return b; +// if (b == null) +// return a; +// a.addAll(b.getAccumulator()); +// return a; +// }); - JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); - JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); - JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); - JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); +// JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); +// JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); +// JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) +// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); +// +// JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); +// +// updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); +// updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); } + private static JavaRDD updatePublicationResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Publication)r); + + } + private static JavaRDD updateDatasetResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Dataset) r); + + } + private static JavaRDD updateORPResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (OtherResearchProduct)r); + + } + private static JavaRDD updateSoftwareResult(JavaRDD result, JavaPairRDD result_communitySet){ + JavaRDD tmp = result.map(r -> (Result)r); + return updateResultDataset(tmp, result_communitySet).map(r -> (Software) r); + + } + private static JavaRDD updateResultDataset(JavaRDD result, JavaPairRDD result_communitySet){ + return result.mapToPair(p -> new Tuple2<>(p.getId(),p)).leftOuterJoin(result_communitySet) + .map(c -> { + Result r = c._2()._1(); + if(c._2()._2().isPresent()){ + Set communitySet = c._2()._2().get().getAccumulator(); + List contextList = r.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); + for(String cId:communitySet){ + if(!contextList.contains(cId)){ + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); + r.getContext().add(newContext); + } + } + } + return r; + }); + + } // private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { // results.leftOuterJoin(toupdateresult) // .map(p -> { From 72c63a326e92e75c9bbd06c244f1564cf1b4638d Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Apr 2020 17:14:51 +0200 Subject: [PATCH 133/259] removed unuseful class --- ...parkResultToCommunityFromOrganization.java | 390 ------------------ 1 file changed, 390 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java deleted file mode 100644 index de14946cc..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganization.java +++ /dev/null @@ -1,390 +0,0 @@ -package eu.dnetlib.dhp.resulttocommunityfromorganization; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -public class SparkResultToCommunityFromOrganization { - - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkResultToCommunityFromOrganization.class.getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json"))); - parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityFromOrganization.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/communitytoresult"; - final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); - boolean writeUpdates = TRUE.equals(parser.get("writeUpdate")); - boolean saveGraph = TRUE.equals(parser.get("saveGraph")); - - System.out.println(new Gson().toJson(organizationMap)); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD relations_rdd_all = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); - JavaRDD publications_rdd_all = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD dataset_rdd_all = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD orp_rdd_all = sc.textFile(inputPath + "/otheresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); - JavaRDD software_rdd_all = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(relations_rdd_all.rdd(), Encoders.bean(Relation.class)); - - relation.createOrReplaceTempView("relation"); -// String query = "SELECT source, target" + -// " FROM relation " + -// " WHERE datainfo.deletedbyinference = false " + -// " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "'"; -// -// org.apache.spark.sql.Dataset result_organization = spark.sql(query); -// result_organization.createOrReplaceTempView("result_organization"); -// -// query = "SELECT source, target " + -// "FROM relation " + -// "WHERE datainfo.deletedbyinference = false " + -// "AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "'" ; - - String query = "SELECT result_organization.source, result_organization.target, org_set " + - "FROM (SELECT source, target " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + - "LEFT JOIN (SELECT source, collect_set(target) org_set " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + - " GROUP BY source) organization_organization " + - "ON result_organization.target = organization_organization.source "; - - org.apache.spark.sql.Dataset result_organizationset = spark.sql(query); - - JavaPairRDD result_communitySet = result_organizationset.toJavaRDD().map(r -> { - String rId = r.getString(0); - List orgs = r.getList(2); - String oTarget = r.getString(1); - TypedRow tp = new TypedRow(); - if (organizationMap.containsKey(oTarget)) { - tp.addAll(organizationMap.get(oTarget)); - } - try{ - for (String oId : orgs) { - if (organizationMap.containsKey(oId)) { - tp.addAll(organizationMap.get(oId)); - } - } - } - catch(Exception e){ - //System.out.println("organizationTargetID: " + oTarget); - } - - if(tp.getAccumulator() == null ){ - return null; - } - tp.setSourceId(rId); - - - return tp; - }) - .filter(tr -> !(tr==null)) - .mapToPair(toPair()).cache(); - - if(writeUpdates){ - result_communitySet.map(c->new ObjectMapper().writeValueAsString(c)).saveAsTextFile(outputPath +"/result_communityset"); - } - - if(saveGraph){ - updatePublicationResult(publications_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); - - updateDatasetResult(dataset_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/dataset"); - - updateORPResult(orp_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); - - updateSoftwareResult(software_rdd_all, result_communitySet) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/software"); - } - - - //relations between organziations and results -// JavaPairRDD organization_result = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> RELATION_RESULT_ORGANIZATION_REL_CLASS.equals(r.getRelClass()) -// && RELATION_RESULTORGANIZATION_REL_TYPE.equals(r.getRelType())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getTarget()); -// tp.setTargetId(r.getSource() ); -// return tp; -// }) -// .mapToPair(toPair()); - - //relations between representative organization and merged Id. One relation per merged organization -// JavaPairRDD organization_organization = relations -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter( r -> RELATION_ORGANIZATIONORGANIZATION_REL_TYPE.equals(r.getRelType()) -// && RELATION_REPRESENTATIVERESULT_RESULT_CLASS.equals(r.getRelClass())) -// .map(r -> { -// TypedRow tp = new TypedRow(); -// tp.setSourceId(r.getSource()); -// tp.setTargetId(r.getTarget()); -// return tp; -// }) -// .mapToPair(toPair()); - - //get the original id of the organizations to be checked against the id associated to the communities -// JavaPairRDD organizationoriginal_result = organization_result.leftOuterJoin(organization_organization) -// .map(c -> { -// if (!c._2()._2().isPresent()) -// return c._2()._1(); -// return c._2()._1().setSourceId(c._2()._2().get().getTargetId()); -// }) -// .mapToPair(toPair()); - - //associates to each result connected to an organization the list of communities related to that organization -// JavaPairRDD to_add_result_communities = organizationoriginal_result.map(o -> { -// List communityList = organizationMap.get(o._1()); -// if (communityList.size() == 0) -// return null; -// TypedRow tp = o._2(); -// tp.setAccumulator(new HashSet<>(communityList)); -// tp.setSourceId(tp.getTargetId()); -// return tp; -// }) -// .filter(r -> !(r == null)) -// .mapToPair(toPair()) -// .reduceByKey((a, b) -> { -// if (a == null) -// return b; -// if (b == null) -// return a; -// a.addAll(b.getAccumulator()); -// return a; -// }); - - -// JavaRDD publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)).cache(); -// JavaRDD datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)).cache(); -// JavaRDD software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)).cache(); -// JavaRDD other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) -// .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)).cache(); -// -// JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); -// -// updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); -// updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME); - - } - - private static JavaRDD updatePublicationResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Publication)r); - - } - private static JavaRDD updateDatasetResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Dataset) r); - - } - private static JavaRDD updateORPResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (OtherResearchProduct)r); - - } - private static JavaRDD updateSoftwareResult(JavaRDD result, JavaPairRDD result_communitySet){ - JavaRDD tmp = result.map(r -> (Result)r); - return updateResultDataset(tmp, result_communitySet).map(r -> (Software) r); - - } - private static JavaRDD updateResultDataset(JavaRDD result, JavaPairRDD result_communitySet){ - return result.mapToPair(p -> new Tuple2<>(p.getId(),p)).leftOuterJoin(result_communitySet) - .map(c -> { - Result r = c._2()._1(); - if(c._2()._2().isPresent()){ - Set communitySet = c._2()._2().get().getAccumulator(); - List contextList = r.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); - for(String cId:communitySet){ - if(!contextList.contains(cId)){ - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - r.getContext().add(newContext); - } - } - } - return r; - }); - - } - // private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { -// results.leftOuterJoin(toupdateresult) -// .map(p -> { -// Result r = p._2()._1(); -// if (p._2()._2().isPresent()){ -// Set communityList = p._2()._2().get().getAccumulator(); -// for(Context c: r.getContext()){ -// if (communityList.contains(c.getId())){ -// //verify if the datainfo for this context contains propagation -// if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ -// c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME)); -// //community id already in the context of the result. Remove it from the set that has to be added -// communityList.remove(c.getId()); -// } -// } -// } -// List cc = r.getContext(); -// for(String cId: communityList){ -// Context context = new Context(); -// context.setId(cId); -// context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); -// cc.add(context); -// } -// r.setContext(cc); -// } -// return r; -// }) -// .map(p -> new ObjectMapper().writeValueAsString(p)) -// .saveAsTextFile(outputPath+"/"+type); -// } - - -} - - -/* -package eu.dnetlib.data.mapreduce.hbase.propagation.communitythroughorganization; - -import com.google.gson.Gson; -import eu.dnetlib.data.mapreduce.hbase.propagation.Value; -import eu.dnetlib.data.mapreduce.util.OafRowKeyDecoder; -import eu.dnetlib.data.proto.OafProtos; -import eu.dnetlib.data.proto.TypeProtos; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.mapreduce.TableMapper; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.io.Text; - -import java.io.IOException; -import java.util.Set; - -import static eu.dnetlib.data.mapreduce.hbase.propagation.PropagationConstants.*; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getEntity; -import static eu.dnetlib.data.mapreduce.hbase.propagation.Utils.getRelationTarget; - -public class PropagationCommunityThroughOrganizationMapper extends TableMapper { - private Text valueOut; - private ImmutableBytesWritable keyOut; - private OrganizationMap organizationMap; - - //seleziono il tipo della entry: - //Result: - //se non e' deleted by inference ed ha organizzazioni a cui e' associato, - // //emetto id della relazione ed id del risultato per ogni organizzazione a cui e' associato - //ORGANIZATION: - //se non e' deleted by inference e non e' deduplicata emetto l'id della organizzazione - //se non e' deleted by inference ed e' deduplicata, emetto id della organizzazione ed id del deduplicato e lista delle organizzazioni mergiate - @Override - protected void setup(final Context context) throws IOException, InterruptedException { - super.setup(context); - valueOut = new Text(); - keyOut = new ImmutableBytesWritable(); - organizationMap = new Gson().fromJson(context.getConfiguration().get("organizationtoresult.community.map"), OrganizationMap.class); - System.out.println("got organizationtoresult map: " + new Gson().toJson(organizationMap)) ; - } - - @Override - protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException { - final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType(); - final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference - if (entity != null) { - switch (type) { - case organization: - DedupedList communityList = getCommunityList(Bytes.toString(keyIn.get()), - getRelationTarget(value, DEDUP_RELATION_ORGANIZATION + REL_DEDUP_REPRESENTATIVE_RESULT, context, COUNTER_PROPAGATION), context); - if (communityList.size() > 0){ - valueOut.set(Value.newInstance( - new Gson().toJson( - communityList, //search for organizationtoresult it merges - DedupedList.class), - ORGANIZATION_COMMUNITY_TRUST, - Type.fromorganization).toJson()); - context.write(keyIn, valueOut); - context.getCounter(COUNTER_PROPAGATION, "emit for organizationtoresult ").increment(1); - }else{ - context.getCounter(COUNTER_PROPAGATION, "community list size = 0 ").increment(1); - } - - break; - case result: - Set result_organization = getRelationTarget(value, RELATION_ORGANIZATION + REL_RESULT_ORGANIZATION, context, COUNTER_PROPAGATION); - for(String org: result_organization) - emit(org, Bytes.toString(keyIn.get()), context); - break; - } - } - } - - private DedupedList getCommunityList(String organizationId, Set relationTarget, Context context) { - DedupedList communityList = new DedupedList(); - relationTarget.stream().forEach(org -> communityList.addAll(organizationMap.get(StringUtils.substringAfter(org, "|")))); - communityList.addAll(organizationMap.get(StringUtils.substringAfter(organizationId,"|"))); - communityList.stream().forEach(c->context.getCounter(COUNTER_PROPAGATION,"found organizationtoresult for " + c).increment(1)); - return communityList; - } - - private void emit(String org, String resId, Context context) throws IOException, InterruptedException { - keyOut.set(Bytes.toBytes(org)); - valueOut.set(Value.newInstance(resId, ORGANIZATION_COMMUNITY_TRUST, Type.fromresult).toJson()); - context.write(keyOut,valueOut); - context.getCounter(COUNTER_PROPAGATION, "emit for result").increment(1); - } - -} - */ \ No newline at end of file From fbf5c27c279fe27dcbf03ec7fe2a5196f7998469 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:09:03 +0200 Subject: [PATCH 134/259] Added preparation classes before actual propagation --- .../PrepareResultCommunitySetStep1.java | 152 ++++++++++++ .../PrepareResultCommunitySetStep2.java | 105 ++++++++ ...parkResultToCommunityThroughSemRelJob.java | 25 +- ...arkResultToCommunityThroughSemRelJob2.java | 232 ++++++++++-------- ...arkResultToCommunityThroughSemRelJob3.java | 105 ++++---- ...t_preparecommunitytoresult_parameters.json | 33 +++ 6 files changed, 481 insertions(+), 171 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java new file mode 100644 index 000000000..d334e518d --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -0,0 +1,152 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.QueryInformationSystem; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareResultCommunitySetStep1 { + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep1.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySetStep1.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); + + final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); + log.info("communityIdList: {}", new Gson().toJson(communityIdList)); + + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + + Class resultClazz = (Class) Class.forName(resultClassName); + + + runWithSparkHiveSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo(spark, inputPath, outputPath, allowedsemrel, resultClazz, resultType, + communityIdList); + }); + } + + private static void prepareInfo(SparkSession spark, String inputPath, String outputPath, + List allowedsemrel, Class resultClazz, String resultType, + List communityIdList) { + //read the relation table and the table related to the result it is using + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + relation.createOrReplaceTempView("relation"); + + log.info("Reading Graph table from: {}", inputPath + "/" + resultType); + Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + + result.createOrReplaceTempView("result"); + + getPossibleResultcommunityAssociation(spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); + + } + + private static void getPossibleResultcommunityAssociation(SparkSession spark, List allowedsemrel, String outputPath, + List communityIdList) { + + String communitylist = getConstraintList(" co.id = '", communityIdList); + String semrellist = getConstraintList(" relClass = '", allowedsemrel ); + + + /* + associates to each result the set of community contexts they are associated to + select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false "+ communitylist + + " group by id + + associates to each target of a relation with allowed semantics the set of community context it could possibly + inherit from the source of the relation + */ + String query = "Select target resultId, community_context " + + "from (select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false "+ communitylist + + " group by id) p " + + "JOIN " + + "(select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + semrellist + ") r " + + "ON p.id = r.source"; + + org.apache.spark.sql.Dataset result_context = spark.sql( query); + result_context.createOrReplaceTempView("result_context"); + + //( target, (mes, dh-ch-, ni)) + /* + a dataset for example could be linked to more than one publication. For each publication linked to that dataset + the previous query will produce a row: targetId set of community context the te=arget could possibly inherit + with the following query there will be a single row for each result linked to more than one result of the result type + currently being used + */ + query = "select resultId , collect_set(co) communityList " + + "from result_context " + + "lateral view explode (community_context) c as co " + + "where length(co) > 0 " + + "group by resultId"; + + spark.sql(query) + .as(Encoders.bean(ResultCommunityList.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java new file mode 100644 index 000000000..6efb2205b --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -0,0 +1,105 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.ResultOrcidList; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import scala.Tuple2; + +import java.util.HashSet; +import java.util.Set; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + + +public class PrepareResultCommunitySetStep2 { + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep2.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySetStep2.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_prepareresulttocommunity2_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + SparkConf conf = new SparkConf(); + + runWithSparkSession(conf, isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + mergeInfo(spark, inputPath, outputPath); + }); + + } + + private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { + + Dataset resultOrcidAssocCommunityList = readResultCommunityList(spark, inputPath + "/publication") + .union(readResultCommunityList(spark, inputPath + "/dataset")) + .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) + .union(readResultCommunityList(spark, inputPath + "/software")); + + resultOrcidAssocCommunityList + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) + .reduceByKey((a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set community_set = new HashSet<>(); + + a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); + + b.getCommunityList().stream().forEach(aa -> { + if (!community_set.contains(aa)) { + a.getCommunityList().add(aa); + community_set.add(aa); + } + }); + return a; + }) + .map(c -> c._2()) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } + + private static Dataset readResultCommunityList(SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map(value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), Encoders.bean(ResultCommunityList.class)); + } + + + + + +} + diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java index 7739ff99d..f5c859280 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp.communitytoresultthroughsemrel; +package eu.dnetlib.dhp.resulttocommunityfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; @@ -33,7 +33,8 @@ public class SparkResultToCommunityThroughSemRelJob { System.out.println(key + " = " + parser.get(key)); } - /* SparkConf conf = new SparkConf(); + + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() @@ -45,7 +46,7 @@ public class SparkResultToCommunityThroughSemRelJob { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/communitytoresultthroughsemrel"; + final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; //final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); @@ -104,8 +105,14 @@ public class SparkResultToCommunityThroughSemRelJob { software.createOrReplaceTempView("software"); other.createOrReplaceTempView("other"); - org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); - publication_context.createOrReplaceTempView("publication_context"); +// org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); +// publication_context.createOrReplaceTempView("publication_context"); + + org.apache.spark.sql.Dataset publication_context = spark.sql( "SELECT relation.source, " + + "publication.context , relation.target " + + "FROM publication " + + " JOIN relation " + + "ON id = source"); org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); dataset_context.createOrReplaceTempView("dataset_context"); @@ -202,7 +209,7 @@ public class SparkResultToCommunityThroughSemRelJob { */ } - /* private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ String query = "SELECT target_id, collect_set(co.id) context_id " + " FROM (SELECT t.id target_id, s.context source_context " + " FROM context_software s " + @@ -223,7 +230,7 @@ public class SparkResultToCommunityThroughSemRelJob { " FROM other_context o " + " JOIN " + table + " t " + " ON o.target = t.id) TMP " + - " LATERAL VIEW EXPLORE(source_context) MyT as co " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + " GROUP BY target_id" ; return spark.sql(query); @@ -413,7 +420,7 @@ public class SparkResultToCommunityThroughSemRelJob { private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ - String query = "SELECT source, context , target " + + String query = "SELECT relation.source, " + table +".context , relation.target " + "FROM " + table + " JOIN relation " + "ON id = source" ; @@ -484,5 +491,5 @@ public class SparkResultToCommunityThroughSemRelJob { return tp; } return null; - }*/ + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java index 2da8d648f..345bd7905 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java @@ -1,6 +1,6 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; -import com.cloudera.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -21,24 +21,19 @@ import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; -public class SparkResultToCommunityThroughSemRelJob { +public class SparkResultToCommunityThroughSemRelJob2 { public static void main(String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob.class + .toString(SparkResultToCommunityThroughSemRelJob2.class .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); parser.parseArgument(args); - for(String key : parser.getObjectMap().keySet()){ - System.out.println(key + " = " + parser.get(key)); - } - - SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() - .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) + .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) .master(parser.get("master")) .config(conf) .enableHiveSupport() @@ -48,128 +43,157 @@ public class SparkResultToCommunityThroughSemRelJob { final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; - //final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); - //final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + //final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + //final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - JavaRDD all_publication_rdd = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD publication_rdd = all_publication_rdd - .filter(p -> relatedToCommunities(p, communityIdList)).cache(); + JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD all_dataset_rdd = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD dataset_rdd = all_dataset_rdd - .filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD all_orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD orp_rdd = all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD all_software_rdd = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD software_rdd = all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + System.out.println(publication_rdd.count()); +// JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") +// .map(item -> new ObjectMapper().readValue(item, Dataset.class)); +// +// JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") +// .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); +// +// JavaRDD software_rdd = sc.textFile(inputPath + "/software") +// .map(item -> new ObjectMapper().readValue(item, Software.class)); JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + + System.out.println(relation_rdd.count()); + +// .filter(r -> !r.getDataInfo().getDeletedbyinference()) +// .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); - org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), - Encoders.bean(Software.class)); - org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); +// org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), +// Encoders.bean(Dataset.class)); +// +// org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), +// Encoders.bean(OtherResearchProduct.class)); +// +// org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), +// Encoders.bean(Software.class)); +// +// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), +// Encoders.bean(Relation.class)); + publication.createOrReplaceTempView("publication"); relation.createOrReplaceTempView("relation"); - dataset.createOrReplaceTempView("dataset"); - software.createOrReplaceTempView("software"); - other.createOrReplaceTempView("other"); +// relation.createOrReplaceTempView("relation"); +// dataset.createOrReplaceTempView("dataset"); +// software.createOrReplaceTempView("software"); +// other.createOrReplaceTempView("other"); -// org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); -// publication_context.createOrReplaceTempView("publication_context"); + String communitylist = getConstraintList(" co.id = '", communityIdList); - org.apache.spark.sql.Dataset publication_context = spark.sql( "SELECT relation.source, " + - "publication.context , relation.target " + - "FROM publication " + - " JOIN relation " + - "ON id = source"); - - org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); - dataset_context.createOrReplaceTempView("dataset_context"); - - org.apache.spark.sql.Dataset software_context = getContext(spark, "software"); - software_context.createOrReplaceTempView("software_context"); - - org.apache.spark.sql.Dataset other_context = getContext(spark, "other"); - other_context.createOrReplaceTempView("other_context"); - - publication = spark.createDataset(all_publication_rdd.rdd(), - Encoders.bean(Publication.class)); - publication.createOrReplaceTempView("publication"); - - dataset = spark.createDataset(all_dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); - dataset.createOrReplaceTempView("dataset"); - - other = spark.createDataset(all_orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); - other.createOrReplaceTempView("other"); - - software = spark.createDataset(all_software_rdd.rdd(), - Encoders.bean(Software.class)); - software.createOrReplaceTempView("software"); + String semrellist = getConstraintList(" relClass = '", allowedsemrel ); - org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); - org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); - - createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + String query = "Select source, community_context, target " + + "from (select id, collect_set(co.id) community_context " + + "from publication " + + "lateral view explode (context) c as co " + + "where datainfo.deletedbyinference = false "+ communitylist + + " group by id) p " + + "JOIN " + + "(select * " + + "from relation " + + "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " + + "ON p.id = r.source"; - updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + org.apache.spark.sql.Dataset publication_context = spark.sql( query); + publication_context.createOrReplaceTempView("publication_context"); - updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + //( source, (mes, dh-ch-, ni), target ) + query = "select target , collect_set(co) " + + "from (select target, community_context " + + "from publication_context pc join publication p on " + + "p.id = pc.source) tmp " + + "lateral view explode (community_context) c as co " + + "group by target"; - updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); + + System.out.println(toupdatepublicationreresult.count()); + + toupdatepublicationreresult.toJavaRDD() + .map(r -> { + TypedRow tp = new TypedRow(); + tp.setSourceId(r.getString(0)); + r.getList(1).stream().forEach(c -> tp.add((String)c)); + return tp; + }) + .map(tr -> new ObjectMapper().writeValueAsString(tr)) + .saveAsTextFile(outputPath + "/community2semrelonpublication"); +// toupdatepublicationreresult.toJavaRDD().flatMap(c -> { +// +// String source = c.getString(0); +// List relation_list = new ArrayList<>(); +// c.getList(1).stream() +// .forEach(res -> { +// Relation r = new Relation(); +// r.setSource(source); +// r.setTarget((String)res); +// r.setRelClass("produces"); +// relation_list.add(r); +// r = new Relation(); +// r.setSource((String)res); +// r.setTarget(source); +// r.setRelClass("isProducedBy"); +// relation_list.add(r); +// }); +// return relation_list.iterator(); +// }).map(tr -> new ObjectMapper().writeValueAsString(tr)) +// .saveAsTextFile(outputPath + "/community2semrel"); +// + +// org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); +// org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); +// org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); +// org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + +// createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + +// createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + +// createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// +// updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// +// updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", +// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// /* JavaPairRDD resultLinkedToCommunities = publication diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java index c55c0e8ea..0e39090dd 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java @@ -1,6 +1,6 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; -import com.cloudera.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -21,11 +21,11 @@ import java.util.stream.Collectors; import static eu.dnetlib.dhp.PropagationConstant.*; -public class SparkResultToCommunityThroughSemRelJob2 { +public class SparkResultToCommunityThroughSemRelJob3 { public static void main(String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob2.class + .toString(SparkResultToCommunityThroughSemRelJob3.class .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); parser.parseArgument(args); @@ -33,7 +33,7 @@ public class SparkResultToCommunityThroughSemRelJob2 { conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); final SparkSession spark = SparkSession .builder() - .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) + .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) .master(parser.get("master")) .config(conf) .enableHiveSupport() @@ -44,9 +44,8 @@ public class SparkResultToCommunityThroughSemRelJob2 { final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - //final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - //final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); @@ -54,21 +53,18 @@ public class SparkResultToCommunityThroughSemRelJob2 { JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") .map(item -> new ObjectMapper().readValue(item, Publication.class)); -// JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") -// .map(item -> new ObjectMapper().readValue(item, Dataset.class)); -// -// JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") -// .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); -// -// JavaRDD software_rdd = sc.textFile(inputPath + "/software") -// .map(item -> new ObjectMapper().readValue(item, Software.class)); + JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + + JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + + JavaRDD software_rdd = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") .map(item -> new ObjectMapper().readValue(item, Relation.class)); -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); - org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); @@ -76,24 +72,21 @@ public class SparkResultToCommunityThroughSemRelJob2 { org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); -// org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), -// Encoders.bean(Dataset.class)); -// -// org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), -// Encoders.bean(OtherResearchProduct.class)); -// -// org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), -// Encoders.bean(Software.class)); -// -// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), -// Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), + Encoders.bean(Software.class)); + publication.createOrReplaceTempView("publication"); relation.createOrReplaceTempView("relation"); -// relation.createOrReplaceTempView("relation"); -// dataset.createOrReplaceTempView("dataset"); -// software.createOrReplaceTempView("software"); -// other.createOrReplaceTempView("other"); + dataset.createOrReplaceTempView("dataset"); + software.createOrReplaceTempView("software"); + other.createOrReplaceTempView("other"); String communitylist = getConstraintList(" co.id = '", communityIdList); @@ -109,7 +102,7 @@ public class SparkResultToCommunityThroughSemRelJob2 { "JOIN " + "(select * " + "from relation " + - "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " + + "where datainfo.deletedbyinference = false " + semrellist + ") r " + "ON p.id = r.source"; @@ -127,37 +120,33 @@ public class SparkResultToCommunityThroughSemRelJob2 { org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); + org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); -// org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); -// org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); -// org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); - -// createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// -// updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); // /* diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json new file mode 100644 index 000000000..e03b3c6a4 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -0,0 +1,33 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"ocm", + "paramLongName":"organizationtoresultcommunitymap", + "paramDescription": "the map for the association organization communities", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true + } + +] \ No newline at end of file From 890ec28f0ff83ae1d8fc31990ba1757d2492ee70 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:09:37 +0200 Subject: [PATCH 135/259] input parameters for preparation step1 --- ...nput_preparecommunitytoresult_parameters.json | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json index e03b3c6a4..19ef290e4 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -18,16 +18,16 @@ "paramRequired": true }, { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", - "paramRequired": true + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false }, { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true + "paramName":"test", + "paramLongName":"isTest", + "paramDescription": "true if it is executing a test", + "paramRequired": false } ] \ No newline at end of file From 98548528785cefbd29dc99121c34a118ea26d1cb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:13:16 +0200 Subject: [PATCH 136/259] refactoring --- .../PrepareResultOrcidAssociationStep2.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index 4ed911c42..e38ba6583 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -20,7 +20,7 @@ public class PrepareResultOrcidAssociationStep2 { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class + String jsonConfiguration = IOUtils.toString(PrepareResultOrcidAssociationStep2.class .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( From 9a8876ac8649491872b1e6fa1ef6b7e5a702d1c4 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:14:08 +0200 Subject: [PATCH 137/259] added needed parameter --- .../SparkOrcidToResultFromSemRelJob3.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index 62f18ec2c..b8fef7ef6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -2,16 +2,21 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Author; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; -import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.List; +import java.util.Optional; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; @@ -46,9 +51,6 @@ public class SparkOrcidToResultFromSemRelJob3 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); - final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) From 437ebbad76fcdbeef651014554da367ea93960f5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:15:09 +0200 Subject: [PATCH 138/259] refactorign --- .../PrepareResultCommunitySet.java | 8 +------- ...SparkResultToCommunityFromOrganizationJob2.java | 14 -------------- 2 files changed, 1 insertion(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index a8a5451d9..bd2d106fe 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -2,16 +2,10 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; -import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3; import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,7 +23,7 @@ public class PrepareResultCommunitySet { public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySet.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser( jsonConfiguration); diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index ff94edbf9..c19c24d28 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -1,26 +1,15 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.ResultOrcidList; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3; import eu.dnetlib.dhp.schema.oaf.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; - import java.util.*; import java.util.stream.Collectors; @@ -58,9 +47,6 @@ public class SparkResultToCommunityFromOrganizationJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); - final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) From e2dfe8b656521bc3fc1186b0d18cf83d9e72f777 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:16:07 +0200 Subject: [PATCH 139/259] removed not used action --- .../countrypropagation/oozie_app/workflow.xml | 31 ------------------- 1 file changed, 31 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 79789e095..0fd60e989 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -196,37 +196,6 @@ - - - - yarn - cluster - countryPropagationForSoftware - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 - dhp-propagation-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - -mt yarn-cluster - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} - --saveGraph${saveGraph} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/country_propagation/publication - --preparedInfoPath${workingDir}/country_propagation/preparedInfo - - - - \ No newline at end of file From 0ff9b1ef054daef31c4477a7081b4b97d0f416c0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:16:29 +0200 Subject: [PATCH 140/259] added needed parameter --- .../orcidtoresultfromsemrel/input_orcidtoresult_parameters.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json index 50869aa0e..d8aa7eb9a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json @@ -9,7 +9,7 @@ "paramName":"sg", "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true + "paramRequired": false }, { "paramName":"h", From e1848b7603526fd9a1dfec310e566615ddbe7ea8 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Sat, 18 Apr 2020 14:16:42 +0200 Subject: [PATCH 141/259] minor --- .../oozie_app/workflow.xml | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index d20b839ee..26e4aa1be 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -174,8 +174,8 @@ --sourcePath${workingDir}/preparedInfo/targetOrcidAssoc --outputPath${workingDir}/preparedInfo/mergedOrcidAssoc - - + + @@ -204,7 +204,8 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${workingDir}/publication @@ -230,7 +231,8 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${workingDir}/dataset @@ -256,7 +258,8 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${workingDir}/otherresearchproduct @@ -282,7 +285,8 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${workingDir}/preparedInfo/mergedOrcidAssoc + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${workingDir}/software From 90c768dde6f3dac01a63ed6970c48bd56c20187e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 21 Apr 2020 16:03:51 +0200 Subject: [PATCH 142/259] added shaded libs module --- dhp-build/dhp-shaded-libs/pom.xml | 52 ++ .../java/eu/dnetlib/dhp/SparkBulkTagJob.java | 59 +- .../java/eu/dnetlib/dhp/SparkBulkTagJob2.java | 161 ++++ .../eu/dnetlib/dhp/community/Community.java | 18 +- .../dhp/community/CommunityConfiguration.java | 135 ++-- .../eu/dnetlib/dhp/community/Datasource.java | 28 +- .../java/eu/dnetlib/dhp/community/Pair.java | 11 +- .../eu/dnetlib/dhp/community/ProtoMap.java | 5 +- .../dnetlib/dhp/community/ResultTagger.java | 184 +++-- .../dhp/community/TagginConstants.java | 30 +- .../dhp/community/ZenodoCommunity.java | 22 +- .../dhp/selectioncriteria/VerbClass.java | 6 +- .../dnetlib/dhp/input_bulkTag_parameters.json | 36 +- .../dnetlib/dhp/oozie_app/config-default.xml | 24 + .../eu/dnetlib/dhp/oozie_app/workflow.xml | 181 ++++- .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 233 ++++++ .../CommunityConfigurationFactoryTest.java | 147 ++++ .../community_configuration.json | 694 ++++++++++++++++++ .../community_configuration.xml | 176 +++++ .../community_configuration_selcrit.json | 37 + .../community_configuration_selcrit.xml | 193 +++++ .../communityconfiguration/tagging_conf.json | 20 + .../dhp/sample/dataset/dataset_10.json.gz | Bin 0 -> 6736 bytes .../otherresearchproduct_10.json.gz | Bin 0 -> 8073 bytes .../sample/publication/publication_10.json.gz | Bin 0 -> 5143 bytes .../dhp/sample/software/software_10.json.gz | Bin 0 -> 6410 bytes ...arkResultToCommunityThroughSemRelJob4.java | 473 ++++++++++++ .../input_communitytoresult_parameters.json | 40 + ..._preparecommunitytoresult2_parameters.json | 50 ++ ..._preparecommunitytoresult_parameters.json} | 12 + .../ResultToCommunityJobTest.java | 4 + .../ResultToCommunityJobTest.java | 4 + 32 files changed, 2764 insertions(+), 271 deletions(-) create mode 100644 dhp-build/dhp-shaded-libs/pom.xml create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java create mode 100644 dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java create mode 100644 dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/software/software_10.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json rename dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/{input_propagationresultcommunityfromsemrel_parameters.json => input_preparecommunitytoresult_parameters.json} (67%) create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java create mode 100644 dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java diff --git a/dhp-build/dhp-shaded-libs/pom.xml b/dhp-build/dhp-shaded-libs/pom.xml new file mode 100644 index 000000000..c091c46f0 --- /dev/null +++ b/dhp-build/dhp-shaded-libs/pom.xml @@ -0,0 +1,52 @@ + + + + dhp-build + eu.dnetlib.dhp + 1.1.7-SNAPSHOT + + 4.0.0 + + dhp-shaded-libs + + + com.google.guava + guava + 23.3-jre + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.3 + + + package + + shade + + + + + com.google.guava + shaded.com.google.guava + + + + + *:* + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java index 13540cefa..140e28f3d 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java @@ -1,7 +1,5 @@ package eu.dnetlib.dhp; -import java.io.File; - import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; @@ -10,32 +8,36 @@ import eu.dnetlib.dhp.community.ProtoMap; import eu.dnetlib.dhp.community.QueryInformationSystem; import eu.dnetlib.dhp.community.ResultTagger; import eu.dnetlib.dhp.schema.oaf.*; +import java.io.File; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.Text; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; - - public class SparkBulkTagJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils.toString(SparkBulkTagJob.class.getResourceAsStream("/eu/dnetlib/dhp/input_bulktag_parameters.json"))); + final ArgumentApplicationParser parser = + new ArgumentApplicationParser( + IOUtils.toString( + SparkBulkTagJob.class.getResourceAsStream( + "/eu/dnetlib/dhp/input_bulktag_parameters.json"))); parser.parseArgument(args); - final SparkSession spark = SparkSession - .builder() - .appName(SparkBulkTagJob.class.getSimpleName()) - .master(parser.get("master")) - .enableHiveSupport() - .getOrCreate(); + final SparkSession spark = + SparkSession.builder() + .appName(SparkBulkTagJob.class.getSimpleName()) + .master(parser.get("master")) + .enableHiveSupport() + .getOrCreate(); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/bulktagging"; final ResultTagger resultTagger = new ResultTagger(); - ProtoMap protoMappingParams = new Gson().fromJson(parser.get("mappingProto"),ProtoMap.class);; + ProtoMap protoMappingParams = + new Gson().fromJson(parser.get("mappingProto"), ProtoMap.class); + ; File directory = new File(outputPath); @@ -43,31 +45,28 @@ public class SparkBulkTagJob { directory.mkdirs(); } - CommunityConfiguration cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); + CommunityConfiguration cc = + QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); - - sc.sequenceFile(inputPath + "/publication", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class)) + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/publication"); - sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class)) + .saveAsTextFile(outputPath + "/publication"); + sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)) .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/dataset"); - sc.sequenceFile(inputPath + "/software", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class)) + .saveAsTextFile(outputPath + "/dataset"); + sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/software"); - sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class) - .map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class)) + .saveAsTextFile(outputPath + "/software"); + sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/otherresearchproduct"); - - - + .saveAsTextFile(outputPath + "/otherresearchproduct"); } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java new file mode 100644 index 000000000..04897dd4a --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java @@ -0,0 +1,161 @@ +package eu.dnetlib.dhp; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.community.*; +import eu.dnetlib.dhp.schema.oaf.*; +import java.util.Optional; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SparkBulkTagJob2 { + + private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob2.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = + IOUtils.toString( + SparkBulkTagJob2.class.getResourceAsStream( + "/eu/dnetlib/dhp/input_bulktag_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = + Optional.ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + Boolean isTest = + Optional.ofNullable(parser.get("isTest")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); + log.info("isTest: {} ", isTest); + + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + ProtoMap protoMappingParams = new Gson().fromJson(parser.get("protoMap"), ProtoMap.class); + ; + log.info("protoMap: {}", new Gson().toJson(protoMappingParams)); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = + (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + CommunityConfiguration cc; + + String taggingConf = parser.get("taggingConf"); + + if (isTest) { + cc = CommunityConfigurationFactory.fromJson(taggingConf); + } else { + cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); + } + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + execBulkTag(spark, inputPath, outputPath, protoMappingParams, resultClazz, cc); + }); + + // runWithSparkSession(conf, isSparkSessionManaged, + // spark -> { + // if(isTest(parser)) { + // removeOutputDir(spark, outputPath); + // } + // if(saveGraph) + // execPropagation(spark, possibleUpdates, inputPath, outputPath, + // resultClazz); + // }); + // + // + // + // + // + // + // sc.textFile(inputPath + "/publication") + // .map(item -> new ObjectMapper().readValue(item, Publication.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/publication"); + // sc.textFile(inputPath + "/dataset") + // .map(item -> new ObjectMapper().readValue(item, Dataset.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/dataset"); + // sc.textFile(inputPath + "/software") + // .map(item -> new ObjectMapper().readValue(item, Software.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/software"); + // sc.textFile(inputPath + "/otherresearchproduct") + // .map(item -> new ObjectMapper().readValue(item, + // OtherResearchProduct.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/otherresearchproduct"); + // + + } + + private static void execBulkTag( + SparkSession spark, + String inputPath, + String outputPath, + ProtoMap protoMappingParams, + Class resultClazz, + CommunityConfiguration communityConfiguration) { + + ResultTagger resultTagger = new ResultTagger(); + Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.map( + value -> + resultTagger.enrichContextCriteria( + value, communityConfiguration, protoMappingParams), + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } + + private static org.apache.spark.sql.Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class resultClazz) { + + return spark.read() + .textFile(inputEntityPath) + .map( + (MapFunction) + value -> OBJECT_MAPPER.readValue(value, resultClazz), + Encoders.bean(resultClazz)); + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java index 47e4b7a12..b0c213c12 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java @@ -1,16 +1,14 @@ package eu.dnetlib.dhp.community; import com.google.gson.Gson; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import java.util.ArrayList; -import java.util.List; - -/** - * Created by miriam on 01/08/2018. - */ -public class Community { +/** Created by miriam on 01/08/2018. */ +public class Community implements Serializable { private static final Log log = LogFactory.getLog(Community.class); @@ -19,14 +17,15 @@ public class Community { private List datasources = new ArrayList<>(); private List zenodoCommunities = new ArrayList<>(); - public String toJson() { final Gson g = new Gson(); return g.toJson(this); } public boolean isValid() { - return !getSubjects().isEmpty() || !getDatasources().isEmpty() || !getZenodoCommunities().isEmpty(); + return !getSubjects().isEmpty() + || !getDatasources().isEmpty() + || !getZenodoCommunities().isEmpty(); } public String getId() { @@ -60,5 +59,4 @@ public class Community { public void setZenodoCommunities(List zenodoCommunities) { this.zenodoCommunities = zenodoCommunities; } - } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java index aa1a9bb8c..1fd5bedd4 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java @@ -3,38 +3,58 @@ package eu.dnetlib.dhp.community; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.gson.Gson; - import com.google.gson.GsonBuilder; - import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; import eu.dnetlib.dhp.selectioncriteria.Selection; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - - +import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; -/** - * Created by miriam on 02/08/2018. - */ -public class CommunityConfiguration { +/** Created by miriam on 02/08/2018. */ +public class CommunityConfiguration implements Serializable { private static final Log log = LogFactory.getLog(CommunityConfiguration.class); + private Map communities; - private Map communities; + // map subject -> communityid + private Map>> subjectMap = new HashMap<>(); + // map datasourceid -> communityid + private Map>> datasourceMap = new HashMap<>(); + // map zenodocommunityid -> communityid + private Map>> zenodocommunityMap = + new HashMap<>(); + public Map>> getSubjectMap() { + return subjectMap; + } - //map subject -> communityid - private transient Map>> subjectMap = new HashMap<>(); - //map datasourceid -> communityid - private transient Map>> datasourceMap = new HashMap<>(); - //map zenodocommunityid -> communityid - private transient Map>> zenodocommunityMap = new HashMap<>(); + public void setSubjectMap(Map>> subjectMap) { + this.subjectMap = subjectMap; + } + + public Map>> getDatasourceMap() { + return datasourceMap; + } + + public void setDatasourceMap( + Map>> datasourceMap) { + this.datasourceMap = datasourceMap; + } + + public Map>> getZenodocommunityMap() { + return zenodocommunityMap; + } + + public void setZenodocommunityMap( + Map>> zenodocommunityMap) { + this.zenodocommunityMap = zenodocommunityMap; + } CommunityConfiguration(final Map communities) { this.communities = communities; @@ -53,65 +73,67 @@ public class CommunityConfiguration { zenodocommunityMap = Maps.newHashMap(); } - - for(Community c : getCommunities().values()) { - //get subjects + for (Community c : getCommunities().values()) { + // get subjects final String id = c.getId(); - for(String sbj : c.getSubjects()){ - Pair p = new Pair<>(id,new SelectionConstraints()); - add(sbj.toLowerCase().trim() , p, subjectMap); + for (String sbj : c.getSubjects()) { + Pair p = new Pair<>(id, new SelectionConstraints()); + add(sbj.toLowerCase().trim(), p, subjectMap); } - //get datasources - for(Datasource d: c.getDatasources()){ + // get datasources + for (Datasource d : c.getDatasources()) { - add(d.getOpenaireId(),new Pair<>(id,d.getSelectionConstraints()),datasourceMap); + add(d.getOpenaireId(), new Pair<>(id, d.getSelectionConstraints()), datasourceMap); } - //get zenodo communities - for(ZenodoCommunity zc : c.getZenodoCommunities()){ - add(zc.getZenodoCommunityId(),new Pair<>(id,zc.getSelCriteria()),zenodocommunityMap); + // get zenodo communities + for (ZenodoCommunity zc : c.getZenodoCommunities()) { + add( + zc.getZenodoCommunityId(), + new Pair<>(id, zc.getSelCriteria()), + zenodocommunityMap); } - - } } - private void add(String key, Pair value, Map>> map){ - List> values = map.get(key); + private void add( + String key, + Pair value, + Map>> map) { + List> values = map.get(key); - if (values == null){ + if (values == null) { values = new ArrayList<>(); - map.put(key,values); + map.put(key, values); } values.add(value); } - public List> getCommunityForSubject(String sbj){ + public List> getCommunityForSubject(String sbj) { return subjectMap.get(sbj); } - public List> getCommunityForDatasource(String dts){ + public List> getCommunityForDatasource(String dts) { return datasourceMap.get(dts); } + public List getCommunityForDatasource( + final String dts, final Map> param) { + List> lp = datasourceMap.get(dts); + if (lp == null) return Lists.newArrayList(); - public List getCommunityForDatasource(final String dts, final Map> param) { - List> lp = datasourceMap.get(dts); - if (lp==null) - return Lists.newArrayList(); - - return lp.stream().map(p -> { - if (p.getSnd() == null) - return p.getFst(); - if (((SelectionConstraints) p.getSnd()).verifyCriteria(param)) - return p.getFst(); - else - return null; - }).filter(st->(st!=null)).collect(Collectors.toList()); - - + return lp.stream() + .map( + p -> { + if (p.getSnd() == null) return p.getFst(); + if (((SelectionConstraints) p.getSnd()).verifyCriteria(param)) + return p.getFst(); + else return null; + }) + .filter(st -> (st != null)) + .collect(Collectors.toList()); } - public List> getCommunityForZenodoCommunity(String zc){ + public List> getCommunityForZenodoCommunity(String zc) { return zenodocommunityMap.get(zc); } @@ -125,7 +147,7 @@ public class CommunityConfiguration { return getContextIds(datasourceMap.get(value.toLowerCase())); } - public List getCommunityForZenodoCommunityValue(String value){ + public List getCommunityForZenodoCommunityValue(String value) { return getContextIds(zenodocommunityMap.get(value.toLowerCase())); } @@ -137,7 +159,6 @@ public class CommunityConfiguration { return Lists.newArrayList(); } - public Map getCommunities() { return communities; } @@ -158,11 +179,11 @@ public class CommunityConfiguration { return communities.keySet().size(); } - public Community getCommunityById(String id){ + public Community getCommunityById(String id) { return communities.get(id); } public List getCommunityList() { return Lists.newLinkedList(communities.values()); } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java index c0047a07e..5acba31d6 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java @@ -1,24 +1,20 @@ package eu.dnetlib.dhp.community; - import com.google.gson.Gson; - import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import java.io.Serializable; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.dom4j.Node; -/** - * Created by miriam on 01/08/2018. - */ -public class Datasource { +/** Created by miriam on 01/08/2018. */ +public class Datasource implements Serializable { private static final Log log = LogFactory.getLog(Datasource.class); private String openaireId; private SelectionConstraints selectionConstraints; - public SelectionConstraints getSelCriteria() { return selectionConstraints; } @@ -43,23 +39,19 @@ public class Datasource { this.openaireId = openaireId; } - private void setSelCriteria(String json, VerbResolver resolver){ + private void setSelCriteria(String json, VerbResolver resolver) { log.info("Selection constraints for datasource = " + json); selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class); selectionConstraints.setSelection(resolver); } - public void setSelCriteria(Node n, VerbResolver resolver){ - try{ - setSelCriteria(n.getText(),resolver); - }catch(Exception e) { + public void setSelCriteria(Node n, VerbResolver resolver) { + try { + setSelCriteria(n.getText(), resolver); + } catch (Exception e) { log.info("not set selection criteria... "); - selectionConstraints =null; + selectionConstraints = null; } - } - - - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java index 1cff8bfef..78ffe860d 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java @@ -1,11 +1,10 @@ package eu.dnetlib.dhp.community; import com.google.gson.Gson; +import java.io.Serializable; -/** - * Created by miriam on 03/08/2018. - */ -public class Pair { +/** Created by miriam on 03/08/2018. */ +public class Pair implements Serializable { private A fst; private B snd; @@ -27,12 +26,12 @@ public class Pair { return this; } - public Pair(A a, B b){ + public Pair(A a, B b) { fst = a; snd = b; } - public String toJson(){ + public String toJson() { return new Gson().toJson(this); } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java index 6bbe50c86..773955d4a 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java @@ -1,10 +1,11 @@ package eu.dnetlib.dhp.community; +import java.io.Serializable; import java.util.HashMap; -public class ProtoMap extends HashMap { +public class ProtoMap extends HashMap implements Serializable { - public ProtoMap(){ + public ProtoMap() { super(); } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java index b9962050c..abedab476 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java @@ -1,67 +1,68 @@ package eu.dnetlib.dhp.community; + +import static eu.dnetlib.dhp.community.TagginConstants.*; + import com.google.gson.Gson; import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.JsonPath; import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.lang3.StringUtils; - +import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.commons.lang3.StringUtils; -import static eu.dnetlib.dhp.community.TagginConstants.*; - - -/** - * Created by miriam on 02/08/2018. - */ -public class ResultTagger { - +/** Created by miriam on 02/08/2018. */ +public class ResultTagger implements Serializable { private String trust = "0.8"; - - private boolean clearContext(Result result){ + private boolean clearContext(Result result) { int tmp = result.getContext().size(); - List clist = result.getContext().stream() - .filter(c -> (!c.getId().contains(ZENODO_COMMUNITY_INDICATOR))).collect(Collectors.toList()); + List clist = + result.getContext().stream() + .filter(c -> (!c.getId().contains(ZENODO_COMMUNITY_INDICATOR))) + .collect(Collectors.toList()); result.setContext(clist); return (tmp != clist.size()); } - private Map> getParamMap(final Result result, Map params) { - Map> param = new HashMap<>(); - String json = new Gson().toJson(result,Result.class); + private Map> getParamMap(final Result result, Map params) { + Map> param = new HashMap<>(); + String json = new Gson().toJson(result, Result.class); DocumentContext jsonContext = JsonPath.parse(json); - if (params == null){ + if (params == null) { params = new HashMap<>(); } - for(String key : params.keySet()) { + for (String key : params.keySet()) { try { param.put(key, jsonContext.read(params.get(key))); } catch (com.jayway.jsonpath.PathNotFoundException e) { param.put(key, new ArrayList<>()); - // throw e; + // throw e; } } return param; - } + public R enrichContextCriteria( + final R result, final CommunityConfiguration conf, final Map criteria) { - public Result enrichContextCriteria(final Result result, final CommunityConfiguration conf, final Map criteria) { + // } + // public Result enrichContextCriteria(final Result result, final CommunityConfiguration + // conf, final Map criteria) { final Map> param = getParamMap(result, criteria); - //Verify if the entity is deletedbyinference. In case verify if to clean the context list from all the zenodo communities - if(result.getDataInfo().getDeletedbyinference()){ - return result; + // Verify if the entity is deletedbyinference. In case verify if to clean the context list + // from all the zenodo communities + if (result.getDataInfo().getDeletedbyinference()) { + if (clearContext(result)) return result; } - //communities contains all the communities to be added as context for the result + // communities contains all the communities to be added as context for the result final Set communities = new HashSet<>(); - - //tagging for Subject + // tagging for Subject final Set subjects = new HashSet<>(); result.getSubject().stream() .map(subject -> subject.getValue()) @@ -73,89 +74,115 @@ public class ResultTagger { communities.addAll(subjects); - - //Tagging for datasource + // Tagging for datasource final Set datasources = new HashSet<>(); final Set tmp = new HashSet<>(); - for(Instance i : result.getInstance()){ - tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(),"|")); - tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(),"|")); + for (Instance i : result.getInstance()) { + tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); + tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); } - result.getInstance() - .stream() + result.getInstance().stream() .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) .map(s -> StringUtils.substringAfter(s, "|")) .collect(Collectors.toCollection(HashSet::new)) - .forEach(dsId -> datasources.addAll(conf.getCommunityForDatasource(dsId,param))); + .forEach(dsId -> datasources.addAll(conf.getCommunityForDatasource(dsId, param))); communities.addAll(datasources); /*Tagging for Zenodo Communities*/ final Set czenodo = new HashSet<>(); - //final ResultProtos.Result.Metadata.Builder mBuilder = builder.getEntityBuilder().getResultBuilder().getMetadataBuilder(); - result.getContext() - .stream() + result.getContext().stream() .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) .collect(Collectors.toList()) - .forEach(c->czenodo.addAll(conf.getCommunityForZenodoCommunityValue(c.getId().substring(c.getId().lastIndexOf("/")+1).trim()))); + .forEach( + c -> + czenodo.addAll( + conf.getCommunityForZenodoCommunityValue( + c.getId() + .substring(c.getId().lastIndexOf("/") + 1) + .trim()))); communities.addAll(czenodo); clearContext(result); /*Verify if there is something to bulktag*/ - if(communities.isEmpty()){ + if (communities.isEmpty()) { return result; - } - result.getContext() - .stream() - .map(c -> { - if(communities.contains(c.getId())){ - List dataInfoList = c.getDataInfo(); - if (subjects.contains(c.getId())) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_SUBJECT, CLASS_NAME_BULKTAG_SUBJECT)); - if (datasources.contains(c.getId())) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_DATASOURCE, CLASS_NAME_BULKTAG_DATASOURCE)); - if (czenodo.contains(c.getId())) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_CZENODO, CLASS_NAME_BULKTAG_ZENODO)); - } - return c; - }) + result.getContext().stream() + .map( + c -> { + if (communities.contains(c.getId())) { + List dataInfoList = c.getDataInfo(); + if (subjects.contains(c.getId())) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_SUBJECT, + CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c.getId())) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_DATASOURCE, + CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c.getId())) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_CZENODO, + CLASS_NAME_BULKTAG_ZENODO)); + } + return c; + }) .collect(Collectors.toList()); + communities.removeAll( + result.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet())); - communities.removeAll(result.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet())); + if (communities.isEmpty()) return result; - if(communities.isEmpty()) - return result; - - List toaddcontext = communities - .stream() - .map(c -> { - Context context = new Context(); - context.setId(c); - List dataInfoList = Arrays.asList(); - if (subjects.contains(c)) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_SUBJECT, CLASS_NAME_BULKTAG_SUBJECT)); - if (datasources.contains(c)) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_DATASOURCE, CLASS_NAME_BULKTAG_DATASOURCE)); - if (czenodo.contains(c)) - dataInfoList.add(getDataInfo(BULKTAG_DATA_INFO_TYPE, CLASS_ID_CZENODO, CLASS_NAME_BULKTAG_ZENODO)); - context.setDataInfo(dataInfoList); - return context; - }) - .collect(Collectors.toList()); + List toaddcontext = + communities.stream() + .map( + c -> { + Context context = new Context(); + context.setId(c); + List dataInfoList = Arrays.asList(); + if (subjects.contains(c)) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_SUBJECT, + CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c)) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_DATASOURCE, + CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c)) + dataInfoList.add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_CZENODO, + CLASS_NAME_BULKTAG_ZENODO)); + context.setDataInfo(dataInfoList); + return context; + }) + .collect(Collectors.toList()); result.getContext().addAll(toaddcontext); return result; } - public static DataInfo getDataInfo(String inference_provenance, String inference_class_id, String inference_class_name){ + public static DataInfo getDataInfo( + String inference_provenance, String inference_class_id, String inference_class_name) { DataInfo di = new DataInfo(); di.setInferred(true); di.setInferenceprovenance(inference_provenance); @@ -171,5 +198,4 @@ public class ResultTagger { pa.setSchemename(DNET_SCHEMA_NAME); return pa; } - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java index 494d955c8..9f681472a 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java @@ -1,27 +1,23 @@ package eu.dnetlib.dhp.community; - public class TagginConstants { + public static final String BULKTAG_DATA_INFO_TYPE = "bulktagging"; - public final static String BULKTAG_DATA_INFO_TYPE = "bulktagging"; + public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; - public final static String DNET_SCHEMA_NAME = "dnet:provenanceActions"; - public final static String DNET_SCHEMA_ID = "dnet:provenanceActions"; - - public final static String CLASS_ID_SUBJECT = "bulktagging:community:subject"; - public final static String CLASS_ID_DATASOURCE = "bulktagging:community:datasource"; - public final static String CLASS_ID_CZENODO = "bulktagging:community:zenodocommunity"; - - public final static String SCHEMA_ID = "dnet:provenanceActions"; - public final static String COUNTER_GROUP = "Bulk Tagging"; - - public final static String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; - - public final static String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; - public final static String CLASS_NAME_BULKTAG_DATASOURCE = "Bulktagging for Community - Datasource"; - public final static String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; + public static final String CLASS_ID_SUBJECT = "community:subject"; + public static final String CLASS_ID_DATASOURCE = "community:datasource"; + public static final String CLASS_ID_CZENODO = "community:zenodocommunity"; + public static final String SCHEMA_ID = "dnet:provenanceActions"; + public static final String COUNTER_GROUP = "Bulk Tagging"; + public static final String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; + public static final String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; + public static final String CLASS_NAME_BULKTAG_DATASOURCE = + "Bulktagging for Community - Datasource"; + public static final String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java index d785a3d94..19d97d221 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java @@ -1,13 +1,11 @@ package eu.dnetlib.dhp.community; import com.google.gson.Gson; +import java.io.Serializable; import org.dom4j.Node; - -/** - * Created by miriam on 01/08/2018. - */ -public class ZenodoCommunity { +/** Created by miriam on 01/08/2018. */ +public class ZenodoCommunity implements Serializable { private String zenodoCommunityId; @@ -29,18 +27,16 @@ public class ZenodoCommunity { this.selCriteria = selCriteria; } - private void setSelCriteria(String json){ - //Type collectionType = new TypeToken>(){}.getType(); + private void setSelCriteria(String json) { + // Type collectionType = new TypeToken>(){}.getType(); selCriteria = new Gson().fromJson(json, SelectionConstraints.class); - } - public void setSelCriteria(Node n){ - if (n==null){ + public void setSelCriteria(Node n) { + if (n == null) { selCriteria = null; - }else{ + } else { setSelCriteria(n.getText()); } } - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java index 9f519f091..9a5fe4e8a 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java @@ -7,7 +7,7 @@ import java.lang.annotation.Target; @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) -public @interface VerbClass { +@interface VerbClass { - public String value(); -} \ No newline at end of file + String value(); +} diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json index 3221924bf..b370467ae 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json @@ -5,12 +5,6 @@ "paramDescription": "URL of the isLookUp Service", "paramRequired": true }, - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -22,6 +16,36 @@ "paramLongName":"protoMap", "paramDescription": "the json path associated to each selection field", "paramRequired": true + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "test", + "paramLongName": "isTest", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "tg", + "paramLongName": "taggingConf", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml index ea3a4d922..73bfe9ae7 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml @@ -19,4 +19,28 @@ hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + sparkExecutorNumber + 1 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml index 1866bb0a0..b4f918bbd 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml @@ -1,13 +1,9 @@ - + sourcePath the source path - - allowedsemrels - the semantic relationships allowed for propagation - sparkDriverMemory memory for driver process @@ -24,38 +20,163 @@ isLookupUrl the isLookup service endpoint + + protoMap + the json path associated to each selection field + - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - ResultToCommunitySemRelPropagation - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${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" - - -mt yarn-cluster - --sourcePath${sourcePath} - - --hive_metastore_uris${hive_metastore_uris} - - - - - + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/relation + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging + eu.dnetlib.dhp.SparkBulkTagJob + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/publication + + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/publication + --proto_map${protoMap} + --isLookupUrl${isLookupUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging + eu.dnetlib.dhp.SparkBulkTagJob + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/dataset + + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/dataset + --proto_map${protoMap} + --isLookupUrl${isLookupUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging + eu.dnetlib.dhp.SparkBulkTagJob + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/otherresearchproduct + + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/otherresearchproduct + --proto_map${protoMap} + --isLookupUrl${isLookupUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging + eu.dnetlib.dhp.SparkBulkTagJob + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/software + + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/software + --proto_map${protoMap} + --isLookupUrl${isLookupUrl} + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java new file mode 100644 index 000000000..4a45d234c --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -0,0 +1,233 @@ +package eu.dnetlib.dhp; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mortbay.util.IO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BulkTagJobTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = eu.dnetlib.dhp.BulkTagJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.BulkTagJobTest.class); + + private static String taggingConf = ""; + + static { + try { + taggingConf = + IO.toString( + BulkTagJobTest.class.getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json")); + } catch (IOException e) { + e.printStackTrace(); + } + } + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = + SparkSession.builder() + .appName(BulkTagJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void test1() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/sample/dataset").getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + // "-preparedInfoPath", + // getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() + }); + } +} + +/* + + +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; + +import static org.apache.spark.sql.functions.desc; + + + + + + @Test + public void test1() throws Exception { + SparkResultToCommunityThroughSemRelJob4.main(new String[]{ + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample").getPath(), + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-preparedInfoPath", getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'propagation'"; + + org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); + Assertions.assertEquals(5, resultExplodedProvenance.count()); + + Assertions.assertEquals(0, resultExplodedProvenance.filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'").count()); + + Assertions.assertEquals(1, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'").count()); + Assertions.assertEquals("dh-ch",resultExplodedProvenance.select("community") + .where(resultExplodedProvenance.col("id").equalTo("50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .collectAsList().get(0).getString(0)); + + Assertions.assertEquals(3, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'").count()); + List rowList = resultExplodedProvenance.select("community") + .where(resultExplodedProvenance.col("id") + .equalTo("50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")).collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); + + + Assertions.assertEquals(1, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'").count()); + Assertions.assertEquals("aginfra", resultExplodedProvenance.select("community") + .where(resultExplodedProvenance.col("id") + .equalTo("50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .collectAsList().get(0).getString(0)); + + + query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; + + org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); + + Assertions.assertEquals(10, resultCommunityId.count()); + + Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'").count()); + rowList = resultCommunityId.select("community") + .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("dh-ch", rowList.get(0).getString(0)); + Assertions.assertEquals("beopen", rowList.get(1).getString(0)); + + Assertions.assertEquals(3, resultCommunityId.filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'").count()); + rowList = resultCommunityId.select("community") + .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); + + Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'").count()); + rowList = resultCommunityId.select("community") + .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("beopen", rowList.get(0).getString(0)); + Assertions.assertEquals("aginfra", rowList.get(1).getString(0)); + + Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'").count()); + rowList = resultCommunityId.select("community") + .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("euromarine", rowList.get(1).getString(0)); + Assertions.assertEquals("ni", rowList.get(0).getString(0)); + + Assertions.assertEquals(1, resultCommunityId.filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'").count()); + Assertions.assertEquals("euromarine", resultCommunityId.select("community") + .where(resultCommunityId.col("id") + .equalTo("50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .collectAsList().get(0).getString(0)); + + + } + */ diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java new file mode 100644 index 000000000..77c448265 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java @@ -0,0 +1,147 @@ +package eu.dnetlib.dhp; + +import com.google.gson.Gson; +import eu.dnetlib.dhp.community.CommunityConfiguration; +import eu.dnetlib.dhp.community.CommunityConfigurationFactory; +import eu.dnetlib.dhp.community.Constraint; +import eu.dnetlib.dhp.community.SelectionConstraints; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.util.*; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.dom4j.DocumentException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +/** Created by miriam on 03/08/2018. */ +public class CommunityConfigurationFactoryTest { + + private static String xml; + private static String xml1; + + private final VerbResolver resolver = new VerbResolver(); + + @Test + public void parseTest() throws DocumentException, IOException { + String xml = + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + Assertions.assertEquals(5, cc.size()); + cc.getCommunityList() + .forEach(c -> Assertions.assertTrue(StringUtils.isNoneBlank(c.getId()))); + } + + @Test + public void applyVerb() + throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, + InstantiationException { + Constraint sc = new Constraint(); + sc.setVerb("not_contains"); + sc.setField("contributor"); + sc.setValue("DARIAH"); + sc.setSelection(resolver.getSelectionCriteria(sc.getVerb(), sc.getValue())); + String metadata = "This work has been partially supported by DARIAH-EU infrastructure"; + Assertions.assertFalse(sc.verifyCriteria(metadata)); + } + + @Test + public void loadSelCriteriaTest() throws DocumentException, IOException { + String xml = + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + Map> param = new HashMap<>(); + param.put("author", new ArrayList<>(Collections.singletonList("Pippo Pippi"))); + param.put( + "description", + new ArrayList<>( + Collections.singletonList( + "This work has been partially supported by DARIAH-EU infrastructure"))); + param.put( + "contributor", + new ArrayList<>( + Collections.singletonList( + "Pallino ha aiutato a scrivere il paper. Pallino lavora per DARIAH"))); + List comm = + cc.getCommunityForDatasource( + "openaire____::1cfdb2e14977f31a98e0118283401f32", param); + Assertions.assertEquals(1, comm.size()); + Assertions.assertEquals("dariah", comm.get(0)); + } + + @Test + public void test4() throws DocumentException, IOException { + final CommunityConfiguration cc = + CommunityConfigurationFactory.fromJson( + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json"))); + cc.toString(); + } + + @Test + public void test5() throws IOException, DocumentException { + + // final CommunityConfiguration cc = + // CommunityConfigurationFactory.newInstance(IOUtils.toString(getClass().getResourceAsStream("test.xml"))); + final CommunityConfiguration cc = + CommunityConfigurationFactory.fromJson( + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration.json"))); + + System.out.println(cc.toJson()); + } + + @Test + public void test6() { + String json = + "{\"criteria\":[{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}]}"; + + String step1 = "{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}"; + + Constraint c = new Gson().fromJson(step1, Constraint.class); + // + // String step2 = + // "{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}"; + // + // ConstraintEncapsulator ce = new + // Gson().fromJson(step2,ConstraintEncapsulator.class); + // + // + // String step3 = + // "{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}"; + // + // Constraints cons = new Gson().fromJson(step3,Constraints.class); + // + // String step4 = + // "{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}"; + // + // ConstraintsList cl = new Gson().fromJson(step4,ConstraintsList.class); + // + // String step5 = + // "{\"cl\":{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}}"; + SelectionConstraints sl = new Gson().fromJson(json, SelectionConstraints.class); + } + + @Test + public void test7() throws IOException { + final CommunityConfiguration cc = + CommunityConfigurationFactory.fromJson( + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json"))); + + System.out.println(cc.toJson()); + } +} diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json new file mode 100644 index 000000000..d21dc4ced --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json @@ -0,0 +1,694 @@ +{"communities": { + "clarin": { + "id": "clarin", + "subjects": [], + "datasources": [ + { + "openaireId": "re3data_____::a507cdacc5bbcc08761c92185dee5cab" + } + ], + "zenodoCommunities": [ + + ] + }, + "ee": { + "id": "ee", + "subjects": [ + "SDG13 - Climate action", + "SDG8 - Decent work and economic\n\t\t\t\t\tgrowth", + "SDG15 - Life on land", + "SDG2 - Zero hunger", + "SDG17 - Partnerships for the\n\t\t\t\t\tgoals", + "SDG10 - Reduced inequalities", + "SDG5 - Gender equality", + "SDG12 - Responsible\n\t\t\t\t\tconsumption and production", + "SDG14 - Life below water", + "SDG6 - Clean water and\n\t\t\t\t\tsanitation", + "SDG11 - Sustainable cities and communities", + "SDG1 - No poverty", + "SDG3 -\n\t\t\t\t\tGood health and well being", + "SDG7 - Affordable and clean energy", + "SDG4 - Quality\n\t\t\t\t\teducation", + "SDG9 - Industry innovation and infrastructure", + "SDG16 - Peace justice\n\t\t\t\t\tand strong institutions" + ], + "datasources": [ + + ], + "zenodoCommunities": [ + + ] + }, + "aginfra": { + "id": "aginfra", + "subjects": [ + "animal production and health", + "fisheries and aquaculture", + "food safety and human nutrition", + "information management", + "food technology", + "agri-food education and extension", + "natural resources and environment", + "food system", + "engineering technology and Research", + "agriculture", + "food safety risk assessment", + "food security", + "farming practices and systems", + "plant production and protection", + "agri-food economics and policy", + "food distribution", + "forestry" + ], + "datasources": [ + { + "openaireId": "opendoar____::1a551829d50f1400b0dab21fdd969c04" + }, + { + "openaireId": "opendoar____::49af6c4e558a7569d80eee2e035e2bd7" + }, + { + "openaireId": "opendoar____::0266e33d3f546cb5436a10798e657d97" + }, + { + "openaireId": "opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06" + }, + { + "openaireId": "opendoar____::41bfd20a38bb1b0bec75acf0845530a7" + }, + { + "openaireId": "opendoar____::87ae6fb631f7c8a627e8e28785d9992d" + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "edenis" + }, + { + "zenodoCommunityId": "efsa-pilot" + }, + { + "zenodoCommunityId": "egene3" + }, + { + "zenodoCommunityId": "efsa-kj" + }, + { + "zenodoCommunityId": "euromixproject" + }, + { + "zenodoCommunityId": "discardless" + }, + { + "zenodoCommunityId": "sedinstcjfst" + }, + { + "zenodoCommunityId": "afinet-kc" + }, + { + "zenodoCommunityId": "2231-4784" + }, + { + "zenodoCommunityId": "2231-0606" + }, + { + "zenodoCommunityId": "solace" + }, + { + "zenodoCommunityId": "pa17" + }, + { + "zenodoCommunityId": "smartakis" + }, + { + "zenodoCommunityId": "sedinstcjae" + }, + { + "zenodoCommunityId": "phenology_camera" + }, + { + "zenodoCommunityId": "aginfra" + }, + { + "zenodoCommunityId": "erosa" + }, + { + "zenodoCommunityId": "bigdatagrapes" + } + ] + }, + "fam": { + "id": "fam", + "subjects": [ + "Stock Assessment", + "pelagic", + "Fish farming", + "EMFF", + "Fisheries", + "Fishermen", + "maximum sustainable yield", + "trawler", + "Fishing vessel", + "Fisherman", + "Fishing gear", + "RFMO", + "Fish Aggregating Device", + "Bycatch", + "Fishery", + "common fisheries policy", + "Fishing fleet", + "Aquaculture" + ], + "datasources": [ + { + "openaireId": "doajarticles::8cec81178926caaca531afbd8eb5d64c" + }, + { + "openaireId": "doajarticles::0f7a7f30b5400615cae1829f3e743982" + }, + { + "openaireId": "doajarticles::9740f7f5af3e506d2ad2c215cdccd51a" + }, + { + "openaireId": "doajarticles::9f3fbaae044fa33cb7069b72935a3254" + }, + { + "openaireId": "doajarticles::cb67f33eb9819f5c624ce0313957f6b3" + }, + { + "openaireId": "doajarticles::e21c97cbb7a209afc75703681c462906" + }, + { + "openaireId": "doajarticles::554cde3be9e5c4588b4c4f9f503120cb" + }, + { + "openaireId": "tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b" + }, + { + "openaireId": "doajarticles::57c5d3837da943e93b28ec4db82ec7a5" + }, + { + "openaireId": "doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1" + }, + { + "openaireId": "doajarticles::e21c97cbb7a209afc75703681c462906" + }, + { + "openaireId": "doajarticles::dca64612dfe0963fffc119098a319957" + }, + { + "openaireId": "doajarticles::dd70e44479f0ade25aa106aef3e87a0a" + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "discardless" + }, + { + "zenodoCommunityId": "farfish2020" + }, + { + "zenodoCommunityId": "facts" + }, + { + "zenodoCommunityId": "climefish" + }, + { + "zenodoCommunityId": "proeel" + }, + { + "zenodoCommunityId": "primefish" + }, + { + "zenodoCommunityId": "h2020_vicinaqua" + }, + { + "zenodoCommunityId": "meece" + }, + { + "zenodoCommunityId": "rlsadb" + } + ] + }, + "instruct": { + "id": "instruct", + "subjects": [ + + ], + "datasources": [ + + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "instruct" + }, + { + "zenodoCommunityId": "west-life" + } + ] + }, + "mes": { + "id": "mes", + "subjects": [ + "marine", + "ocean", + "fish", + "aqua", + "sea" + ], + "datasources": [ + + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "adriplan" + }, + { + "zenodoCommunityId": "devotes-project" + }, + { + "zenodoCommunityId": "euro-basin" + }, + { + "zenodoCommunityId": "naclim" + }, + { + "zenodoCommunityId": "discardless" + }, + { + "zenodoCommunityId": "assisibf" + }, + { + "zenodoCommunityId": "meece" + }, + { + "zenodoCommunityId": "facts" + }, + { + "zenodoCommunityId": "proeel" + }, + { + "zenodoCommunityId": "aquatrace" + }, + { + "zenodoCommunityId": "myfish" + }, + { + "zenodoCommunityId": "atlas" + }, + { + "zenodoCommunityId": "blue-actionh2020" + }, + { + "zenodoCommunityId": "sponges" + }, + { + "zenodoCommunityId": "merces_project" + }, + { + "zenodoCommunityId": "bigdataocean" + }, + { + "zenodoCommunityId": "columbus" + }, + { + "zenodoCommunityId": "h2020-aquainvad-ed" + }, + { + "zenodoCommunityId": "aquarius" + }, + { + "zenodoCommunityId": "southern-ocean-observing-system" + }, + { + "zenodoCommunityId": "eawag" + }, + { + "zenodoCommunityId": "mossco" + }, + { + "zenodoCommunityId": "onc" + }, + { + "zenodoCommunityId": "oceanbiogeochemistry" + }, + { + "zenodoCommunityId": "oceanliteracy" + }, + { + "zenodoCommunityId": "openearth" + }, + { + "zenodoCommunityId": "ocean" + }, + { + "zenodoCommunityId": "calcifierraman" + }, + { + "zenodoCommunityId": "bermudabream" + }, + { + "zenodoCommunityId": "brcorp1" + }, + { + "zenodoCommunityId": "mce" + }, + { + "zenodoCommunityId": "biogeochem" + }, + { + "zenodoCommunityId": "ecc2014" + }, + { + "zenodoCommunityId": "fisheries" + }, + { + "zenodoCommunityId": "sedinstcjfas" + }, + { + "zenodoCommunityId": "narmada" + }, + { + "zenodoCommunityId": "umr-entropie" + }, + { + "zenodoCommunityId": "farfish2020" + }, + { + "zenodoCommunityId": "primefish" + }, + { + "zenodoCommunityId": "zf-ilcs" + }, + { + "zenodoCommunityId": "climefish" + }, + { + "zenodoCommunityId": "afrimed_eu" + }, + { + "zenodoCommunityId": "spi-ace" + }, + { + "zenodoCommunityId": "cice-consortium" + }, + { + "zenodoCommunityId": "nemo-ocean" + }, + { + "zenodoCommunityId": "mesopp-h2020" + }, + { + "zenodoCommunityId": "marxiv" + } + ] + }, + "ni": { + "id": "ni", + "subjects": [ + "brain mapping", + "brain imaging", + "electroencephalography", + "arterial spin labelling", + "brain fingerprinting", + "brain", + "neuroimaging", + "Multimodal Brain Image Analysis", + "fMRI", + "neuroinformatics", + "fetal brain", + "brain ultrasonic imaging", + "topographic brain mapping", + "diffusion tensor imaging", + "computerized knowledge assessment", + "connectome mapping", + "brain magnetic resonance imaging", + "brain abnormalities" + ], + "datasources": [ + { + "openaireId": "re3data_____::5b9bf9171d92df854cf3c520692e9122" + }, + { + "openaireId": "doajarticles::c7d3de67dc77af72f6747157441252ec" + }, + { + "openaireId": "re3data_____::8515794670370f49c1d176c399c714f5" + }, + { + "openaireId": "doajarticles::d640648c84b10d425f96f11c3de468f3" + }, + { + "openaireId": "doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a" + }, + { + "openaireId": "rest________::fb1a3d4523c95e63496e3bc7ba36244b" + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "neuroinformatics" + }, + { + "zenodoCommunityId": "hbp" + }, + { + "zenodoCommunityId": "from_neuroscience_to_machine_learning" + }, + { + "zenodoCommunityId": "ci2c" + }, + { + "zenodoCommunityId": "opensourcebrain" + }, + { + "zenodoCommunityId": "brainspeak" + }, + { + "zenodoCommunityId": "braincom" + }, + { + "zenodoCommunityId": "nextgenvis" + }, + { + "zenodoCommunityId": "meso-brain" + }, + { + "zenodoCommunityId": "neuroplasticity-workshop" + }, + { + "zenodoCommunityId": "bionics" + }, + { + "zenodoCommunityId": "brainmattrain-676408" + }, + { + "zenodoCommunityId": "repronim" + }, + { + "zenodoCommunityId": "affectiveneuro" + }, + { + "zenodoCommunityId": "con" + }, + { + "zenodoCommunityId": "lab_neurol_sperim_irfmn_irccs_milano_it" + } + ] + }, + "dariah": { + "id": "dariah", + "subjects": [ + + ], + "datasources": [ + { + "openaireId": "opendoar____::7e7757b1e12abcb736ab9a754ffb617a", + "sc": { + "cl": { + "criteria": [ + { + "ce": { + "constraint": [ + { + "verb": "contains", + "field": "contributor", + "value": "DARIAH" + } + ] + } + } + ] + } + } + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "dimpo" + } + ] + }, + "rda": { + "id": "rda", + "subjects": [ + + ], + "datasources": [ + + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "rda" + } + ] + }, + "dh-ch": { + "id": "dh-ch", + "subjects": [ + "modern art", + "metadata", + "monuments", + "sites", + "field walking", + "frescoes", + "excavation", + "ontologies", + "mapping", + "cities", + "temples", + "lithics", + "roads", + "digital cultural heritage", + "interoperability", + "archaeological reports", + "churches", + "standards", + "archaeological stratigraphy", + "buidings", + "digital humanities", + "survey", + "archaeological sites", + "CIDOC CRM", + "decorations", + "classic art", + "stratigraphy", + "digital archaeology", + "walls", + "data science", + "chapels", + "paintings", + "archaeology", + "fair data", + "mosaics", + "data visualization", + "burials", + "medieval art", + "castles", + "statues", + "natural language processing", + "inscriptions", + "vaults", + "open data", + "contemporary art", + "3D", + "pottery", + "site", + "metadata schema", + "architectural", + "vessels" + ], + "datasources": [ + { + "openaireId": "re3data_____::9ebe127e5f3a0bf401875690f3bb6b81" + }, + { + "openaireId": "doajarticles::c6cd4b532e12868c1d760a8d7cda6815" + }, + { + "openaireId": "doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b" + }, + { + "openaireId": "doajarticles::6eb31d13b12bc06bbac06aef63cf33c9" + }, + { + "openaireId": "doajarticles::0da84e9dfdc8419576169e027baa8028" + }, + { + "openaireId": "re3data_____::84e123776089ce3c7a33db98d9cd15a8" + }, + { + "openaireId": "openaire____::c5502a43e76feab55dd00cf50f519125" + }, + { + "openaireId": "re3data_____::a48f09c562b247a9919acfe195549b47" + }, + { + "openaireId": "opendoar____::97275a23ca44226c9964043c8462be96" + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "storm" + }, + { + "zenodoCommunityId": "crosscult" + }, + { + "zenodoCommunityId": "wholodance_eu" + }, + { + "zenodoCommunityId": "digcur2013" + }, + { + "zenodoCommunityId": "gravitate" + }, + { + "zenodoCommunityId": "dipp2014" + }, + { + "zenodoCommunityId": "digitalhumanities" + }, + { + "zenodoCommunityId": "dimpo" + }, + { + "zenodoCommunityId": "adho" + }, + { + "zenodoCommunityId": "chc" + }, + { + "zenodoCommunityId": "wahr" + }, + { + "zenodoCommunityId": "ibe" + }, + { + "zenodoCommunityId": "ariadne" + }, + { + "zenodoCommunityId": "parthenos-hub" + }, + { + "zenodoCommunityId": "parthenos-training" + }, + { + "zenodoCommunityId": "gandhara" + }, + { + "zenodoCommunityId": "cmsouthasia" + }, + { + "zenodoCommunityId": "nilgirihills" + }, + { + "zenodoCommunityId": "shamsa_mustecio" + }, + { + "zenodoCommunityId": "bodhgaya" + } + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml new file mode 100644 index 000000000..8fec18593 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml @@ -0,0 +1,176 @@ + + + + + + + + + + + + + + + + + + + + SDG13 - Climate action + SDG8 - Decent work and economic growth + SDG15 - Life on land + SDG2 - Zero hunger + SDG17 - Partnerships for the goals + SDG10 - Reduced inequalities + SDG5 - Gender equality + SDG12 - Responsible consumption and production + SDG14 - Life below water + SDG6 - Clean water and sanitation + SDG11 - Sustainable cities and communities + SDG1 - No poverty + SDG3 - Good health and well being + SDG7 - Affordable and clean energy + SDG4 - Quality education + SDG9 - Industry innovation and infrastructure + SDG16 - Peace justice and strong institutions + + + + + 123 + + + + + + + + + + + + + + + + + brain mapping + brain imaging + electroencephalography + arterial spin labelling + brain fingerprinting + brain + neuroimaging + Multimodal Brain Image Analysis + fMRI + neuroinformatics + fetal brain + brain ultrasonic imaging + topographic brain mapping + diffusion tensor imaging + computerized knowledge assessment + connectome mapping + brain magnetic resonance imaging + brain abnormalities + + + + re3data_____::5b9bf9171d92df854cf3c520692e9122 + + + + doajarticles::c7d3de67dc77af72f6747157441252ec + + + + re3data_____::8515794670370f49c1d176c399c714f5 + + + + doajarticles::d640648c84b10d425f96f11c3de468f3 + + + + doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a + + + + + + + + marine + ocean + fish + aqua + sea + + + + re3data_____::9633d1e8c4309c833c2c442abeb0cfeb + + + + + + + + animal production and health + fisheries and aquaculture + food safety and human nutrition + information management + food technology + agri-food education and extension + natural resources and environment + food system + engineering technology and Research + agriculture + food safety risk assessment + food security + farming practices and systems + plant production and protection + agri-food economics and policy + food distribution + forestry + + + + opendoar____::1a551829d50f1400b0dab21fdd969c04 + + + + opendoar____::49af6c4e558a7569d80eee2e035e2bd7 + + + + opendoar____::0266e33d3f546cb5436a10798e657d97 + + + + opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 + + + + opendoar____::41bfd20a38bb1b0bec75acf0845530a7 + + + + opendoar____::87ae6fb631f7c8a627e8e28785d9992d + + + + + + + oac_clarin + + + + re3data_____::a507cdacc5bbcc08761c92185dee5cab + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json new file mode 100644 index 000000000..6aa4275d6 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json @@ -0,0 +1,37 @@ +{ + "communities": { + "dariah": { + "id": "dariah", + "subjects": [ + + ], + "datasources": [ + { + "openaireId": "opendoar____::7e7757b1e12abcb736ab9a754ffb617a", + "sc": { + "cl": { + "criteria": [ + { + "ce": { + "constraint": [ + { + "verb": "contains", + "field": "contributor", + "value": "DARIAH" + } + ] + } + } + ] + } + } + } + ], + "zenodoCommunities": [ + { + "zenodoCommunityId": "dimpo" + } + ] + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml new file mode 100644 index 000000000..ad31e1763 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml @@ -0,0 +1,193 @@ + + + + + + + + + + + + + + + + + + + + SDG13 - Climate action + SDG8 - Decent work and economic growth + SDG15 - Life on land + SDG2 - Zero hunger + SDG17 - Partnerships for the goals + SDG10 - Reduced inequalities + SDG5 - Gender equality + SDG12 - Responsible consumption and production + SDG14 - Life below water + SDG6 - Clean water and sanitation + SDG11 - Sustainable cities and communities + SDG1 - No poverty + SDG3 - Good health and well being + SDG7 - Affordable and clean energy + SDG4 - Quality education + SDG9 - Industry innovation and infrastructure + SDG16 - Peace justice and strong institutions + + + + + 123 + + + + + + + + + + + + + + + + + brain mapping + brain imaging + electroencephalography + arterial spin labelling + brain fingerprinting + brain + neuroimaging + Multimodal Brain Image Analysis + fMRI + neuroinformatics + fetal brain + brain ultrasonic imaging + topographic brain mapping + diffusion tensor imaging + computerized knowledge assessment + connectome mapping + brain magnetic resonance imaging + brain abnormalities + + + + re3data_____::5b9bf9171d92df854cf3c520692e9122 + + + + doajarticles::c7d3de67dc77af72f6747157441252ec + + + + re3data_____::8515794670370f49c1d176c399c714f5 + + + + doajarticles::d640648c84b10d425f96f11c3de468f3 + + + + doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a + + + + + + + + marine + ocean + fish + aqua + sea + + + + re3data_____::9633d1e8c4309c833c2c442abeb0cfeb + + + + + + + + animal production and health + fisheries and aquaculture + food safety and human nutrition + information management + food technology + agri-food education and extension + natural resources and environment + food system + engineering technology and Research + agriculture + food safety risk assessment + food security + farming practices and systems + plant production and protection + agri-food economics and policy + food distribution + forestry + + + + opendoar____::1a551829d50f1400b0dab21fdd969c04 + + + + opendoar____::49af6c4e558a7569d80eee2e035e2bd7 + + + + opendoar____::0266e33d3f546cb5436a10798e657d97 + + + + opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 + + + + opendoar____::41bfd20a38bb1b0bec75acf0845530a7 + + + + opendoar____::87ae6fb631f7c8a627e8e28785d9992d + + + + + + + oac_clarin + + + + re3data_____::a507cdacc5bbcc08761c92185dee5cab + + + + + + + oaa_dariah + + + + openaire____::1cfdb2e14977f31a98e0118283401f32 + {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} + + + + + + dimpo + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json new file mode 100644 index 000000000..c150e4386 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json @@ -0,0 +1,20 @@ +{"communities": + {"ee": + {"id":"ee", + "subjects":["SDG13 - Climate action","SDG8 - Decent work and economic growth","SDG15 - Life on land","SDG2 - Zero hunger","SDG17 - Partnerships for the goals","SDG10 - Reduced inequalities","SDG5 - Gender equality","SDG12 - Responsible consumption and production","SDG14 - Life below water","SDG6 - Clean water and sanitation","SDG11 - Sustainable cities and communities","SDG1 - No poverty","SDG3 - Good health and well being","SDG7 - Affordable and clean energy","SDG4 - Quality education","SDG9 - Industry innovation and infrastructure","SDG16 - Peace justice and strong institutions"], + "datasources":[], + "zenodoCommunities":[], + "organizationCommunity":[] + }, + "instruct": + {"id":"instruct", + "subjects":[], + "datasources":[], + "zenodoCommunities":[{"zenodoCommunityId":"instruct"},{"zenodoCommunityId":"west-life"}],"organizationCommunity":[]}, + "egi":{"id":"egi","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"zenodo"}],"organizationCommunity":[]}, + "covid-19":{"id":"covid-19","subjects":["COVID-19","SARS-CoV-2","2019-nCoV","Severe acute respiratory syndrome coronavirus 2","2019 novel coronavirus","coronavirus disease 2019","coronavirus disease-19","HCoV-19","mesh:COVID-19","mesh:C000657245"], + "datasources":[{"openaireId":"opendoar____::358aee4cc897452c00244351e4d91f69","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"re3data_____::7b0ad08687b2c960d5aeef06f811d5e6","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"} + ]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"driver______::bee53aa31dc2cbb538c10c2b65fa5824","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::437f4b072b1aa198adcbc35910ff3b98","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::081b82f96300b6a6e3d282bad31cb6e2","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::9e3be59865b2c1c335d32dae2fe7b254","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]} + ,{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"opendoar____::8b6dd7db9af49e67306feb59a8bdc52c","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"share_______::4719356ec8d7d55d3feb384ce879ad6c","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"share_______::bbd802baad85d1fd440f32a7a3a2c2b1","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"opendoar____::6f4922f45568161a8cdf4ad2299f6d23","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}}],"zenodoCommunities":[{"zenodoCommunityId":"chicago-covid-19"},{"zenodoCommunityId":"covid-19-senacyt-panama-sample"},{"zenodoCommunityId":"covid-19-tx-rct-stats-review"},{"zenodoCommunityId":"covid_19_senacyt_abc_panama"}],"organizationCommunity":[]}, + "dariah":{"id":"dariah","subjects":[],"datasources":[{"openaireId":"opendoar____::7e7757b1e12abcb736ab9a754ffb617a","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}},{"openaireId":"opendoar____::96da2f590cd7246bbde0051047b0d6f7","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}}],"zenodoCommunities":[{"zenodoCommunityId":"dimpo"}],"organizationCommunity":[]},"rda":{"id":"rda","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"rda"}],"organizationCommunity":[]},"clarin":{"id":"clarin","subjects":[],"datasources":[{"openaireId":"re3data_____::a507cdacc5bbcc08761c92185dee5cab"}],"zenodoCommunities":[],"organizationCommunity":[]},"aginfra":{"id":"aginfra","subjects":["animal production and health","fisheries and aquaculture","food safety and human nutrition","information management","food technology","agri-food education and extension","natural resources and environment","food system","engineering technology and Research","agriculture","food safety risk assessment","food security","farming practices and systems","plant production and protection","agri-food economics and policy","Agri-food","food distribution","forestry"],"datasources":[{"openaireId":"opendoar____::1a551829d50f1400b0dab21fdd969c04"},{"openaireId":"opendoar____::49af6c4e558a7569d80eee2e035e2bd7"},{"openaireId":"opendoar____::0266e33d3f546cb5436a10798e657d97"},{"openaireId":"opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06"},{"openaireId":"opendoar____::41bfd20a38bb1b0bec75acf0845530a7"},{"openaireId":"opendoar____::87ae6fb631f7c8a627e8e28785d9992d"}],"zenodoCommunities":[{"zenodoCommunityId":"edenis"},{"zenodoCommunityId":"efsa-pilot"},{"zenodoCommunityId":"egene3"},{"zenodoCommunityId":"efsa-kj"},{"zenodoCommunityId":"euromixproject"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"sedinstcjfst"},{"zenodoCommunityId":"afinet-kc"},{"zenodoCommunityId":"2231-4784"},{"zenodoCommunityId":"2231-0606"},{"zenodoCommunityId":"solace"},{"zenodoCommunityId":"pa17"},{"zenodoCommunityId":"smartakis"},{"zenodoCommunityId":"sedinstcjae"},{"zenodoCommunityId":"phenology_camera"},{"zenodoCommunityId":"aginfra"},{"zenodoCommunityId":"erosa"},{"zenodoCommunityId":"bigdatagrapes"}],"organizationCommunity":[]},"fam":{"id":"fam","subjects":["Stock Assessment","pelagic","Acoustic","Fish farming","Fisheries","Fishermen","maximum sustainable yield","trawler","Fishing vessel","Fisherman","Fishing gear","mackerel","RFMO","Fish Aggregating Device","Bycatch","Fishery","common fisheries policy","Fishing fleet","Aquaculture"],"datasources":[{"openaireId":"doajarticles::8cec81178926caaca531afbd8eb5d64c"},{"openaireId":"doajarticles::0f7a7f30b5400615cae1829f3e743982"},{"openaireId":"doajarticles::9740f7f5af3e506d2ad2c215cdccd51a"},{"openaireId":"doajarticles::9f3fbaae044fa33cb7069b72935a3254"},{"openaireId":"doajarticles::cb67f33eb9819f5c624ce0313957f6b3"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::554cde3be9e5c4588b4c4f9f503120cb"},{"openaireId":"tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b"},{"openaireId":"doajarticles::57c5d3837da943e93b28ec4db82ec7a5"},{"openaireId":"doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::dca64612dfe0963fffc119098a319957"},{"openaireId":"doajarticles::dd70e44479f0ade25aa106aef3e87a0a"}],"zenodoCommunities":[{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"h2020_vicinaqua"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"rlsadb"},{"zenodoCommunityId":"iotc_ctoi"}],"organizationCommunity":[]},"beopen":{"id":"beopen","subjects":["Green Transport","City mobility systems","Vulnerable road users","Traffic engineering","Transport electrification","Mobility","Intermodal freight transport","Clean vehicle fleets","Intelligent mobility","Inflight refueling","District mobility systems","Navigation and control systems for optimised planning and routing","European Space Technology Platform","European Transport networks","Green cars","Inter-modality infrastructures","Advanced Take Off and Landing Ideas","Sustainable urban systems","port-area railway networks","Innovative forms of urban transport","Alliance for Logistics Innovation through Collaboration in Europe","Advisory Council for Aeronautics Research in Europe","Mobility services for people and goods","Guidance and traffic management","Passenger mobility","Smart mobility and services","transport innovation","high-speed railway","Vehicle design","Inland shipping","public transportation","aviation’s climate impact","Road transport","On-demand public transport","Personal Air Transport","Transport","transport vulnerability","Pipeline transport","European Association of Aviation Training and Education Organisations","Defrosting of railway infrastructure","Inclusive and affordable transport","River Information Services","jel:L92","Increased use of public transport","Seamless mobility","STRIA","trolleybus transport","Intelligent Transport System","Low-emission alternative energy for transport","Shared mobility for people and goods","Business model for urban mobility","Interoperability of transport systems","Cross-border train slot booking","Air transport","Transport pricing","Sustainable transport","European Rail Transport Research Advisory Council","Alternative aircraft configurations","Transport and Mobility","Railways applications","urban transport","Environmental impact of transport","urban freight delivery systems","Automated Road Transport","Alternative fuels in public transport","Active LIDAR-sensor for GHG-measurements","Autonomous logistics operations","Rational use of motorised transport","Network and traffic management systems","electrification of railway wagons","Single European Sky","Electrified road systems","transportation planning","Railway dynamics","Motorway of the Sea","smart railway communications","Maritime transport","Environmental- friendly transport","Combined transport","Connected automated driving technology","Innovative freight logistics services","automated and shared vehicles","Alternative Aircraft Systems","Land-use and transport interaction","Public transport system","Business plan for shared mobility","Shared mobility","Growing of mobility demand","European Road Transport Research Advisory Council","WATERBORNE ETP","Effective transport management system","Short Sea Shipping","air traffic management","Sea hubs and the motorways of the sea","Urban mobility solutions","Smart city planning","Maritime spatial planning","EUropean rail Research Network of Excellence","Transport governance","ENERGY CONSUMPTION BY THE TRANSPORT SECTOR","Integrated urban plan","inland waterway services","European Conference of Transport Research Institutes","air vehicles","E-freight","Automated Driving","Automated ships","pricing for cross-border passenger transport","Vehicle efficiency","Railway transport","Electric vehicles","Road traffic monitoring","Deep sea shipping","Circular economy in transport","Traffic congestion","air transport system","Urban logistics","Rail transport","OpenStreetMap","high speed rail","Transportation engineering","Intermodal travel information","Flight Data Recorders","Advanced driver assistance systems","long distance freight transport","Inland waterway transport","Smart mobility","Mobility integration","Personal Rapid Transit system","Safety measures \\u0026 requirements for roads","Green rail transport","Electrical","Vehicle manufacturing","Future Airport Layout","Rail technologies","European Intermodal Research Advisory Council","inland navigation","Automated urban vehicles","ECSS-standards","Traveller services","Polluting transport","Air Traffic Control","Cooperative and connected and automated transport","Innovative powertrains","Quality of transport system and services","door-to- door logistics chain","Inter-modal aspects of urban mobility","travel (and mobility)","Innovative freight delivery systems","urban freight delivery infrastructures"],"datasources":[{"openaireId":"doajarticles::1c5bdf8fca58937894ad1441cca99b76"},{"openaireId":"doajarticles::b37a634324a45c821687e6e80e6f53b4"},{"openaireId":"doajarticles::4bf64f2a104040e4e055cd9594b2d77c"},{"openaireId":"doajarticles::479ca537c12755d1868bbf02938a900c"},{"openaireId":"doajarticles::55f31df96a60e2309f45b7c265fcf7a2"},{"openaireId":"doajarticles::c52a09891a5301f9986ebbfe3761810c"},{"openaireId":"doajarticles::379807bc7f6c71a227ef1651462c414c"},{"openaireId":"doajarticles::36069db531a00b85a2e8fb301f4bdc19"},{"openaireId":"doajarticles::b6a898da311ded96fabf49c520b80d5d"},{"openaireId":"doajarticles::d0753d9180b35a271d8b4a31f449749f"},{"openaireId":"doajarticles::172050a92511838393a3fe237ae47e31"},{"openaireId":"doajarticles::301ed96c62abb160a3e29796efe5c95c"},{"openaireId":"doajarticles::0f4f805b3d842f2c7f1b077c3426fa59"},{"openaireId":"doajarticles::ba73728b84437b8d48ae287b867c7215"},{"openaireId":"doajarticles::86faef424d804309ccf45f692523aa48"},{"openaireId":"doajarticles::73bd758fa41671de70964c3ecba013af"},{"openaireId":"doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4"},{"openaireId":"doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7"},{"openaireId":"doajarticles::ca61df07089acc53a1569bde6673d82a"},{"openaireId":"doajarticles::237dd6f1606600459d0297abd8ed9976"},{"openaireId":"doajarticles::fba6191177ede7c51ea1cdf58eae7f8b"}],"zenodoCommunities":[{"zenodoCommunityId":"jsdtl"},{"zenodoCommunityId":"utc-martrec"},{"zenodoCommunityId":"utc-uti"},{"zenodoCommunityId":"stp"},{"zenodoCommunityId":"c2smart"},{"zenodoCommunityId":"stride-utc"},{"zenodoCommunityId":"crowd4roads"},{"zenodoCommunityId":"lemo"},{"zenodoCommunityId":"imov3d"},{"zenodoCommunityId":"tra2018"},{"zenodoCommunityId":"optimum"},{"zenodoCommunityId":"stars"},{"zenodoCommunityId":"iecteim"},{"zenodoCommunityId":"iccpt2019"}],"organizationCommunity":[]},"science-innovation-policy":{"id":"science-innovation-policy","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"risis"}],"organizationCommunity":[]},"mes":{"id":"mes","subjects":["marine","ocean","fish","aqua","sea"],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"adriplan"},{"zenodoCommunityId":"devotes-project"},{"zenodoCommunityId":"euro-basin"},{"zenodoCommunityId":"naclim"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"assisibf"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"aquatrace"},{"zenodoCommunityId":"myfish"},{"zenodoCommunityId":"atlas"},{"zenodoCommunityId":"blue-actionh2020"},{"zenodoCommunityId":"sponges"},{"zenodoCommunityId":"merces_project"},{"zenodoCommunityId":"bigdataocean"},{"zenodoCommunityId":"columbus"},{"zenodoCommunityId":"h2020-aquainvad-ed"},{"zenodoCommunityId":"aquarius"},{"zenodoCommunityId":"southern-ocean-observing-system"},{"zenodoCommunityId":"eawag"},{"zenodoCommunityId":"mossco"},{"zenodoCommunityId":"onc"},{"zenodoCommunityId":"oceanbiogeochemistry"},{"zenodoCommunityId":"oceanliteracy"},{"zenodoCommunityId":"openearth"},{"zenodoCommunityId":"ocean"},{"zenodoCommunityId":"calcifierraman"},{"zenodoCommunityId":"bermudabream"},{"zenodoCommunityId":"brcorp1"},{"zenodoCommunityId":"mce"},{"zenodoCommunityId":"biogeochem"},{"zenodoCommunityId":"ecc2014"},{"zenodoCommunityId":"fisheries"},{"zenodoCommunityId":"sedinstcjfas"},{"zenodoCommunityId":"narmada"},{"zenodoCommunityId":"umr-entropie"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"zf-ilcs"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"afrimed_eu"},{"zenodoCommunityId":"spi-ace"},{"zenodoCommunityId":"cice-consortium"},{"zenodoCommunityId":"nemo-ocean"},{"zenodoCommunityId":"mesopp-h2020"},{"zenodoCommunityId":"marxiv"}],"organizationCommunity":[]},"ni":{"id":"ni","subjects":["brain mapping","brain imaging","electroencephalography","arterial spin labelling","brain fingerprinting","brain","neuroimaging","Multimodal Brain Image Analysis","fMRI","neuroinformatics","fetal brain","brain ultrasonic imaging","topographic brain mapping","diffusion tensor imaging","computerized knowledge assessment","connectome mapping","brain magnetic resonance imaging","brain abnormalities"],"datasources":[{"openaireId":"re3data_____::5b9bf9171d92df854cf3c520692e9122"},{"openaireId":"doajarticles::c7d3de67dc77af72f6747157441252ec"},{"openaireId":"re3data_____::8515794670370f49c1d176c399c714f5"},{"openaireId":"doajarticles::d640648c84b10d425f96f11c3de468f3"},{"openaireId":"doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a"},{"openaireId":"rest________::fb1a3d4523c95e63496e3bc7ba36244b"}],"zenodoCommunities":[{"zenodoCommunityId":"neuroinformatics"},{"zenodoCommunityId":"hbp"},{"zenodoCommunityId":"from_neuroscience_to_machine_learning"},{"zenodoCommunityId":"ci2c"},{"zenodoCommunityId":"opensourcebrain"},{"zenodoCommunityId":"brainspeak"},{"zenodoCommunityId":"braincom"},{"zenodoCommunityId":"nextgenvis"},{"zenodoCommunityId":"meso-brain"},{"zenodoCommunityId":"neuroplasticity-workshop"},{"zenodoCommunityId":"bionics"},{"zenodoCommunityId":"brainmattrain-676408"},{"zenodoCommunityId":"repronim"},{"zenodoCommunityId":"affectiveneuro"},{"zenodoCommunityId":"con"},{"zenodoCommunityId":"lab_neurol_sperim_irfmn_irccs_milano_it"}],"organizationCommunity":[]},"dh-ch":{"id":"dh-ch","subjects":["modern art","monuments","europeana data model","sites","field walking","frescoes","LIDO metadata schema","art history","excavation","Arts and Humanities General","cities","coins","temples","numismatics","lithics","roads","environmental archaeology","digital cultural heritage","archaeological reports","history","CRMba","churches","cultural heritage","archaeological stratigraphy","religious art","buidings","digital humanities","survey","archaeological sites","linguistic studies","bioarchaeology","architectural orders","palaeoanthropology","fine arts","europeana","CIDOC CRM","decorations","classic art","stratigraphy","digital archaeology","intangible cultural heritage","walls","humanities","chapels","CRMtex","Language and Literature","paintings","archaeology","fair data","mosaics","burials","architecture","medieval art","castles","CARARE metadata schema","statues","natural language processing","inscriptions","CRMsci","vaults","contemporary art","Arts and Humanities","CRMarchaeo","pottery","site","architectural","vessels"],"datasources":[{"openaireId":"re3data_____::9ebe127e5f3a0bf401875690f3bb6b81"},{"openaireId":"doajarticles::c6cd4b532e12868c1d760a8d7cda6815"},{"openaireId":"doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b"},{"openaireId":"doajarticles::6eb31d13b12bc06bbac06aef63cf33c9"},{"openaireId":"doajarticles::0da84e9dfdc8419576169e027baa8028"},{"openaireId":"re3data_____::84e123776089ce3c7a33db98d9cd15a8"},{"openaireId":"openaire____::c5502a43e76feab55dd00cf50f519125"},{"openaireId":"re3data_____::a48f09c562b247a9919acfe195549b47"},{"openaireId":"opendoar____::97275a23ca44226c9964043c8462be96"}],"zenodoCommunities":[{"zenodoCommunityId":"storm"},{"zenodoCommunityId":"crosscult"},{"zenodoCommunityId":"wholodance_eu"},{"zenodoCommunityId":"digcur2013"},{"zenodoCommunityId":"gravitate"},{"zenodoCommunityId":"dipp2014"},{"zenodoCommunityId":"digitalhumanities"},{"zenodoCommunityId":"dimpo"},{"zenodoCommunityId":"adho"},{"zenodoCommunityId":"chc"},{"zenodoCommunityId":"wahr"},{"zenodoCommunityId":"ibe"},{"zenodoCommunityId":"ariadne"},{"zenodoCommunityId":"parthenos-hub"},{"zenodoCommunityId":"parthenos-training"},{"zenodoCommunityId":"gandhara"},{"zenodoCommunityId":"cmsouthasia"},{"zenodoCommunityId":"nilgirihills"},{"zenodoCommunityId":"shamsa_mustecio"},{"zenodoCommunityId":"bodhgaya"}],"organizationCommunity":[]}}} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/dataset_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..bd29d59ae948f260084f492e11d2439e22d96d1e GIT binary patch literal 6736 zcmV-W8n5LaiwFp=uZ>;+17u-zVRL14UokK)YIARH0PS7flG`?xzMrQ+scNcbs?=(d z67_f2X*-!t*@??`rYbw0N`WLKp+f=+NOq!RUA1|JeF9(l!WUDy%dKwn6#FE54giXj z`ajbiGB$Qy8MOou2j@FD0C7GX{M1WmLC?fO_U1qJxVYdsk5bl~FKC*xK76xaO0i`3 zMZ!{6uq4{(UzmvNvV|)1qBrk(gIUkk#NXFSUNAv*$sYVr|4rk9OMwl?Da~^Zt@QFu z&a<^t#XMFr&%dYXw{ykTGUtU<8>p^}3YxLI6wc!n%b0#}B3Lo6+dk24=esiYiPw~R z{kq?Sc8hWi>S6`YSYFU<-JAR2H1xu7I-btF8FU(33cdf9$v67{hO_BxKKFto36mfW zg212okr&OPWV9F#<0+es!vM7<6<=~e(=&L$KYI79{!h?~Oj8yEFbgF!c*IY=Z`lT9 zk!vOrNfkbAKF8-lV;3|nv9piZdix9NiadhW3z`yk^wsI-r%ub8rMeYW7ch_>L2Eb5L(_t3BmpNtE2263>iH9> zvnq=9pNA)Z4nBo{JN0onDC6^Zz>*Ta23hov-<8l7qMI;LSOY~t=U3I62i_p`yn&BW z9f#wqeqbV@TCuVkNm4i8|^rY%tY znV0ICD)1v9`YkvuJDPE5Lwkz_Pr1cdkM{mv%P&=V&eK$qPniIwm>r(E4|{p)L-LP}A9>(K)^FQaqt6nh^3l!H&?G$#ooDJ7>eD+P~f%#96P zG8?|43C}a>U}zJD3$8%-2~8Xft-`Q;mjM7Gk^hJ`?v4l z&H9;4x9iRZ2~hM`rg1Fl9z7z)atf%!3$JPV-xCeO1MZkI4?v4fFZGj|FFDJ zv|vkq!}~@?xgh3=`<>eR;jjaU)+sO8H~R&e(jOFY5dWSbch*FefG?zifE-YZ=PV%> z-y7Bp4&Khet9Q~x6Jz60o1*JYG%!hi;j7>X(#l|!{h7!QU)k>`pp zS4CY&)G)yZjCiKc*-nA@H5a)ZNdQ;MWAJEuqsm5&W5s9zZVWtP29oKZu`Qh+N1G!N zABp66H1Licw)Yr(d!RLC{_@%5FAgO8at+hUiLP*k80JaeWHPxX#=2MFbq)OG;Iiih zql30T0m`glClBF zb9{+afpd$U+EK_<#ueG5cZt;W;G()&?crxNCDGiuSjH=u%^1y)RZuy`!(Zc4VfQ!H z?@IefmefRy9E1dS+nf;-cERI%e&j#yKI_$9MdL7py^4@yhpE{`vY(KRNs#iX$10uH{IE$uypX{%9O5W>GXvg2)@s!f~>Q;xL{>y<0evw^jcS zawP9Tx0}rFNw*7zv-e52GxD@ew+n(v?-Jc^a^(9*e)!TK&x6@K4Bs!^?m50rKHI)k zNOpTlRnZ`sRe#nrGgF%V_Ooh`+HPanIhxs>XlCeMF7!M6r|_B&iy5+kUs3`C6Py}z z`?76r+C7F}3yDx++x$YhxJ9dYJZFW2pj`-@U+Pr6=!~37vBY;@@|uLkfde!u;uGu=g|CBuMft_1T$fk1-?S*4#ZNx>$t*x-BXAGdBKQc7_8a;LP~)` zlnKW^TGd}jvO+GyE-nL_fHwk?H%BB+y>4AhI+yyZ+sMZPN&@n^|&y_hio`5OwoZb z1@1<5IZWX^vI|qVFvSCgDQ4bma&4GmF`bRZqj*H)Q4C)#m={m{(Qp{~laT6T;x~pV zZmZtSFva-3VTy3{KBE-lPDClD-jzuKFiJ7PqyR&2K6=kd0d9O9?UQ@&q}#{t>OI-3 z#DDu)*ZO(@Rhrf6K*-esVNL+cxpScr7aI9+XO`wZ@D3bSZ=$$k9W8UrGjP8!M=tbm zac4SO1>>3B7|k#5j5=W21SS~yJ}k0e!iTpGb23d&E052aih0c4xamr4s~$8`Fug&4 zzZC*GJw5sKQB4kafv-Q1J={I;^$Dyp0s7}MV@mdGUz zOeliqOZ(iMq}z9b(JMTE6(m|wj6~&BDR?=x@i&Iypf!@ObjZjWFFXRRcaA^v{rSCW zE#Vq`cu^zt%2gS33&Tg*_H7}plo)YZGi9a9wr^l~Eil^^RK%zfHpeQ5ST#6!C9JMN z#)xm<>P{3Cz5qQL3}cT| ze>RI|Vd5v@csvb#x|k%xI1FzIVcuH3gCWdtFb;?J3}H^j?=ObAlc)U{rhj7$)1S?U zQy0T@F-#Z3{8*-z{SXb(z+FhDbs+x669rmxNVJZG^Glrp0p4aEp`zvUzqsmFGUe{r zX(xsRv$3lj#)e*IIZzyv!ohPZlj6yKbk@9@tQw0h43ow(TeE^&vPEN%^$AoZpO-pj zosv_T-vF#5Q*_ZJ?oOuIgL$7!0k5v``~}P5qY3bTR-GNbXHXvZy}+u##_$L)x%n6| zQYKc^p2yYg-At5#_Sf}`9exjeW#5%N<&ols{y^c49%N#T`MsiMBve#Krjx5D9Rh7r zUA!)&7i&hw7;5KbO~t_4rJy9Ef1L^cX#De&DJeEV#kschjj`YwK(u40v4o2lpa9j0S-} zeP5)9TAubv4>noGm86H^d^DT~eygO1zk>&YAs?f6(rLuVnGo_~7yq+QIVe_@xpeos zV>CEM!$&-CxbSVqKscofwtXjDaH`$d?Lwcvb|AExB_f5TAM`EZI$Ng;2)65(nh`z+ zMVnB3!UUeT<<3Dl3WI~^4itv7{h;s>vGY68GUx=(8zR_}n$L(~nO8^8W4j`&kJ;b6 zN_HfME*iz%Nes_!W)tKLQxI(5G;=VByCp#Hz=rIHgBX6d&a}zW+*SHpZhJ8zbRnJ4BoJ6B^I8vP$G|On zGC~g;o5~{I{$ehj-o7C@$NfE>;Nh|+YyDxm^*t^GvMdf(f0}WD1@kh~VpYmBspWfL zl=`0Q!-fzpkx><6$3&q_;`x|@>Dj(54*FhauFx&Y=$z@C?I>~qGjFDoP&87to3s=b zJ6%d=ls|AC2Xqf)uy7IcHBB>}VI-#!Q=l+>HZnfwrc9usT7qP61#XoX3(M#RFhm9m zt5UmHRFghthL?4IjVg)#cO@VNKb@C)(LmS_nvG(pC=?`454k9spF&Ala5O(#62w(F z+6w40D5iyjk~e`U0> zn+i~{0p3?&Ohj*-%7%St=aAcBfyly8H(a{DwJH1X6-BUg&lJhnxm0~y%luB!#6b6U zqN2ubtg|SX$~Zh68))|TRZpw`85#zPb~ru+_?hi5Y1M|_-3BSZbmFwk4V{k*+?39Z zN;hQtD<}*;X*C>Ljv;wLJ#$;ZtKPmIcJt?zNUmj~hu7}*1^6887Q09xt3f%KoV6w^ z!?&7(5e=W4I1#ioRw5hCXuA|UfnY!^5niZJ)2w8rza8Y(XKC*N9b-4K&+J4I%T&8t ztAU5~y_lVcl9OeFTLyo_6jrdfzO#ao7dn5AS;5rla6ZIdxtQ?-Qqd+yzIWt}old(( zMf*2{WDZS7Mf=#XbyT#DP|>_jQ_y z)LSN}RLDNO;FhX!(|Pr0O(koT?m6DrgYw2sWme)Is7vzEdeHMJRo{YN7fX^DSLonq z6%MA8VdoM?OX`Rjt&fe^rCn*6x|iY6aEC$%P!I`Aq;gKFT`11Z$P>QQi8~&#h~mvA zXQwBRZ@t;XVe|g68TC=>ruYTL3yCncnK0bF9-jacSrbr=Wan=5bQSnm0rKqGkOn79>(Fsi@dPNcj-Rx=Q%2L9OVe!UDF# ziRmx}eQw`oc(oCEoD#EVr>Q=;J(w5RC%`$~zRmjp0rZh7!$!fTzWf0yB&Y`o@Fh}c zJ`6MuJv!h10;X%oNs5)i@K{W{Tv?C`mhHAjG+!mABGHT>7?Gvz^M|!Ex*LlL*nM1km26}xC8P`U*u(#Bop|o z1w#)5K!ajtgdPz0!0vY0^n$yWCS?wHX&^0EQQ4_mEK|CsyJ* z1((jz7V`+6R%eg0dE9EH_L~~FR=;kN4~@ZAG(_+?Och2(TN?LcxBrp|qP9_J6scW2 zF{Pl(;CsPK8;Y6Gm-W$9TiYXA>{woBzuVoou-yZ)fu=IKsg3%obM$hn0a&tb( z#a!CW@)FCHbS$E_Q-9&+rMnfSZ6K=T1Nr51mkyW?xGA1zE~`tcKs;wQwspZ3#LRV-C?m@%6oK%VYZ7Z`U@{{m}Y|slV_$esOl6oNxd@A0ZDq>a?P<` zx@a3+N`Hib^Mb{q@*7kWRlpFz?vSUDNKs{pxW@XB1@m0ylzd*|OY|N&XYbZ2THWk) zb>gHtSB=TD0-Xcl&da5y&Cq(vuc#A;;{@}*JmrUa!X!nQVaEi*D=jI_LnyXyW6g}v z>TQljOod`77ezzgCYO+AKtLy}eWx9LL8m}jLwU<<;9|uiOJ1pOnxZhq;D2Vg}ujm6SiVw2+gc`!*rwfx)<{*WWZP4aesaQ?> z+Uyh1I73E?<{sRhjLMoPk`y@RR}yF-(C(RlhZbdFw`TSb`-IDE`!+GSq~vchU+aB6 zKfuCY!lZ`oaC6xDGraw)0q9ejD4z7Io}RKt05RGKbWP~|R4`D!EBX4W(Pd~ThR@L_ zG+~-Ck(l%>Oj;mNb^ToiC!OtwTD}Ol7h~{y7Gc4&mSOU4OTg2nRkoE5rf@CW7r(j*u(J)t>SnoC1dn<5JaIy{WU zOeceq=y~v0Fhrzs?NWX%aBI0Df#t+B$8`FyS3RAf>GBXnO(&S-yp%>@O?N`UkkPlI zBx5mdaAnmyUOiyz1ysKtyvW}(eM629w)CSRp~YO;{@L;KHFg!Ci(nA0w(soCcrb>g zW+W{r^t$R%N(#Kc(XyU28EYX;jxpOP_DJjSRd26hP2mAcF1dCGn9coFk5GKx#rCZs zcqzG5)Y8*5X-d#tqOc~dY)3)_d(WA&x#9aDYKGgGn8*&A|GWfOr*m9Mq@|%Z+r9y~ z@=HFnk6_RU29+>F0y>t#tf$?OL0S(g_yY64ZS=FCLv;HNpPT?HT$CGzS+iV-@4@p3 zg-)lAIrLGY>6k+wF1C(2)Dh;;wA0KXHw8N8kYf%#By(sIde<_C7IDOe)Efp^_rr4p< zuCv1ZEcnN8{pzsj%EWJzx5RA~hH&P4123qu_vqX- z&5R_P58@ILc7_c3xl~I=KBGv*O5`<3y{-q#@fy1b^WAxk`|#J4L1O`mAvxskB*s2@ zB=u#AP-TDEpOoMZ0JGAh_WI-mi&RNTo4;Z*uNP(1xRy~T3pcS9=@?Bn_OjZfQ?3)aH zVI=zfIeI#s60Be<*6MI6!CXo(mlDjS1pCM)SeFmwk3{TqJhKnka2(I96FjqVr+H@1 zBRig%c%5W?%gMvd7~pgccjfr+)&}UrT6(4QmaLP+q8&N!;!1P=p-66`BCyPwR3e&qg zAX)|DF}xy~H;}{}S#)r;J1#T`Ia!q%x+{lHhXZ&b`u zJeaQ|){R$1Lh<;bKKoeISHs-#1;()_x@Zn}XHT3;(PY8yTTN=MrcW?TfQRh4Iq{4q zWuH-%(oEB3jUnO9DduJtnypLiB2LnbE|t6wQ6i$}5+72)I~yJ-f7Sav28(yC)MxBh zc>57kdUiMGl+ArSsJN@^o66XS-P(+7Y+ZA>9}iVysWnSEb}`jX(au=*fNp$wsXZVs z@&1(UTfEVyzD6gJrax7#0!s_&lNU5A;g_dSaQjXieL_{llyr2Fdku%X9P09Yr&C84 mX+%97UBuBvK2*90Mg_Y~7jYih(M4YW>Hh(z`8FgaPyqm7df$-% literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..20b6a4dba3f02516796283f817134026a083e7a3 GIT binary patch literal 8073 zcmV;4A9mm$iwFosWPDx#18;O_WpZ+5b7f(2V`y-4Z)A02bYC$rE^2dcZUF6F&5j#K zlD_X#BpMi?8%tJK@t-sngSI5wn$b^>)Sd-)JTS7Vl4RP&N;R|Nkfs7Z@NIwz>qcM++Xuy-P&{~AWgHS{w5uv}NHzE$ovWZSjaxgk@o#8=N6W`CZ zT8Nla$pd&%{^mgyg5@=C#5r>LvY;82c_Mo!`k5U}$Bg?AQPg*K= zONuyI{Wz9^VxeMpW0S}vO%yv-5tR8Mi0O@Bzn4a&&`PN|aT-O>fQUH`{||qpS3-?5 znn=V4(Dk6{cAM_t493(NjJkbi&}sk6L0&r1JT_ycXS}R2YpI@>xYO;oGA?9F zg~auVxbB(T8oA!6Jw(E0BH_sO2cWBF&S^qrG@YNHJ4ApErV@tfC_OoTcyjl+<+w=Q zmtk;BTFldAsx)cH7{r#T{iiCF@l-X~eXb=(`QKXofI`mR+sC)zv3U6$pPN*#SA}OF zz^`FOWO{%~0o{$qGLo4{NlzMU9p6$g<_5c^C8%4>*~yG+RrY(~dD$=3EP8G2=oPd? z3zJ4kvYd-`i=&b$$Z6fM6EK1_Lrm&_1yhm4p3nYMP&XOKkYi)tq&&(Lwn`otk#nWP z4HCigAyDJJ7YOf=YqT8euiMVO7Sup(j4*O!ZW#%0WG`|Cqod$Bnjl-(_vh%eU* zRL8PWq!##f1g>H13mzvS?LBCRRNb3Y6yb#NiIxG(tyFUcy0CfyGh8fW!WSZf-V(uJ z7%Gr3;~8h^j0GzG3QRri%@(juuU;&KG%Qw31TZ#OK*bD}SQW#2Y>;e55T&ahQQtt% z=9*7X-9Z0Dy!sygjNyM74fuQxvz^{x7b;Eg=^PfD1pfU7^-|5|Aln%zZ=@LLCDI9- ziPZ~Oj-Y;&K6Tg~paZ&_;1CQmNCq@J2PS0%gE*i8jhY|hyEwG%U>6N|1Uo~^fCghR zR;M;#7bLMk+jYh}rJ+Bz`k_IMO5j(62sHxQ4HW1l<7;zm1ch;!frM!|N8>sXnpL>~ zTO!bRYy`OmJ75D`lruR|#`z*X<1>Ht!wg$l;1GaMSd_0x6JElDL@p#12$87^72!`y z{%2|iOvH*XTd@=L$y+7SJEAy?lk8rPbdfGlC8%$Ld*wzR@9RzJdio+BSYz;*y20@&aH6(r~eY8Urq z_BH@DSiJ!Lq6W-(oF=G3LmREM&>LNqb~wN4gDBk32f0G(DlcJo zqJz}bjJyUT#mwh=0!waaw*~g-YE-I&Y?FZJwtU)U*_E0F3xQQfgISFlO`i6Z`=9SI7oGbp2N%?Gz}wX%HuE+4v>7j?rTE32x}(k z4-6twLGE%pxZp^97elFy_M11_TH)N0@g$gn(Gh%x-l2RBZ@vvujl*B&?=_hN5thC* zv>L#IIqoLo(j>v7iB{>{K1q~9o&4jofB3c*)tf;X+P#){8Oq@K;jr&>G4_U`AGSij z-E!N$=e2x)(DsAw4wPYM^*uouhE2ECY;_ScXuG3s&lwIo*FhP&1E_!yWU!qKO}Zu3}*h?kE3@& zfa|ia{AbxwR5xf16-d3!qD#yS5k9AF!^OC`u$G{dbts;RBrvi2gQ+gc91?>F+kJuR~1A@FT4HHxdR1Op1K3e|YzyJ4l0lMsGX^c7q`)D{lgi9(DlzPVI zYMw^4IHwGiW?rNizla8DW_&f}{z!8WD7py7wVQ}>P-*NEzfh0stcg5zfcXqQn0WoDpap~G&= zF)2ZU8jsEZZ09MB!eC?2OoPZqKNs@O4lrX;h87@PhD1{h9xhNWA;M6_M4N&}YBdqD z$cUsSqmbAgg-9qfY9N6eaOJvm5NNRG0-4YL8&RIvyQaxmuHRsB(JgX=eZNCs9XwdxYc8rYfQs_XTYH zLm1x)mM=-}4q5qO%lZJ$r;5#C=Pd&{hp7>=jG?Dzp6h=35GN>pcT_6#W8@klRWZ&H zTLI28ZFY``3MdCCp(X{&=T|lYIJ7hkXd^NRIc7iv=g6+2DHy>cTJ2y2_Z29>8l|OB ztYR0U0sUJtG91_yOo~`&cGz{D!LOLMV@eQB{Q0~8m?#zg{M~;xSPx&?XH-3VL4>xQ zPX10j2?QK);{@IveZ*P=tX2|Iw41rh!20Hmc&^jRTk$!z?XU-GO)cOgnBWIi0gtM3 zv`iVpjxvP**VdX8d?k<>J3@KiQV1oZkpPZWS2kt@p>!E|oUo~!Oc_`}5z$m!Q}6~T zfefY$hyvS`1Kh~BbBSvo*eU^&j=)80=HP@nPwXFvy`i=zX=WN|f~p`$k#en4=-0|C zqBARd023XZ!P$HOt5Vha++5`J%NVBhzop2Sa%RqROOvSpO?9ipU}tqcS80AD*I`}A zjoeIWM4^qqLp0bF_62B*2vMmN1CGOJlH4>QDb^lKGAfyIDmuN)$b(^p1u0stZ6xMq z8BFPT5@aJLE=OzftWn6fk3x2uu2(y>UmJzA4uBd>tI@Q#ji!-Lbx3hcQ_pM+ufk!@oqPu8yl}g({@$gUo@?EEi`TDbi2J9 zMbmok1WjvgxwEr@rrq$Jowtmpy*^U&9s+5nJdGL*4zIZ9S$$vWD0>#X{C>SdtZ`*b zt;U|#v8RWHH?0OP_F>`NfSpWQ15W!Px2=HFKTpYbmO13Be%R8>{cCo@CXw=aWWDK9 zoU_yr+C=haD1jPr+AmeOI^y&(55C5;MjU<-pJnieuA)ZLFqs$;jTwhs+N5D&2iUg= zf+Tsoj;%<lD4JP9duf>3J(G{VP4cvcZz?sZ< z+;w{FSGo5yQ3+^ig*GIm)pNRK5q#NX*=urUQ=`2ul|=>5FaytFpK&_^yj4RO2;R|v z1s#}>`*q8%&Jm!Rpp)Oq5uNDhGR=0tjBa5qM5N{xvIDU+8NGo^=Ws`#N@#Pa8Zwt`~V&>ca@r+aszB1iQ zIL9gl9%o?+2)dIUSU?TfVeh)cNkwvp57Z}n#ew#dC_2?db&FykeT5thn=3|!E_*a)#?%g3}88wvSVp`G^OCs7^{SR9YZ@)_+Nq-?|6884kU^r z(7kieTtOhPh1TeNX@L+j?bcz3gRx9zhr^!J209C77XyxH+Tcu(3ouB~Uuwh@R1)YN z44gqWT;O7-?X-Y6>o`fGW&YtqhIv@=o)gdszn50Qe5GNstg-m8-*#N21#PTGIEh|e z5IHQ5k-%A?Vo{NQF6Jfs`MWK26}jWeT|8AQNp*oWM0Sr3B{IaMnAy zR9AbHYdEWhv-S_pYJCWo!)oJG!&x7ssCL0w9k<`Q49;qKZfoELT|e-GetXD0FMuf| zM8F4buRZMTfU|a1-xD}%fZ$YXi11OXHF5_|tJl3UoYitiU?_X;8@wEL89i?q&hp&B z@FJXLiR+zt-BH^cwFWl}XPv;wmFwmsWq5;qtkbv>1}^_u!GFrqSH`p6S5)R5#(+E1 zaMYwJ^&3>)+ReWg3qcj0xeGQ?avM?RHpWxZ2^{!qIqsVr?+c7*gr=ev-rfoCwhV2k zt6Fq>A-b(F>E>EEVy9Zh!Rp7kRE9m&xH;va(&81XWQ~KqkFvfx4tiS!X%TdgVg*Ks zh8hN&eF~R15fpeYHP*}JvZ?3cLgD<4ZK z7$;8mV!md3a#~mN?;UDB*J1K%B;-9tLdYMs=SWCx(`zK;LlxCdB*g2rFGWIHT|Q_J zT7HkWL%-h}`+V3BT4CUIdgEbxS2CWR)%OMoDU;vy@FGpC>s<>8@f@#pL&|27m*MOIqG%tWIQ(cB=VY}DzF)OO;ioq~!^?Rc4`#qnB!LHNWuIhVodK)%f_-~KG zAPh&XA)J=3d3fu!y*C<_VH-Jb`ztnY+u6<1X3uW!sN>!6*-gaaDaPo&m#%&^2W9+% zRK9#pQ>6HWEh~EP3&Sy-fU5lz%gWbs@I<|QZXdm=s0{ZpwjBbRCoyNIJi&Be)%Xng zV;E(y5vn1pdhmLY8Xl<-R4_%B-a=Tj8mfW6e7+fuyeNL~dhyRcRWt1BGce6@rK{q1 zHSg`L(#!Rto!a(!K?U1w$+Qf`&zB)@08w)@09L5zVrKcKWehc_Btc<@fb3%7TN7w+7wCp+Zqmcp~!wNpag-mLkW zF7DTi1bjWm_Unf=zMdcZH)=k=Vx(p)`kL$$q&AnKy z>r|S1ExoPY_RALV+T#F%Bo+E4^M6gB~5oX&iqFQi7KXv0kmp(-ft zrHliNwTr0%KY@*?O@B!NDJeiI9HL&)ToIrWTLQ@_%>Jli+|}_YS~8wa=#DBh{wamm zwOR8p58RV1X2i9(tE#j)!WQmrx2@xW3 zHcEUJQaKBF3zRJck0@Ot5cP>h(PL<6%Nbf*EekN1DNsgXQ8x)_urbLBn(>$HZo(s* zeDqT|^_y z0x`NQ0Z7y;RiMdF78TZeOwcdjw8Zh#SFndiD}2?}5^RBJWO z=^TTeYMb1&i0W%4SWU^O;U|sB}b`{YJ&r(+cY-R%$5(Pi35cDNp7xC~{h^hW&QEMuw- z?0rh`ZSiEhLS?;Wd z48BUMz&y%risN12bB)*jiD$k@&P8VGF0sKyu;I0S&nSlbXEtc|ohRs#0FG*U_v!6Z zq+j_MJ{ZkuCFc3u#yU5bgkLX{9AvSht0B=#1GZ*eZ01pEa=RF2bG^xg1{yCI;Y$-M zlcBux`3VjA#7JJpAATnwVZU}c*Y`GRylemPF7kTrIo?$ppBnG_P(`&9@9MYROYtt= z@Av&~;CW-W@4|+-+v$b44c*N##axw%BseC0! z)<6d>`f4+WJ0eq&2w@iFF50y$a1FMg__9e}9?0O>^b~YVtBHsOL1^eE$5UXFS;Xoo z`OfoEJM1%cju{UviU)^ukUqg8AdEQ3+Lqd;Fl8p6uO)y+3HNXSp@oBTm;8Fd%_4he z-UBG23SfI8tSHKsFIa*oq;zH7)*T$y4i| zZFYEi20j1$-Jc)Y0HhVLoIS=GM-7IM6b1lJWFY70O-IxSh64v~i??x>h^WcLe2Tdd zZH!>d4UBAFvL2hk!Fo#AJ1j$}v?{C4KrF`u$yp~qVlvWVF&2Amf|_D)d4y!lfN4<+ zJX^9uXwo)*RKN@{9B7G>dWPJhfS@PvNa+$6uF{(w`yE!=W{U$E>Uu6Tp0=#ZAEk;A z$7{+J`oL7CxIYXsm<<}Hzw_n@!YhaCFt6jH$eOGISyQgmTmaaL=XKfPpyhbKI%1_} zV%s>~a*Bv&+tdxm*&bx9tTKYi;<|+8GqJCYmIs*WGXzrsO2!U}xzx-N=C_Q-O@egh z78x0Xy#T)Aa0^LW0cjb3ou$T1QnQBUpgP4NT>!fDi7M}e9u zDr8bjIg1c+E~JaOF3nV>AR{ouk;*{9Y{-bT(IS%Bd{fvnO5l4gqKLwYn`JOnBscs$ zhkaZn=s2=H1NDf5U{1jH(zgrP7412)Oth#zzE~=Pmj#=GEvz$WzAFT4P&<)p1gl1{ z_7TBq4fY+us*O*LV11~f+J#{ChP}aM2v%nx{6Wue4@6J2g0VmB_+yuMdcF3bGYC9) z2ZFV;`u-wV*9tx6wVj^#W3iGLpOWc4Dd zb_7rO%jeN17)Rw_6R-Lh^E6SWq$^$2qT>AJpRNvDZ7}eufKx2=cAvgX{*8jJ%O3NG zSqBx4+G+={7kljWF}ZyWKlZLLxm)OkNbKRbMs?o%%|R8EtKrmrmE0$3OwZ9%N8z86 zWi7aOlEbmzGOCJ1OH?lu|Sa4{wogd}Wu?E}0?L@=u%U=4?>$nGF~K)^wXwX8#z z8RW4^#3lw*z;V^-WHHIjN#(Hn0!}5N8Nuc`;dq@!*gG-nko6j7Nf0GFycD4%PUUch zS-Xrg;kbXK%Y|B{lPQitG!xate@mp&T)bqN-${}hnfC;Nm@p-<1deQN$9By`1jkQb z60bqk5N_rr5L@zqkUm6jO3cjyL)FVv*#LnfYXf6m+_Y%+*l=7o5CB=^(}>ZjYb3*U*2Bc z^|gno+RFy!3(hXG=J;lDJ$U1L4;Bl%RQtI`;BC9DUtQq!-ykiT5f*XXGKxldIWQLI zTdxI@4V>eh=N#Y3uJ+RBB2YXh4~B_tm_S+=P=FCbja7SVhu4Nv!=ChgQnMGceI zFzG(Rq+V~|Vba?8)G+CXDym&DY2WQ!nxLt}JwFJ;VXx~C`opo$gF%n?!?r){^m#AX z71(lT^*w<}tLbbzqb}xK>R%Hk?cY>--6}nA873VLyBAT?4rSNvp1B@oXlmauO8OAb zL4=ZG&LRTY4%X5b<&R1rEU$89ob-J~!royb9&+T`)ChvzleX%tcFpg@dQinzPVFkq z(dPgrlGn1_Gg)qhS#qgrRuL%Uu<|TcKi~)YhCN0z4}Xjp{DGGg;OXjn%;`d@TxPIk zOZ6*~Y}Lrh`=|=PJWaNXt9N}6`sF!U%3WxPDkz|Z X(VqLm>SbW>{9*qOJ`$9O{P6$)?e=jl literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..257e0db3a47f976dff3deeade640e652f9f2ac60 GIT binary patch literal 5143 zcmV+y6zJ<8iwFo|V|-o!18{X>Y-wX*bZKvHUokK)YIARH0PS7da@)9;eV?yD>0#=e zDqRTPMEN|)vK%XsZI$h$yHD3t3M3&5Hbv-wq#dUx5A!+m9rHRLH57hJ?g zp5RgRNP_z#K^Ht?F&9tpdMX-{Io(A#@@8~S(+~S0jgLy*{Zx0fl*zyNjMTSp%?9X= zlLb^4joF+=F_|wKN7!;K&9X2ydZrGAmYOW>KdJY5y6Da*3F6t&kp}-9%X4~D)9u;5 zr|Wjl^^Go6qhT4E4VmDA-LjAbBly7YjXhug52zI$1k{6unF>CKPyErir%#XZ!QB_v=|M#XAl`<$({XqK|kY|Q^0KP~7p z&9(gXTT_|yZssf-P?|EI=9ZJ@yaBCdw=}(l@eOv4uy&;BZKr4avr$OLf`rkOi#b_& z!)R&F4YrR={Ro?FtBd6u^b1I5ZSgy(mhbY~|v=&$vh}e+oSzujny-WKk21DG5W0hHd!5@|Q9xN<>v2QlL8T z!HeX*Np>yJFohzpyn@r-)?CqbOjy`N?-QTJGjv{2RZS{YN%5s7eJ@R8{z#!zgPCAa z4DP8^RZXgtO3E*F8GGq^#~+*Mg0cYh+gFuB^sVfFMMBV`Rg}13MF9-o+|C{QR3c#V_6v^o*xR$11mqQmsU3IfW`Flsz9}_l{J3 z?opp~?h`s&LmIUY>g^w{WO8wU{lVjx5KhP&0AhV|`ejooK zCsQ%Z?lV`Btp>8cJr3hPbCWooA!jq8v_Z1DP}W2qd{v}V5-H)zSvU9@n3uELSzHQ* zr3(0i!elJCqRYCL$t;X==LD?{(8RsA zb&R&wQ3kL_&s%A(YMw7A!;7_?T`p*d`soYb$wv02&vWiSGre39_*ORg=GA2XbpfDf9`;b3|=&IUqYq(3Zk%&M-M7eGIU)nxrsYySNzsT_%R%>a; zgfax&D^~A~4dm#xfg?4wc#Wxr;If;r@Yb8bm`KT7F2o8~y8f$|2wD86;&+jHRqB`P z*E=rPzJV-%u6@fGDr*$VJrN8%wx0rlPPc^})5?Fj3K_Ew-a& zzro+Kg`zaEEB}1pkc~z0MjvwXkVfT|-d;^|Rz% zsQ`@I4#s;^0ln9v0?PctqT_lE?=PedME#GU>qHF7aZ=H7vc{BTGCL#v zYvo`BR+v!R+nb~mTk5g*q8hVO_Q7p`x8e=zEn2%u!g~3_~)>CLSZegwEQ<$O2 z@d#aG2Ki&5MoP0|69LX>kUtSMS)+%ri z@CXJyMKt^E;ZO=iI|w3hAI+Rn@ZHoDf5TCfUg0UNfpL8c{h+P6 z8aAkb9o^|V8g&)NP(O>_iDS@uN2b#@4SQdX!Fd&mVI@2Dq8LuO@S8A7&cYHe&7Vrd zR0j>g2E0)@gpog&T2jda$%Xe^w*3Vc zE){X9rA#Q*NYR}MhE*+RTs%MvG|@kK04t7C&+@rclgpNUQ`@s?VsN_m&^jLil2p29 z@mOu0Ote(BpC9-{gy2oo4}C!@ik8on7Lt;snmsv20)860%|{U>LFJIq^10G=QqfY; zOP)Z{K#B&c*EDF820BaTBt%0wEs_UrAUmnacqB--*-c=QGAG_LZ305zxxl0lpQ2xf z7p)TwYxoHD!OK0+0QFd$@SM^%d_^N4xp+ypN$! zufK-(8ILBD(Z%5M<_7&T8jnZ)^K10)Fqi*_eEcxFxK>sC8stxo3QDgxAi`(-fkYlm z5Jc5Q=0|#LW#)T%7sUuEM(EEnLeIc)Z;BaGV`Lqfq1`-xW1TM2u&uRp)3h7A(nTiz z4qIeuX;(?fqI6tQWRW6^YExG)l0~{^K2H{5@X)qFv4g2Wd)Rec+xKi2QoL^O<%1|M zyj~-+C_NZS?iM7T$+MA0TeI65?mfQ>R%B7F$s&0G#BSTp-dvIXYUfdk?9_`Zny{~5 zdAgCmAKua`yt;fY;nhO8aTss(3yh|cnYfE8YKq4Sox{Qs&EF3yKI6gMC$pf5uGl>h z6>ehrTm{-Q;l{4KkwS|VT3jzbxQQ0`(Yqk=U~Sv7TN79YE@t2!&?w?_Sb;|rO<-XI z9t4d-KY@uiy2;LJqX7+q*2(g8!C)qM$UM{+@l3#UR)&B=1FL&HhSeN8hn08br$+{! zC5d5qNHRqE^2^gL4bk<2dJ>108U}p62xO^Zg)F}<8&k+~ZII>R_{+2y^Pau3mAp5<{Q(VuUQV(39%i+hj+R?+jM=;e;KFW{(60I zdaOI0^TBZ&oPvQ{SXcOb{VaJ`d|t@Sl;NKqO89Iexae-X$pv6}82p#i~qH``5Gk%)|pp~28W6Hu- z#&Y;5{Z=s+pm-*Day!fS!)GiBlqmo=?W5~C@o~p$$^RQUZSjSwI7n~Ip4254wP7um z!H>>IgR4s!{pj-DZ>m~fgSI&5FyT&jKBku1$_Y{s0Lp691tkoquM$X>h^vdv`<1k|soy#6$9qO*#IvsDeb z%z;4q3A?+GW}J(#iN@6nrB{`dVh4igpO`!xP~`;PuVdSl0|EO6MDmEF9IAZ7DlQKK zJHzG*)K;K&y?pj2sC`AigwrWIthW*&_vRO_Ei*#78|IdVTweMb<@@L5WFd{}-}hkB z9&;D!fePdj85t&I4E?R|{vF*4o-C02gwFfplTqvD@)Vt?9FfAU-;l{D+`2Zn^)akq z?;qve=9BE_>|C0v)xR9J9t12*gIAq1$d{X87^#Zs z*oMU@qTna*m#@c=&umu?RoL1K~e%+ZZz*sX%wg4Awfeic9D0kf}wm`83b*ihE z*@Eu2;5as>mQFpVt9c%b3>MLdsd<*~IMmu9IL`Lh>%$gkI_?lpm+W=-PSFGPGvl4F zPw2IKU43tQ;PviLSmvi*^uVu46J5`Ea>q!z;xF*}N>T|?|C$^~5sVqA`E5nRZ|ffi znFHy8Ux;}@GQ*xJ3$sR_yPid4WEXW(#<||BJSa@6KI)YDzD)FH)L3U9`46# z)%_Q4Qy-iS&qss)SngL>Rr(rOe;*b^PjQM!q<%1h;VuLBRV99*KyDx`}-m+MtmFQd!7Z6RG; z!=sjNldf%8MvvN#Ni5HCsfTrahrQ_AU#}6mY&e}z%q2NxXm(fO%k?wlowuKx?QYlF z8(;3d+V=Bhe(HrU4`f)AKwfQo&S0#iK?$9f3DrAiGw}m)W}l*fpn*ufHqoh^zlVP1 zI@L-_1*^^E4}_jio_qqT1g(T$&z~y$(n07z(D{sc_wzJ*3@qY#h>oj4WGYG3*eK~6 zI|I;bH>&awCNxZHd`Dcts_j)KRIXJmx`%P{V}ODRrXNoS;{q3*cSz? zzs+G~6IeeMl*s9SihAF17RtZnswAMTNRB8z_0h>i|0vxNfA;YVQE*>j;ssR3w~vwq z6;m>2!IQ$m-9{0P3YZHt3M7N2p>Dae5$EUVu6LSZ6eCz8qg2%Dl zxz;fF1G{V3f5ZCUZC%&&mZO03`kD03q0f752Y2@c%6s_LLZ2`5Q!k+W4620&O>`sZ z0~hF~#6pXdHJmq&u3v41O1vpxno8`G%ht(5?7TL?;VJi%hQ(8j662cgvzK6 z?@N_=!m7GLerNDpfxZg#t(S}31bwsFaDG+jT;5+w0>tE|Xo|p$7Vs`juONjkIGYoN z?7poASIBN{kX?0!=#RWYbev*rdE9MBnXiIz6^#3Xud@99;#mQ@KehMxQFmEp{(S?# zU7Fm^_^s0|H#qkkyH`NCj*XWPZZ?1@3|C>ewW+I@Fk+f(Xt?V+ zzR`0v&34@B%NXwV*XxDh-nHqMMMx9DgFM8O)7I1lmUT4hUBhWuGEUdtAg=ye(KMFX zsTYQOPQDU(EWU~Qp)daDzn@BcQzp>={jaALe6x^05XC(wP!Qfk=Om69DRnKMtD2Ds@Tmoe|r$0NnKJR}!Z62Ps5UOI1y(MC8uiXdJfPc|b(K z|UbQaxupRGE+?Zs(6&)JNo{E%!-fR1y6a@?;*%c z1osm!R##2FF(XnjJpP~2?QFc+8JHb@Sx^Csl`tj&x=5$w%dNJ$x*bu0vxvegT`7MsQruhrG{h(8Fg{XdhsTv?~; F002Dh3zYx> literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/software/software_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/software/software_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..a5b8c8774c5650641feb3a3c257b4bc182777b38 GIT binary patch literal 6410 zcmV+l8TIBLiwFqDh>%_Y19NX?ba!ELWnVEcE^2dcZUF6FUvuL&l7GKXfm7Aq+Eubl ziu#|uz4F+e8E5Somp!|=T(XsdNKnEINoWbs9wnEDeY=mce%cS=*Zok}4e+02JF-2Q zksKp6u}u&_1L#Hr-Tfo*QzM{?9wnh@jDBkHDgc`ihGL3HuS1k^_~7oJ$_hM5;!9 zDx|!m(T7pWWl*LR}93)JQD%yLRwk-4HHQtfy=bxJmpsTVO z3>Q;8W>V33))=|nLD%Vad&7Y@=tHNGQrG!k5&WS4gV*iRsN;I=u*>{mXV?p!phpA0 zKXx42g@3&;>_8(#$|pRb(GgVeUE_hp{|cibqKNsxm@pMFRPj^e1Diuz#Ed0@pefci z!q&Ed*aeL;?CfbOf}MRw*d2OT|xl_Yau#Q$8z%VoK8s80Q11wNBG} z=MF|*XXJHTo$e6n(nv8aqacAGhhr+)pU;k=%v7n_Xm=OLwSZp0p|36dBTRQgg+dyz zMDdWb6vvA4(Q>v?r_=8CdwJ(_rej+MJFdIqw%<92vY=f8Y7o%OpX$e_C${-jX)q`)OT4-4r6}I35X_6@xNs>%H_J=w-_Xa7&pGs6*Czkc0m&rXicYeZe0^rN|^f4ZB;1ELRygJ zs2Lr%VTb*`!wwD4#yG2SiWo)3;xSDp0v1fe2$Kq`E%kG@ppI0qmgP@+jWd7y0wy;f zXG-gCqu_KwqvA|eKC~m%`UZ21aXA5oN(qbKR8^UhiUjz){9P_UG9$OJ4vwX!TMvx<^f`}z_NOq-<}@6S!&r^ zSnoA1I<5proJ8nh`Aer=Drv<3d;mRXhi>#N*U zphWXz;=>MS!f1?}qC)d6zxA^eyFV}9r&-9gkQQQXRI^JzYvE%KuoiAh*pMuOOa@P8~)g)&u zo>x|C{#2Q$c~kRB0{33jFe7GD356{ zBGcM@B}%|QMI%y7={ecTs|d(=PF^>4HSFU|TUlfZtVzX>^9hc?yh%igxz0>3*sd_J z%n9ya64=0Ow}@`$LL=Z=&LRw61q3DSQowr{;sh)bIU?|z1R2u}gDR_L?3zL>^8}} zy;o%DG)bq|-s$w=kIN}^3#yKyLY4^nAp@EQ1ZJm5rEc#cN8tU9(VkIg63R$EvOJk( zih$+CknxnD-`hNHNrrNzM{9)Rh-PL>eN@}n9H>)M3wbIPiFkrl1?Z>*DaRK{7LTES zFm}5CXu?UdrOQD#ftEYiZZ74ljq*a)4&=nnXep^NRP%5P^2n*kqJShqAy*>wT;X6) zGtx3f;eck==2cA!z9MCBZd8u#bSmOyO?4rHJZ(o$B@m@hM3J~OLx6f!vt@ZwZ{c|1 zFczXx4S~*Dju^_;jlzLQey>pUV8%o`wg7X8fen)QV6bkF2n;RhK)Z=c%>R!5?UGij zwFU2d5oK{Aw{>b;zk$Cz_yHMo=7Ij{kzSsL8HRaUAD+P0M>|;-^8iF8(aHiRe5Oi~ z+A72b67?v5g)?@HbAu;ZU5S+c2-*jRbq+&FUa^FwpzU%M2NDUKnLrm4BWWW&W&w?Fm!DU_xw67$&MTuwVHM#ZHx=P>t}ZK zs%X@Ei}SU44~5mN=JN!X(MnYsrMfU(4qnS9Ie4w$2XwQEEUBp(7%|v!NS0ohR=wcp z-mSxjETIL>#zVLX9pGa(@SJmTY&XK{iy_u{nK~4(lfQ=7A?jeR>e!= zBNBtG@>z5nH-W`a6BN-EZ2=2$TLA(Hxse}>xnY!R%YTZ`H+C4NueS_`)PfOYK1_{; z7VFuCJB~~hN*Tx%!-U%rtVOnL)aM}^*I$mLIXxRmv*3k?gBK*V-~|g_cmm*sJX*Fv z;03F4EqLMCvvd`BVbB}6cY+rNZm;dOy*BOk+;N``#+_leJ$47!Tr-x@Je||^y-W-s6@;*ucd2zV^_T-SfeRKRR{3zeH zz8j~zec$z{HYLNRPj)!Tte$G9*QW9nGrUT8d9dpqTaK^y#t_Y z+EyekV8^(;1vezg)>@aTALB67fU5CUHBRk=yqAJ8rwvH8I$&7g;)nc7n{)sD=54 zO>!b4a_Ga40hH+uy&bpbw71JnBhbkNVyl=w24ZiA82NQTR}tE;At5HFjT#oyG;#SG za5(hNBfrU*sY?Wv)WAcCPr9HyqA)LV+yX*zpdyWiFvSY!#dW@dF?+ zP_D|Orc{xjh7aHg#_ysfcYoHb1yKJbYLOv5DVIT)&(XkOwwR5|VlnlP~v zgQCen1!IPoiM{jvqoW*MgRl~uFGY?iyGt{q4dBw_ zl9%C$A7w!f=F(6P1Xz^2JCTeEX?R&dy^6FqC2q-|tGr)tZiREza#pRw5d>-(ms2E) zO}L;bhq(g-he7*E_b_DfyS_EH5!t=cjWkR@BAI{>I3}mHGV9h*h=(n zAskSPUSPB`d?S?^d2AKr2D%lhN@pvS8^$_F_jmfG{@eYXk;ys^=q3%+dx&{8oUKh= z>{Wz|PL9Pz#F8PnHJ)p|13Lk|aH8xBqb|82BgO~@10rwALO^F3IjNCkmpFK+^rqqx zCPOm6rHu%MS}t5C(IQ7?Vbm%L#^BE0P7rc%Yc=IqzQI}fAx zXDb&lZK8eB3U>A@uV80ieg!jK0E5(M&xy!CZ~@WYLDiaDQ(L^nFkdm$Q0$L-5Ro*t z4-)=>Dccr+`yii%J<7%ca4i6L6U4>>aMuOE?QaqQ*Q#6#z-#tyPoIV3BaYy zcilnH>2^HF9rt>^<97q!>kj*E+8y9%{UQK&Rq@ZeKmX}r9awJqcby*Q z4!;lp_axJwBywR6)#gh_YV*YdwaeTMi^2UA3@+?OuKy9?7p3^sbM9`i7=?90MGBV8 zra)f{jzv|iAL|CYnJKgQ&rRWoPtkZ{)N)~+TnOnTlSJZ;7YZy(%Z$yyww)flA_sM2 zs!4S8P9((MMe)SA`{{R4^sQ^ddXud;*?NofgF}--1lzA zC3-2IFL**z&UU^QlGVs9Ce{c-k&>GCAZo8*^f_&kmmn-G<^G3?T0Nnr(9v2>NMWY`%D?(~&-!=W>zK}dsP(02x&GYH!h2C?sXtV5kuzLHhNpQEqD z8FhQDLHn~Gec5pa1MjY*FTWV_?gaxOjaUkXw}3&b8Q%G0Wq21^AL%YRf!T%o_^%mF zc1}4?{)*aSVh!r23q8_dvIn(f!Om+clAuY>#ca9&EG(WYYV`t-bep^eO~I6El9zPM zS#3Pa=ZajO_mM6W?ekHJLC@6n)oeGmE(_b(I#PXN(57DiEnuCY=zLa482Nt1-Ml4C{BAP$fykB|dlGf0Aza9a7^&?Pk5my$; zw^j@p%Y2&7g=NEf*^pwN0`q}2F;l~GWHUg=V>-XSlCX0~EJxPLk?SkptY+lm`f750 zrR*G8`!T@g6i3U2wJzaFASN{Cb~3DY!N`>02_6jtoBuJ`0ALeZj%)@be;;;Khf?tq zsu*!w+TU%1>raTK_YbbuS=}U*yr6R?IZa5+)Kr+eOo8?28k}PUJJknK;n7~XK{{VP z{cFM!eKu4;sKQGxO=w*zHj@p@#9uEuZEs<-(P{gog#A+Dw;KJ=!%dqM{kJODM*p8X zOIJkyy+PmW-5LF7eL5VsUD^&A3-OFvZ!mN^UYmwqr`_o^Rz&|-6@QM=e{a<7w+4Op z^GE+Zx8G^3Ir@LVFu=(2p!~NNtcJX=C)Hz}QSOU=3< zSdqQ)xcL7|EYu|bO)t1cE#%4Nb45t!Y&zx^;hQbRAuCG#iy4Si{5=PlaJd$uif3%F)1V-S^;M`tbxhc`a zmyx&}i;s7+!^RLdk&zGkKo@;(2T!sCdU&x@ZsZbu#Sh-hGs7>18TMt$CsVu^iYW?N0}^xOaIpuL`5iZN zsTlf_d=gSQg+AyTY$Usp9WV8SreIrB%>rohw7C2Vb`5wG0*z`GC=P+S8B4_IA)QU< z_DkXiO_mjbgRTu6*jTNN)!JC?Z#Gt|g9DostFq`CIu?_)ao-rN4QQ@7j%^==Ug=QH zY>Mt^6W)ZBOExAaN5_96M~U7im(bH$OMac>-8*(yW)0zo8^TFo z4Pk2tKLtbB?7TL~5Vk7U8p6+=rK=3#uGhWW5C-dYIHr9sY=<7}4?7Gj*?}{5`d!a+ zSh&g%URC@V8p0iS!#r~COq%YE0q{m8QJPJ8f$4B?mbBEF$=UVQWWkYe#{Tv9$&Ewum&!FBcV5{G3hEu~U;{R9Ea1!+9T$1V#^a7C zJ3c&n`)!lFp=t`YaIJH-l$!H`rDEQA+@WL#*-T_?o}=fFGk>vDc}fjCZ}K6z`yTydn~nWu~p0Mu+EDIZ0YKkaiX&-(5I_M((+=R zyqH{HiS+s^yuQ*MjLr8nmLY3n$OULF*v?r!wSHFMCtkhmf((}m%Zbf^ z6CW(>rSw9smvqFB`uf~KNk#;={uzr1>|aw7QL>=dG)DaQIdAiQ0LH8$ePz1XSz3{F zo|q>Y(s041B#z_gIw;9E6Tc^iO4}RnMS8cK*r3)XN|?Q~*ao#My7P&MjSXt81>LE= zsGAhjvMSdGwVpdmR|T~OgMoKvP>c4c&%B^dgHD_J1J`4nVYeIhdbHE&c&qL#URnGZ z2DQ5FQM=vhIiDZh>GXU3USsVi@yPcq``9Gk()tY``P0H(Bbcx8NK_|<0SrUdBun?U z+`O@+$l9>D&QA$doGSqffW*%XU#74d2088ccsBt#ALJ{ge?I=LG%)T?Rdvg4rh$25)DX!f(d)iT^ASnY3RuKS-vm5oY~f*6+0mVq^VY>+yS` Yp=JGEPkZh>5x>{}19s*i?oIsw0IUmUV*mgE literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java new file mode 100644 index 000000000..0e39090dd --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java @@ -0,0 +1,473 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.QueryInformationSystem; +import eu.dnetlib.dhp.TypedRow; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import scala.Tuple2; + +import java.util.*; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.PropagationConstant.*; + +public class SparkResultToCommunityThroughSemRelJob3 { + public static void main(String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils + .toString(SparkResultToCommunityThroughSemRelJob3.class + .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + final SparkSession spark = SparkSession + .builder() + .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; + + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + + final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + + createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + + + JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); + + JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + + JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + + JavaRDD software_rdd = sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); + + JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); + + + org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), + Encoders.bean(Publication.class)); + + org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), + Encoders.bean(Relation.class)); + + org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), + Encoders.bean(Dataset.class)); + + org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), + Encoders.bean(OtherResearchProduct.class)); + + org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), + Encoders.bean(Software.class)); + + + publication.createOrReplaceTempView("publication"); + relation.createOrReplaceTempView("relation"); + dataset.createOrReplaceTempView("dataset"); + software.createOrReplaceTempView("software"); + other.createOrReplaceTempView("other"); + + String communitylist = getConstraintList(" co.id = '", communityIdList); + + String semrellist = getConstraintList(" relClass = '", allowedsemrel ); + + + String query = "Select source, community_context, target " + + "from (select id, collect_set(co.id) community_context " + + "from publication " + + "lateral view explode (context) c as co " + + "where datainfo.deletedbyinference = false "+ communitylist + + " group by id) p " + + "JOIN " + + "(select * " + + "from relation " + + "where datainfo.deletedbyinference = false " + semrellist + ") r " + + "ON p.id = r.source"; + + + org.apache.spark.sql.Dataset publication_context = spark.sql( query); + publication_context.createOrReplaceTempView("publication_context"); + + //( source, (mes, dh-ch-, ni), target ) + query = "select target , collect_set(co) " + + "from (select target, community_context " + + "from publication_context pc join publication p on " + + "p.id = pc.source) tmp " + + "lateral view explode (community_context) c as co " + + "group by target"; + + + + org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); + org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + + createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + + updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); +// + +/* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); + + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); + + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla +*/ + } + + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ + String query = "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + table + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + table + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + table +" t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + table + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id" ; + + return spark.sql(query); + } + + private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return toupdateresult.map(r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + contextList.add(newContext); + } + + } + + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }).filter(r -> r != null); + } + + private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r-> (Dataset)r) + .map(d -> new ObjectMapper().writeValueAsString(d)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map(r -> (Publication)r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); + } + + private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) + .map( r -> (OtherResearchProduct)r) + .map( o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); + } + + + + private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ + return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map(c -> { + if(! c._2()._2().isPresent()){ + return c._2()._1(); + } + + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for(Object cId: toAddContext){ + String id = (String)cId; + if (communityIdList.contains(id)){ + context_set.add(id); + } + } + for (Context context: c._2()._1().getContext()){ + if(context_set.contains(context)){ + context_set.remove(context); + } + } + + List contextList = context_set.stream().map(co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + return newContext; + + }).collect(Collectors.toList()); + + if(contextList.size() > 0 ){ + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }).filter(r -> r != null); + + +// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) +// .join(result) +// .map(c -> { +// List toAddContext = c._2()._1(); +// Set context_set = new HashSet<>(); +// for(Object cId: toAddContext){ +// String id = (String)cId; +// if (communityIdList.contains(id)){ +// context_set.add(id); +// } +// } +// for (Context context: c._2()._2().getContext()){ +// if(context_set.contains(context)){ +// context_set.remove(context); +// } +// } +// +// List contextList = context_set.stream().map(co -> { +// Context newContext = new Context(); +// newContext.setId(co); +// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); +// return newContext; +// +// }).collect(Collectors.toList()); +// +// if(contextList.size() > 0 ){ +// Result r = new Result(); +// r.setId(c._1()); +// r.setContext(contextList); +// return r; +// } +// return null; +// }) +// .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, + JavaRDD result, String class_id, String class_name) { + return result + .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map(c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())){ + if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); + //community id already in the context of the result. Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for(String cId: contexts){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); + cc.add(context); + } + oaf.setContext(cc); + + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair(c -> { + + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } + } + + return new Tuple2<>(c.getString(0) ,contextList); + }); + } + + + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ + String query = "SELECT relation.source, " + table +".context , relation.target " + + "FROM " + table + + " JOIN relation " + + "ON id = source" ; + + return spark.sql(query); + } + + private static Boolean relatedToCommunities(Result r, List communityIdList) { + Set result_communities = r.getContext() + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + return true; + } + } + return false; + } + + private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + results.leftOuterJoin(toupdateresult) + .map(p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()){ + Set communityList = p._2()._2().get().getAccumulator(); + for(Context c: r.getContext()){ + if (communityList.contains(c.getId())){ + //verify if the datainfo for this context contains propagation + if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ + c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + //community id already in the context of the result. Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } + } + List cc = r.getContext(); + for(String cId: communityList){ + Context context = new Context(); + context.setId(cId); + context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); + } + return r; + }) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath+"/"+type); + } + + + + private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { + Set result_communities = context + .stream() + .map(c -> c.getId()) + .collect(Collectors.toSet()); + TypedRow tp = new TypedRow(); + tp.setSourceId(id); + tp.setType(type); + for (String communityId : result_communities) { + if (communityIdList.contains(communityId)) { + tp.add(communityId); + } + } + if (tp.getAccumulator() != null) { + return tp; + } + return null; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json new file mode 100644 index 000000000..27bb097f9 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json @@ -0,0 +1,40 @@ +[ + + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json new file mode 100644 index 000000000..f61db6f34 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json @@ -0,0 +1,50 @@ +[ + { + "paramName":"is", + "paramLongName":"isLookupUrl", + "paramDescription": "URL of the isLookUp Service", + "paramRequired": true + }, + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"as", + "paramLongName":"allowedsemrels", + "paramDescription": "the allowed semantic relations for propagation", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json similarity index 67% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json rename to dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json index 034205b88..90b5974ea 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json @@ -28,6 +28,18 @@ "paramLongName":"hive_metastore_uris", "paramDescription": "the hive metastore uris", "paramRequired": true + }, + { + "paramName":"wu", + "paramLongName":"writeUpdate", + "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java new file mode 100644 index 000000000..0051ac01b --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +public class ResultToCommunityJobTest { +} diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java new file mode 100644 index 000000000..9c4d37fb5 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.resulttocommunityfromsemrel; + +public class ResultToCommunityJobTest { +} From 676ba493241c12772bdc336204065e2c2d76da4b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 21 Apr 2020 16:09:23 +0200 Subject: [PATCH 143/259] changed pom of dhp-build --- dhp-build/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-build/pom.xml b/dhp-build/pom.xml index 3b54b4e61..040842000 100644 --- a/dhp-build/pom.xml +++ b/dhp-build/pom.xml @@ -11,6 +11,7 @@ dhp-build-assembly-resources dhp-build-properties-maven-plugin + dhp-shaded-libs From 30e53261d0ee9c6d7e6561fd9c249dc4a48cbece Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 21 Apr 2020 18:00:53 +0200 Subject: [PATCH 144/259] minor --- .../dhp/selectioncriteria/VerbResolver.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java index a8df69ea6..f6d5394e3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java @@ -1,25 +1,29 @@ package eu.dnetlib.dhp.selectioncriteria; - -import org.reflections.Reflections; - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.util.Map; import java.util.stream.Collectors; +import org.reflections.Reflections; public class VerbResolver implements Serializable { private final Map> map; - public VerbResolver(){ - this.map = new Reflections("eu.dnetlib").getTypesAnnotatedWith(VerbClass.class).stream() - .collect(Collectors.toMap(v -> v.getAnnotation(VerbClass.class).value(), v->(Class)v)); + public VerbResolver() { + + this.map = + new Reflections("eu.dnetlib") + .getTypesAnnotatedWith(VerbClass.class).stream() + .collect( + Collectors.toMap( + v -> v.getAnnotation(VerbClass.class).value(), + v -> (Class) v)); } - - public Selection getSelectionCriteria(String name, String param) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + public Selection getSelectionCriteria(String name, String param) + throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, + InstantiationException { return map.get(name).getDeclaredConstructor((String.class)).newInstance(param); - } -} \ No newline at end of file +} From 3b2e4ab6702e7f75fefda0d7b2279e85b21fbe16 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 10:00:10 +0200 Subject: [PATCH 145/259] test for bulktag --- .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 833 +++++++++-- .../CommunityConfigurationFactoryTest.java | 3 - .../communityconfiguration/tagging_conf.xml | 1256 +++++++++++++++++ .../{ => no_updates}/dataset_10.json.gz | Bin .../dataset_10.json.gz | Bin 0 -> 6883 bytes .../contextnoprovenance/dataset_10.json.gz | Bin 0 -> 6888 bytes .../nocontext/dataset_10.json.gz | Bin 0 -> 6755 bytes .../dataset_10.json.gz | Bin 0 -> 6968 bytes .../otherresearchproduct_10.json.gz | Bin 8073 -> 0 bytes .../otherresearchproduct_10.json.gz | Bin 0 -> 8300 bytes .../sample/publication/publication_10.json.gz | Bin 5143 -> 0 bytes .../update_datasource/publication_10.json.gz | Bin 0 -> 5257 bytes .../dhp/sample/software/software_10.json.gz | Bin 6410 -> 6727 bytes 13 files changed, 1957 insertions(+), 135 deletions(-) create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml rename dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/{ => no_updates}/dataset_10.json.gz (100%) create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject_datasource/dataset_10.json.gz delete mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz delete mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz create mode 100644 dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/update_datasource/publication_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index 4a45d234c..d04504253 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -1,13 +1,24 @@ package eu.dnetlib.dhp; +import static eu.dnetlib.dhp.community.TagginConstants.ZENODO_COMMUNITY_INDICATOR; + import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Software; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.mortbay.util.IO; @@ -31,7 +42,7 @@ public class BulkTagJobTest { taggingConf = IO.toString( BulkTagJobTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json")); + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); } catch (IOException e) { e.printStackTrace(); } @@ -66,7 +77,7 @@ public class BulkTagJobTest { } @Test - public void test1() throws Exception { + public void noUpdatesTest() throws Exception { SparkBulkTagJob2.main( new String[] { "-isTest", @@ -74,7 +85,7 @@ public class BulkTagJobTest { "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/sample/dataset").getPath(), + getClass().getResource("/eu/dnetlib/dhp/sample/dataset/no_updates").getPath(), "-taggingConf", taggingConf, "-resultTableName", @@ -92,142 +103,700 @@ public class BulkTagJobTest { // "-preparedInfoPath", // getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() }); - } -} - -/* - - -import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; -import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import org.apache.commons.io.FileUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.List; - -import static org.apache.spark.sql.functions.desc; - - - - - - @Test - public void test1() throws Exception { - SparkResultToCommunityThroughSemRelJob4.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample").getPath(), - "-hive_metastore_uris", "", - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-preparedInfoPath", getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() - }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); verificationDataset.createOrReplaceTempView("dataset"); - String query = "select id, MyT.id community " + - "from dataset " + - "lateral view explode(context) c as MyT " + - "lateral view explode(MyT.datainfo) d as MyD " + - "where MyD.inferenceprovenance = 'propagation'"; - - org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); - Assertions.assertEquals(5, resultExplodedProvenance.count()); - - Assertions.assertEquals(0, resultExplodedProvenance.filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'").count()); - - Assertions.assertEquals(1, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'").count()); - Assertions.assertEquals("dh-ch",resultExplodedProvenance.select("community") - .where(resultExplodedProvenance.col("id").equalTo("50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) - .collectAsList().get(0).getString(0)); - - Assertions.assertEquals(3, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'").count()); - List rowList = resultExplodedProvenance.select("community") - .where(resultExplodedProvenance.col("id") - .equalTo("50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) - .sort(desc("community")).collectAsList(); - Assertions.assertEquals("mes", rowList.get(0).getString(0)); - Assertions.assertEquals("fam", rowList.get(1).getString(0)); - Assertions.assertEquals("ee", rowList.get(2).getString(0)); - - - Assertions.assertEquals(1, resultExplodedProvenance.filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'").count()); - Assertions.assertEquals("aginfra", resultExplodedProvenance.select("community") - .where(resultExplodedProvenance.col("id") - .equalTo("50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) - .collectAsList().get(0).getString(0)); - - - query = "select id, MyT.id community " + - "from dataset " + - "lateral view explode(context) c as MyT " + - "lateral view explode(MyT.datainfo) d as MyD "; - - org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); - - Assertions.assertEquals(10, resultCommunityId.count()); - - Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'").count()); - rowList = resultCommunityId.select("community") - .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("dh-ch", rowList.get(0).getString(0)); - Assertions.assertEquals("beopen", rowList.get(1).getString(0)); - - Assertions.assertEquals(3, resultCommunityId.filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'").count()); - rowList = resultCommunityId.select("community") - .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("mes", rowList.get(0).getString(0)); - Assertions.assertEquals("fam", rowList.get(1).getString(0)); - Assertions.assertEquals("ee", rowList.get(2).getString(0)); - - Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'").count()); - rowList = resultCommunityId.select("community") - .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("beopen", rowList.get(0).getString(0)); - Assertions.assertEquals("aginfra", rowList.get(1).getString(0)); - - Assertions.assertEquals(2, resultCommunityId.filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'").count()); - rowList = resultCommunityId.select("community") - .where(resultCommunityId.col("id").equalTo("50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("euromarine", rowList.get(1).getString(0)); - Assertions.assertEquals("ni", rowList.get(0).getString(0)); - - Assertions.assertEquals(1, resultCommunityId.filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'").count()); - Assertions.assertEquals("euromarine", resultCommunityId.select("community") - .where(resultCommunityId.col("id") - .equalTo("50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) - .collectAsList().get(0).getString(0)); - + String query = + "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + Assertions.assertEquals(0, spark.sql(query).count()); } - */ + + @Test + public void bulktagBySubjectNoPreviousContextTest() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + Assertions.assertEquals(5, spark.sql(query).count()); + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions.assertEquals( + 5, + idExplodeCommunity.filter("name = 'Bulktagging for Community - Subject'").count()); + + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); + + Assertions.assertEquals( + 1, + idExplodeCommunity + .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + Assertions.assertEquals( + 1, + idExplodeCommunity + .filter( + "community = 'covid-19' and id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter("id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") + .count()); + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter( + "(community = 'covid-19' or community = 'aginfra') and id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") + .count()); + + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter("id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") + .count()); + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter( + "(community = 'mes' or community = 'fam') and id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") + .count()); + } + + @Test + public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyT.id = 'covid-19' "; + + Assertions.assertEquals(3, spark.sql(query).count()); + + org.apache.spark.sql.Dataset communityContext = spark.sql(query); + + Assertions.assertEquals( + 2, + communityContext + .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + Assertions.assertEquals( + 1, + communityContext + .filter( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'community:subject'") + .count()); + Assertions.assertEquals( + 1, + communityContext + .filter( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'propagation:community:productsthroughsemrel'") + .count()); + + query = + "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from dataset " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + Assertions.assertEquals( + 2, + spark.sql(query) + .select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' a" + + "nd community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagByDatasourceTest() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/publication/update_datasource") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", + workingDir.toString() + "/publication", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/publication") + .map(item -> OBJECT_MAPPER.readValue(item, Publication.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Publication.class)); + + verificationDataset.createOrReplaceTempView("publication"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from publication " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + Assertions.assertEquals(5, idExplodeCommunity.count()); + Assertions.assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions.assertEquals( + 5, + idExplodeCommunity + .filter("name = 'Bulktagging for Community - Datasource'") + .count()); + + Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); + + Assertions.assertEquals( + 3, + idExplodeCommunity + .filter( + "community = 'fam' and (id = '50|ec_fp7health::000085c89f4b96dc2269bd37edb35306' " + + "or id = '50|ec_fp7health::000b9e61f83f5a4b0c35777b7bccdf38' " + + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") + .count()); + + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter( + "community = 'aginfra' and (id = '50|ec_fp7health::000c8195edd542e4e64ebb32172cbf89' " + + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") + .count()); + } + + @Test + public void bulktagByZenodoCommunityTest() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", + "-outputPath", + workingDir.toString() + "/orp", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/orp") + .map(item -> OBJECT_MAPPER.readValue(item, OtherResearchProduct.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(OtherResearchProduct.class)); + + verificationDataset.createOrReplaceTempView("orp"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from orp " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(8, idExplodeCommunity.count()); + + Assertions.assertEquals( + 8, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); + Assertions.assertEquals( + 8, + idExplodeCommunity.filter("name = 'Bulktagging for Community - Zenodo'").count()); + + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'beopen'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'mes'").count()); + + Assertions.assertEquals( + 1, + idExplodeCommunity + .filter( + "id = '50|od______2017::0750a4d0782265873d669520f5e33c07' " + + "and community = 'covid-19'") + .count()); + Assertions.assertEquals( + 3, + idExplodeCommunity + .filter( + "id = '50|od______2017::1bd97baef19dbd2db3203b112bb83bc5' and " + + "(community = 'aginfra' or community = 'mes' or community = 'fam')") + .count()); + Assertions.assertEquals( + 1, + idExplodeCommunity + .filter( + "id = '50|od______2017::1e400f1747487fd15998735c41a55c72' " + + "and community = 'beopen'") + .count()); + Assertions.assertEquals( + 3, + idExplodeCommunity + .filter( + "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' and " + + "(community = 'beopen' or community = 'fam' or community = 'mes')") + .count()); + + query = + "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from orp " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + Assertions.assertEquals( + 2, + spark.sql(query) + .select("datainfosize") + .where( + "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' a" + + "nd community = 'beopen'") + .collectAsList() + .get(0) + .getInt(0)); + + // verify the zenodo community context is not present anymore in the records + query = + "select id, MyT.id community " + + "from orp " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions.assertEquals( + 0, + tmp2.select("community") + .where(tmp2.col("community").contains(ZENODO_COMMUNITY_INDICATOR)) + .count()); + } + + @Test + public void bulktagBySubjectDatasourceTest() throws Exception { + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject_datasource") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(7, idExplodeCommunity.count()); + + Assertions.assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions.assertEquals( + 2, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); + + query = + "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from dataset " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions.assertEquals( + 2, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " + + "community = 'aginfra'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions.assertEquals( + 1, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions.assertEquals( + 2, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " + + "community = 'fam'") + .collectAsList() + .get(0) + .getInt(0)); + Assertions.assertEquals( + 2, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions.assertEquals( + 1, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " + + "community = 'fam'") + .collectAsList() + .get(0) + .getInt(0)); + Assertions.assertEquals( + 1, + tmp2.select("datainfosize") + .where( + "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " + + "community = 'mes'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagBySubjectDatasourceZenodoCommunityTest() throws Exception { + + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/sample/software/").getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath", + workingDir.toString() + "/software", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/software") + .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); + + verificationDataset.createOrReplaceTempView("software"); + + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from software " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(10, idExplodeCommunity.count()); + + idExplodeCommunity.show(false); + Assertions.assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions.assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions.assertEquals( + 4, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); + + Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dh-ch'").count()); + Assertions.assertEquals(4, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dariah'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); + + Assertions.assertEquals( + 2, + idExplodeCommunity + .filter( + "provenance = 'community:zenodocommunity' and " + + "id = '50|od______1582::4132f5ec9496f0d6adc7b00a50a56ff4' and (" + + "community = 'dh-ch' or community = 'dariah')") + .count()); + + query = + "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from software " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions.assertEquals( + 2, + tmp2.select("datainfosize") + .where( + "id = '50|od______1582::501b25d420f808c8eddcd9b16e917f11' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions.assertEquals( + 3, + tmp2.select("datainfosize") + .where( + "id = '50|od______1582::581621232a561b7e8b4952b18b8b0e56' and " + + "community = 'aginfra'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagDatasourcewithConstraintsTest() throws Exception { + + SparkBulkTagJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + String query = + "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + idExplodeCommunity.show(false); + Assertions.assertEquals(2, idExplodeCommunity.count()); + + Assertions.assertEquals( + 2, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + } +} diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java index 77c448265..4a855bd3a 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java @@ -18,9 +18,6 @@ import org.junit.jupiter.api.Test; /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactoryTest { - private static String xml; - private static String xml1; - private final VerbResolver resolver = new VerbResolver(); @Test diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml new file mode 100644 index 000000000..dfea7b588 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml @@ -0,0 +1,1256 @@ + + + + + + + zenodo + + + + + + + + + + re3data_____::a507cdacc5bbcc08761c92185dee5cab + + + + + + + + + SDG13 - Climate action + SDG8 - Decent work and economic + growth + SDG15 - Life on land + SDG2 - Zero hunger + SDG17 - Partnerships for the + goals + SDG10 - Reduced inequalities + SDG5 - Gender equality + SDG12 - Responsible + consumption and production + SDG14 - Life below water + SDG6 - Clean water and + sanitation + SDG11 - Sustainable cities and communities + SDG1 - No poverty + SDG3 - + Good health and well being + SDG7 - Affordable and clean energy + SDG4 - Quality + education + SDG9 - Industry innovation and infrastructure + SDG16 - Peace justice + and strong institutions + + + + + + + + modern art + monuments + europeana data model + sites + field walking + frescoes + LIDO metadata schema + art history + excavation + Arts and Humanities General + cities + coins + temples + numismatics + lithics + roads + environmental archaeology + digital cultural heritage + archaeological reports + history + CRMba + churches + cultural heritage + archaeological stratigraphy + religious art + buidings + digital humanities + survey + archaeological sites + linguistic studies + bioarchaeology + architectural orders + palaeoanthropology + fine arts + europeana + CIDOC CRM + decorations + classic art + stratigraphy + digital archaeology + intangible cultural heritage + walls + humanities + chapels + CRMtex + Language and Literature + paintings + archaeology + fair data + mosaics + burials + architecture + medieval art + castles + CARARE metadata schema + statues + natural language processing + inscriptions + CRMsci + vaults + contemporary art + Arts and Humanities + CRMarchaeo + pottery + site + architectural + vessels + + + + re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 + + + + doajarticles::c6cd4b532e12868c1d760a8d7cda6815 + + + + doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b + + + + doajarticles::6eb31d13b12bc06bbac06aef63cf33c9 + + + + doajarticles::0da84e9dfdc8419576169e027baa8028 + + + + re3data_____::84e123776089ce3c7a33db98d9cd15a8 + + + + openaire____::c5502a43e76feab55dd00cf50f519125 + + + + re3data_____::a48f09c562b247a9919acfe195549b47 + + + + opendoar____::97275a23ca44226c9964043c8462be96 + + + + + + storm + + + + crosscult + + + + wholodance_eu + + + + digcur2013 + + + + gravitate + + + + dipp2014 + + + + digitalhumanities + + + + dimpo + + + + adho + + + + chc + + + + wahr + + + + ibe + + + + ariadne + + + + parthenos-hub + + + + parthenos-training + + + + gandhara + + + + cmsouthasia + + + + nilgirihills + + + + shamsa_mustecio + + + + bodhgaya + + + + + + + + Stock Assessment + pelagic + Acoustic + Fish farming + Fisheries + Fishermen + maximum sustainable yield + trawler + Fishing vessel + Fisherman + Fishing gear + mackerel + RFMO + Fish Aggregating Device + Bycatch + Fishery + common fisheries policy + Fishing fleet + Aquaculture + + + + doajarticles::8cec81178926caaca531afbd8eb5d64c + + + + doajarticles::0f7a7f30b5400615cae1829f3e743982 + + + + doajarticles::9740f7f5af3e506d2ad2c215cdccd51a + + + + doajarticles::9f3fbaae044fa33cb7069b72935a3254 + + + + doajarticles::cb67f33eb9819f5c624ce0313957f6b3 + + + + doajarticles::e21c97cbb7a209afc75703681c462906 + + + + doajarticles::554cde3be9e5c4588b4c4f9f503120cb + + + + tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b + + + + doajarticles::57c5d3837da943e93b28ec4db82ec7a5 + + + + doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1 + + + + doajarticles::e21c97cbb7a209afc75703681c462906 + + + + doajarticles::dca64612dfe0963fffc119098a319957 + + + + doajarticles::dd70e44479f0ade25aa106aef3e87a0a + + + + + + discardless + + + + farfish2020 + + + + facts + + + + climefish + + + + proeel + + + + primefish + + + + h2020_vicinaqua + + + + meece + + + + rlsadb + + + + iotc_ctoi + + + + + + + + brain mapping + brain imaging + electroencephalography + arterial spin labelling + brain fingerprinting + brain + neuroimaging + Multimodal Brain Image Analysis + fMRI + neuroinformatics + fetal brain + brain ultrasonic imaging + topographic brain mapping + diffusion tensor imaging + computerized knowledge assessment + connectome mapping + brain magnetic resonance imaging + brain abnormalities + + + + re3data_____::5b9bf9171d92df854cf3c520692e9122 + + + + doajarticles::c7d3de67dc77af72f6747157441252ec + + + + re3data_____::8515794670370f49c1d176c399c714f5 + + + + doajarticles::d640648c84b10d425f96f11c3de468f3 + + + + doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a + + + + rest________::fb1a3d4523c95e63496e3bc7ba36244b + + + + + + neuroinformatics + + + + hbp + + + + from_neuroscience_to_machine_learning + + + + ci2c + + + + opensourcebrain + + + + brainspeak + + + + braincom + + + + nextgenvis + + + + meso-brain + + + + neuroplasticity-workshop + + + + bionics + + + + brainmattrain-676408 + + + + repronim + + + + affectiveneuro + + + + con + + + + lab_neurol_sperim_irfmn_irccs_milano_it + + + + + + + + marine + ocean + fish + aqua + sea + + + + + adriplan + + + + devotes-project + + + + euro-basin + + + + naclim + + + + discardless + + + + assisibf + + + + meece + + + + facts + + + + proeel + + + + aquatrace + + + + myfish + + + + atlas + + + + blue-actionh2020 + + + + sponges + + + + merces_project + + + + bigdataocean + + + + columbus + + + + h2020-aquainvad-ed + + + + aquarius + + + + southern-ocean-observing-system + + + + eawag + + + + mossco + + + + onc + + + + oceanbiogeochemistry + + + + oceanliteracy + + + + openearth + + + + ocean + + + + calcifierraman + + + + bermudabream + + + + brcorp1 + + + + mce + + + + biogeochem + + + + ecc2014 + + + + fisheries + + + + sedinstcjfas + + + + narmada + + + + umr-entropie + + + + farfish2020 + + + + primefish + + + + zf-ilcs + + + + climefish + + + + afrimed_eu + + + + spi-ace + + + + cice-consortium + + + + nemo-ocean + + + + mesopp-h2020 + + + + marxiv + + + + + + + + + + + instruct + + + + west-life + + + + + + + + animal production and health + fisheries and aquaculture + food safety and human nutrition + information management + food technology + agri-food education and extension + natural resources and environment + food system + engineering technology and Research + agriculture + food safety risk assessment + food security + farming practices and systems + plant production and protection + agri-food economics and policy + Agri-food + food distribution + forestry + + + + opendoar____::1a551829d50f1400b0dab21fdd969c04 + + + + opendoar____::49af6c4e558a7569d80eee2e035e2bd7 + + + + opendoar____::0266e33d3f546cb5436a10798e657d97 + + + + opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 + + + + opendoar____::41bfd20a38bb1b0bec75acf0845530a7 + + + + opendoar____::87ae6fb631f7c8a627e8e28785d9992d + + + + + + edenis + + + + efsa-pilot + + + + egene3 + + + + efsa-kj + + + + euromixproject + + + + discardless + + + + sedinstcjfst + + + + afinet-kc + + + + 2231-4784 + + + + 2231-0606 + + + + solace + + + + pa17 + + + + smartakis + + + + sedinstcjae + + + + phenology_camera + + + + aginfra + + + + erosa + + + + bigdatagrapes + + + + + + + + + + opendoar____::7e7757b1e12abcb736ab9a754ffb617a + {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} + + + opendoar____::96da2f590cd7246bbde0051047b0d6f7 + {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} + + + + + dimpo + + + + + + + + Green Transport + City mobility systems + Vulnerable road users + Traffic engineering + Transport electrification + Mobility + Intermodal freight transport + Clean vehicle fleets + Intelligent mobility + Inflight refueling + District mobility systems + Navigation and control systems for optimised planning and routing + European Space Technology Platform + European Transport networks + Green cars + Inter-modality infrastructures + Advanced Take Off and Landing Ideas + Sustainable urban systems + port-area railway networks + Innovative forms of urban transport + Alliance for Logistics Innovation through Collaboration in Europe + Advisory Council for Aeronautics Research in Europe + Mobility services for people and goods + Guidance and traffic management + Passenger mobility + Smart mobility and services + transport innovation + high-speed railway + Vehicle design + Inland shipping + public transportation + aviation’s climate impact + Road transport + On-demand public transport + Personal Air Transport + Transport + transport vulnerability + Pipeline transport + European Association of Aviation Training and Education Organisations + Defrosting of railway infrastructure + Inclusive and affordable transport + River Information Services + jel:L92 + Increased use of public transport + Seamless mobility + STRIA + trolleybus transport + Intelligent Transport System + Low-emission alternative energy for transport + Shared mobility for people and goods + Business model for urban mobility + Interoperability of transport systems + Cross-border train slot booking + Air transport + Transport pricing + Sustainable transport + European Rail Transport Research Advisory Council + Alternative aircraft configurations + Transport and Mobility + Railways applications + urban transport + Environmental impact of transport + urban freight delivery systems + Automated Road Transport + Alternative fuels in public transport + Active LIDAR-sensor for GHG-measurements + Autonomous logistics operations + Rational use of motorised transport + Network and traffic management systems + electrification of railway wagons + Single European Sky + Electrified road systems + transportation planning + Railway dynamics + Motorway of the Sea + smart railway communications + Maritime transport + Environmental- friendly transport + Combined transport + Connected automated driving technology + Innovative freight logistics services + automated and shared vehicles + Alternative Aircraft Systems + Land-use and transport interaction + Public transport system + Business plan for shared mobility + Shared mobility + Growing of mobility demand + European Road Transport Research Advisory Council + WATERBORNE ETP + Effective transport management system + Short Sea Shipping + air traffic management + Sea hubs and the motorways of the sea + Urban mobility solutions + Smart city planning + Maritime spatial planning + EUropean rail Research Network of Excellence + Transport governance + ENERGY CONSUMPTION BY THE TRANSPORT SECTOR + Integrated urban plan + inland waterway services + European Conference of Transport Research Institutes + air vehicles + E-freight + Automated Driving + Automated ships + pricing for cross-border passenger transport + Vehicle efficiency + Railway transport + Electric vehicles + Road traffic monitoring + Deep sea shipping + Circular economy in transport + Traffic congestion + air transport system + Urban logistics + Rail transport + OpenStreetMap + high speed rail + Transportation engineering + Intermodal travel information + Flight Data Recorders + Advanced driver assistance systems + long distance freight transport + Inland waterway transport + Smart mobility + Mobility integration + Personal Rapid Transit system + Safety measures & requirements for roads + Green rail transport + Electrical + Vehicle manufacturing + Future Airport Layout + Rail technologies + European Intermodal Research Advisory Council + inland navigation + Automated urban vehicles + ECSS-standards + Traveller services + Polluting transport + Air Traffic Control + Cooperative and connected and automated transport + Innovative powertrains + Quality of transport system and services + door-to- door logistics chain + Inter-modal aspects of urban mobility + travel (and mobility) + Innovative freight delivery systems + urban freight delivery infrastructures + + + + doajarticles::1c5bdf8fca58937894ad1441cca99b76 + + + + doajarticles::b37a634324a45c821687e6e80e6f53b4 + + + + doajarticles::4bf64f2a104040e4e055cd9594b2d77c + + + + doajarticles::479ca537c12755d1868bbf02938a900c + + + + doajarticles::55f31df96a60e2309f45b7c265fcf7a2 + + + + doajarticles::c52a09891a5301f9986ebbfe3761810c + + + + doajarticles::379807bc7f6c71a227ef1651462c414c + + + + doajarticles::36069db531a00b85a2e8fb301f4bdc19 + + + + doajarticles::b6a898da311ded96fabf49c520b80d5d + + + + doajarticles::d0753d9180b35a271d8b4a31f449749f + + + + doajarticles::172050a92511838393a3fe237ae47e31 + + + + doajarticles::301ed96c62abb160a3e29796efe5c95c + + + + doajarticles::0f4f805b3d842f2c7f1b077c3426fa59 + + + + doajarticles::ba73728b84437b8d48ae287b867c7215 + + + + doajarticles::86faef424d804309ccf45f692523aa48 + + + + doajarticles::73bd758fa41671de70964c3ecba013af + + + + doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4 + + + + doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7 + + + + doajarticles::ca61df07089acc53a1569bde6673d82a + + + + doajarticles::237dd6f1606600459d0297abd8ed9976 + + + + doajarticles::fba6191177ede7c51ea1cdf58eae7f8b + + + + + + jsdtl + + + + utc-martrec + + + + utc-uti + + + + stp + + + + c2smart + + + + stride-utc + + + + crowd4roads + + + + lemo + + + + imov3d + + + + tra2018 + + + + optimum + + + + stars + + + + iecteim + + + + iccpt2019 + + + + + + + + COVID-19 + Severe acute respiratory syndrome coronavirus 2 + SARS-CoV-2 + COVID19 + 2019 novel coronavirus + coronavirus disease 2019 + HCoV-19 + mesh:C000657245 + 2019-nCoV + coronavirus disease-19 + mesh:COVID-19 + COVID2019 + + + + opendoar____::358aee4cc897452c00244351e4d91f69 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"sars-cov-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + driver______::bee53aa31dc2cbb538c10c2b65fa5824 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::437f4b072b1aa198adcbc35910ff3b98 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::081b82f96300b6a6e3d282bad31cb6e2 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::9e3be59865b2c1c335d32dae2fe7b254 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + opendoar____::8b6dd7db9af49e67306feb59a8bdc52c + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + share_______::4719356ec8d7d55d3feb384ce879ad6c + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + share_______::bbd802baad85d1fd440f32a7a3a2c2b1 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + opendoar____::6f4922f45568161a8cdf4ad2299f6d23 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + re3data_____::7980778c78fb4cf0fab13ce2159030dc + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + + + re3data_____::978378def740bbf2bfb420de868c460b + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + + + + + chicago-covid-19 + + + + covid-19-senacyt-panama-sample + + + + covid-19-tx-rct-stats-review + + + + covid_19_senacyt_abc_panama + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/dataset_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/no_updates/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/dataset_10.json.gz rename to dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/no_updates/dataset_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..2eb33c5a49a7c5e59ed47771793657eb2d55f6e7 GIT binary patch literal 6883 zcmV<98XV;xiwFovaiCrR17u-zVRL14UokK)YIARH0PS60Z{s$$f8S3*7+7Eza2!dt z{O>Jw(r${Gc95Aa(7Ur3P!er3l|+}4(z@LNZa>5Q1oJo#d06zVuYT>PxS!;nLsFLP zIEgcxT)8vc9Tc%miihWScu4X5c=&TGqB%WJr=m6fxy92fp79`Nt?`t`8EeBgQzj*g z)?Y*{W;u(3rTT@XVO2JlMV7b5EvGYTnVR_fLW(PvQdP1AKUAM-m~)Y0!(mLbj6*A} zY?<+7A!I%drO2`$X#Cw+vW3WaF60ubtD-4Q*gl1`aLy8@9vr1CA6IRksJ63p8S}(j zO1*vCZb7?wu>f_koF^>HX|iaIU4Q61zCY{_N6rX3jV<}kzeMy#{jWP3j>cohiy}Yr zLeKNuksCO{D2RH~ZZ{mVQQ!AaTSD>~PicG(5BNvxfz|&BS`l%~LI7qeMFNlbx%C}e zf-GXe(nwH=PaBW%dC=GujSKATDO;?5MO~3oXg#MfVMpIgzMMdj$|}+#j^9EH3+x*_ zx2)_sot``Nk1w80ULJXlJ3Ml|UKgrTYmi0#_}S8tv_kmS4%@lE<+ipEH@R-lY-y2^-H=@5u#C zR==io88Y_!FZi|i{g-q@n2o1U_!SPVHtXHQl7Mg3h)CU|D<&hP5TY z%LPq&Hs2vY`IOxaSHHf+DWphqa6M`OXGL%co?^q}PdNBQPBRiQ5>qk}Ns;o9hFsgg zEq;GX)J++;rLtaqf3M!IkXiCMjd+#_3qzAITyY7yk7#6JXcUIkb;RsYXcPnZeL-h5 zji|+;NgQC^EMZ}5r-nnp!NSla43}o1WKn2-os=&v3eBQmS4j;+;E$XxIR_ZvwSRRD zZ`LnFyt>Zer8={~Gzm<<>pC5`OJ3RKQ={9~InPlwwN0qAnD8{>;JgqG0EWCgxp;Xr z5r04ONGRYl%W_F`Hsg1^dL)#mL_d79e;Yq+_XUBQ@SMHbEKu*rRuPN#hm5}SI=}?n zB25X%0kwF>B4Y5pWBFj=JveyPrh3e-XqrbV-h;c$Q9T}18{{t;Kn}?MM9q+n$+zVe zCxQRwk_W|-k*pxlbVe^3DK{Ke*BQ*8P$ebbih{f<0@it*ev|SmCNtcdBNLkNSeOXL zgvID4EQ$||B+?N2l$X)H61Orc=D1d;`#ZkYq6om#&C!eLm=sF_DrbVFksd4h+e{QP zG$RMYVLAITH=k6HR{tG}R1J^Z#1Kntp0kM4XEY#34Y51^mr8v*UQZt+qesak8uL=sQ9czJ60$>F)0DDhNW%MW&EP0W@dGb^f(Q8}f&g zv0fMsM}wa4g(zLG>$|KMjofMf!ze}VRv}{~I~ntyM{#FP(t3F z3EsFqVMgqVht>4Se_H>o7K0_>LsR=o>W-%PmA{nVnhI%=1l2S>!;PEL4FB=Zw}1NC z;{RA|O22n2o6?Vl;lOu$eQ!Dnf??zZPJiV0qiGQO;UH+;!=}8i`u`)F@&UxW!RUd+ zJg+-?N@AXtr)grI=M7pnhn^0s|AA8nZjNP44NP!LPYM zsIYB*C2WMFQ9LeKZXswE0_&F!DzbD=CL*2Tt1RD8y|BajPk`NeB-QB#l)30hT0p#vMAZQ~m!_CixXs!qp1My@Z7hejorP zENuBqbL`3M)-y^#Ert1v%N~^H#?*?fpTVkZ^>0cjffs0q+GQt0_SW{2F0a!xC1g(21CV+EHyx!m_!PM&gD+27ktpDV(4BWCh2TER_re|xdm(%^WllJ7d);o}4t%Oko!=SDxUc$Q#xnX3jb-?~rwnEE4=R;H_dKtoLXR-1jg&R zE-bQO!iTpGeb7x!D`%HXhCJkU+#E^(tQs^TWorMvzCy^T%#K2Fb>!`_SRj+h$>*mP z*W8B4{y?U&e~9ceSa1UL7b0O&v@5q838@T1gzz53(0U%W*4!t2);7_j0G=<*b2AdJ zu2V*LOaf&%X-+X_1;#AlJ=H|&7zU0ONY+xpC1bqs2(;cY{>(S0`m(i%EAZi+jnJJ7 zH0Txvnv&J~To@@aHnm{VNR_PK!AP589xbSd@h5DKmAZ$>c&L7V=(}_}h`OQg z-xCtOxB3BwMEy?R?>;ajI_N)LOmr8h3{_CTe4%HYWP1Of8!sA7q`| z5KrSk{EbHvv}TcL90}`}4h95x*|h|Xh7TR%s#}Vf+q13*F(jA?fMqZ?)H2J0Vwo8h zo_m=Y&o)E6`qgA!TXbQpH5AE$<=l`rYJ;pzpep&YP=W85Ohk4Eu$GZ=h$gW=BVz;R zO-2U1y2A6PEQ3EaOW=$0sPY3N`?&7~Rs}YOCx*%0r|~o5Ke@9(C5JI6lh5gmh|wLq=(+{)^p$<>rT79 zkc}MQbEcGaeTNR1>-L7xAObbC?j}9lTmA4z58Ym8tnCcNe3q_Mqe8y5dw9Bo7 zvJ?gj&mI(p^Ua{}5i#>SQ8K7}(MyuD8P$IyhDBDMUk}ZStU9ZI_bS&Cd?`Jd+lB%5`?B}f>%#4A1_GD5uXiPqgaP_Oc z2z&L8WE}VRRF;UFnymDP+1)p|5XiFFTKy@;1s2STM2S@@PiL}keO0Ktx3?QYxYC5m z5IdIUQs=o3DVUz=+jOh%Rr(9nq6jXTN)eAD=P>goDw9PmGrZ1iVX#xBR08`g*Re(S zKn4p>L%yJKqSBOPG++`GhR;UE2VE8sRFpH2?0t$`CECIgx&#c7!NST+@+H-zjfv<* zm87H0ng4wONWo8+g<3Qa_M>8>XetT?iQ{c9isGkGQsf-X&y)mlC62Znx(te`W3x`Y zkNNxAj|+@^txZn>^ey&lO?#oJstiC+iX{O9NVMl2lU!=XOh}>gFb*_ecN@l9%DTpB zYPdJby3EoQ4PpsxhI4q-6~m+_U>~MF=sq3ZBtACPSM0b_hzD^+HQ=q$eBzy zJRBRS_xE*6ss9BU28y;lK2q>At6$@?4Yj)sQh@1%agk{{A5U>pI@2m$lGSgZF!)QU zVcT*v$qVY~%Mo6;HubQcKX)Rz5Rn>Q>&qVC?_j;yMG9F3%GTtpG+7$H(G-km_`8l1 zK}%x=veC4*i*zjz42T)R3l%Dwl_>SMhTQlp@N11nJ7XLD|c%&@R+_AQ~XeJ zGOKaR;4@5NISZ>RGAMbelJMviOqEmTW9*fU89yQwZE)l|M^4}B^w6ki|6!2KqG_pU zpE|adigqAWH0PkHXm$#;R5VLPdsHggXm~3XZQ7f5!=dMdJ>Ll;XBg2C{^JZn_%aHo zt$V0w_f9oXx05Wip|uXu}I` zA#2yGmme!CS*>)-^2Q#OH#QMTfqS4f$w%Ws&*xNr2Yx-Bkx08j3rC}HFqKg|6EIq0 zOU!6|Y=k!LO2gE>1doPW6b=A|G-8pE)+sd$#rZjT#%C&V$0-XaUXOA!f_akG4*yw%p2X&hYcW01Up2m_a}nYZJ@o*UPB+k^jea$Y{778 zuGj}qvwD3hcaoOGtl?jpHM|aZE6x&f9$0-U?GR)moTZw}A)o}l5u444wXCTk`Dhjn zi%G_3DaTuWbSMxkfmLq|$b5w2a~j}XJ&KuT=#Pr6qjTCI5u?ZoiYaguAL8ywrA$K; zE6zYjS;bYu%Fp)J9Q8%kIkG#MW}c)2RbwOhKQk_X*yQ zM9yNO_v{qa2e${a9Qy<~$E){Q8z6u_QmNS}*woiQLWKzRAOOBdq>2v%%|nkaSHFVk zYI2ffMXGr$x?L^}NI6T^+arpv64EqMj35}1h3WH>EI?lg^e>=_52mSjc`d(@l3Oa8 z9Lb>&SU(1hsWJ&3>r16N2DV+8tBuf-wN_drS^cIsYio`WjHyJ^1VN;fnFcXNN|9xP zqRqicQN;v;#!-}#XcUS>aR?2+#Lz>+`ks*H-vPKC@c%TTOf$aMxjxpcJWA;f-Zya1uv~B zW~sXTkH*T{PHDbod7Xc-yRl)rM`QyHF%;@`txhc)=#xa#vVlHcY%Lq;AlN_{^gU!Y zkevc88_2SO9*+$)aQvZn8yjfgd(+4Xyda>V%cvWMzQgdlUY`a1K90h>*g*GIKS(x^ z-*+Fx2J$-txBrA}AR|xnY@lK5E;f)q7<+x2n9352_9Pg+z>CfG`6L^2X*SDCDCWYl zh?-9Qg`1c5R+OfJD3cFlH_u&KU=F}d@hq`fT^a@Af|=OX6_*(2vS>8wuc~Df2E?Kw zw3r+Yi{(b%qjL|~N>LM4D+3jfr1ve?EbHYEZR3#AA7S7; zXJK0U4XTJLV2EIM$Ww?!T4ss3#rlv1b0IQHzAW%1dV`#^aqAQcsXq!Mrce`L>=gNfBn)(Sh(%OH%O=^40rLF(Z_E>!T4#f-iQBE=@zxe2zAu5mStbNT+9E z!T^D)Yp*jnscb*g@I}DAXoKId2m_w644t2ar8>z=ZX_v<1G6_pDt<{xZG4sNpPRD; z2lH(sZEE#fx*khk&_Hq!`=SUq$a=}9iYx+41?W1>WtkFYx0W$ykh0uJjTpc&6N)_2 z)&{LCNsU$zCsc$HES43QELS~)510o?kzmmDQpL$qTpAji6j?CP;bAOfDjAeO&x5~$ zAtIG)o$_mnTgxR0EF-!(rqX}CZmA4SH-{iIF9h0p>kx)Tb9jJ`P~2@7$9D=FXc zssURqpxV{oMgE@d8*+TGrJpnjE#%Vl&y1gMv8w=G0E2M8x;8iC!59{bku;^y>#|2N z$?^V1!+KI=tf|mB#!REwBc;RFt&N5ig$FFT;L06fHuu*pLh*T5tM{7VCB;lqLr+tr zsg$l0h1F?gYZ4;Zd&Z>64c`V)6WqqcM0U{pmj$>wmE%evEe*Zd>K(w9-SDAp1cL@J zs8T&7pkoQldddxHq}8B;FEIbxQauYgL|51NG!)~6tQ3$b;Drf zIbC;viDm9&4&7J%@R&oM-|71gyVB0@jQpn}4prhbPaHD2+innt93O@4Vn!uP9J0hA zo0;ejI7w_qhU;5O7Knw*^-QvKRECAH*gi zJQy%U?$ua`woyOvQU3%eBL#p*Wp?C?J(S;@$W$U~^i6|3>nKv2uLQC}FIeHE{ zC0Nd6sMKLog4vW{HYJ!%3HFIiur?pcABos!d1fE8;aHy8LGaA_2hB6H9@+BDEYIu_ zd1jvDxVQ1lqQS`R!k_&?z`}miWr629y@;`($9ki_+q#=)c5n5=BAvsY?lRd4=eqTZ9uuEaKW zHpI|ioYVqwAVAoBNr!@kp~+ehngrohm1V;MaTrl(wK=UeXI|p@`jC-Rx+J(#w4vue z7Pk9`p3h=JcH>^l?y^ogk%%nUc_lF8gvp7(b|eHOF$#cTIZV$G2o;E_XTM=(uw<{$ zDpjGeOvPErc z5#uJ_tv7o_BPZ%ociQjK$m{n4p9L%$(ykwPL#Nxj?`DsCs~=?CWaK{VeEy&_=nbAQ zZerwVI&Lx;?Tnk?0sa0f*B|$~W7n~96B{?N2lW4dvqXJ65)2==EzSxFW7Y0R+Ox`pgI6T| z1`_Ebix!S%$AtnRBl99bcV*GA2l6dS`%)n;Xo9djD|kl!PGz`yCqowF!F(05uDvP} zipLk#*~h%P8fMQI7|WhGM041mJuwk!odvsTHL-%_6al7m5%GihX+KensU46J84^?BS1&cX$F;-5|%vk1tZg_L4-65~={*={wywRt+ zMkf-wKV_~0Lknq>mozEhm*-G$b)6o4MrFXHuym0J4TpUWb-Aw9silk5q8^qmV(B6u dD_sPmf`?5Pu^!pdMc)4D{{S7|8zsb00RW%H8sGo` literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..ee62cd7913e67677f1d81e880f137d1545656608 GIT binary patch literal 6888 zcmVxLlHK z`R}e$UCC6?oiw_82FOki0!pH7RwUA)q)MD@1Dj{qC+N$*=!-$_a_ieX#XiZNLsFLP z`p;BHO2XYnrfl=)@O+2A=i}k`y@VF@Tr6a7{(X;&OP=#6Wxe@=ra9}wHw&f|OLku* zEM)~tqK*EAiMUQ%s4_2l^PV@D^=wZ3eXZmr6I7?{!4LIk8W&s&tT;|-o^zt*3ZAjNpxL@N_rq!Eh2eBOoq028G}aV)|1Fa*_5TfL)7gCP1xXSnK^z2u zKl39mnnlTIF&xHIHXDZlYD+4<P{2QX56u2A(27h`76UK~B{L}E``%Y<1G309 z6N#h>OPkNJJgDrFrX@D^30rS}L|u_5P1ZkvD{Yul(WM3+KUj;LZI1>Q$)=MMb`lDx=ru7|+Auz#r=z z>|01}u=pobCn<00?t5$BPgou+zOGcaqUsVl(i5odc6n%8FpVVO1ZYJxXMcb36!NT! zV*Tgg$)AId;onYuTn@_kA|9}$gs(vs{o~gq)P?9KOcd5YQPBBy`R0K)2t9A$qg2P? zIIvo=-M*&{-t&(F5H>_rcoP&fV+BR4{0tz;*HqgL$~Zqh{^px+2G|>^idP((J&AtprzGu<>gr+G~i99-`8I4!6Pdcl#P+R7*mqcYzI{ht&}{po zXv&bYpMSuw<MJkO+qp-mVrxdPoMG;uJr3d8n&!dzEq6$AKvO_x+8)Zx%34zNa6u!wb2 zLsxKcFtiE7goA&sF;MdiM$aI4&2p6_&e z|ADW!E+ZJ}Ryc|orxVK}s+N+8#Pk*OZ7xd{+n$5|u7Z7A*iYIEYyOU<(A}f3PQnVC zXDs3LDUHYxx#Glk0Npus8RZ+7pMfKWq)L0>Q0)==exXz3x#G)JQ6~}>rtcg2#t%1Z z^LqW+0Z!|!iH`AkU&ZfsWj9Kj!w{R50yg2;{G^PFyjUq&E>}6rV9Y+yJb7McC2~QG zcalUbm14=ok`#I!W=bTsBmzaj$j@U)o>pymG>Do8I1PIUnDK_<4DIy(tLuG1GpG1A z6c0lZPtXOA-@vegT#EDs+vytzDlT$6Zv!YPk750^H-}q3N~{=&eaWz8P^YG&*wp!P zv^f&-kw}h51Mk?)BoCPC_oj%Z%%49$`|Lon&)2YsJ<%Di5yO=9WhRpw*c*EZ-qgTf zrLA~gFgmFFW1eQ$wH;R~v9{+gt2y!d=3kc(gjmU3BaZ?+Dh09MD^-f*NNc~VBps}% zNK!Tcd5*8}wQwbCHxejhD&vaA*1JNNd~{LW8v60q&`P4Y!+0F8VC~FkhJ2-J8Fu(< zTq1KN{5dpB9iJ|aF(L{SHQ%Y|2P!3)T^Z!qn;+L2Sa6y;n z2G%h6fdG`SX5~v7^X-p8Iho>eBe%K9?^3g9(m)Vz;in7hUfTkyq?Yv z(M`i~euxhEA@I1V%YFzqB0E2X^FutMA7bXsCO7&a7Sq{iJc>s&9>wt0f_d@89}S0* zKMARhFmbCN;;!u7^h1mv>W2tNAJPpm?t~j+>RlU81lJZ(o^Lu}+5{#T z`93U_VA_VggSoD%XZy1YreYp**KfMwp{hHL6igqpKWyPfPESuhep25Uaqg?%kUiYr zef1Hn6#@DeGGj{i>#;VGN*jcXVGG01WFBZf_i%0Xt!F2K@}(^|C+YURVDy@izjF1g zDEgCfsuXN7trv}4vGr4 z&Dp+%RE3q3T?~b^^YHS|+NW{VWYhJ%} zj>s-z>l~4ta75yPbC(^FZccQLNau)rOh;tkhc`JQ{YfxG58~9H&7xVD_(?b(PeY$B zCdn`k!#f<2cV_RPBQhL}!{GxRk(2R<^F;3CY2Op+-|C6BppT60LWj)WVRIs*diW?k^yc7Zs z^fJqV;+PZ;o;#ToPxrmEW^1x)EV|H58p~|W3U0|3jZW4lkd=H|YM*sVPGx=zu#Qa8 zMU%KcnPLy-eKG}XU7`F1%i&Ms;Qyq$0sKIxJf3@jRe_D+MOx~_L4{c>%mptW>;)eb}VMh-#vBvyfQ8N-Ms*BIb^^*>Px~WcHC(?^GBV!D? z^RlL5VD3^-l2O4RpXOTRY*{M27yS%Bfx+QvotAe_8s|>csn#(%5stm#@lG{^!ChgY zolDoTAbvwS{gqe{|H+8u&~&5+hpi($bcOWbJJQ4NtYbOS!yg#`J4p|t>5aFq12$X? zM=_guVc;z&8-^a8Fy9|dlSvZmdwI8!9`4NEJ<`K)G?@9pgGdjfLEuk66zQRsr+w0c z4S8`b>0vk@4d;R1D(T@LFaklBk8yO;!NAD55b|>8|FfkWB&*WMpMTJ=5jaMJV>GyV z!?|xe2f`^`u6#Jxd(;eeBUX2MC|-dvTxf?)f$8G}LGFO)g?3m#wnmE_ASXdp6}@Z4_7r=>kosb?{OiJWpS|j(~JvDn3tIrt5Ti`5a0U@553ft ziV!Z5Q59puM4?Rh_?Uv}*|seX+FnPl&^5~Fg6YWXC~^TaZ>ED#Gy<-huoM3D zlRn0GmvwxNDirT^R+WQ4bT_Zt{v@#qN;KLJu5c^3?MTj@0=9MFlJ&3jfZ|<0K45V zc2YJqPKD*(XzQ{|S2TzX)ETd!s7r?NO29sBe$c(JD4>iDeWBRvk|hRzWwf!I3Q({C zzB*t`L?4{WhJEP4A-CNEk%g{qxO9DMQ?~FGMX+?s6v^0yRDGMv{7%utK=*c{qQ-9Q zU{NrYaVQ)sXtwutPpkhq8U~7X*gph}Gut22st$d+4N`#V#A%ruIv*E!D4iRXZpij0 zP#FBB)o^GzhU5kL%$ozRd;5CW&7apIxt59UUb{CG;O}U+*hLCi4a&jftTkB~zSR`; zX!yH{6G2O3C9=_swo9=S2s*?P;e`w}%}Q4K+d*!}EIoQa!`N-?GdoelGS$Pa)xcwp zy%?Q`l9OeFTLzzD3M*J#zdS+7OC3MQtYGSJIKRhMIiK+(Qqd+yzIWt}old((Mf(q( zWDZS7Mf;;;>!@fQp`v-6rlPqi&{5GG741=}XtU{!RJ6rtF^s2y7mq?OO1x=8WB89Z ziQ&s6Ui9vuqTQ9flT@^+H-0D;E%;y$lGpOIO+}l`u6dArM@0+f@GlsCSSs3SsgF!f zsgQlx;FhZKAbIt%rjj*M_Z)BRQF&vhGAr>6)CKuy-Rb#^s;^*N7fX_uq0qt6DjZA) z!_Fo2medh5TJIaN3%k-XbuYuK;SPlkpdb>KNaY5lcA+>wCr|lO2kv;nB8o4YoS&YY z-TAVK!{)-4VN^JNjiE$PvN1i1q!6U`-q7@1obxhcLZ1 z%ylWNXDOj*rU!^}_btvzG`XhC;g|5KT=C2H%r;+GU3BtqY2!tnHtn|F-1Y4z&YK%%liNUw2>;qM!~8+{}wVNs0Rt~B~oZU3{(#- zy4d~*rfbMaij~6fSWLZKS&#~r?T$w@UnQm@(TpJIk)>_(hO9wf8MH5=nh$2Ects<= zk(OI&nj9%05m-Mujj1*Xo|^Zd1{SvKFxLm66>F`sNV5G&bJlhoA?QZLopNju|Aq=YkNYA9n0(d7Y{eiZTE<5peedSgQ3%@V*~vm(R6H}-!Ha~4b%xX z5ITLk%m#8(pko6$Hqhg-fhJx!4Q^rsO~PQ2cu^2VH1-+w<2dvfem5GkXgtPVcpDq& z&g`9J1BGM%L2RIKF!9G9kPT$zX`c-=?cK%(3Mcbm>;h9cg3&z*M$hq0a&te)`CQtK z@)FCHbS$E_Lx17nrF#{nZ6K=P1Nqf^mkyW?3{yPMTvV4`2 zafXZ(%{{m~7?m|oBq?x=uO!ewpx$!pO zA~E4vn6yBk>iW9~PCD8TwR{m6UW~!-S%d}8T84?w!h{L(QdmhUCNK3=fZ2K1A%CC%} zeFTF>(5Zy!640;=W<5O&8KiZmf-f-s+eViK4WirkSaJfWa8Yg;M$K|AzDJKE6gr(c z=FlG!O~)Mi{bK8wLmgobO*_pTa#NsV4msw~V={*(p?4#5Xc0$jNWEb&^~U32oFpt- zB*QqG1>Vq~U|^YBnL~GF?;dj~2nXZvVNcqHgIV~Yh(onF?GuM=?6xb!AumK>hZs@G z5r-Uc$VDdl4NemKp5f+^lEb1+EL^wf3>2uY>WZgBqFt69Y*ri?-2mgJJH-x_b_Xjw z&w_sp*RKwXu1p;0HsvzezRwGo)g3Nv;_@;5iV?xSaZB7*VF+iwH}HZgdXJ7x(~L-> z`5-P3VQ0vYpGvi4K{YeSK0bo{|&|V*%V3H~*Y4cYM=Jm3S8qYH7U||=MJm3C=jy5gN2?lGTWJo@f zNN1xrMgTi7?`AwN!*bc@+ZYPfiV9Mn6q}bCGm4L(SzQ={QND`{l*uY!nBB?17e=CA z9;2tzA;Ai!VyzAr63m4Jb0NW8NU%TH1nc6V{FaD)j%W6JHXO$@>jcki+-aVf8<8E) z%<;?~k!Kcoo_`b1ESb#wA^bU>L@XXBLlydgEo|@adOk}DxgPgMbeCPwiA?5&i7SB-Cu~dvc1=J)lA-|Umc#H2k<^ZuX7n3Y zIZN*4TBX`GmTSKaEHbl5?VK%`zy(BjT)zqK3q7Y(=QnYF6X!Si1Ncqwc&yug6E`9| zzlrmkJfhzuxbbC=WacG9>MzD4ngruf6tajVQ#uTzVCoG=cfIU!XZB9|O=kYX?&nVi zlhNb@`c14nZTn3ovuph(ctL;s$`9wG;oSF}-^BS%+y(vL;4INRjs(NUV~ev=L0@$z zlI||D1H;DLmZ|2LyFSq_4Em0%d*Y*Q98P%&Ya@#H6PTVWq`O4f@???8SYdc~2Slqt zoWT~!>_8H8WzoUWZokkVCRJ zv2JV?3B~J+`tD;WKZM2U!ANGzm)JsVyqf8F~f28(a4)OYMx`1%o3 zdUiMWl+AlQsJQF%o6OjT-P(+7Y+iG@A1_s7sx?bFHZj$MqMfnq1>N}SQhPvN;rl7u zcle@D{ftf`O?#?X1(p`lCogGM!Y|Jt;r6{a`iQECDe34U59$v0xzy$RPN$A8((rmX ix`?BT{9frI=oRcXUBr#ZjxO@%PyY{%{0-$MQ~?0JZc1qY literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext/dataset_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..cf3c3aa7b6412355f6172a0b70a3929ebdcc2735 GIT binary patch literal 6755 zcmV-p8l2@HiwFqS2%ug717u-zVRL14UokK)YIARH0PS7Pj^j4A-p^AI1_qb`TrNqr z{I{#PDwztplSX$ZKyG>vP!eskB8d(qRpR6}aPtiJ33}O!UJQDdt+#oK`y}@qlCor% z>n~F^Qo1=)=-3u19-i;;_k2A3r4!Mdo~Lusnf}t@=_Sv25VOv7PUDPq;g>m+l11Ae zA{MipMZsD=RXXwoPN+culI; zue%*6H!oJ8E|&9zWjRe&ovG`OeaH95!|}wKK&7!H-}!G5eW(7{pNuEdspCbFA9NK#ovTEy{dC}D+lgZGx1 zU8gs2$Nus2C$kqvp5u;>TyM~atUnYq=5x;EF0hf{d+6z3u0X`2@ACDET?5Vb>FcgF z-Mg7CqK9TH=Eh2toPoFk_Xe=pts)bXd@u;)zN|LT;fb4k-|E@VOv=NL}?e$O4M z9PB4Zt+DtPiW8LAb@z+0?^BkAlCMhDEvdYOhI9(0-7F7{bEc34oB*wWX6&oyPaw}S z&sTr#pZwMP6#lmA|tx9GF)<~c2CH~ z@s}s($6n7l1~I;i!eiCX&>~+7soL5c+MYr8GaAQKM&f8j6B;f>mwXN{U{Iz>P=3t{ zc}*602#9_UotA0M*tDU%`JBhxV60kucc=3#s0N%B!3FdbJ3apF%dgK*kKD=ief*wSyYlQyV0-Yp z_u$B zXlM%#7KSEaxG)nXi$e4Bqk!K>_yvun=GYd?U zzzqAY({uaerJX)Cx<#Gy992`>a5{?#Pa_VU7oq{ckWj!sEXyU$*@EA&-bg4O#=Vsk_W|_k*pw4bVe^2DR&t*?=l!P zAxlcW5e0c!1g!Te{VL^`OlG(tM`kqPu`n)*8H=%-uqfWsRY*gqQ(n67O594fm*ZNU z-o4}Ntcn17x+V5v+UdlSfXannX{6hV{xuVY3{A^He^<_a%FQ?Bh1H)!k*emA8z*6j z%~KX}`h*7Lh+J{vTYzruy0r4O%TL1*LQ<(baH#eGeZSBsvP|;DGOrS)6-@8}y_@lK zwpAeh%+t)YB!DaCA@pcFt;%?aL&<0k-5B(U2}q_K#3pri9ITJh@F{iV(2%0mx$;Zf9g(x*A?(r9m$U44fe|Zl*h@Tx?|rF zD*yC%rIjA8{-J~*#8PAmc@*G5NkQFCA!DQ~8T(yEac@b}C}us7=lBYP0>>6Jw4;!* z2uor~=L&)8!9}&B+QW}(QlPo>bP+CLG-EVD20_^|?mrC+iPc}1&!y@kSyYiOG7u8F z+xmzYu}dCS<0Joh`&rHQO3H>N=au9eP0}l$l)st`X^{lgIK05^n9>Zt{pahy{9^Hc zC_bb=xRwv;N8@nhyMv)Op9H};@&ac#@rTho2>oyrbZ+58-j@A8$cMZKwQe-IC$-M& zPacz6r{!syTIYGA&J}9i=*V@CT>qszoO+X~?>{cJ?l~?epKabsB(^=JGOrQJ%C8lr zOs8hQ|0-()w%ZtWmQr?@QWgp+QhLdur^IDHgF0J+S;L!ibRl^N##2aUfoKwl=PVc2 zH`FKuFe896Z98a*L+G5$M7qGmE%~Bgmm(vojo@Bz4F5TU*Wg!;8o#`$^;i!=8zH~D z2jSkP&?OI0DUgjmBI2EP!166UOYK|{WIpOROd zCzOC%3iBOOpdic$5^V*+($8SRwfPq%l)&8na`P6Sqq75Dn?(piFU`nW|CW&@*awIQ z@1fbNP8W=mrA+%x=D0>u6$nKN>urhkx~C95@`4e`&>ORPPl^XnW0aJdG)sz79IQSW1#)9Pznj$hFbXD^*8B~zx zpc;4sOd=T(psK8AcW55am*XEf!z0hJI&Ft9$NC{YWW%w3h!*%EaJ#9^ehAwmTR(*L zLp-1#V&Y6j*ZLvm4q4#!VNKY4xaEwH^cx>`1hUZ;E_-G-?$v@k{PV}LQd665!OrC0xnhBPhnvOgR z#-!S@GOyj{d$N;>|Ng7W^-T}5)U#Edj>7`s0e?fPv%Zgi$M-Q;s|b5ArqK=FjM+o0 z%~HV==tNT9=&K3jAv$hGrPLKX@O}H7;oUZFjazK8`ci`0GS%?&n}n@dB|Q7`3cXwZX0&_)x{<%n)6y2(?jf7MNAwpQf zFf^Han$I0vTV3PX3E=(0yf-89=3UC@fu6r~^(-m+lQJqLEHRB2jiFO#g&Zm6_%X%{ zuR!aa;$yx$zL%v%T!9bEEkes0vC2WE++XMfN;YqEVWdRA(TYhURkC>l&1()C2q7c- zhOjtRc8Fzxy;rHxHOLt7&0AH8oWc*FCv6-Q6>O@rc>}3(BPYoKu1LGkv{UR4v}lmv z;W4xugb(Nj8{(#fXUX3!af4OZJoJ``h1VX4`q;6xj>uLxB5}jH&5lSrCR#_Nbwoa- zBhqvI>l~5p$eW-CaqLbe!NiZ;$R7^JzDwt$s2}?NEsn@rv$xO@>Gy_y|DKM>(eUv+ zkz0A1_e8okdLrG)v_H0DeN zK!DY(4QJUb5N9~+mLlf%@MtTV1T(&?4BCd8W?4`ylfuGtE0f~Mu6I^1O_sGu7rIG9 zk*rwG4cVgB$hriwlFtj}vyRD3WH$h7$rNogiMx|2c3|EmQ^3*{-als(*ccwfB{v@iM#{vJnp3r^yqk#=nyEIs^nE7HCfX#hLAfeDk=u%E>cPonli|zxE2{(6cW!xKgDlg zaJXBiEQ!wSeEqg5#xU=>0vOw_Vl&K`t$xEWE02toH=EE-=QPsx`S~vib8cR?&6W6;J>0!|G-0@?P9x8d7Cq0;u7Y9iX{pp}T_1s2D4_`wM1Wi7~-bn=m zBj;%1zu@lOUYL zF!h7FMO?+)v<|^$JEppa&qC286rZpZ&)ag_L0Jleh35_whVxyg@DVZNJ5e&IFwAR` zvIW(j5yK)YkDiBSMphlOzj>ByNepc?io25-p54qQ$QXto*u1I7U`X#4%AEWSx3B(A zG7-pK(^{+e;>=iOSg1E!|k&Lb%d|$`C7-=2C}`4=I?QsoQ+7?p5RpRiX$k zn2NlPBIhvjCMpO;E#SHiOJT56sZ?C}y{==A?tu&zo`!ry<3vRm$!NeNC=B0?j1Rgl zBFHEgAlcg#w@S2yC3FoKB7=pM0oqHdNf+b0iz>cG8H)ag0+5177loQM5cZQ|qi8A$ z1&QN*E{fu(P*UU^&CjF+aV5649I6b8sbjN^XOCg=*-tCR z%QDpNHb?=c6UIfR>3lrJP3cUlbWJwDfx_UEQp3LGXp$G?({~QM>g?)aJANKSawQ_w zyta2Jz~^8)*+mLj1{&tR%> zI3HrItk3uXsc54k*Ew>AR;O*FqWzmrGK;3AqJ8YxS}IygsAx{Bsc3cxv{W=pMSD;x z+GKn!6>UD4_rtO0gah9RB4-@Y5dLvSA^aGH^Uf_)wA-?`l8QEVhWDkSd5?A>c_mNN zRJ76Lz=h;nDw;QiKd=9|RJ2*4woGO;6Vd1Et?6u1X! zgM2jZ^n6O?_t3AW3lizB(8AFu983kn&IGiU*b*}uZyTWvyV5XqFTtbX7KIj|kVY&L z(soMCLUDdhp74bV+;PeRiZ`2_&rZ&6z1hTK^Z3{dU3buP`(!(*vT(G6F{V~`#H{F! z{#XODM6fnueRm>Q-3Geb=r!~{Os_RL%T^4h=8Am)IjiMUxs$XeW)At0BN5`~5B1VxF6hq)BKE%y~N*RVGR-A#5vWlyQnV;NQd&mM%f^NP3GZC`| zYU`QgmlQ7~Lf>Y@aQBYudyM|?fBtv2c_+g{(-~%>(iWb>QitZg|$(54bi69kb`W(q_fDMgkEinfGKiYg`$6po^lM59n7ibH7lC59dn z*7t-o{|>=$eL6KRu0!!n0QE#O^a*d|2b=+w87(q(k$E72!A;~V&92}^4%Ji*QijF39IH_bC zdkU;J;VBw`=7pI(WYoxsk+=%MrDC*&Jb<@V(W6Wpw@Rtqy2g#wuj=GOZLlQ`5qcaZ zb1kDujr*~ie+dLp*(el>)Gi+BR8VE;d!d)s6th%a)<sy^#Hqb|jreyCfBv$v8B9PB8AvQ@*b! z3{r#=0afXZ(#XYz^7?m+kBq?x=uau&JK)L5ByfiOz zvo*7W*e6^ho41k1B_{t6*-Gu}xgI9|5(YKw4%hpoKf~LpXgtm%xM0>o$| zP_AwXg)`m(17|O3rMNUSHX*X0p~K5q$W$;WfjtjC1w%wC*EZzW9JiKB5?DrbaZH8(deu=8 znyxlM)O3VF&I_RhR&*y63>ke(N)i_023Jz9@u~(}O`y6}<3;|St{ZZEu%w?g2`%K( z)X%h^Z?LKWT>y=6xp`-9#)CF26eDR)q1I)MVv^(ijfVB4$XIisV~m+Xu|`UVuR1#g zD+&)-a=}%1fYIDvbqK}xU2fiLf|nEvNew+sk)~3*4HQX3n{D0zT-lXAw2NTS02)=Qn*>xWfl*I&LmFw-sNe^T|F%}| zf(p^iJA88lsPME{GmM&LU3?E7N65E2walT95>3k-`f#ze%%PSrhsLdD4%s2lGKVa4 z=pmUyBj34}IW!Lg)~8P28#}{cKa3(4%%gr7OgyLWjxeyyjm)9jvbT>pYe6T9sSameve*gi&7vcw@v9I}y#{)B_Xu4lNurDU;a5)0cbS_1{D zt32Xqk!Y4>2a6R8MmxZ`?o6>orP+~G2jNYSS)6^r9C_ab{MA#ZK#YtW|n96fIKtLaop>8X3=Qk_Tl4j6tHj@^;zJ# z&LCne7_h-)=yq=AncbSbeLORa2sOUneRZSW$a5bvLe;nx_tgythTOYLG;;<=uJ_Uz zP5sH#8(E^6C7RulX!cUItZL;?W@?>mb|v<)vo3}P{iGI%mH=VnCA9?$LzA^2Gzr4X zD$0fhq8(mnwK=UeXI|j>`jC-Rx+XYNw65nr7q)kIJ)gyd9LBvC-DMkeA`w}x<4Rz} z2@?~69S8_WViW+~au}W=5Xuo#kAA~SXUUyht5mthGUc~{NoEqrt+NFa*nkKR>o>u5 zp<{Jw{U+9LV*Mr`f!_qT$J*^Tu|2Z&n^?ce1Nu$8Yj5_5CQj6+?tD0)kvAL!J_}eh zrhPx~#!i25+sz)gW^bk6Wa8fMeEz668jK#%Z(`(W+HW$N9Q2#u0sY}i*PjmhQ`fP6 z6YDpz2lW4hqeOi>5)2==EzSxFZPo5b+OxPdx)~(!6Ff%#PIGGh(>`pgC&w)fkgVqqJ^W`cA-GX$g)VVyRvB51Nj!E zyHX*Z(*$98Qt*s?O=Y-wBSRMB!F=Vht}PV_#p8?W>|(H!p1o|uWW z&Vt>wn%G!Pm!O*f57~2l;u%lMKBF?GiK5GDL&BR=^vx_ZSrw{_IEfRwkm5c>iGW@R zd`S*#Hat-Ns`Ez-7VlcA&e$*U_9G_M=&sKx>-%_6afkEk%vgu*+>ESEUVXS94^?BT z6^l7mF;<xLlHK z`R}e$UCCC^oiw_82FOki0!pN9RwU7(q)IB81~$*IPtccr(HDch%dKzo6#FE54oO+E z>%XTeQWEYqGG&`Thvz%|Js%H$>@ix<(`+Go^B;R6I~Tc#65g9HXp-|je6!$6akl+} z@q`zgMHl)Pp2crL?`tK`c}8{09{f;$rg0%;h84#N&2s^@^zw^b zq-&{)d8}lfe@ByV=8CUnE()nGAiGYQ(UjjMaUQRD%5}jkpCadjVr13*sOM z{Fxtl(JW%4#c&u;`D`2ps4c0)Qe-qag#!N3yJz-)f>vab@ECwuD49YLKlZ-i7a)sV z^NdNVu(bIc%Y(|!X;NZipYZkO1L}%Af!Yh25PtBto(6U%brTf|f`kqDiGnyn+F?n!IQyQ;ipL`AlaH!KJs6OVU zx+V+!2#9_IgO=^h*tMa)#X=;)Vyt_6f2ZbGsyq`(BFU#*Wt+Dd<3C{K`Q{yYLDS8L ztSLjzfB6Z&mcRUz?Fh3|3WZ;xn5{s;y#&%NB}WNZH>lw#qqP9$LtSe#>9)4vEwzFD zDxQ}~fdkFnAO_gxyx1Sew%_h|T2l^=STd1Iam}E1IhCGnvSmq^djjx0IRwMn6X4~V zW+Go*AwX5i)fLyjzQHM^%nBHKbO+AM=nO`Roe}@|%fFvKIq+xKkMVnA?aH$+fg8c^ zK7#ja6fVSyGLfg!!O$iQ=R$$*8D$QJR$Xcq;yN@^K`aOMrkDZl_*{LOosljkzoye~vda|cYD zz>J5!H}Hq#m0LcwdPH4_0#(yEa5_nd$e4h^3(){zNG#z$JTDY2_)^@k-$<#*h$(Tm zQ9D2EHvrK(76t!$H$hYSy(A9ekLhx!O;ibtg)}1|2jmhtXT;)r!WB zX;v`p(7|J5)^zwZdiF~WkOQ(mP}`*w@>O-NN#MVQ5>a`Bu$&$P(BR+VY{VfycT;+HmM~-PK5@}r&$2`Gd!l8IiS0Rm|O-1Ftt8lB_UY_rC zdiQ~^w=N?X=~g(38K)CZBdV5?XUy~!^KC9m72BSJ{;q<5U)WFD3v2$4Wv07FVV#5( zHqST{^eK(V0lDPFcL3cwbQ$Fvm!E+nhNMb+;85)m`hKBP4JWAj-V7kRN#vRtlmp2C>Dr+Mo8Lyu_X~G3PyerL-M3*!-GN8G{9-tL%@tT6lZ9s_g`J_3z|B` zx1o3#k_1B+JbnYi4st1y7ksO49H>N=+j$#6Nkk0mr=2<6@=;>NLF{vmErU8W9mS^3 z52K5NEI!EC;b`C;x|!r2Q~k~q(Ukei=O=~~n1(~2E+RJNQOL%lxc4y(#eT0;334Z`d9PS9Ska6nd;sDeUZQj1 zLf7syunmcfE0$aD5}WdY$&_x1{TNGZC9yZaa2>B;Y0YVhbf#*Rw*PBfDs28m^}EtT zC}%YTEC(TB?blqMGkz}Oda*8k*#51zo)v%0Foi27pkWMGzf|Aa3|W~*^`>BnXTy~l z{^Or-{`8~6|Dnj);pkd&cF3mjB=kq)U@?oLDGMTRJPXHc5yfFViF&t?vv14(E0VMC z!SkKW?#c5FhO@`y`5JlJ=J^J}q<4wuJ2~+E13!G_kLST`9)^$0^L>G5ThBM|6tZle z!KB$>w^ko(eyd5%fB8u@*tECNdmX>^YJO`hmCWe5fRPfU8~}Bi`v#axAkY?tRG@B|n!r(R~CRQwjXf2^51L4Qk^2 zrrzV+nO%hZ?(WQcn?j$&Pi0Fj#=}P&=xPD_9X%(kMI&$l;GwnbHiA z0io-bSIMA(q5#!E39yLdM1rdFfjgmjK*w@;;EfLg&*`)qj^&4OiaIw%Hx0+RDXwr+ z?4HN@Gz|S|Fk_*&@JF5(d5lJZzhG=Op2gni(cKhy8r5Ytg&VM)o5Hy%9?(rO^JbH4 z-4u)IY&0IlBN~rl_-etuc;b(S!^oe6RL7^d(M@q%_HMc<#`kqogrmpwQj9y{rI>nG zMk7Hl#R#L3488g2k)x5^xLfU$W7rhNg6fSVY^SgXoa?lo!oCfrW4)bBNPVDXi{0jX zx|2!#@{`W>)c~?Iv(=7}s|CVfB$fc|TqFO9Yhi9GV&VEtR~%S%r;!=g zr||b%xRK-IqfejI_f4F4>o;T%clU060&7Kp{)J4rlKpzDWm0K_kTGmwIGW5o&F2oD zufBEsL{Pr8<>n;Wyw5njV&tzpK`V;xrkpATTTJU@E>M_t(53iT61NkN;hwzdo4f%F=Ry75mv{m0kNuZ z@H(@)1{ouENf^Bm)Zy{A-=4#f*o|_{=_{M}q_pkD)std_XVR z7!M@^#|`g@m#oI-p^royyuaq82XX4pX3;EUJ`2a=Y3S3%gbm{`yu}fDYxWL0BE!Kr9NyCrIT=5m zCvq!K`<_VuMo*+an-8bX6X`sW5BLbdeF1ex%`LiUkn8PSGOYvgUxF#nnnR*>B;2^v z84zGM>q76g3&aVox|K|XyIb0cF2SwuDu=$Imst)J$E0xZ+{&bQy6c@aTa#5|(S>f( zSf*=U2urqTbh18ytmN}j`>YdkEb|+Hb!3Vzn#A476gx2Qk||*83gs_&4u2X4|7X<& z;d?se@!Sin3TzDT>5`l83L|A=MeU_qUEj?_38;TvZ|v}UXen!#vR-HT>dPdWtZraF0@NH5lmj4|ZS%bJRTxyy`_lx7_AX|6@im!-mM)6eh| z7#yC~X?f?Qaqd)|Y8|2z;m{i%ZdEfF+!j>YxpW;1;y0wzUx@|rpNv=zO-Fig*gDcf zS4a=OBR%}iI+i0n{DJYmmGm&0UVHmG;KRjm6!V!E2Ht}5Vd&8b_x;h7O<1h&<=sSj zxHWtCNDsr&VCDz+B0Y=-fj@mL(nBpz`=kdO^5RO;!*D(t&I7+y(!<|j1cELfI1lt{$njSs}MVnB3!ZW;YE8GC(C=3psJ5U%-cb&oq#Ln+T%b=q) zUyzLBD|zSIE1QrqudbfQc12cSv%h(j>_`kW^E7@ptECdfI4AlSTZ#$d?q7U`V) z1&^=(N^%*gUDLYI;m5hP%DB`e=2ft8S{dCy`}gy!9sJY7gdRjue2{DZ{_9M$F`jJM z7)+Ss5N+O1X(ScZCqdZGYez8K2W~l&$#kc&p)BIf2lI~o<}Jwup6}@Z50^Dr>knh9 z?{FcIWwE#V(~JvDn3t&*t5TkcAK&`|553f-iV&_Wr7FgTWrZ@)eYkM8J zLf0sxGp+-(qsRr!yr~XC(TK-x!cti5bSfP-esAd5qkABOMP#v9(@3M}sQAMi%wgjZ$r?XNo8VLVh zvr!Bcg@VM%J{LvvQz$75f#zpZg18EMTLDc5#Wb= zP*hb8pr_>p0Ru?Q$U7&6GK`s+LgS$y7{G2ejIESSjnm9>Z?tvUr7If51=Ja@pr~_> z@k+oxY<|#vW>G*H8~Q@A*Cl5Le`U0>oeEH}0X|A#Ohg}?%7%UD!6CQZ0+EHTZn$)P zYg4xH6-BUg%M?lZnN)q7%luB!#6b6UqN2ub>|jxFl?o^vD`>X&bx*7RIT{9vw%BLEy8#2C|U9kcZ40S)6fvCr&85z9mmw^joWIrd_79!idu4Q?5HhAFJzas4m_B`!-%`cAaOctr7Klhfm)lUrXlao9XQHe=r(4g4Y5E~*?H-C&IA z-5s$zx`U7FK#mC3MXc{m1Z&zre;d7q-iPURK~C~D$ECSuA3)A}`&6AIT@bs5e`(k7 z#^J3wOUQZP&8c)ikd1JX87_x}6u2X{hZ9>_Gfna_EF7MYTr4wz5A+yUAXoygcMQmU zgybt4;WIm$nP%yanyq7E+8`05%u9+Pa5Nv{=1HXrLz8IEKumefRm03r@2ov!0cb(D zUjJj6@Fi;Nxf16TA0$HGCgXT|C(JuW|Mx%td%StC;?mF=jzRfY%?T@G&6^>A(X##( z6B4PGG^^M{NcqT+b;ZQCLapef!UVR#%ruySJ~!`DeAw4s<~`msKmXlr{yi!ICR^rMFx z=eB!5HqaDZp~2AU)Ukp7kZ3wK(C-&p#|G*I8wj1gU1kHhDbTTj92@B2*gz96oCepi zfhJ+FU|tkN5siIL{WuOij^B;OJQ|O&7v97Mx;1+z*+AjgzZV-Q98CQ2BeH?4JngfA zroEfkK;dK_j9p+VM=-i0!RR@@Np9{ZIiE|rQC?!Xl8!~xcIYoWymYUkv<*ZRd?3Gk z@6rL&fniGIsf+5;DiANY^=+LCg?=uFMyumhy^KPK7%O9kNq1N*m*O6sqMPl!^8Ugm z4#RBl%*2^#bgDW;4XalM+964ATdq0QOBZdUOW}{uabEB^tHuqgi7H@-V0XwT^%^7j#XpgtU%*HxU+JpX*0B*@+<1Zem}vu zFVDojo-jxeM%Xcq@JdTc^AL*7yI3fC8dU(g{? z){x%v8br1dktMIxFHKP$&L)=`Z3HOFuG21(2{Kcx4aS3RM~N-S%W9O@4WJ;>_oAW? ztSA;_;|Vo{$G^@^Oqsn9PPRfDW2Is>?P{}6K;s-4DVlq5doU_%o=8#<7+)zv1A%%^ zGbpqu3wt!PgV-lRrki)n;F6I4lKEPn>-hmD{t_lN91bt`Yk!Wfe>DJoMwt?ZmD)!IiSwevM_K`NV`8nJ6rO#-j1c?2j zj0DJf#uu6_0!syGIxSQc66R_xW6U6JxlH#Mz%iGaJkmD?tu0CSRuCuFgb_@ZmuI}t zErSmj2S}4((DX9R$} zf4%PM2u+u}AZnUnkn>U+fi>L;1w%&PijtJac)*obd%U{C)(fb9-FcC}XWE7wAFSyI zLqdy%vhB0|=PPU~Ko>zLTy5Ulm+_zvOU+1HP-u14qJ$LqexqeQX)@MAniylYQf!gd z;p^T`#hStcmRt%w9AGr}*F8e9yz|XFL-0~^si>u=Y0^|iw}HZ%w6ZM;5$rwZ%EpH8 zgQzJUV`3mXsQ&X3hB_VNN+K-{N3+dafGfW=hV~H*8bPPZOqYO$r7-L1VaOn@I~9C^ z@!u|VS^c$Qcc0I$*BPEALn^?GR z(HSUEUDXv&heW$9JJ_r^FuDQ8b$5y#D(wzdc%B9SIIdqE7G0S*&}}Lt+q};UnAIIF zZQ}AN{f3i_f9ICCt-=t_d~e_dRrDSmo2D6&MDsyhAi~a&AwQRD$;oFFsaQ;26V&T^ zupFR@4Kl04u1jE*)f&k+V|qGU+EkVt2v*G2%_Gw)_RFT--#<=Yqv)rw}M zJ}EXYHKr6FL9@Cr1fzT#7bulgz%aX$fiH|iA0MNq(;>kMu41hY7ZS{c1al$5Tu87# z*aYk1q5PJJeU4}Ldo~=$GwTG;Y}{#{nH!NE&&=`69*}1ic%FY9&x}oG{t*5gPa+8Ay?zxi0-ltI+4n} zFmWX?;)IQfz^@1hND>qP-EtV7A(Gk=(~N$@D`&}_T&q;O#&YeqfkkE-sjagG6S#l~ z59>Fij0oZ{qwWe*nJ;9*=d~Z{kK|=QnYFlLz#h1lPXo!Db#CQhzZXQ5K9x zQOG0CrgRua!PFa$ZhP6|*6f}1o6P+C-Ory4CZowC`c14nZTn3ovn%~3ctL;s$`9wG z;oSF}-^BS%+y(vL;4INRjs(NUV~dkgL0@$zlI||D1H;DLmWk$=yFSq_4Em0%d*Y*Q z9F9c^Ya@#H6S$r$q`O4f@???9SYdc~2SlqtoWK^z>_C{gvgqJww_j)wa8@0W7c@m!o|Ymff2S(myj3wz@M6C9SU0wcgyQu@efP1bpN6^P2F9@`x@Zn} zXHOiWK$ zM2U!=Ni3v*JsVyqf8F~SgT=R2>O1x;eEoN}aQhPvN;rl6@cle@D{frKirae`x0!s_&lb19t;g@HS zaPvMp_=Kv6E9vMW_v#LJxzy$RPN$A8((rmXx`?BT{9frI=oRcXUBr#ZjxO@%PyY{f KBST|kQ~?0ZJb}sp literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/otherresearchproduct_10.json.gz deleted file mode 100644 index 20b6a4dba3f02516796283f817134026a083e7a3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8073 zcmV;4A9mm$iwFosWPDx#18;O_WpZ+5b7f(2V`y-4Z)A02bYC$rE^2dcZUF6F&5j#K zlD_X#BpMi?8%tJK@t-sngSI5wn$b^>)Sd-)JTS7Vl4RP&N;R|Nkfs7Z@NIwz>qcM++Xuy-P&{~AWgHS{w5uv}NHzE$ovWZSjaxgk@o#8=N6W`CZ zT8Nla$pd&%{^mgyg5@=C#5r>LvY;82c_Mo!`k5U}$Bg?AQPg*K= zONuyI{Wz9^VxeMpW0S}vO%yv-5tR8Mi0O@Bzn4a&&`PN|aT-O>fQUH`{||qpS3-?5 znn=V4(Dk6{cAM_t493(NjJkbi&}sk6L0&r1JT_ycXS}R2YpI@>xYO;oGA?9F zg~auVxbB(T8oA!6Jw(E0BH_sO2cWBF&S^qrG@YNHJ4ApErV@tfC_OoTcyjl+<+w=Q zmtk;BTFldAsx)cH7{r#T{iiCF@l-X~eXb=(`QKXofI`mR+sC)zv3U6$pPN*#SA}OF zz^`FOWO{%~0o{$qGLo4{NlzMU9p6$g<_5c^C8%4>*~yG+RrY(~dD$=3EP8G2=oPd? z3zJ4kvYd-`i=&b$$Z6fM6EK1_Lrm&_1yhm4p3nYMP&XOKkYi)tq&&(Lwn`otk#nWP z4HCigAyDJJ7YOf=YqT8euiMVO7Sup(j4*O!ZW#%0WG`|Cqod$Bnjl-(_vh%eU* zRL8PWq!##f1g>H13mzvS?LBCRRNb3Y6yb#NiIxG(tyFUcy0CfyGh8fW!WSZf-V(uJ z7%Gr3;~8h^j0GzG3QRri%@(juuU;&KG%Qw31TZ#OK*bD}SQW#2Y>;e55T&ahQQtt% z=9*7X-9Z0Dy!sygjNyM74fuQxvz^{x7b;Eg=^PfD1pfU7^-|5|Aln%zZ=@LLCDI9- ziPZ~Oj-Y;&K6Tg~paZ&_;1CQmNCq@J2PS0%gE*i8jhY|hyEwG%U>6N|1Uo~^fCghR zR;M;#7bLMk+jYh}rJ+Bz`k_IMO5j(62sHxQ4HW1l<7;zm1ch;!frM!|N8>sXnpL>~ zTO!bRYy`OmJ75D`lruR|#`z*X<1>Ht!wg$l;1GaMSd_0x6JElDL@p#12$87^72!`y z{%2|iOvH*XTd@=L$y+7SJEAy?lk8rPbdfGlC8%$Ld*wzR@9RzJdio+BSYz;*y20@&aH6(r~eY8Urq z_BH@DSiJ!Lq6W-(oF=G3LmREM&>LNqb~wN4gDBk32f0G(DlcJo zqJz}bjJyUT#mwh=0!waaw*~g-YE-I&Y?FZJwtU)U*_E0F3xQQfgISFlO`i6Z`=9SI7oGbp2N%?Gz}wX%HuE+4v>7j?rTE32x}(k z4-6twLGE%pxZp^97elFy_M11_TH)N0@g$gn(Gh%x-l2RBZ@vvujl*B&?=_hN5thC* zv>L#IIqoLo(j>v7iB{>{K1q~9o&4jofB3c*)tf;X+P#){8Oq@K;jr&>G4_U`AGSij z-E!N$=e2x)(DsAw4wPYM^*uouhE2ECY;_ScXuG3s&lwIo*FhP&1E_!yWU!qKO}Zu3}*h?kE3@& zfa|ia{AbxwR5xf16-d3!qD#yS5k9AF!^OC`u$G{dbts;RBrvi2gQ+gc91?>F+kJuR~1A@FT4HHxdR1Op1K3e|YzyJ4l0lMsGX^c7q`)D{lgi9(DlzPVI zYMw^4IHwGiW?rNizla8DW_&f}{z!8WD7py7wVQ}>P-*NEzfh0stcg5zfcXqQn0WoDpap~G&= zF)2ZU8jsEZZ09MB!eC?2OoPZqKNs@O4lrX;h87@PhD1{h9xhNWA;M6_M4N&}YBdqD z$cUsSqmbAgg-9qfY9N6eaOJvm5NNRG0-4YL8&RIvyQaxmuHRsB(JgX=eZNCs9XwdxYc8rYfQs_XTYH zLm1x)mM=-}4q5qO%lZJ$r;5#C=Pd&{hp7>=jG?Dzp6h=35GN>pcT_6#W8@klRWZ&H zTLI28ZFY``3MdCCp(X{&=T|lYIJ7hkXd^NRIc7iv=g6+2DHy>cTJ2y2_Z29>8l|OB ztYR0U0sUJtG91_yOo~`&cGz{D!LOLMV@eQB{Q0~8m?#zg{M~;xSPx&?XH-3VL4>xQ zPX10j2?QK);{@IveZ*P=tX2|Iw41rh!20Hmc&^jRTk$!z?XU-GO)cOgnBWIi0gtM3 zv`iVpjxvP**VdX8d?k<>J3@KiQV1oZkpPZWS2kt@p>!E|oUo~!Oc_`}5z$m!Q}6~T zfefY$hyvS`1Kh~BbBSvo*eU^&j=)80=HP@nPwXFvy`i=zX=WN|f~p`$k#en4=-0|C zqBARd023XZ!P$HOt5Vha++5`J%NVBhzop2Sa%RqROOvSpO?9ipU}tqcS80AD*I`}A zjoeIWM4^qqLp0bF_62B*2vMmN1CGOJlH4>QDb^lKGAfyIDmuN)$b(^p1u0stZ6xMq z8BFPT5@aJLE=OzftWn6fk3x2uu2(y>UmJzA4uBd>tI@Q#ji!-Lbx3hcQ_pM+ufk!@oqPu8yl}g({@$gUo@?EEi`TDbi2J9 zMbmok1WjvgxwEr@rrq$Jowtmpy*^U&9s+5nJdGL*4zIZ9S$$vWD0>#X{C>SdtZ`*b zt;U|#v8RWHH?0OP_F>`NfSpWQ15W!Px2=HFKTpYbmO13Be%R8>{cCo@CXw=aWWDK9 zoU_yr+C=haD1jPr+AmeOI^y&(55C5;MjU<-pJnieuA)ZLFqs$;jTwhs+N5D&2iUg= zf+Tsoj;%<lD4JP9duf>3J(G{VP4cvcZz?sZ< z+;w{FSGo5yQ3+^ig*GIm)pNRK5q#NX*=urUQ=`2ul|=>5FaytFpK&_^yj4RO2;R|v z1s#}>`*q8%&Jm!Rpp)Oq5uNDhGR=0tjBa5qM5N{xvIDU+8NGo^=Ws`#N@#Pa8Zwt`~V&>ca@r+aszB1iQ zIL9gl9%o?+2)dIUSU?TfVeh)cNkwvp57Z}n#ew#dC_2?db&FykeT5thn=3|!E_*a)#?%g3}88wvSVp`G^OCs7^{SR9YZ@)_+Nq-?|6884kU^r z(7kieTtOhPh1TeNX@L+j?bcz3gRx9zhr^!J209C77XyxH+Tcu(3ouB~Uuwh@R1)YN z44gqWT;O7-?X-Y6>o`fGW&YtqhIv@=o)gdszn50Qe5GNstg-m8-*#N21#PTGIEh|e z5IHQ5k-%A?Vo{NQF6Jfs`MWK26}jWeT|8AQNp*oWM0Sr3B{IaMnAy zR9AbHYdEWhv-S_pYJCWo!)oJG!&x7ssCL0w9k<`Q49;qKZfoELT|e-GetXD0FMuf| zM8F4buRZMTfU|a1-xD}%fZ$YXi11OXHF5_|tJl3UoYitiU?_X;8@wEL89i?q&hp&B z@FJXLiR+zt-BH^cwFWl}XPv;wmFwmsWq5;qtkbv>1}^_u!GFrqSH`p6S5)R5#(+E1 zaMYwJ^&3>)+ReWg3qcj0xeGQ?avM?RHpWxZ2^{!qIqsVr?+c7*gr=ev-rfoCwhV2k zt6Fq>A-b(F>E>EEVy9Zh!Rp7kRE9m&xH;va(&81XWQ~KqkFvfx4tiS!X%TdgVg*Ks zh8hN&eF~R15fpeYHP*}JvZ?3cLgD<4ZK z7$;8mV!md3a#~mN?;UDB*J1K%B;-9tLdYMs=SWCx(`zK;LlxCdB*g2rFGWIHT|Q_J zT7HkWL%-h}`+V3BT4CUIdgEbxS2CWR)%OMoDU;vy@FGpC>s<>8@f@#pL&|27m*MOIqG%tWIQ(cB=VY}DzF)OO;ioq~!^?Rc4`#qnB!LHNWuIhVodK)%f_-~KG zAPh&XA)J=3d3fu!y*C<_VH-Jb`ztnY+u6<1X3uW!sN>!6*-gaaDaPo&m#%&^2W9+% zRK9#pQ>6HWEh~EP3&Sy-fU5lz%gWbs@I<|QZXdm=s0{ZpwjBbRCoyNIJi&Be)%Xng zV;E(y5vn1pdhmLY8Xl<-R4_%B-a=Tj8mfW6e7+fuyeNL~dhyRcRWt1BGce6@rK{q1 zHSg`L(#!Rto!a(!K?U1w$+Qf`&zB)@08w)@09L5zVrKcKWehc_Btc<@fb3%7TN7w+7wCp+Zqmcp~!wNpag-mLkW zF7DTi1bjWm_Unf=zMdcZH)=k=Vx(p)`kL$$q&AnKy z>r|S1ExoPY_RALV+T#F%Bo+E4^M6gB~5oX&iqFQi7KXv0kmp(-ft zrHliNwTr0%KY@*?O@B!NDJeiI9HL&)ToIrWTLQ@_%>Jli+|}_YS~8wa=#DBh{wamm zwOR8p58RV1X2i9(tE#j)!WQmrx2@xW3 zHcEUJQaKBF3zRJck0@Ot5cP>h(PL<6%Nbf*EekN1DNsgXQ8x)_urbLBn(>$HZo(s* zeDqT|^_y z0x`NQ0Z7y;RiMdF78TZeOwcdjw8Zh#SFndiD}2?}5^RBJWO z=^TTeYMb1&i0W%4SWU^O;U|sB}b`{YJ&r(+cY-R%$5(Pi35cDNp7xC~{h^hW&QEMuw- z?0rh`ZSiEhLS?;Wd z48BUMz&y%risN12bB)*jiD$k@&P8VGF0sKyu;I0S&nSlbXEtc|ohRs#0FG*U_v!6Z zq+j_MJ{ZkuCFc3u#yU5bgkLX{9AvSht0B=#1GZ*eZ01pEa=RF2bG^xg1{yCI;Y$-M zlcBux`3VjA#7JJpAATnwVZU}c*Y`GRylemPF7kTrIo?$ppBnG_P(`&9@9MYROYtt= z@Av&~;CW-W@4|+-+v$b44c*N##axw%BseC0! z)<6d>`f4+WJ0eq&2w@iFF50y$a1FMg__9e}9?0O>^b~YVtBHsOL1^eE$5UXFS;Xoo z`OfoEJM1%cju{UviU)^ukUqg8AdEQ3+Lqd;Fl8p6uO)y+3HNXSp@oBTm;8Fd%_4he z-UBG23SfI8tSHKsFIa*oq;zH7)*T$y4i| zZFYEi20j1$-Jc)Y0HhVLoIS=GM-7IM6b1lJWFY70O-IxSh64v~i??x>h^WcLe2Tdd zZH!>d4UBAFvL2hk!Fo#AJ1j$}v?{C4KrF`u$yp~qVlvWVF&2Amf|_D)d4y!lfN4<+ zJX^9uXwo)*RKN@{9B7G>dWPJhfS@PvNa+$6uF{(w`yE!=W{U$E>Uu6Tp0=#ZAEk;A z$7{+J`oL7CxIYXsm<<}Hzw_n@!YhaCFt6jH$eOGISyQgmTmaaL=XKfPpyhbKI%1_} zV%s>~a*Bv&+tdxm*&bx9tTKYi;<|+8GqJCYmIs*WGXzrsO2!U}xzx-N=C_Q-O@egh z78x0Xy#T)Aa0^LW0cjb3ou$T1QnQBUpgP4NT>!fDi7M}e9u zDr8bjIg1c+E~JaOF3nV>AR{ouk;*{9Y{-bT(IS%Bd{fvnO5l4gqKLwYn`JOnBscs$ zhkaZn=s2=H1NDf5U{1jH(zgrP7412)Oth#zzE~=Pmj#=GEvz$WzAFT4P&<)p1gl1{ z_7TBq4fY+us*O*LV11~f+J#{ChP}aM2v%nx{6Wue4@6J2g0VmB_+yuMdcF3bGYC9) z2ZFV;`u-wV*9tx6wVj^#W3iGLpOWc4Dd zb_7rO%jeN17)Rw_6R-Lh^E6SWq$^$2qT>AJpRNvDZ7}eufKx2=cAvgX{*8jJ%O3NG zSqBx4+G+={7kljWF}ZyWKlZLLxm)OkNbKRbMs?o%%|R8EtKrmrmE0$3OwZ9%N8z86 zWi7aOlEbmzGOCJ1OH?lu|Sa4{wogd}Wu?E}0?L@=u%U=4?>$nGF~K)^wXwX8#z z8RW4^#3lw*z;V^-WHHIjN#(Hn0!}5N8Nuc`;dq@!*gG-nko6j7Nf0GFycD4%PUUch zS-Xrg;kbXK%Y|B{lPQitG!xate@mp&T)bqN-${}hnfC;Nm@p-<1deQN$9By`1jkQb z60bqk5N_rr5L@zqkUm6jO3cjyL)FVv*#LnfYXf6m+_Y%+*l=7o5CB=^(}>ZjYb3*U*2Bc z^|gno+RFy!3(hXG=J;lDJ$U1L4;Bl%RQtI`;BC9DUtQq!-ykiT5f*XXGKxldIWQLI zTdxI@4V>eh=N#Y3uJ+RBB2YXh4~B_tm_S+=P=FCbja7SVhu4Nv!=ChgQnMGceI zFzG(Rq+V~|Vba?8)G+CXDym&DY2WQ!nxLt}JwFJ;VXx~C`opo$gF%n?!?r){^m#AX z71(lT^*w<}tLbbzqb}xK>R%Hk?cY>--6}nA873VLyBAT?4rSNvp1B@oXlmauO8OAb zL4=ZG&LRTY4%X5b<&R1rEU$89ob-J~!royb9&+T`)ChvzleX%tcFpg@dQinzPVFkq z(dPgrlGn1_Gg)qhS#qgrRuL%Uu<|TcKi~)YhCN0z4}Xjp{DGGg;OXjn%;`d@TxPIk zOZ6*~Y}Lrh`=|=PJWaNXt9N}6`sF!U%3WxPDkz|Z X(VqLm>SbW>{9*qOJ`$9O{P6$)?e=jl diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..ea9e212bd4d8e9090360692f6338774c360e065f GIT binary patch literal 8300 zcmV-yAd}x8iwFpOKcHR!18;O_WpZ+5b7f(2V`y-4Z)A02bYC$rE^2dcZUF6FOOG5k zl0NVL3Zj7ly0KN+nf0zR7K64V+w$tsBgwPCjt2%alT}4KE14!Ut5xzEuy=mI9_G4x z5cXhr00Rqn_cF)ntA2%l!~TGc2lg(5phuEO?76{gUNyGj1TgX_&w2T zDPm4J58y-fn+K^>F*Y1T+!zV19GI1ni$v*k5@==28IQi2Xptx*Q>9l>oU+DzA+D0y z1aq+vRB;%KbW*i_LT#HePhR6CxxW3jbpY+ASpw`5sayz?@?!T(=3;IvT^@fv)JM6o>7rMhyr!(k}hTU*5823Bwv@g2dz#Re+O3Rsyd2|~p z_`|_9i~kY0qM}FyAegCE3#j6ogRjI2XiAj=?|Szulp|tMebDd*i4mP@T$bP?Ro94`^fE0yz!(raz^7J zmJp92w{07nQ(V@S?3km6Q7jmUS z;)X67xZb2YM#2^%<;eAWpsOa~G@x>t&d$ypB0vXo3Egy*o*mymxqaMmT%_)c zFgPYH=2<#dnlxkzV$0S3V-?DHu3GFa*OH_BZ=C;#Le9V2xQLB+q56HjxV4R6{(pN+wI52Tx34tS;l7@jf3&pX!;b)SvJdr z5nI|XD%sv{_4Qk{2w9wh5u_1fGXF~$iX`@8^q+#d$v}o2EBiX*QLeCc^1z6kDIIQ* zSfyUVpQ5?vFsI1!ZL67qt~LYV$SGx~RVvPZIR7ujWSokb2D{KmaWA9@qY)-nWKC@> z-e;M?Y1go%9F>Ko83X#-H@sHVm(8> zFB@4JfnP`98v4HEaT?OvgH}lOwMk78P8gqQ8Nk@eG-sd-=TBjTi=|BYQbf>NDj0M_ z1rkO)<1AaSK*e8zsi(Es66Wdir%NFXixm?A^bICZv4AO7#qb>~B%2XL+4)bXZ=h$1 z<}*|`&_5BM{{Vl+@V}e}Oiy66(-&+)Whtg6FxjN=@7Ji8YLS3!7ofb6VxX5uCoCq; zpTcwm<)iG0!)^f`(A*5WV3*vqhL!;>#$qf^ zb-*e}VuiNpoOen?d+hwj7F8;NUo9fk2&gwupqGrVjkO76#%=}@hGBxnbtW{ca{-n_ zpzl}-at&6%2Dm5}a;A*)dHjek{PQ0dSj!T-0DQu%d`+705fW0ll$0SvrY=>4KP~y6 zs0A<)3&Lo{Mi>WL9?J!{japHnc(6W+qJe}KDujf8Ko=4Y6|n%gqIbx1nYT@d;|g84uA+?fdf>Kpj)V2+?V;+ z0My|8DexCHV8P=oMHM1KfGnUMRAJ$;`x zCz&5fGZ*3Su?esijp4n9;k8>!9zdVcVsluSTG(U_+Ai#~yoRS&UxOYL3aRV7gw=@- zGSe{f8jKVRpX(V+xuIPa*sZHknGW(r0`8Mj@i@(Yk#EHQ4Rd~~rJtp=mA9K)DrmR2 zwyFS{)wJWp96nlS7zoB_Ih&_drZ{in8|%Hw9SToG3>KE$2rwlqvRJ0Dkv$IPV9;l( z4Ry<~?Trs`ltFm`+w$mvC}oH(%Gl(C4(g61%u1%DZH-O2T4l*5~!!Z~7JdB(O>T!hAzC4;1yR#{?G zQl(HQ|M>0Sf76KS<$x32LC3ocobdc`JoLGkdgIU!JE7n0xLx1#I=(;Z`aypOII*+% zo`4hMw%ciU`Uo|2-ARAojK{s}fD`?Z(;IYO9yoCsJ+B#@==Z$-MR39r*L~!65uE6B zUoANCAuvCL9aD?lzB~AJN1U{Y%~G6vS;6aS@o!Uy+$HndW${ey4FVitJVnSDqHAX ztA$5GJ{A(U@^?5`pmeP>RZ7)ZQJ`$Ef35Z&NFPcA+wPly>#D8dXVp-ivutOur$CX2 zXB$o5if6bHd&MZx@6dCSu9%v#NKQoqGVH^8pbiz?K$mHZ_-)((efuH3tq@UNj(__f z>(Q*nC4)JSfh}TMMFJD&0#*uhN{-|b*66I*={MNXUTMPBv7_78VNRtd>=7sfV#6>D zQ&b344in%yTK?a^|MzzRE8IFO2J2|pJ%lUD6I6P}5|v~T&5mr&4fBPF8SK=~P@Td| zU_q)@&SG0B(8^QHX>xqZR_;rwqE5 z&N)()!6)EHL_=zhXvCB_2c{C{e@3YiVL1bRL%7Rf_sM|b!WpHCzz_+G%Y{J+aM+ox zo(QT9cGiJ_oFVoSE91<-s@K z*74}b%GH{jLsigAsk!~fpF|z$?Gg4nAF2p`+!e6!521f2n7<;qKW5c=nDq{v%@s>v zH;KBWjk{_@>_%#;d$`R+ekY=9qa8Wqn!5TRYCS9}wX0|6V{IE8OVAF$2{i&cb_ z_2%v}u)aklp6hgqT6~UmJM5lXQw!J$rg$zZ;89(UmMK$MQHJpU+FFy6uY^rwLn!YX z3K3{D62P(g%*Ko$a4*N4Q#O~gIRgtQA{vTo3f=-Gkinb*QDB=2Tpam!CUNcqTP0x7 z5hiKP9PCh&)c%2x9cp`;<)(oKs0xA%Dc354cCEZ3+Ox8IFwoH%oR0^vDs`34)a1`{{0Ehgg-@AiOl(LJY-jog1= zz?s~W-FF7;H-#@YRVip`jW#5uGjRG<7W~*|`Db#iQ>BAG?4?h%S9};N~k}Mpwy}B2tM35J4<$Mqi-PndS?N9_4FI z>n-M5a08wFI+NP+3x*_5Q_P}b#IV{dhk>?&SSWpaW2Pj0dig-dD^JvenEAGS+#^+; zuMPJKT(U}m+gTU_f-aT^7TyDP*b6&xT9e%2J@${R-|e(I-uNgNK+HiRD+4(SU;{gg z4sq2E#R9pL%w@2~&cm5N_k4>jxRxk&gikWlVj|3f-0zM{EF>i-BM>dTU>8uDFLgRc z?E>`^M4E4<6e<|J`$pe@+E|U? z7Ja%Pa+n??fulghq9*@B%q#ZGcYkI+2V)S=jzevoJ3uR#?>56Uys+n!I;XIkL|k16 z$^vSk>5m0yxLzF-td2Jft?m-|)Mf-fVR70Bm}4Ic7s!-S;X3MP^@zYdLdN10EiOo) zhM-pr+rsTaGFFMK^lH7P(*QZJmnbk}#aV^I|$+F9ze6gFI@JbYk~iMAz9P#>C7f|eU_ z?Ty?FY&^>E9=KKs6MtMTs_nvo9|@mXA4y&G)?If|Ii4@cT5Xfk=1dZMgv;bd5Ma2& z0(e8L?6fMHuNq(;fW<1FsY1I0)~0s4D?iLoSPsY@6@7SV@~><6s87hDb=C-iDI31O zX1-7PqG9}-V0?8k_G;*S&@r$tNE@vE{lMBg@4`LWW@Xf1?eCxy8 z+bHX+AJVv~f~@z{Y8w+6F%U0$^SJ)VE@%L{%J5y)n2uPWn?bR|kbn|oSli%jYI;t#cw3j~ zmFCE4JUlNEPjnlv*{{J9YbQAs^b?1BIbQQMIn67@VXTJFd;Jh;gIc|2)XEp=S85Q$ z2C99J7=K-;wstZ4idU&CaQE*TboZS{cge@U=jd*;PHoWLcU4q7(OqxQy%gQ;^!cbe z>i7fR4gKL@>htk1=!AjS8%)RDT_GHI7T+6mw+fj!z?;jRzIQEj*K@qiD+=LQr{^`J zyWP&%y@>A8i}PNu2;umOUrD|z@x3pZElN-~-^Ue*Zf9J2^dR{Mh?5JzT|)-CP`aPcIJSyW-|$toLX?N3;5# z@8=qG)a>WpRZ;ES&vnM*OZRgyMX(Kzf32V&^^1D}V%uKnDu;(M~6 z8@FBf?|@#S8c#Z7*gsu!J2&WhFZE%pZREV}v%224-JGS(?&jP{&wItYIT4E|7_{Y1 zcK(w&sGg9@_9SvwO9DeBqxHtA(OjCB_uJH}%hCoyNI zJjJlG^(V55WawqE@mf}1^x(xF%X*+fP{9mYdIJw{)L+Zu&%WP`Pg7<;c(H&uAFBm6 z^$D2fxYktpyK0GY9=uo>oO@~sl6#19t4H=BsjatyIS=+d2F@Ei#(fuz!J;{VxIc@7 zwb#Em*yj9j6;5jS^gEZ$QT+W7PYo`O4?X8^e}xQV`UmnHwHQ7lq->ab z3{-*vkg0DQb@(}BbeIxJicXMRnFPN3^s_;9_ze?uOa0J+_tUm8?|aqW4;xqF>gJ* z_4|g4dwtgO3dfkH{43G8yx#|xH@QCc+~wUE_r~S@u8L}>%X`qdG#Er@G!^}R*X<8R zt{3+BaL^MW4+g#ny4{}G>GIxHe1BZto52@G*YbIHo#A-=N`pa+-bgTr?(Sd^y;tk> z4mrD_aGQ7SaN;*NOTM5t`~~AST+avo;z1~`=koqkCCL@zYGBqE^mPBqC=|OK-fycf z`b|ZlxRq&@G-25HVzzF=uwmP2Ig4N2H?HZ4b+xcyj>ve*vSMdcx5%5sh;Nfw$ z@P)n*R-0l&wDG}vK_Sy+9AIPw3@iN+EJSVSbb3CVp0kHd)N`sUpA*NLKr#v=hpQM@ zbv%kzjAt{tv=5bkOu-Or#AS>j{5TJIoQJ5cA_vpk^mUS340wUnaJ^cKTTn!fplHId zAg7oZw0(oxPvpOdaaL@zR7+ch2$47&(B4f~ZioOC8je*~L zRmgF6xE>Osx)!VSMzjcN_rW#{%TcLjty0Hu$CA!7n^-U#I(~_^y7suo!q<&Dcc-tVe+*uFsaFtqtdH9Zt#QrWZT^IL)nJ>~ak(;_pY;YMGVy)dr z6mjPh8{*{FWAsP>M>T!>_~t3nuY3&lfabIop$4ZA8QRxWwgk(mLQ>_CKI#E%Dg(A= zU2N2PX$rd-=3~9hga#Tf7~#v}wOd1Z>$4N;@`;hWi5Brj@Wf%`a(?$NeKdI2yN`E~ z*K^PDuEzK@c-Olss-1Y(uo$1TOTfE+B|>z~dmb+gt~K${uZKiE%=A(&>8-^E&3Nl) z(mEEk<_&Dbw-@5u3Pu$}knt%@o^>zp)JCw5>+zF=D?yMAVD&c2{_21gVhR?eq4>5J zLXy{_ha1dDh}0BT*^ojmb_;75LD*6~7p6rXmr-C@5UCYcYFUY)KDpr$v4s zgJacm&@ruMA{O?6N}dv4)n*p4dP2VQqSp@lM4e%LYK!8*CLN?tum}hv4)VICwgKmv zDaLCBpiy{a>_BMYpu#1;9v;1nwO+IUil_ouo(L<7s^$xpAPOluG&3A=LG0lrCc$AR zRMrM=mKdrT<84#nif?U;zghCsx@Vgmo<4$>fBEju_w9?!HL#pL#1cm>hL9A#h@Q$o zCg@E^)Cjr*8*YoYah8gx%|tTCNY?hzbScglvU$Y@YylhVIbrWG4WZJiyf_1~91|pG zo&1Q&NQ=c-?6nPQimeq-#$&v4i(25`k{v>ow(_GAW`OQMO%#GXpD_pov zZ@%q!SY?|n4rHk5L}=V?S(iUb6(Nonlqf#-33800V;R7`&t)e(eOb=P5D&qa~7 zSq-wL$o7Q*uolnjv%^ux@qTl}D$T^Yakv!}5zn@%8}_q1%2`=w1m(qf3G?S-U+XOo zFfMxtrUI0V9TIb?nj?%x9!=W>=_D2znS#9lzG8PvNm~JF8Gn@rGEXzJhUTC;#Ub4l zyD+8-NDNqrz+OBP6gM5T0)MnI^m|yhs1Cx781mdu9D`X5#$ij0@uGTyu!^4MDrzab zymM`Kv3oFS3n;jP!F(8l?ni-|E6QZDg*YOJxDe9C$mC|OGLR7%;z;G7VBTd!+GrBV zZN4e(Sqq2BnTR5auV7ZeT#?-HcLM9US`-Fkdk*Rm2f>_x?WMB|*c7cf5a66z_3_0@ z5xgwe9Bg3|@BB?6Sfj>?+#pyDg0+tbR%f*D2v%c!8U*WI71b^TYcL*+E<><-BjJw* zes?4Wq7zL0anGN+yf+whN4-(txjPW7oyGST!MfHfWnR}AcrW*rvaR&Ib_B~C>_V_Q zulSj=YXevNfLGnjxW-6m_jr2#G^#fQDg4>@(Iyy2`CkyP`Va%xQRuiUUDTrN{Mnzb z4qI(7@QHw3EVOo)P9}eb^?uqb#O$^wiNyQR%7?;$8@G`whRshLX2+S_&5f{UloWNjP}l;IK&L%z)&v2q#N9Td8!pCVf{=s_uYG_w zi3nyj1gv3q71s*F~POX`m5-r*c-bC>J{v&dg~s9JxSeEANmmLM_8K=zDB= zU*0#}gmLp4a)>+-$K0oXNiVI@UK02QyJrMlcz7d$T*B})QOxnPt`v{YIgMEid8{^Z!r0=SzcEO}Ww|8kQrylqGAPmQY zzCRj{r#=rx13nD9{ zL9jd07F{&1`F)rV>iEj3ouxVY9Kb~KMwWXf%dId=Ayv(40%aU#p5^(EctGE<&!gnSO5_Vz@2ZbAMPf4eXsi?Ee9U`?Xtd`~U#n?eH=H literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/publication_10.json.gz deleted file mode 100644 index 257e0db3a47f976dff3deeade640e652f9f2ac60..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5143 zcmV+y6zJ<8iwFo|V|-o!18{X>Y-wX*bZKvHUokK)YIARH0PS7da@)9;eV?yD>0#=e zDqRTPMEN|)vK%XsZI$h$yHD3t3M3&5Hbv-wq#dUx5A!+m9rHRLH57hJ?g zp5RgRNP_z#K^Ht?F&9tpdMX-{Io(A#@@8~S(+~S0jgLy*{Zx0fl*zyNjMTSp%?9X= zlLb^4joF+=F_|wKN7!;K&9X2ydZrGAmYOW>KdJY5y6Da*3F6t&kp}-9%X4~D)9u;5 zr|Wjl^^Go6qhT4E4VmDA-LjAbBly7YjXhug52zI$1k{6unF>CKPyErir%#XZ!QB_v=|M#XAl`<$({XqK|kY|Q^0KP~7p z&9(gXTT_|yZssf-P?|EI=9ZJ@yaBCdw=}(l@eOv4uy&;BZKr4avr$OLf`rkOi#b_& z!)R&F4YrR={Ro?FtBd6u^b1I5ZSgy(mhbY~|v=&$vh}e+oSzujny-WKk21DG5W0hHd!5@|Q9xN<>v2QlL8T z!HeX*Np>yJFohzpyn@r-)?CqbOjy`N?-QTJGjv{2RZS{YN%5s7eJ@R8{z#!zgPCAa z4DP8^RZXgtO3E*F8GGq^#~+*Mg0cYh+gFuB^sVfFMMBV`Rg}13MF9-o+|C{QR3c#V_6v^o*xR$11mqQmsU3IfW`Flsz9}_l{J3 z?opp~?h`s&LmIUY>g^w{WO8wU{lVjx5KhP&0AhV|`ejooK zCsQ%Z?lV`Btp>8cJr3hPbCWooA!jq8v_Z1DP}W2qd{v}V5-H)zSvU9@n3uELSzHQ* zr3(0i!elJCqRYCL$t;X==LD?{(8RsA zb&R&wQ3kL_&s%A(YMw7A!;7_?T`p*d`soYb$wv02&vWiSGre39_*ORg=GA2XbpfDf9`;b3|=&IUqYq(3Zk%&M-M7eGIU)nxrsYySNzsT_%R%>a; zgfax&D^~A~4dm#xfg?4wc#Wxr;If;r@Yb8bm`KT7F2o8~y8f$|2wD86;&+jHRqB`P z*E=rPzJV-%u6@fGDr*$VJrN8%wx0rlPPc^})5?Fj3K_Ew-a& zzro+Kg`zaEEB}1pkc~z0MjvwXkVfT|-d;^|Rz% zsQ`@I4#s;^0ln9v0?PctqT_lE?=PedME#GU>qHF7aZ=H7vc{BTGCL#v zYvo`BR+v!R+nb~mTk5g*q8hVO_Q7p`x8e=zEn2%u!g~3_~)>CLSZegwEQ<$O2 z@d#aG2Ki&5MoP0|69LX>kUtSMS)+%ri z@CXJyMKt^E;ZO=iI|w3hAI+Rn@ZHoDf5TCfUg0UNfpL8c{h+P6 z8aAkb9o^|V8g&)NP(O>_iDS@uN2b#@4SQdX!Fd&mVI@2Dq8LuO@S8A7&cYHe&7Vrd zR0j>g2E0)@gpog&T2jda$%Xe^w*3Vc zE){X9rA#Q*NYR}MhE*+RTs%MvG|@kK04t7C&+@rclgpNUQ`@s?VsN_m&^jLil2p29 z@mOu0Ote(BpC9-{gy2oo4}C!@ik8on7Lt;snmsv20)860%|{U>LFJIq^10G=QqfY; zOP)Z{K#B&c*EDF820BaTBt%0wEs_UrAUmnacqB--*-c=QGAG_LZ305zxxl0lpQ2xf z7p)TwYxoHD!OK0+0QFd$@SM^%d_^N4xp+ypN$! zufK-(8ILBD(Z%5M<_7&T8jnZ)^K10)Fqi*_eEcxFxK>sC8stxo3QDgxAi`(-fkYlm z5Jc5Q=0|#LW#)T%7sUuEM(EEnLeIc)Z;BaGV`Lqfq1`-xW1TM2u&uRp)3h7A(nTiz z4qIeuX;(?fqI6tQWRW6^YExG)l0~{^K2H{5@X)qFv4g2Wd)Rec+xKi2QoL^O<%1|M zyj~-+C_NZS?iM7T$+MA0TeI65?mfQ>R%B7F$s&0G#BSTp-dvIXYUfdk?9_`Zny{~5 zdAgCmAKua`yt;fY;nhO8aTss(3yh|cnYfE8YKq4Sox{Qs&EF3yKI6gMC$pf5uGl>h z6>ehrTm{-Q;l{4KkwS|VT3jzbxQQ0`(Yqk=U~Sv7TN79YE@t2!&?w?_Sb;|rO<-XI z9t4d-KY@uiy2;LJqX7+q*2(g8!C)qM$UM{+@l3#UR)&B=1FL&HhSeN8hn08br$+{! zC5d5qNHRqE^2^gL4bk<2dJ>108U}p62xO^Zg)F}<8&k+~ZII>R_{+2y^Pau3mAp5<{Q(VuUQV(39%i+hj+R?+jM=;e;KFW{(60I zdaOI0^TBZ&oPvQ{SXcOb{VaJ`d|t@Sl;NKqO89Iexae-X$pv6}82p#i~qH``5Gk%)|pp~28W6Hu- z#&Y;5{Z=s+pm-*Day!fS!)GiBlqmo=?W5~C@o~p$$^RQUZSjSwI7n~Ip4254wP7um z!H>>IgR4s!{pj-DZ>m~fgSI&5FyT&jKBku1$_Y{s0Lp691tkoquM$X>h^vdv`<1k|soy#6$9qO*#IvsDeb z%z;4q3A?+GW}J(#iN@6nrB{`dVh4igpO`!xP~`;PuVdSl0|EO6MDmEF9IAZ7DlQKK zJHzG*)K;K&y?pj2sC`AigwrWIthW*&_vRO_Ei*#78|IdVTweMb<@@L5WFd{}-}hkB z9&;D!fePdj85t&I4E?R|{vF*4o-C02gwFfplTqvD@)Vt?9FfAU-;l{D+`2Zn^)akq z?;qve=9BE_>|C0v)xR9J9t12*gIAq1$d{X87^#Zs z*oMU@qTna*m#@c=&umu?RoL1K~e%+ZZz*sX%wg4Awfeic9D0kf}wm`83b*ihE z*@Eu2;5as>mQFpVt9c%b3>MLdsd<*~IMmu9IL`Lh>%$gkI_?lpm+W=-PSFGPGvl4F zPw2IKU43tQ;PviLSmvi*^uVu46J5`Ea>q!z;xF*}N>T|?|C$^~5sVqA`E5nRZ|ffi znFHy8Ux;}@GQ*xJ3$sR_yPid4WEXW(#<||BJSa@6KI)YDzD)FH)L3U9`46# z)%_Q4Qy-iS&qss)SngL>Rr(rOe;*b^PjQM!q<%1h;VuLBRV99*KyDx`}-m+MtmFQd!7Z6RG; z!=sjNldf%8MvvN#Ni5HCsfTrahrQ_AU#}6mY&e}z%q2NxXm(fO%k?wlowuKx?QYlF z8(;3d+V=Bhe(HrU4`f)AKwfQo&S0#iK?$9f3DrAiGw}m)W}l*fpn*ufHqoh^zlVP1 zI@L-_1*^^E4}_jio_qqT1g(T$&z~y$(n07z(D{sc_wzJ*3@qY#h>oj4WGYG3*eK~6 zI|I;bH>&awCNxZHd`Dcts_j)KRIXJmx`%P{V}ODRrXNoS;{q3*cSz? zzs+G~6IeeMl*s9SihAF17RtZnswAMTNRB8z_0h>i|0vxNfA;YVQE*>j;ssR3w~vwq z6;m>2!IQ$m-9{0P3YZHt3M7N2p>Dae5$EUVu6LSZ6eCz8qg2%Dl zxz;fF1G{V3f5ZCUZC%&&mZO03`kD03q0f752Y2@c%6s_LLZ2`5Q!k+W4620&O>`sZ z0~hF~#6pXdHJmq&u3v41O1vpxno8`G%ht(5?7TL?;VJi%hQ(8j662cgvzK6 z?@N_=!m7GLerNDpfxZg#t(S}31bwsFaDG+jT;5+w0>tE|Xo|p$7Vs`juONjkIGYoN z?7poASIBN{kX?0!=#RWYbev*rdE9MBnXiIz6^#3Xud@99;#mQ@KehMxQFmEp{(S?# zU7Fm^_^s0|H#qkkyH`NCj*XWPZZ?1@3|C>ewW+I@Fk+f(Xt?V+ zzR`0v&34@B%NXwV*XxDh-nHqMMMx9DgFM8O)7I1lmUT4hUBhWuGEUdtAg=ye(KMFX zsTYQOPQDU(EWU~Qp)daDzn@BcQzp>={jaALe6x^05XC(wP!Qfk=Om69DRnKMtD2Ds@Tmoe|r$0NnKJR}!Z62Ps5UOI1y(MC8uiXdJfPc|b(K z|UbQaxupRGE+?Zs(6&)JNo{E%!-fR1y6a@?;*%c z1osm!R##2FF(XnjJpP~2?QFc+8JHb@Sx^Csl`tj&x=5$w%dNJ$x*bu0vxvegT`7MsQruhrG{h(8Fg{XdhsTv?~; F002Dh3zYx> diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/update_datasource/publication_10.json.gz b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/update_datasource/publication_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..99c4015e715ae7540f024ef729623d3e0484d739 GIT binary patch literal 5257 zcmV;46n5($iwFocB%odZ18{X>Y-wX*bZKvHUokK)YIARH0PS7fa@#nTzTc-n>0)c9 zNA8|+J60mwD%(kSPuEroBq0knMQA|Ej+4p7KF&VJ-tI%~0V#@<6w8tlXXp%4 zUDYOZz=NN|1K|6BgD;Jcq@*8@c;nzpgT?nOVSYp#2O|g(5L1N5e| zDKr;N*@Px3nM@l8*m5n+vM@G0Q-?}RO_ugg8hjj0oiQa*IzBki;Gb&+t~WA$&kh4! zw>>{J9B4+vGBg_s!3DcvF^T%{fiDt6ar^nsF^(i_TI%qiaBNmXX+UN*u!zT#^(b7!fSrmN*l~Y++ zP*l7GYi+DyyK~zf-S}-?!+Kk@+M4D-?tjck#72yY<<1T{Tj|j}Wv$y-_%VH)(kHrW z@$1*7GUx5gS$3c_Wj>l)PFC^;^qSq$^cKd~*ge47fu^@z&-i6a`?6B0c@nx@t`&$c6q!3IIu;M?2D*xfq&0+( zUsU?4npWz{axAo2&u9Zv7c`kgsq{(9-e6if*OgCy`Ipu2Jb#~bay;TU>6f)V! zAOU$1sDfUkqOjN}dUXQ62j%LQTVUyj)!NFg?Xi@8To$NSbwxzk!!Gt9pzil1PQz@F zmEUSu9fbc}%ne8Y)lQ4?x-cF&wQFe@{%fda-?rJCxr#z;PT^P@l7y>k&=>$T%HvUcJ&=i{C}&YiU@nYHzt*5l6EAbL|^O zR@tIZ@1bDe$HUB^T&IOy)0+Qs5l1YZH?e#ku=oxNlP6G#!q~2s{|0}{9!j!ANB()w zp%{&9+@>L`8uOwTqYSfI3?}pB>Qq5l(sq{g^*_F-w)!zFfr)i@154n0)W##%7+J*h zwZO0($MGFM2*Q!!KF<oL%h;dK3ytWL;2qsXF=Nl9A1Y7uwt%YxMaum;HZI z(Q~p|p-g6>!BgCf;$~je7`x8RypJ=04Aw%hoA!VFE0N9g>dbw{RCf)0_O zftay$B9l{&(lO1X1(EE`gv00uX(UA~K~N((zY7tI(baYL+CZaOoRxz_(IX0J#{S$> zGzBw<_a#pogQ9fmXhz>p>1YaJ`7xBv$??(gW%s&&agMrY$Cv2pqKA%uAELwl#Xz<0 zOAtKQ;Q3CvSI39wZ4Oc8w2p47=KghPo%xjRDe9p&xVSn#T4}&u#1k0w{%z=o?*Z+9 z1pBax>+C~3yc8Ct2EbQIF|7Iabz;FXw3hAK+SbGZHXLigELa7zfDPMPPz%}Ep{NB# zE!3v1o~IVaOEz)K?Tk{VaMbib3-ZOt)(prVP61q;Tdn&PokGnd?xU&s$IG|@E|FpI7< zS(#A5vNPd5IfhGqLj|UEt%w=+`?Nf`~I8@pNh29 zQYMsYr0CZC!m^fAF7BZRn&|I5f)z)pXYpLBnah^nrnY0(q~LV#pm*LyWLEh+i^pp3 z>NO2oXMs*<6B46iIW3Z>aG*Gu$pj?IcG(SKk}@H|B5epl;JLu05Fer6kI!32 z8rJXu>VlWMrxEI=f|00oMWZ>>^X?wEtxmu7ZYEGaKl8)I)4i`~pE}yj_tQRxLcM+r z?K9{PhyC;3#q~A%tv?v_yJuJEKVdHaH~ILke}1K!_$9cXIVvclQX52>bESTh}83+<89_`!tG z6KLF?CsLN{f{A6pE4|RtM%dIqNl1JhkHXNiy+AX+3tv?OF-Wy$kmR8dyKTEW6G-}tolz-@ zQ!fH($UcAO+2;Okd_$|4>*Be@TnpjGeq_>bFq%rb;x?(MDIO~v4~t8Pe>eR2lt+`0 zjH4#HWOqbV2#dvY6@AZz8{3je3PM&8a=rZEI)vOsZ=zWM3*MI98p7IeItKTE`U#)F zvOJ+^2n!qVAgCXQGnjaz>-_vS>d`1_9W72A^u~h6EI?h6js;9-Ww0pJvz)^Ltnttp zEXON9JuvVrvlJGMB!`(VK0V&h7+pJ4s#Ra=bU_zfrVNyaD)_<>76=t^8M}sR|6d1PUW+32@bJ*4&@=Zi zR<#2@>Bx0yE7#eXB~;9_cCZR3rNT-58s|Blftf zoc_R83Ei6obdRxVF0lK2ELY&Y0`F_nR?ma?n<7hLtcAwN3pLBdIw7`egpkMaY|{^e z@OkilbN2dx_gHs3XT8HVI0XZ@u&zM+`dRX}puLXUmbr6eDc5)@)V?fFyNR4l(;L^tcpxJsfq#2Q$LIvA#Rc-ggBF|G6LiR9iq5#0j`>X< zsa9^6PbrIAIrR~u>|MoJfYPzxvzzgJ=X_3`K%F9R(;>Q=kPvsQmi)gl=Q2J~6)@?Q z*^{~hqc&8=B1+O(zjt{d!z5k2`9n49OK=%y946e^cF2r~l|ew|!_sXShOXNF*WoeL zF8SwS_J8%Q4Qr^ytBRYX#=$xw>6<nRm0$uk&GIT7%*FD35F+tP45%%oRcpgUIoV`XcI>w#bHa2aq zrP&^q+%IlxhV9wyPYP16pH=U$4_?!DzRlUSUvMM*ygc=S)Q^*?{E9b1wsV<10re{? zZ%<8|=(M8YbXmh(W=|mfh~3^LW6nj~M1$&m)XPdru{}Zb4@{mIsB!{tSF!ENDS=%> zCj~@O4pqKk8J7ovtzmNoYAaB?UOsyr)V?HO!r7D^)>|_nH|!U#EjL2B8|H?_T;3U* z%s17`$wC(XzZ<}$J>@>M0~N?8GKNgd82U@s|0}u?d^Sb?BRcC2j{2?ZixYH~aYPEY zenlpuaO>LO)`zfyy?c;XqYv|+^TTSYRev~a-HTY9g}FLoP%b~mFfuO*4V_PSi%s~d z1AZOsTJ6DCd+-%GunCJ%M8PlqUA{a~u*KNA#ui}gxh=;ue&3ocz*sX%wg4Awfs7E< zD0kf}wm`83b!w~U*#c)%#2gz_OQ(VBXh8rYgGDrAYJnBHF15CZn6o*1eb@p`#~l*r zlD&3r6+KWtGv4~51+VQn`p)#g%iXlFC{MlUf!}9MbT#I)TSl@Ke}UIml1hmB_t~Bl z!GM99KbAE7vHEh5*^?gljmQ8z*MyCERYRFj3Bp4vL3nl}2L`N(-oi{^SxF_OT9vIy z0tMtNAirMTbsfkbN>wrd&XA|l_rL@j*3Dvp;S|G4Fv|7R@Fw zr%nRdP^dqd%wnFb`vw%KPw#2x%~1U*ws zcVR*Fm}ZDX<_8lP?s9Nn1=#tO zzJ@ZZE6`=b#U;9|Lb@n) zxn6DcJi6>{3hCk-0kw3SIJRLK9<^PQSb^))0PFe|d(k&%uMxUzxSd!`BspbhwxjUn z`Wf=p+t1Cm<5)Z6%ifD^KVOumUiflPhBb-gCAen{##$Pc&}o@ay`44}-xFsJDT)Z{ ziP`5SI+64DIILW}T1ly3wYmJB(DTt_NI;dKm6+`Fr^>!`5ZV)TK4rn(B#Ryci+CQR z!)g$jN>Vj8N|s}50D9%_RUX5HhDnWYh%Z>RzsiKl1+0a9W50ke*F0{W@(jN}B$ckp zA4)w7xrPe+qJZ_+IjpP$>xY68IsMO2?_17d`L|q^M6{L25hbJ{Iy&zjWIN(d-=88G zbTpm?5tTvjli8Gt5t*>)QDNb)%NA7h`PyLN=iTe~m&a)M{&3L0I_;moL2oWE-Vas7 zz62tEPSTm6D2E=CrG0wyUGK`G6)>)V@t+BduYeM-3L00VU>%|H4PJf?A;*rXwXmgc zAFahQ+yy}H7WlZOTMKv`tLsV?JYK7|dKMlxbbAv#9y*3^6N`jIw~4LkIJ7l4q;_Bv za|?KUWA+-sMzD*EPN6Dxkc6CcSm&bFb}UXIG%y!!H*4d{Lfy z0p+LAEHrAOYeDb1K-VP}TBNMtyqWx-IQd&Xj`JAkR}|`*{cY?A8X^RriYi%zH%#8OAn`yX`9TRWPoCaepR^`{d$T0lL4m z_xM?NS?2zI4ZmHO+}8N5>y#UuJ=gXM2-mgo0>aG)5QX6?47WCI^*n}a=^HTI5ZeS} zJn{o#daj0@zzq%0*EHMpN6%xpo3qyo!@X_OPt%yr1drw+p4_&kF0iblQEwYg!;*2j zb_Q|vmx`vbC{DdF+%xi-$Yb$M)Qv;&zyI@C;+ryo{@;H;F5#Pn{GKT88G(xMCORW& z%1Ehe@m$p;MfZel`_M&A^npaOt>TKJYEr37if@hBt_0wwpFfj0Yd=UKqFSqJQY9jH zo};nfcIO@u5tCcf#Vx5;tW2nEO_#EbZMQiqBw8WS^>Xv;Nc1Tz87s$*o?aSg(hVSTx_p&G< zdJ{d)qkbQQ%uw)f7NqK`$ya7XDu&0u7~Rgto2`M_<4;p6V6hUXBtqxelzg$(R#&$p zDv(xzv_Bi9eQI_5s$g2>#p?*ue%srxL$(?=Tc+!7581-Euz+m!0jiwFo3Y@l8M19NX?ba!ELWnVEcE^2dcZUF6FUvt|yl7GKXfm7Aq*j2Jj ziljto_V&uLoy2o?lCrbAH@7pDf=Ez8LJ?XLw4==CVc+hfte^Hn_;o+jbp!k-S&nQ^ z5-Vd&Wo#1!&_H*ifo}W={L%`kq(@04TEkyjJUQnXAH}RSjA)#(Hl#!>O<7nage+!~ zg`>H8!;+xRE7LratzpaR3|d7=d_POYIZLR@*@74KGYupc2@(!snq?fQw6b}|r!$et zVUUU}yP)yAVajGA<5Hw^D6X<5bjog%ISVFi%2dT+!sM`~y{BkrRi2{88*+X8*lqz` zna_Y-O!6tqB%RJ$L)-H`%kz2z-|_pcPKW6uHm{k@7K5XnX_}{H1lr;y(jdL>#jK1QVrV3RV2lddKF_7BOQRR?-$Fg_2?rXa@ zbiJX~|KC=bQ>HY@B9TsMksIEy_mHaV$0`NoO|AH)=>7rAf|SonrI^t49Qb?ywKi!w zuWf(mxI@S5xZVKi(pWO3qo9GHha;M?Kb{^#nTeFM;qESw>j1rwLti`UN0jbH5|uP$ ziR2MyDe{W);c~X2>vp|iW7|7+_qAmW)jw>goDOsK_=7%&8#ye2cLY^pGpbFc@Fh)%{i9WOP3 zCF3RPhsikR*<=OrvWC?}>$e|qgyx9^qof8vmXF@TXi{P>Mk8sRv2##kLW5i~0u7$= zl%gFY8L%9NH6wvYMMBRx7+Qh@`L8*R3+*M1E|At@2h?-j^u}ZLo_uWD0J1O@kZW@R z`NU*E7H0$Usp){Mo(~rLa+ew57P*^zxYe9+W44bUQIjQ?$fR13t$i?DAi;4;XA}O{ zoDn7okZ0tAjmS{~wklDhfyOYLr`WW7N)D&|m=iE8OLA>$<`l*`(exG-bm7Iaj-pui zf&81r3{6E{%CevfW>xTsRe?5TF*~P;4AnfPX3XY1R!^ZcfNslj$}6=6MUL~DWw#95 zZ_9RQcs@d#j6+0EGi*Ad=~%#WM+-uy!o-vYg;;Qglduvgp419O{~ZW0&3u$gHS_A} zNfp!+l&&g(cBE>yz#Jo741hqh43;hvS!YTL62IuhU@`xHV2KEYn)L%Qo#qLbFbVC3 z$gvh`5fnnZ5TCM2l1PRpS)nmdVax-TWDDAZ)n`zDkTS3-=L{b{Gfz0t=Y!4pGfqXaqcGvl#a?LxP%iA>ci*I04@z8xr_U z!knpwL6vp0iVoFn*E9wENXrsjW>6whY9aNipP-AVa;6Mbn7(Lo{~Kncy01}?-?0bZQ}O2J8mETxR^k?HPq8p^g-s<;g6U1bi@tf~N$7>H2XiW2je(TSMd{I-DK(UT$M^aN`uIMLf+U ziFtxmg&5HUEyov0J{>{-fOo3@=*(qgOO=Ce0xjF$ZZG9*aCsrC3puefN=q67RWjUy zJlRC#aYzy&Q7SRUu#g#4b2^)%azHa1p01^a5~xxn z;#gd07NA{KVp$&4TgWeDW2q`N3slyMW2jr#3Wp;3PNM3;h>6uE1dJgDK1dRP>$*K8 zz*^!$yNONo|Cah~lTN3z1@8h8=hGzHR;gX}2L5v32V_v0ef85Jy&?@W42rZqJb~Ab z9&=WX15lMj84DcnxvW%bt5h3ER9yZZN9+j422Yf^5-I-x)(6-+2NsfNEMX~FyTZhQ zMuKF<(8WY++EDQfC^Z<;u#C?$)Tw2A`CJzI^9&?cRoE_MrR^xpA{_LlCR~lxWy4Yk zdMA7yz6ONINd1G8P=YWjtLk;^-;eNQlrRy1$m0a#g$;3tD4ddh@#d#&s zr4p^@JOn<1`G6XgeJ{-_=1KA(W*`;J0Ih*(KgX~!c@BJ>@BEqZ1cuF+fb#R0k5Zmb zJLI{zKqXT#ZK&n~g(cMpN6}}9`2h7mALYNuflxL>(s!C!&~OmAT1hBn!Di~G{bsLK zzvimG6!{5FgE-!Q@LZ8EyfLBvoK6j2pzAtnZ+tmn`YC6C42z~#pDpW^ei z6^8EX3U*{lF~U56p|Q|nHM(%cfv6m9l!08yCR~nSEm_Cx|J-Ha`sqD2t7rDq9w{Os zz}4g%d+-Jp!Zeu6H*^%dnNjFJZDB}3RiQHN*n_WuoXDs_P8j6GXF^UaZaDgV%6ieL z=h#s{@M*8xXFlutecuZQgF!d^tjLLCt8Rmk6UNFL)KtX zOTC^w0^>1q2VQq%`y+p3F|W4{IkB$z<3&#Nov(tN=o{q3qeD)B%bcUfU#=B0<&rY6 zkf}(}5}PeCgQYM6F<6R?lm_nKTwP-4iVd9c)n#&ZDa|-}Xvr9q%R@_m&SqsKfIv6( zO4W>u2e()Td$fsiQJZa7{|wFW*boTVF*r;CXK_Yev2>bC{A@Xxot>TDEFc34+eFdc z0TlN9ki0s5`RsJ>guFUDJ^ABnviIVEG?VvH1IUxZ{g)?)${;K?9gE)@@WMowkMvTtnw!`)e;CLx>Oz1fyKEcR13~AJoDSgkU`E*D?8(go~_Z6N@@ueuklOWE+ z0$Zk#Cy4i`R(B#9mfG;NMxK>vFDe*SFqcKYUaUlyHFDOiLkJO^k6W6(t08=i>eXPS72X1v zKBXZmMN@)&3YZ?*`st@b{imK%e)>t_`;g0u$=6S1;b$5uB7A%5pP1Wn0M zy2vG3z_Ut@lPwJ|+eXY$`LYRLOT`8B{V4{?0OO#>n?szN|#8OW)CZ5PNqGbSqPqI^*L6mT@p4Lly|Sc9rx?Wj5nP&Vdx$29_+DZ_-!LZ<$w>Vj5}LP?5wTQ44lUv0u{PI7@l ztJbyx7F@u(dNxwfup8&i>`P6Yjdp%wek|lwPL3d6HDl^k!Qw=Ty4A?HV&43%&zMTd zDzM&tr!aDMZ;+^Uv};S#T?5?xu!4^*y;)F^xK}mPS;Lg%0>HC zGuY{8p21GP{tTwO01m0Po?}sbZv(2mgQm6crnY#8VZM^pQ0xziiAWmT2MNE&lx>3t zzEjM?oUvo@z$@{uQB5W@cZaRx(4vuwH?R05%5b{VB3Dr@?6KVN4;KP1zs39 z-k{&5o{!x6B;a>l@ka;v^%3yf^*i2|KPt_I*{FZhQEB$zD*=8_a`n|E7y2l9zI2d0 zUpz*>%H1-U-(PiFnh3A{5#vX*_+fVu!Z5gphlHP$BAHEqzET{6)OvXQ9IR$0%wS44 zMIr%3--=Nqg@=?vM8|nXGQ8(Sg5zm~u^D)|R|n6?LDQLP6SX5KlI6Qu3}kN8rTgpe z@aUV+#&0h|ZGz34s5U0pyeimy@0jf^FW%7Mi%p6q8!QiNbMwU;y>4$5 zSu`39d_VGSoAtT_E9?zi&+o$+_SQs`*A{<_(d7Qnb2?xCR#IosbNb$@V6xr)dZ$P1 zPiTnnup=5STuE9z*O6qM_10nJ_A(vd0QgtWavsMN3{Fj}nY2I-OSEo#Bk2;o6pv6m zp($rOKZuMq$So!|2ttvHnsX;=&w%@!w#ida7MAkhT}`c-RMXN0Dr)YXsGZT#2*rcv zeT;e5Y8?2XA8R^g){2s6g8#P;{;v)h(BK`EBPR#1q!95)yeH=Cx^HNRfhMgoA4-*$z(n zH&Ypsh$ndUY;mx0yTpP@mKHH97&ttbH&YPeOH(ypMXifP6_d1ym3;kJNwGRNv664x zB$!yqZ%U-=VkHCD_iv1qID>&TpkYMALD;u^$MT~t1t#_#hq=^R7b{s;{4vH#tfANI z_}wpl)TV3szH`%2n_mri_k@9x#w-QLTL9Ati{h3*+J*@Ig0aOXJ{N!TW5Vm4U-7M9O7wPt~PhE1M>rC?IF$x}MwtijLf zxh7ZSy=TZo#eAe>&@*{;Ih)XAp?8w~>+e-!lcBoQt)nMPQ6Y?Lu_*>>bj<0P=IOOS zRGHWa4wdC9y1hHm{SyVn6F_th&fT;@uY9ga>8ur<{+;N6p^#!a0;cXyQdlJkeY9g0 zziXt`6w5*tP5*AOFDY2?j>V{=F-wxBiWbj}=vR@sqBStluX{jWgBZ+P%%wr{Jt&5Z zPWnM6NCaJ)4p9tIP4#r8HyY z!H*W!Cqo)3Jm?gjgknsm+zf_?oiK7Kc!I~xz~X-dJ^=WHMk1R5$=`<+)nO+236+f4 z9To33o9o|@NN>Nn-V}A4Q1XP%S;lEXrc6$RzOEHSkDxcaeKWh$=1U3lrNnP__rKU~+N9loV{*;z|6>>F zn%#fL?>oI4cmG+R4n|#@b|c0jJonb|2bSw}Y2>(F*KMuY{a;u7G4B35L$BZQ`}UXL z{derX+j`*L{{w~_jI7v{|M8qPkoV1`=3Zx9#G>zm0XW5L3?r7e$)7lgNZl$~ZrfG~ zHL)^#>wf$HPq9#&{0}|n3bjxqSI;#em9urf9sg4rzNJ_H#2x>1Le+j|wdG$XG`fVj z`M>_Jb=ysUvwyp4|Mou9pU90j_P}}# ztP16+3D-YtxW4>y#mM1dM@D6#@DhZp%ha6i_0S@T;`}`xNjRn-j372a5O~`-Hy2`V z%5LJ*SX_+6`G8`y<5##mgJBqA6uJKHR^M1f%MWj(Slum> zihPXs(ZFy0{cEw_OUV>(H{rnR&QD3B-s+M{yb1=IGUXrO**PG`R7xH&@B7i1O$_uq%)qqDK(754&A`A3wTVje4 z(%EEgz9hcWU|F+q;F*mBvsY{OYRz8lZ+5R%Z4PYGUac{?X0P_Ki*(Iit!wvf`=-6x zz8!{^6*7zUt$xo9yS`=7KHhaQvU^@QYJGaIc5U%TxL50VL&xs8{ue*i&UL$9Z{=RC zZSUCKul1!imy+eFz9EG!oNU)fX?(7&^SAD|Q+o^}ipICPjzwo}-FA=GY-p}Ij%^>5 zUaGB{*#yJUdV3R6F4%~i93B5RIZD(z8JP->=h+))Que0AGi!WS>GC2 zea~?$7OiuH*A;(+j<9PFoqor0y)O@1?%Ve2Bg?#Fb0hA(gvzKcMnH zSOwg{+&KZe$AoN?)2qwzLLlLk{p;`JYlq}7m22Jays{@0%scqO8f@NLz?m-wE{cq; z`yEkse0ciu$2NIEQQ=WpHi zXtK?Yltd}rjjpxeDNb`=8vPxDft0j_f8QMTSZdp1tCrhgf){sK(lsyR#9&pZPFIgJ zBa4TW#rWz{q*s^G)urm-RDVxn1oB`4IS0!H%Q<7G9$pmqiC17itPd)KJi}{d!W^dx zBZdJJn zvos^AJX4Y$`pOngTUrSdo4iu7hZF`HVOsA1;LVza4b(4F6i+L%qP z2SInLDC#C{Y8jJjHnkqRNY`y@`M&SmxT!^ZG+<8Hr=i=Wfp0s^9e7^U>rvNroppB> zuPy!vH?_R(u-onQtS^u5wEDe%ul3+3@nmmV{=Q9qq|F;Zil>FUMlfINo~TX=42&Yy zCQJ9WT)(lU%-S-z&R?@taV7;!02zK}_%wypFeoC~12X`2EhAfJPxrRJKmDGZl9|4P z*F;4&BPxnE)L0PPD+D} zp*0$J$QG{Kx5@7hULFl?&@gM@?9#WC?NWMS_4|LokD_5d%_Y19NX?ba!ELWnVEcE^2dcZUF6FUvuL&l7GKXfm7Aq+Eubl ziu#|uz4F+e8E5Somp!|=T(XsdNKnEINoWbs9wnEDeY=mce%cS=*Zok}4e+02JF-2Q zksKp6u}u&_1L#Hr-Tfo*QzM{?9wnh@jDBkHDgc`ihGL3HuS1k^_~7oJ$_hM5;!9 zDx|!m(T7pWWl*LR}93)JQD%yLRwk-4HHQtfy=bxJmpsTVO z3>Q;8W>V33))=|nLD%Vad&7Y@=tHNGQrG!k5&WS4gV*iRsN;I=u*>{mXV?p!phpA0 zKXx42g@3&;>_8(#$|pRb(GgVeUE_hp{|cibqKNsxm@pMFRPj^e1Diuz#Ed0@pefci z!q&Ed*aeL;?CfbOf}MRw*d2OT|xl_Yau#Q$8z%VoK8s80Q11wNBG} z=MF|*XXJHTo$e6n(nv8aqacAGhhr+)pU;k=%v7n_Xm=OLwSZp0p|36dBTRQgg+dyz zMDdWb6vvA4(Q>v?r_=8CdwJ(_rej+MJFdIqw%<92vY=f8Y7o%OpX$e_C${-jX)q`)OT4-4r6}I35X_6@xNs>%H_J=w-_Xa7&pGs6*Czkc0m&rXicYeZe0^rN|^f4ZB;1ELRygJ zs2Lr%VTb*`!wwD4#yG2SiWo)3;xSDp0v1fe2$Kq`E%kG@ppI0qmgP@+jWd7y0wy;f zXG-gCqu_KwqvA|eKC~m%`UZ21aXA5oN(qbKR8^UhiUjz){9P_UG9$OJ4vwX!TMvx<^f`}z_NOq-<}@6S!&r^ zSnoA1I<5proJ8nh`Aer=Drv<3d;mRXhi>#N*U zphWXz;=>MS!f1?}qC)d6zxA^eyFV}9r&-9gkQQQXRI^JzYvE%KuoiAh*pMuOOa@P8~)g)&u zo>x|C{#2Q$c~kRB0{33jFe7GD356{ zBGcM@B}%|QMI%y7={ecTs|d(=PF^>4HSFU|TUlfZtVzX>^9hc?yh%igxz0>3*sd_J z%n9ya64=0Ow}@`$LL=Z=&LRw61q3DSQowr{;sh)bIU?|z1R2u}gDR_L?3zL>^8}} zy;o%DG)bq|-s$w=kIN}^3#yKyLY4^nAp@EQ1ZJm5rEc#cN8tU9(VkIg63R$EvOJk( zih$+CknxnD-`hNHNrrNzM{9)Rh-PL>eN@}n9H>)M3wbIPiFkrl1?Z>*DaRK{7LTES zFm}5CXu?UdrOQD#ftEYiZZ74ljq*a)4&=nnXep^NRP%5P^2n*kqJShqAy*>wT;X6) zGtx3f;eck==2cA!z9MCBZd8u#bSmOyO?4rHJZ(o$B@m@hM3J~OLx6f!vt@ZwZ{c|1 zFczXx4S~*Dju^_;jlzLQey>pUV8%o`wg7X8fen)QV6bkF2n;RhK)Z=c%>R!5?UGij zwFU2d5oK{Aw{>b;zk$Cz_yHMo=7Ij{kzSsL8HRaUAD+P0M>|;-^8iF8(aHiRe5Oi~ z+A72b67?v5g)?@HbAu;ZU5S+c2-*jRbq+&FUa^FwpzU%M2NDUKnLrm4BWWW&W&w?Fm!DU_xw67$&MTuwVHM#ZHx=P>t}ZK zs%X@Ei}SU44~5mN=JN!X(MnYsrMfU(4qnS9Ie4w$2XwQEEUBp(7%|v!NS0ohR=wcp z-mSxjETIL>#zVLX9pGa(@SJmTY&XK{iy_u{nK~4(lfQ=7A?jeR>e!= zBNBtG@>z5nH-W`a6BN-EZ2=2$TLA(Hxse}>xnY!R%YTZ`H+C4NueS_`)PfOYK1_{; z7VFuCJB~~hN*Tx%!-U%rtVOnL)aM}^*I$mLIXxRmv*3k?gBK*V-~|g_cmm*sJX*Fv z;03F4EqLMCvvd`BVbB}6cY+rNZm;dOy*BOk+;N``#+_leJ$47!Tr-x@Je||^y-W-s6@;*ucd2zV^_T-SfeRKRR{3zeH zz8j~zec$z{HYLNRPj)!Tte$G9*QW9nGrUT8d9dpqTaK^y#t_Y z+EyekV8^(;1vezg)>@aTALB67fU5CUHBRk=yqAJ8rwvH8I$&7g;)nc7n{)sD=54 zO>!b4a_Ga40hH+uy&bpbw71JnBhbkNVyl=w24ZiA82NQTR}tE;At5HFjT#oyG;#SG za5(hNBfrU*sY?Wv)WAcCPr9HyqA)LV+yX*zpdyWiFvSY!#dW@dF?+ zP_D|Orc{xjh7aHg#_ysfcYoHb1yKJbYLOv5DVIT)&(XkOwwR5|VlnlP~v zgQCen1!IPoiM{jvqoW*MgRl~uFGY?iyGt{q4dBw_ zl9%C$A7w!f=F(6P1Xz^2JCTeEX?R&dy^6FqC2q-|tGr)tZiREza#pRw5d>-(ms2E) zO}L;bhq(g-he7*E_b_DfyS_EH5!t=cjWkR@BAI{>I3}mHGV9h*h=(n zAskSPUSPB`d?S?^d2AKr2D%lhN@pvS8^$_F_jmfG{@eYXk;ys^=q3%+dx&{8oUKh= z>{Wz|PL9Pz#F8PnHJ)p|13Lk|aH8xBqb|82BgO~@10rwALO^F3IjNCkmpFK+^rqqx zCPOm6rHu%MS}t5C(IQ7?Vbm%L#^BE0P7rc%Yc=IqzQI}fAx zXDb&lZK8eB3U>A@uV80ieg!jK0E5(M&xy!CZ~@WYLDiaDQ(L^nFkdm$Q0$L-5Ro*t z4-)=>Dccr+`yii%J<7%ca4i6L6U4>>aMuOE?QaqQ*Q#6#z-#tyPoIV3BaYy zcilnH>2^HF9rt>^<97q!>kj*E+8y9%{UQK&Rq@ZeKmX}r9awJqcby*Q z4!;lp_axJwBywR6)#gh_YV*YdwaeTMi^2UA3@+?OuKy9?7p3^sbM9`i7=?90MGBV8 zra)f{jzv|iAL|CYnJKgQ&rRWoPtkZ{)N)~+TnOnTlSJZ;7YZy(%Z$yyww)flA_sM2 zs!4S8P9((MMe)SA`{{R4^sQ^ddXud;*?NofgF}--1lzA zC3-2IFL**z&UU^QlGVs9Ce{c-k&>GCAZo8*^f_&kmmn-G<^G3?T0Nnr(9v2>NMWY`%D?(~&-!=W>zK}dsP(02x&GYH!h2C?sXtV5kuzLHhNpQEqD z8FhQDLHn~Gec5pa1MjY*FTWV_?gaxOjaUkXw}3&b8Q%G0Wq21^AL%YRf!T%o_^%mF zc1}4?{)*aSVh!r23q8_dvIn(f!Om+clAuY>#ca9&EG(WYYV`t-bep^eO~I6El9zPM zS#3Pa=ZajO_mM6W?ekHJLC@6n)oeGmE(_b(I#PXN(57DiEnuCY=zLa482Nt1-Ml4C{BAP$fykB|dlGf0Aza9a7^&?Pk5my$; zw^j@p%Y2&7g=NEf*^pwN0`q}2F;l~GWHUg=V>-XSlCX0~EJxPLk?SkptY+lm`f750 zrR*G8`!T@g6i3U2wJzaFASN{Cb~3DY!N`>02_6jtoBuJ`0ALeZj%)@be;;;Khf?tq zsu*!w+TU%1>raTK_YbbuS=}U*yr6R?IZa5+)Kr+eOo8?28k}PUJJknK;n7~XK{{VP z{cFM!eKu4;sKQGxO=w*zHj@p@#9uEuZEs<-(P{gog#A+Dw;KJ=!%dqM{kJODM*p8X zOIJkyy+PmW-5LF7eL5VsUD^&A3-OFvZ!mN^UYmwqr`_o^Rz&|-6@QM=e{a<7w+4Op z^GE+Zx8G^3Ir@LVFu=(2p!~NNtcJX=C)Hz}QSOU=3< zSdqQ)xcL7|EYu|bO)t1cE#%4Nb45t!Y&zx^;hQbRAuCG#iy4Si{5=PlaJd$uif3%F)1V-S^;M`tbxhc`a zmyx&}i;s7+!^RLdk&zGkKo@;(2T!sCdU&x@ZsZbu#Sh-hGs7>18TMt$CsVu^iYW?N0}^xOaIpuL`5iZN zsTlf_d=gSQg+AyTY$Usp9WV8SreIrB%>rohw7C2Vb`5wG0*z`GC=P+S8B4_IA)QU< z_DkXiO_mjbgRTu6*jTNN)!JC?Z#Gt|g9DostFq`CIu?_)ao-rN4QQ@7j%^==Ug=QH zY>Mt^6W)ZBOExAaN5_96M~U7im(bH$OMac>-8*(yW)0zo8^TFo z4Pk2tKLtbB?7TL~5Vk7U8p6+=rK=3#uGhWW5C-dYIHr9sY=<7}4?7Gj*?}{5`d!a+ zSh&g%URC@V8p0iS!#r~COq%YE0q{m8QJPJ8f$4B?mbBEF$=UVQWWkYe#{Tv9$&Ewum&!FBcV5{G3hEu~U;{R9Ea1!+9T$1V#^a7C zJ3c&n`)!lFp=t`YaIJH-l$!H`rDEQA+@WL#*-T_?o}=fFGk>vDc}fjCZ}K6z`yTydn~nWu~p0Mu+EDIZ0YKkaiX&-(5I_M((+=R zyqH{HiS+s^yuQ*MjLr8nmLY3n$OULF*v?r!wSHFMCtkhmf((}m%Zbf^ z6CW(>rSw9smvqFB`uf~KNk#;={uzr1>|aw7QL>=dG)DaQIdAiQ0LH8$ePz1XSz3{F zo|q>Y(s041B#z_gIw;9E6Tc^iO4}RnMS8cK*r3)XN|?Q~*ao#My7P&MjSXt81>LE= zsGAhjvMSdGwVpdmR|T~OgMoKvP>c4c&%B^dgHD_J1J`4nVYeIhdbHE&c&qL#URnGZ z2DQ5FQM=vhIiDZh>GXU3USsVi@yPcq``9Gk()tY``P0H(Bbcx8NK_|<0SrUdBun?U z+`O@+$l9>D&QA$doGSqffW*%XU#74d2088ccsBt#ALJ{ge?I=LG%)T?Rdvg4rh$25)DX!f(d)iT^ASnY3RuKS-vm5oY~f*6+0mVq^VY>+yS` Yp=JGEPkZh>5x>{}19s*i?oIsw0IUmUV*mgE From 5153d88bd34f9ea14142955f62d6c0e1a0ab9bb0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:04:53 +0200 Subject: [PATCH 146/259] defiition of workflow and properties for bulktagging --- .../dhp/{ => bulktag}/input_bulkTag_parameters.json | 0 .../dhp/{ => bulktag}/oozie_app/config-default.xml | 0 .../dnetlib/dhp/{ => bulktag}/oozie_app/workflow.xml | 12 ++++++------ 3 files changed, 6 insertions(+), 6 deletions(-) rename dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/{ => bulktag}/input_bulkTag_parameters.json (100%) rename dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/{ => bulktag}/oozie_app/config-default.xml (100%) rename dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/{ => bulktag}/oozie_app/workflow.xml (96%) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/input_bulkTag_parameters.json rename to dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/config-default.xml rename to dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml similarity index 96% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml rename to dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index b4f918bbd..390311846 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -55,12 +55,12 @@ - + - + ${jobTracker} @@ -84,7 +84,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${workingDir}/publication - --proto_map${protoMap} + --protoMap${protoMap} --isLookupUrl${isLookupUrl} @@ -113,7 +113,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${workingDir}/dataset - --proto_map${protoMap} + --protoMap${protoMap} --isLookupUrl${isLookupUrl} @@ -142,7 +142,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${workingDir}/otherresearchproduct - --proto_map${protoMap} + --protoMap${protoMap} --isLookupUrl${isLookupUrl} @@ -171,7 +171,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${workingDir}/software - --proto_map${protoMap} + --protoMap${protoMap} --isLookupUrl${isLookupUrl} From 04fc22334629a0d37b76a9565aa429cb6fb8dc08 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:07:44 +0200 Subject: [PATCH 147/259] add method addPid --- .../main/java/eu/dnetlib/dhp/schema/oaf/Author.java | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java index eb122b14e..dec1e463f 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java @@ -2,7 +2,6 @@ package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; import java.util.*; -import java.util.stream.Collectors; public class Author implements Serializable { @@ -86,17 +85,12 @@ public class Author implements Serializable { public void addPid(StructuredProperty pid) { - if (pid == null) - return; + if (pid == null) return; - if(this.pid == null){ + if (this.pid == null) { this.pid = Arrays.asList(pid); - }else{ + } else { this.pid.add(pid); } - - } - - } From ab2a24cc2b62f55da3f36bbc874556afaac8cb9a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:08:47 +0200 Subject: [PATCH 148/259] changed the dependency to use reflections to find annotated classes --- dhp-workflows/dhp-bulktag/pom.xml | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-bulktag/pom.xml index 3fa311315..4844ef287 100644 --- a/dhp-workflows/dhp-bulktag/pom.xml +++ b/dhp-workflows/dhp-bulktag/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.6-SNAPSHOT + 1.1.7-SNAPSHOT 4.0.0 @@ -35,6 +35,10 @@ dom4j dom4j + + jaxen + jaxen + com.jayway.jsonpath json-path @@ -45,6 +49,17 @@ 0.9.11 compile + + com.google.guava + guava + 23.3-jre + + + io.github.classgraph + classgraph + 4.8.71 + + \ No newline at end of file From ac7ec349cf4fe162e075776120fe2273c904d020 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:43:03 +0200 Subject: [PATCH 149/259] removed the shaded lib module --- dhp-build/dhp-shaded-libs/pom.xml | 52 ------------------------------- dhp-build/pom.xml | 1 - 2 files changed, 53 deletions(-) delete mode 100644 dhp-build/dhp-shaded-libs/pom.xml diff --git a/dhp-build/dhp-shaded-libs/pom.xml b/dhp-build/dhp-shaded-libs/pom.xml deleted file mode 100644 index c091c46f0..000000000 --- a/dhp-build/dhp-shaded-libs/pom.xml +++ /dev/null @@ -1,52 +0,0 @@ - - - - dhp-build - eu.dnetlib.dhp - 1.1.7-SNAPSHOT - - 4.0.0 - - dhp-shaded-libs - - - com.google.guava - guava - 23.3-jre - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.2.3 - - - package - - shade - - - - - com.google.guava - shaded.com.google.guava - - - - - *:* - - - - - - - - - - \ No newline at end of file diff --git a/dhp-build/pom.xml b/dhp-build/pom.xml index 040842000..3b54b4e61 100644 --- a/dhp-build/pom.xml +++ b/dhp-build/pom.xml @@ -11,7 +11,6 @@ dhp-build-assembly-resources dhp-build-properties-maven-plugin - dhp-shaded-libs From 24fa81d7e824a92c03489504b0186531eba50d14 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:44:19 +0200 Subject: [PATCH 150/259] implementation parametrized for result type --- .../src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java index 04897dd4a..3ca22de7c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java @@ -28,7 +28,7 @@ public class SparkBulkTagJob2 { String jsonConfiguration = IOUtils.toString( SparkBulkTagJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/input_bulktag_parameters.json")); + "/eu/dnetlib/dhp/bulktag/input_bulktag_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); @@ -74,7 +74,7 @@ public class SparkBulkTagJob2 { String taggingConf = parser.get("taggingConf"); if (isTest) { - cc = CommunityConfigurationFactory.fromJson(taggingConf); + cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); } From e431fe4f5bf8762ed217b044d6d0de974f230f49 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:48:47 +0200 Subject: [PATCH 151/259] added the implements Serializable to each class --- .../CommunityConfigurationFactory.java | 47 ++++------ .../eu/dnetlib/dhp/community/Constraint.java | 25 ++--- .../eu/dnetlib/dhp/community/Constraints.java | 40 +++----- .../dhp/community/QueryInformationSystem.java | 92 +++++++++---------- .../dnetlib/dhp/community/ResultTagger.java | 15 ++- .../dhp/community/SelectionConstraints.java | 23 ++--- .../dhp/selectioncriteria/ContainsVerb.java | 9 +- .../dhp/selectioncriteria/EqualVerb.java | 9 +- .../selectioncriteria/InterfaceAdapter.java | 17 ++-- .../selectioncriteria/NotContainsVerb.java | 9 +- .../dhp/selectioncriteria/NotEqualVerb.java | 9 +- .../dhp/selectioncriteria/Selection.java | 2 +- .../dhp/selectioncriteria/VerbResolver.java | 46 ++++++++-- .../VerbResolverFactory.java | 5 +- 14 files changed, 173 insertions(+), 175 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java index 772aea51d..866ec28a1 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java @@ -4,11 +4,14 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.gson.Gson; import com.google.gson.GsonBuilder; - import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; import eu.dnetlib.dhp.selectioncriteria.Selection; import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import eu.dnetlib.dhp.selectioncriteria.VerbResolverFactory; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -17,15 +20,7 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; - -import java.io.StringReader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Created by miriam on 03/08/2018. - */ +/** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactory { private static final Log log = LogFactory.getLog(CommunityConfigurationFactory.class); @@ -38,9 +33,9 @@ public class CommunityConfigurationFactory { final Document doc = new SAXReader().read(new StringReader(xml)); - final Map communities = Maps.newHashMap(); + final Map communities = Maps.newHashMap(); - for(final Object o : doc.selectNodes("//community")) { + for (final Object o : doc.selectNodes("//community")) { final Node node = (Node) o; @@ -54,8 +49,7 @@ public class CommunityConfigurationFactory { log.info(String.format("loaded %s community configuration profiles", communities.size())); log.debug(String.format("loaded community configuration:\n%s", communities.toString())); - - return new CommunityConfiguration(communities); + return new CommunityConfiguration(communities); } public static CommunityConfiguration fromJson(final String json) { @@ -88,9 +82,9 @@ public class CommunityConfigurationFactory { final List subjects = Lists.newArrayList(); - final List list = node.selectNodes("./subjects/subject"); + final List list = node.selectNodes("./subjects/subject"); - for(Node n : list){ + for (Node n : list) { log.debug("text of the node " + n.getText()); subjects.add(StringUtils.trim(n.getText())); } @@ -98,11 +92,10 @@ public class CommunityConfigurationFactory { return subjects; } - private static List parseDatasources(final Node node) { - final List list = node.selectNodes("./datasources/datasource"); + final List list = node.selectNodes("./datasources/datasource"); final List datasourceList = new ArrayList<>(); - for(Node n : list){ + for (Node n : list) { Datasource d = new Datasource(); d.setOpenaireId(n.selectSingleNode("./openaireId").getText()); d.setSelCriteria(n.selectSingleNode("./selcriteria"), resolver); @@ -115,23 +108,21 @@ public class CommunityConfigurationFactory { private static List parseZenodoCommunities(final Node node) { final Node oacommunitynode = node.selectSingleNode("./oacommunity"); String oacommunity = null; - if (oacommunitynode != null){ + if (oacommunitynode != null) { String tmp = oacommunitynode.getText(); - if(StringUtils.isNotBlank(tmp)) - oacommunity = tmp; + if (StringUtils.isNotBlank(tmp)) oacommunity = tmp; } - final List list = node.selectNodes("./zenodocommunities/zenodocommunity"); final List zenodoCommunityList = new ArrayList<>(); - for(Node n : list){ + for (Node n : list) { ZenodoCommunity zc = new ZenodoCommunity(); zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); zc.setSelCriteria(n.selectSingleNode("./selcriteria")); zenodoCommunityList.add(zc); } - if(oacommunity != null){ + if (oacommunity != null) { ZenodoCommunity zc = new ZenodoCommunity(); zc.setZenodoCommunityId(oacommunity); zenodoCommunityList.add(zc); @@ -139,8 +130,4 @@ public class CommunityConfigurationFactory { log.info("size of the zenodo community list " + zenodoCommunityList.size()); return zenodoCommunityList; } - - - - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java index 271493ca8..491f3de05 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java @@ -1,22 +1,17 @@ package eu.dnetlib.dhp.community; - - import eu.dnetlib.dhp.selectioncriteria.Selection; import eu.dnetlib.dhp.selectioncriteria.VerbResolver; - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; - public class Constraint implements Serializable { private String verb; private String field; private String value; private Selection selection; - public Constraint() { - } + public Constraint() {} public String getVerb() { return verb; @@ -42,21 +37,17 @@ public class Constraint implements Serializable { this.value = value; } - - - public void setSelection(Selection sel){ + public void setSelection(Selection sel) { selection = sel; } - public void setSelection(VerbResolver resolver) throws InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException { - selection = resolver.getSelectionCriteria(verb,value); + public void setSelection(VerbResolver resolver) + throws InvocationTargetException, NoSuchMethodException, InstantiationException, + IllegalAccessException { + selection = resolver.getSelectionCriteria(verb, value); } - - public boolean verifyCriteria(String metadata){ + public boolean verifyCriteria(String metadata) { return selection.apply(metadata); } - - - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java index a33f993f0..9b2974c27 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java @@ -1,32 +1,25 @@ package eu.dnetlib.dhp.community; - import com.google.gson.Gson; - import com.google.gson.reflect.TypeToken; import eu.dnetlib.dhp.selectioncriteria.VerbResolver; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Type; import java.util.Collection; import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; -/** - * Created by miriam on 02/08/2018. - */ +/** Created by miriam on 02/08/2018. */ public class Constraints implements Serializable { private static final Log log = LogFactory.getLog(Constraints.class); - //private ConstraintEncapsulator ce; + // private ConstraintEncapsulator ce; private List constraint; + public Constraints() {} - public Constraints() { - } public List getConstraint() { return constraint; } @@ -35,14 +28,13 @@ public class Constraints implements Serializable { this.constraint = constraint; } - public void setSc(String json){ - Type collectionType = new TypeToken>(){}.getType(); + public void setSc(String json) { + Type collectionType = new TypeToken>() {}.getType(); constraint = new Gson().fromJson(json, collectionType); - } void setSelection(VerbResolver resolver) { - for(Constraint st: constraint){ + for (Constraint st : constraint) { try { st.setSelection(resolver); @@ -56,24 +48,20 @@ public class Constraints implements Serializable { log.error(e.getMessage()); } } - } - - //Constraint in and + // Constraint in and public boolean verifyCriteria(final Map> param) { - for(Constraint sc : constraint) { + for (Constraint sc : constraint) { boolean verified = false; - for(String value : param.get(sc.getField())){ - if (sc.verifyCriteria(value.trim())){ + for (String value : param.get(sc.getField())) { + if (sc.verifyCriteria(value.trim())) { verified = true; } } - if(!verified) - return verified; + if (!verified) return verified; } return true; } - -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java index 00edf7fdc..7dd9339ef 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java @@ -4,63 +4,59 @@ import com.google.common.base.Joiner; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import java.util.List; import org.dom4j.DocumentException; -import java.util.List; - - public class QueryInformationSystem { - private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " + - " let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() " + - " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " + - " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " + - " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " + - " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " + - " return " + - " " + - " { $x//CONFIGURATION/context/@id} " + - " " + - " {for $y in tokenize($subj,',') " + - " return " + - " {$y}} " + - " " + - " " + - " {for $d in $datasources " + - " where $d/param[./@name='enabled']/text()='true' " + - " return " + - " " + - " " + - " {$d//param[./@name='openaireId']/text()} " + - " " + - " " + - " {$d/param[./@name='selcriteria']/text()} " + - " " + - " } " + - " " + - " " + - " {for $zc in $communities " + - " return " + - " " + - " " + - " {$zc/param[./@name='zenodoid']/text()} " + - " " + - " " + - " {$zc/param[./@name='selcriteria']/text()} " + - " " + - " } " + - " " + - " "; + private static final String XQUERY = + "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " + + " let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() " + + " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " + + " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " + + " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " + + " return " + + " " + + " { $x//CONFIGURATION/context/@id} " + + " " + + " {for $y in tokenize($subj,',') " + + " return " + + " {$y}} " + + " " + + " " + + " {for $d in $datasources " + + " where $d/param[./@name='enabled']/text()='true' " + + " return " + + " " + + " " + + " {$d//param[./@name='openaireId']/text()} " + + " " + + " " + + " {$d/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " " + + " {for $zc in $communities " + + " return " + + " " + + " " + + " {$zc/param[./@name='zenodoid']/text()} " + + " " + + " " + + " {$zc/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " "; - public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) throws ISLookUpException, DocumentException { + public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) + throws ISLookUpException, DocumentException { ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); final List res = isLookUp.quickSearchProfile(XQUERY); final String xmlConf = "" + Joiner.on(" ").join(res) + ""; - return CommunityConfigurationFactory.newInstance(xmlConf); - } - - } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java index abedab476..4e58bb1a2 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java @@ -56,7 +56,8 @@ public class ResultTagger implements Serializable { // Verify if the entity is deletedbyinference. In case verify if to clean the context list // from all the zenodo communities if (result.getDataInfo().getDeletedbyinference()) { - if (clearContext(result)) return result; + clearContext(result); + return result; } // communities contains all the communities to be added as context for the result @@ -118,7 +119,15 @@ public class ResultTagger implements Serializable { .map( c -> { if (communities.contains(c.getId())) { - List dataInfoList = c.getDataInfo(); + Optional> opt_dataInfoList = + Optional.ofNullable(c.getDataInfo()); + List dataInfoList; + if (opt_dataInfoList.isPresent()) + dataInfoList = opt_dataInfoList.get(); + else { + dataInfoList = new ArrayList<>(); + c.setDataInfo(dataInfoList); + } if (subjects.contains(c.getId())) dataInfoList.add( getDataInfo( @@ -153,7 +162,7 @@ public class ResultTagger implements Serializable { c -> { Context context = new Context(); context.setId(c); - List dataInfoList = Arrays.asList(); + List dataInfoList = new ArrayList<>(); if (subjects.contains(c)) dataInfoList.add( getDataInfo( diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java index 2c09ecaf1..530861425 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java @@ -3,8 +3,6 @@ package eu.dnetlib.dhp.community; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import eu.dnetlib.dhp.selectioncriteria.VerbResolver; - - import java.io.Serializable; import java.lang.reflect.Type; import java.util.Collection; @@ -14,9 +12,7 @@ import java.util.Map; public class SelectionConstraints implements Serializable { private List criteria; - public SelectionConstraints() { - } - + public SelectionConstraints() {} public List getCriteria() { return criteria; @@ -26,24 +22,25 @@ public class SelectionConstraints implements Serializable { this.criteria = criteria; } - public void setSc(String json){ - Type collectionType = new TypeToken>(){}.getType(); + public void setSc(String json) { + Type collectionType = new TypeToken>() {}.getType(); criteria = new Gson().fromJson(json, collectionType); } - //Constraints in or - public boolean verifyCriteria(final Map> param){ - for(Constraints selc : criteria) { - if(selc.verifyCriteria(param)){ + // Constraints in or + public boolean verifyCriteria(final Map> param) { + for (Constraints selc : criteria) { + if (selc.verifyCriteria(param)) { return true; } } return false; } + public void setSelection(VerbResolver resolver) { - for(Constraints cs : criteria){ + for (Constraints cs : criteria) { cs.setSelection(resolver); } } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java index eb7f059d8..fc6456a8c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java @@ -1,12 +1,13 @@ package eu.dnetlib.dhp.selectioncriteria; +import java.io.Serializable; + @VerbClass("contains") -public class ContainsVerb implements Selection { +public class ContainsVerb implements Selection, Serializable { private String param; - public ContainsVerb() { - } + public ContainsVerb() {} public ContainsVerb(final String param) { this.param = param; @@ -24,4 +25,4 @@ public class ContainsVerb implements Selection { public void setParam(String param) { this.param = param; } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java index 9b35e9583..32171001e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java @@ -1,19 +1,18 @@ package eu.dnetlib.dhp.selectioncriteria; +import java.io.Serializable; @VerbClass("equals") -public class EqualVerb implements Selection { +public class EqualVerb implements Selection, Serializable { private String param; - public EqualVerb() { - } + public EqualVerb() {} public EqualVerb(final String param) { this.param = param; } - @Override public boolean apply(String value) { return value.equalsIgnoreCase(param); @@ -26,4 +25,4 @@ public class EqualVerb implements Selection { public void setParam(String param) { this.param = param; } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java index 7cd261b85..a4a1494b3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java @@ -1,7 +1,6 @@ package eu.dnetlib.dhp.selectioncriteria; import com.google.gson.*; - import java.lang.reflect.Type; public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { @@ -9,8 +8,11 @@ public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { private static final String CLASSNAME = "CLASSNAME"; private static final String DATA = "DATA"; - public Object deserialize(JsonElement jsonElement, Type type, - JsonDeserializationContext jsonDeserializationContext) throws JsonParseException { + public Object deserialize( + JsonElement jsonElement, + Type type, + JsonDeserializationContext jsonDeserializationContext) + throws JsonParseException { JsonObject jsonObject = jsonElement.getAsJsonObject(); JsonPrimitive prim = (JsonPrimitive) jsonObject.get(CLASSNAME); @@ -18,20 +20,21 @@ public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { Class klass = getObjectClass(className); return jsonDeserializationContext.deserialize(jsonObject.get(DATA), klass); } - public JsonElement serialize(Object jsonElement, Type type, JsonSerializationContext jsonSerializationContext) { + + public JsonElement serialize( + Object jsonElement, Type type, JsonSerializationContext jsonSerializationContext) { JsonObject jsonObject = new JsonObject(); jsonObject.addProperty(CLASSNAME, jsonElement.getClass().getName()); jsonObject.add(DATA, jsonSerializationContext.serialize(jsonElement)); return jsonObject; } - /****** Helper method to get the className of the object to be deserialized *****/ + /** **** Helper method to get the className of the object to be deserialized **** */ public Class getObjectClass(String className) { try { return Class.forName(className); } catch (ClassNotFoundException e) { - //e.printStackTrace(); + // e.printStackTrace(); throw new JsonParseException(e.getMessage()); } } } - diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java index ecfabd7de..f07540d35 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java @@ -1,12 +1,13 @@ package eu.dnetlib.dhp.selectioncriteria; +import java.io.Serializable; + @VerbClass("not_contains") -public class NotContainsVerb implements Selection { +public class NotContainsVerb implements Selection, Serializable { private String param; - public NotContainsVerb() { - } + public NotContainsVerb() {} public NotContainsVerb(final String param) { this.param = param; @@ -24,4 +25,4 @@ public class NotContainsVerb implements Selection { public void setParam(String param) { this.param = param; } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java index c2c9e73ad..d1716588e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java @@ -1,18 +1,17 @@ package eu.dnetlib.dhp.selectioncriteria; +import java.io.Serializable; @VerbClass("not_equals") -public class NotEqualVerb implements Selection { +public class NotEqualVerb implements Selection, Serializable { private String param; - public NotEqualVerb(final String param) { this.param = param; } - public NotEqualVerb() { - } + public NotEqualVerb() {} public String getParam() { return param; @@ -26,4 +25,4 @@ public class NotEqualVerb implements Selection { public boolean apply(String value) { return !value.equalsIgnoreCase(param); } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java index cd9888a7e..c9b30790f 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java @@ -3,4 +3,4 @@ package eu.dnetlib.dhp.selectioncriteria; public interface Selection { boolean apply(String value); -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java index f6d5394e3..c2a9b4544 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java @@ -1,29 +1,57 @@ package eu.dnetlib.dhp.selectioncriteria; +import io.github.classgraph.ClassGraph; +import io.github.classgraph.ClassInfo; +import io.github.classgraph.ClassInfoList; +import io.github.classgraph.ScanResult; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.util.Map; import java.util.stream.Collectors; -import org.reflections.Reflections; public class VerbResolver implements Serializable { - private final Map> map; + private Map> map = null; // = new HashMap<>(); + private final ClassGraph classgraph = new ClassGraph(); public VerbResolver() { - this.map = - new Reflections("eu.dnetlib") - .getTypesAnnotatedWith(VerbClass.class).stream() - .collect( - Collectors.toMap( - v -> v.getAnnotation(VerbClass.class).value(), - v -> (Class) v)); + try (ScanResult scanResult = // Assign scanResult in try-with-resources + classgraph // Create a new ClassGraph instance + .verbose() // If you want to enable logging to stderr + .enableAllInfo() // Scan classes, methods, fields, annotations + .whitelistPackages( + "eu.dnetlib.dhp.selectioncriteria") // Scan com.xyz and subpackages + .scan()) { // Perform the scan and return a ScanResult + + ClassInfoList routeClassInfoList = + scanResult.getClassesWithAnnotation( + "eu.dnetlib.dhp.selectioncriteria.VerbClass"); + + this.map = + routeClassInfoList.stream() + .collect( + Collectors.toMap( + value -> + (String) + ((ClassInfo) value) + .getAnnotationInfo() + .get(0) + .getParameterValues() + .get(0) + .getValue(), + value -> + (Class) + ((ClassInfo) value).loadClass())); + } catch (Exception e) { + e.printStackTrace(); + } } public Selection getSelectionCriteria(String name, String param) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + // return Class.forName(tmp_map.get(name)). return map.get(name).getDeclaredConstructor((String.class)).newInstance(param); } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java index 8879e2d3b..c878c7f58 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java @@ -2,9 +2,8 @@ package eu.dnetlib.dhp.selectioncriteria; public class VerbResolverFactory { - public static VerbResolver newInstance(){ + public static VerbResolver newInstance() { return new VerbResolver(); } - -} \ No newline at end of file +} From 540f70298b5d2da357c62829f0655f53a0ece108 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:51:48 +0200 Subject: [PATCH 152/259] added missing property --- .../dhp/bulktag/oozie_app/config-default.xml | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml index 73bfe9ae7..fe82ae194 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml @@ -24,12 +24,20 @@ http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 - spark2EventLogDir - /user/spark/spark2ApplicationHistory + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener sparkExecutorNumber - 1 + 4 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory sparkDriverMemory From fa2ff5c6f5e8521af6dcc743bbba8d8a75da272c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 11:58:26 +0200 Subject: [PATCH 153/259] refactoring --- .../PrepareDatasourceCountryAssociation.java | 128 ++++---- .../SparkCountryPropagationJob2.java | 296 ++++++++++-------- 2 files changed, 228 insertions(+), 196 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index 2de536222..0604bb019 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -1,47 +1,43 @@ package eu.dnetlib.dhp.countrypropagation; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.Arrays; +import java.util.List; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.SaveMode; - -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; /** - * For the association of the country to the datasource - * The association is computed only for datasource of specific type or having whitelisted ids - * The country is registered in the Organization associated to the Datasource, so the - * relation provides between Datasource and Organization is exploited to get the country for the datasource + * For the association of the country to the datasource The association is computed only for + * datasource of specific type or having whitelisted ids The country is registered in the + * Organization associated to the Datasource, so the relation provides between Datasource and + * Organization is exploited to get the country for the datasource */ - public class PrepareDatasourceCountryAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareDatasourceCountryAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareDatasourceCountryAssociation.class.getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -57,73 +53,83 @@ public class PrepareDatasourceCountryAssociation { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - - - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - prepareDatasourceCountryAssociation(spark, + prepareDatasourceCountryAssociation( + spark, Arrays.asList(parser.get("whitelist").split(";")), Arrays.asList(parser.get("allowedtypes").split(";")), inputPath, outputPath); - }); - - } - - - private static void prepareDatasourceCountryAssociation(SparkSession spark, - List whitelist, - List allowedtypes, - String inputPath, - String outputPath) { + private static void prepareDatasourceCountryAssociation( + SparkSession spark, + List whitelist, + List allowedtypes, + String inputPath, + String outputPath) { String whitelisted = ""; - for (String i : whitelist){ + for (String i : whitelist) { whitelisted += " OR id = '" + i + "'"; } final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + Dataset datasource = + spark.createDataset( + sc.textFile(inputPath + "/datasource") + .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)) + .rdd(), + Encoders.bean(Datasource.class)); - Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + Dataset relation = + spark.createDataset( + sc.textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); - Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); - - Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + Dataset organization = + spark.createDataset( + sc.textFile(inputPath + "/organization") + .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)) + .rdd(), + Encoders.bean(Organization.class)); datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); - String query = "SELECT source dataSourceId, named_struct('classid', country.classid, 'classname', country.classname) country " + - "FROM ( SELECT id " + - " FROM datasource " + - " WHERE (datainfo.deletedbyinference = false " + whitelisted + ") " + - getConstraintList("datasourcetype.classid = '", allowedtypes) + ") d " + - "JOIN ( SELECT source, target " + - " FROM relation " + - " WHERE relclass = '" + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + "' " + - " AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source " + - "JOIN (SELECT id, country " + - " FROM organization " + - " WHERE datainfo.deletedbyinference = false " + - " AND length(country.classid)>0) o " + - "ON o.id = rel.target"; + String query = + "SELECT source dataSourceId, named_struct('classid', country.classid, 'classname', country.classname) country " + + "FROM ( SELECT id " + + " FROM datasource " + + " WHERE (datainfo.deletedbyinference = false " + + whitelisted + + ") " + + getConstraintList("datasourcetype.classid = '", allowedtypes) + + ") d " + + "JOIN ( SELECT source, target " + + " FROM relation " + + " WHERE relclass = '" + + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + + "' " + + " AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source " + + "JOIN (SELECT id, country " + + " FROM organization " + + " WHERE datainfo.deletedbyinference = false " + + " AND length(country.classid)>0) o " + + "ON o.id = rel.target"; spark.sql(query) .as(Encoders.bean(DatasourceCountry.class)) .toJavaRDD() .map(c -> OBJECT_MAPPER.writeValueAsString(c)) .saveAsTextFile(outputPath, GzipCodec.class); - - } - - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 643373a1c..37d4a5271 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -1,28 +1,22 @@ package eu.dnetlib.dhp.countrypropagation; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; -import scala.Tuple2; - -import java.util.*; - -import static eu.dnetlib.dhp.PropagationConstant.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import scala.Tuple2; public class SparkCountryPropagationJob2 { @@ -30,14 +24,14 @@ public class SparkCountryPropagationJob2 { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkCountryPropagationJob2.class - .getResourceAsStream("/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkCountryPropagationJob2.class.getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -56,182 +50,214 @@ public class SparkCountryPropagationJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + final String resultType = + resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); - final Boolean writeUpdates = Optional - .ofNullable(parser.get("writeUpdate")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean writeUpdates = + Optional.ofNullable(parser.get("writeUpdate")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("saveGraph: {}", saveGraph); - Class resultClazz = (Class) Class.forName(resultClassName); + Class resultClazz = + (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { - //createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + // createOutputDirs(outputPath, + // FileSystem.get(spark.sparkContext().hadoopConfiguration())); removeOutputDir(spark, outputPath); - execPropagation(spark, datasourcecountrypath, inputPath, outputPath, resultClazz, resultType, - writeUpdates, saveGraph); + execPropagation( + spark, + datasourcecountrypath, + inputPath, + outputPath, + resultClazz, + resultType, + writeUpdates, + saveGraph); }); - } - private static void execPropagation(SparkSession spark, String datasourcecountrypath, - String inputPath, String outputPath, Class resultClazz, String resultType, - boolean writeUpdates, boolean saveGraph){ + private static void execPropagation( + SparkSession spark, + String datasourcecountrypath, + String inputPath, + String outputPath, + Class resultClazz, + String resultType, + boolean writeUpdates, + boolean saveGraph) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - //Load parque file with preprocessed association datasource - country - Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); - //broadcasting the result of the preparation step - Broadcast> broadcast_datasourcecountryassoc = sc.broadcast(datasourcecountryassoc); + // Load parque file with preprocessed association datasource - country + Dataset datasourcecountryassoc = + readAssocDatasourceCountry(spark, datasourcecountrypath); + // broadcasting the result of the preparation step + Broadcast> broadcast_datasourcecountryassoc = + sc.broadcast(datasourcecountryassoc); - Dataset potentialUpdates = getPotentialResultToUpdate(spark, inputPath, resultClazz, - broadcast_datasourcecountryassoc).as(Encoders.bean(ResultCountrySet.class)); + Dataset potentialUpdates = + getPotentialResultToUpdate( + spark, inputPath, resultClazz, broadcast_datasourcecountryassoc) + .as(Encoders.bean(ResultCountrySet.class)); - if(writeUpdates){ + if (writeUpdates) { writeUpdates(potentialUpdates, outputPath + "/update_" + resultType); } - if(saveGraph){ + if (saveGraph) { updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); - } } - private static void updateResultTable(SparkSession spark, Dataset potentialUpdates, - String inputPath, - Class resultClazz, - String outputPath) { + private static void updateResultTable( + SparkSession spark, + Dataset potentialUpdates, + String inputPath, + Class resultClazz, + String outputPath) { log.info("Reading Graph table from: {}", inputPath); Dataset result = readPathEntity(spark, inputPath, resultClazz); - Dataset> result_pair = result - .map(r -> new Tuple2<>(r.getId(), r), - Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); + Dataset> result_pair = + result.map( + r -> new Tuple2<>(r.getId(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); -// Dataset> potential_update_pair = potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), -// pu), -// Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); + // Dataset> potential_update_pair = + // potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), + // pu), + // Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); - Dataset new_table = result_pair - .joinWith(potentialUpdates, result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), "left_outer") - .map((MapFunction, ResultCountrySet>, R>) value -> { - R r = value._1()._2(); - Optional potentialNewCountries = Optional.ofNullable(value._2()); - if (potentialNewCountries.isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : r.getCountry()) { - countries.add(country.getClassid()); - } - Result res = new Result(); - res.setId(r.getId()); - List countryList = new ArrayList<>(); - for (CountrySbs country : potentialNewCountries.get().getCountrySet()) { - if (!countries.contains(country.getClassid())) { - countryList.add(getCountry(country.getClassid(), country.getClassname())); - } - } - res.setCountry(countryList); - r.mergeFrom(res); - } - return r; - }, Encoders.bean(resultClazz)); + Dataset new_table = + result_pair + .joinWith( + potentialUpdates, + result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), + "left_outer") + .map( + (MapFunction, ResultCountrySet>, R>) + value -> { + R r = value._1()._2(); + Optional potentialNewCountries = + Optional.ofNullable(value._2()); + if (potentialNewCountries.isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : r.getCountry()) { + countries.add(country.getClassid()); + } + Result res = new Result(); + res.setId(r.getId()); + List countryList = new ArrayList<>(); + for (CountrySbs country : + potentialNewCountries + .get() + .getCountrySet()) { + if (!countries.contains(country.getClassid())) { + countryList.add( + getCountry( + country.getClassid(), + country.getClassname())); + } + } + res.setCountry(countryList); + r.mergeFrom(res); + } + return r; + }, + Encoders.bean(resultClazz)); - log.info("Saving graph table to path: {}", outputPath); - //log.info("number of saved recordsa: {}", new_table.count()); - new_table - .toJSON() - .write() - .option("compression", "gzip") - .text(outputPath); -// .toJavaRDD() -// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) -// .saveAsTextFile(outputPath , GzipCodec.class); + log.info("Saving graph table to path: {}", outputPath); + // log.info("number of saved recordsa: {}", new_table.count()); + new_table.toJSON().write().option("compression", "gzip").text(outputPath); + // .toJavaRDD() + // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + // .saveAsTextFile(outputPath , GzipCodec.class); - } + } - - - private static Dataset getPotentialResultToUpdate(SparkSession spark, String inputPath, - Class resultClazz, - Broadcast> broadcast_datasourcecountryassoc) { + private static Dataset getPotentialResultToUpdate( + SparkSession spark, + String inputPath, + Class resultClazz, + Broadcast> broadcast_datasourcecountryassoc) { Dataset result = readPathEntity(spark, inputPath, resultClazz); result.createOrReplaceTempView("result"); - //log.info("number of results: {}", result.count()); + // log.info("number of results: {}", result.count()); createCfHbforresult(spark); return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); } + // private static void createCfHbforresult(SparkSession spark) { + // String query; + // query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + // "FROM ( SELECT id, instance " + + // "FROM result " + + // " WHERE datainfo.deletedbyinference = false) ds " + + // "LATERAL VIEW EXPLODE(instance) i AS inst"; + // Dataset cfhb = spark.sql(query); + // cfhb.createOrReplaceTempView("cfhb"); + // //log.info("cfhb_number : {}", cfhb.count()); + // } -// private static void createCfHbforresult(SparkSession spark) { -// String query; -// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + -// "FROM ( SELECT id, instance " + -// "FROM result " + -// " WHERE datainfo.deletedbyinference = false) ds " + -// "LATERAL VIEW EXPLODE(instance) i AS inst"; -// Dataset cfhb = spark.sql(query); -// cfhb.createOrReplaceTempView("cfhb"); -// //log.info("cfhb_number : {}", cfhb.count()); -// } - - - private static Dataset countryPropagationAssoc(SparkSession spark, - Broadcast> broadcast_datasourcecountryassoc){ + private static Dataset countryPropagationAssoc( + SparkSession spark, + Broadcast> broadcast_datasourcecountryassoc) { Dataset datasource_country = broadcast_datasourcecountryassoc.value(); datasource_country.createOrReplaceTempView("datasource_country"); - log.info("datasource_country number : {}",datasource_country.count()); + log.info("datasource_country number : {}", datasource_country.count()); - String query = "SELECT id resultId, collect_set(country) countrySet "+ - "FROM ( SELECT id, country " + - "FROM datasource_country " + - "JOIN cfhb " + - " ON cf = dataSourceId " + - "UNION ALL " + - "SELECT id , country " + - "FROM datasource_country " + - "JOIN cfhb " + - " ON hb = dataSourceId ) tmp " + - "GROUP BY id"; + String query = + "SELECT id resultId, collect_set(country) countrySet " + + "FROM ( SELECT id, country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON cf = dataSourceId " + + "UNION ALL " + + "SELECT id , country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON hb = dataSourceId ) tmp " + + "GROUP BY id"; Dataset potentialUpdates = spark.sql(query); - //log.info("potential update number : {}", potentialUpdates.count()); + // log.info("potential update number : {}", potentialUpdates.count()); return potentialUpdates; } - - - private static Dataset readAssocDatasourceCountry(SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map(value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), Encoders.bean(DatasourceCountry.class)); + private static Dataset readAssocDatasourceCountry( + SparkSession spark, String relationPath) { + return spark.read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), + Encoders.bean(DatasourceCountry.class)); } - private static void writeUpdates(Dataset potentialUpdates, String outputPath){ + private static void writeUpdates( + Dataset potentialUpdates, String outputPath) { potentialUpdates .toJSON() .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .text(outputPath); -// map(u -> OBJECT_MAPPER.writeValueAsString(u)) -// .saveAsTextFile(outputPath, GzipCodec.class); + // map(u -> OBJECT_MAPPER.writeValueAsString(u)) + // .saveAsTextFile(outputPath, GzipCodec.class); } - - - -} \ No newline at end of file +} From 19cd5b85c044b11159a7dfcb1d4c6f0ddea8c97c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:07:41 +0200 Subject: [PATCH 154/259] changed the classname to execute --- .../eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 390311846..4015843f3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -68,7 +68,7 @@ yarn-cluster cluster bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob + eu.dnetlib.dhp.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -97,7 +97,7 @@ yarn-cluster cluster bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob + eu.dnetlib.dhp.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -126,7 +126,7 @@ yarn-cluster cluster bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob + eu.dnetlib.dhp.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -155,7 +155,7 @@ yarn-cluster cluster bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob + eu.dnetlib.dhp.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} From 6f35f5ca422c08c98c77e2544fd2d2b177ead6a2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:12:07 +0200 Subject: [PATCH 155/259] added the steps of reset output dir and copy information not changed by the propagation step --- .../countrypropagation/oozie_app/workflow.xml | 30 +++++++-- .../oozie_app/workflow.xml | 24 ++++++-- .../projecttoresult/oozie_app/workflow.xml | 61 ++++++++++++++++++- .../oozie_app/workflow.xml | 58 +++++++++++++++++- 4 files changed, 156 insertions(+), 17 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 0fd60e989..3a8aa949a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -39,12 +39,30 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/country_propagation/relation + + + + yarn @@ -65,7 +83,7 @@ --whitelist${whitelist} --allowedtypes${allowedtypes} --hive_metastore_uris${hive_metastore_uris} - --outputPath${workingDir}/country_propagation/preparedInfo + --outputPath${workingDir}/preparedInfo @@ -101,7 +119,7 @@ --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${workingDir}/country_propagation/publication - --preparedInfoPath${workingDir}/country_propagation/preparedInfo + --preparedInfoPath${workingDir}/preparedInfo @@ -131,7 +149,7 @@ --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${workingDir}/country_propagation/dataset - --preparedInfoPath${workingDir}/country_propagation/preparedInfo + --preparedInfoPath${workingDir}/preparedInfo @@ -161,7 +179,7 @@ --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${workingDir}/country_propagation/otherresearchproduct - --preparedInfoPath${workingDir}/country_propagation/preparedInfo + --preparedInfoPath${workingDir}/preparedInfo @@ -191,7 +209,7 @@ --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${workingDir}/country_propagation/software - --preparedInfoPath${workingDir}/country_propagation/preparedInfo + --preparedInfoPath${workingDir}/preparedInfo diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index 26e4aa1be..e26c8f28a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -27,11 +27,25 @@ - + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/orcid_propagation/relation + + + @@ -208,7 +222,7 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/publication + --outputPath${workingDir}/orcid_propagation/publication @@ -235,7 +249,7 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/dataset + --outputPath${workingDir}/orcid_propagation/dataset @@ -262,7 +276,7 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/otherresearchproduct + --outputPath${workingDir}/orcid_propagation/otherresearchproduct @@ -289,7 +303,7 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/software + --outputPath${workingDir}/orcid_propagation/software diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index d8df18c58..011ff70e3 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -37,11 +37,19 @@ - + - + + + + + + + + + ${jobTracker} @@ -49,10 +57,57 @@ ${nameNode}/${sourcePath}/relation ${nameNode}/${workingDir}/projecttoresult_propagation/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/publication + ${nameNode}/${workingDir}/projecttoresult_propagation/publication + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/dataset + ${nameNode}/${workingDir}/projecttoresult_propagation/dataset + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/otherresearchproduct + ${nameNode}/${workingDir}/projecttoresult_propagation/otherresearchproduct + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/software + ${nameNode}/${workingDir}/projecttoresult_propagation/software + + + + + + + + yarn diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 19e7e6507..13c3a2fee 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -22,11 +22,20 @@ - + - + + + + + + + + + + ${jobTracker} @@ -34,10 +43,53 @@ ${nameNode}/${sourcePath}/relation ${nameNode}/${workingDir}/resulttoorganization_propagation/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/publication + ${nameNode}/${workingDir}/resulttoorganization_propagation/publication + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/dataset + ${nameNode}/${workingDir}/resulttoorganization_propagation/dataset + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/otherresearchproduct + ${nameNode}/${workingDir}/resulttoorganization_propagation/otherresearchproduct + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/software + ${nameNode}/${workingDir}/resulttoorganization_propagation/software + + + + + yarn From 15656684b9f71ffc3d032f3136ad11914c3fd2da Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:13:34 +0200 Subject: [PATCH 156/259] added proeprties for the preparation step and actual propagation. Added the new parametrized workflow --- .../input_communitytoresult_parameters.json | 50 ++++- ...t_preparecommunitytoresult_parameters.json | 6 + .../oozie_app/config-default.xml | 40 ++++ .../oozie_app/workflow.xml | 196 +++++++++++++++--- 4 files changed, 253 insertions(+), 39 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json index ec2549d04..eebc1a0ca 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json @@ -1,21 +1,51 @@ [ - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"ocm", - "paramLongName":"organizationtoresultcommunitymap", - "paramDescription": "the map in json format assocaition the organization original id with the list of communities", + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, + { + "paramName":"test", + "paramLongName":"isTest", + "paramDescription": "true if it is executing a test", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", "paramRequired": true } + ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json index 19ef290e4..de472417d 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -28,6 +28,12 @@ "paramLongName":"isTest", "paramDescription": "true if it is executing a test", "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml index 2e0ed9aee..2744ea92b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml @@ -15,4 +15,44 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index cb85030d8..20ce6ddda 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -9,47 +9,185 @@ organization community map - sparkDriverMemory - memory for driver process + writeUpdate + writes the information found for the update. No double check done if the information is already present - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor + saveGraph + writes new version of the graph after the propagation step - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - ProjectToResultPropagation - eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob - dhp-propagation-${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" - - -mt yarn-cluster - --sourcePath${sourcePath} - --organizationtoresultcommunitymap${organizationtoresultcommunitymap} - - + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/projecttoresult_propagation/relation + + + + + + + + yarn + cluster + Prepare-Community-Result-Organization + eu.dnetlib.dhp.resulttocommunityfromorganization.PrepareResultCommunitySet + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/relation + --hive_metastore_uris${hive_metastore_uris} + --outputPath${workingDir}/preparedInfo/resultCommunityList + --organizationtoresultcommunitymap${organizationtoresultcommunitymap} + + + + + + + + + + + + + + + + yarn + cluster + community2resultfromorganization-Publication + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList + --sourcePath${sourcePath}/publication + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/publication + + + + + + + yarn + cluster + community2resultfromorganization-Dataset + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList + --sourcePath${sourcePath}/dataset + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/dataset + + + + + + + yarn + cluster + community2resultfromorganization-ORP + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList + --sourcePath${sourcePath}/otherresearchproduct + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/otherresearchproduct + + + + + + + yarn + cluster + community2resultfromorganization-Software + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/software + + + + + + + \ No newline at end of file From 8c5dac5cc362c10220016344b862ce0ef33e88cd Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:30:58 +0200 Subject: [PATCH 157/259] removed unuseful classes --- .../java/eu/dnetlib/dhp/SparkBulkTagJob.java | 72 ------------------- .../dhp/{ => bulktag}/SparkBulkTagJob2.java | 0 2 files changed, 72 deletions(-) delete mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/{ => bulktag}/SparkBulkTagJob2.java (100%) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java deleted file mode 100644 index 140e28f3d..000000000 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob.java +++ /dev/null @@ -1,72 +0,0 @@ -package eu.dnetlib.dhp; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.community.CommunityConfiguration; -import eu.dnetlib.dhp.community.ProtoMap; -import eu.dnetlib.dhp.community.QueryInformationSystem; -import eu.dnetlib.dhp.community.ResultTagger; -import eu.dnetlib.dhp.schema.oaf.*; -import java.io.File; -import org.apache.commons.io.IOUtils; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SparkSession; - -public class SparkBulkTagJob { - - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkBulkTagJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/input_bulktag_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkBulkTagJob.class.getSimpleName()) - .master(parser.get("master")) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/bulktagging"; - - final ResultTagger resultTagger = new ResultTagger(); - ProtoMap protoMappingParams = - new Gson().fromJson(parser.get("mappingProto"), ProtoMap.class); - ; - - File directory = new File(outputPath); - - if (!directory.exists()) { - directory.mkdirs(); - } - - CommunityConfiguration cc = - QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); - - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/publication"); - sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/dataset"); - sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/software"); - sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/otherresearchproduct"); - } -} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/SparkBulkTagJob2.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java From d8dc31d4afb8326d7deeb1fadb0f2af7f3486024 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:35:49 +0200 Subject: [PATCH 158/259] refactoring --- .../PrepareResultOrcidAssociationStep1.java | 109 ++++++------ .../PrepareResultOrcidAssociationStep2.java | 87 +++++----- .../SparkOrcidToResultFromSemRelJob3.java | 162 ++++++++++-------- 3 files changed, 197 insertions(+), 161 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index 8f4ecb649..9bc34eb73 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -1,38 +1,38 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; +import java.util.Arrays; +import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class PrepareResultOrcidAssociationStep1 { - private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkOrcidToResultFromSemRelJob3.class.getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -51,33 +51,44 @@ public class PrepareResultOrcidAssociationStep1 { final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + final String resultType = + resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); - - Class resultClazz = (Class) Class.forName(resultClassName); + Class resultClazz = + (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - prepareInfo(spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); + prepareInfo( + spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); }); } - private static void prepareInfo(SparkSession spark, String inputPath, - String outputPath, Class resultClazz, - String resultType, - List allowedsemrel) { + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + Class resultClazz, + String resultType, + List allowedsemrel) { - //read the relation table and the table related to the result it is using + // read the relation table and the table related to the result it is using final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset( + sc.textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); relation.createOrReplaceTempView("relation"); log.info("Reading Graph table from: {}", inputPath + "/" + resultType); @@ -86,38 +97,38 @@ public class PrepareResultOrcidAssociationStep1 { result.createOrReplaceTempView("result"); getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath + "/" + resultType); - } - private static void getPossibleResultOrcidAssociation(SparkSession spark, List allowedsemrel, String outputPath){ - String query = " select target resultId, author authorList" + - " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + - " from ( " + - " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + - " from result " + - " lateral view explode (author) a as MyT " + - " lateral view explode (MyT.pid) p as MyP " + - " where MyP.qualifier.classid = 'ORCID') tmp " + - " group by id) r_t " + - " join (" + - " select source, target " + - " from relation " + - " where datainfo.deletedbyinference = false " + - getConstraintList(" relclass = '" ,allowedsemrel) + ") rel_rel " + - " on source = id"; + private static void getPossibleResultOrcidAssociation( + SparkSession spark, List allowedsemrel, String outputPath) { + String query = + " select target resultId, author authorList" + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from result " + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false " + + getConstraintList(" relclass = '", allowedsemrel) + + ") rel_rel " + + " on source = id"; spark.sql(query) .as(Encoders.bean(ResultOrcidList.class)) .toJavaRDD() .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(outputPath, GzipCodec.class); -// .toJSON() -// .write() -// .mode(SaveMode.Append) -// .option("compression","gzip") -// .text(outputPath) -// ; + // .toJSON() + // .write() + // .mode(SaveMode.Append) + // .option("compression","gzip") + // .text(outputPath) + // ; } - - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index e38ba6583..658c97f6c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -1,7 +1,12 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import java.util.HashSet; +import java.util.Set; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -9,22 +14,20 @@ import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.Tuple2; -import java.util.HashSet; -import java.util.Set; -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; public class PrepareResultOrcidAssociationStep2 { - private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep2.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareResultOrcidAssociationStep2.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultOrcidAssociationStep2.class - .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareResultOrcidAssociationStep2.class.getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -39,56 +42,60 @@ public class PrepareResultOrcidAssociationStep2 { SparkConf conf = new SparkConf(); - runWithSparkSession(conf, isSparkSessionManaged, + runWithSparkSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - mergeInfo(spark, inputPath, outputPath); + mergeInfo(spark, inputPath, outputPath); }); } private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath + "/publication") - .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) - .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) - .union(readAssocResultOrcidList(spark, inputPath + "/software")); + Dataset resultOrcidAssoc = + readAssocResultOrcidList(spark, inputPath + "/publication") + .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) + .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) + .union(readAssocResultOrcidList(spark, inputPath + "/software")); resultOrcidAssoc .toJavaRDD() .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) - .reduceByKey((a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } - Set orcid_set = new HashSet<>(); - a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); + .reduceByKey( + (a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set orcid_set = new HashSet<>(); + a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); - b.getAuthorList().stream().forEach(aa -> { - if (!orcid_set.contains(aa.getOrcid())) { - a.getAuthorList().add(aa); - orcid_set.add(aa.getOrcid()); - } - }); - return a; - }) + b.getAuthorList().stream() + .forEach( + aa -> { + if (!orcid_set.contains(aa.getOrcid())) { + a.getAuthorList().add(aa); + orcid_set.add(aa.getOrcid()); + } + }); + return a; + }) .map(c -> c._2()) .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(outputPath, GzipCodec.class); } - private static Dataset readAssocResultOrcidList(SparkSession spark, String relationPath) { - return spark - .read() + private static Dataset readAssocResultOrcidList( + SparkSession spark, String relationPath) { + return spark.read() .textFile(relationPath) - .map(value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), Encoders.bean(ResultOrcidList.class)); + .map( + value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), + Encoders.bean(ResultOrcidList.class)); } - - - - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index b8fef7ef6..75527552a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -1,10 +1,15 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import java.util.List; +import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -15,30 +20,25 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; -import java.util.Optional; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class SparkOrcidToResultFromSemRelJob3 { - private static final Logger log = LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); + private static final Logger log = + LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkOrcidToResultFromSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkOrcidToResultFromSemRelJob3.class.getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); Boolean isSparkSessionManaged = isSparkSessionManaged(parser); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); @@ -51,96 +51,116 @@ public class SparkOrcidToResultFromSemRelJob3 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("saveGraph: {}", saveGraph); - Class resultClazz = (Class) Class.forName(resultClassName); + Class resultClazz = + (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { - if(isTest(parser)) { + if (isTest(parser)) { removeOutputDir(spark, outputPath); } - if(saveGraph) + if (saveGraph) execPropagation(spark, possibleUpdates, inputPath, outputPath, resultClazz); }); - } - private static void execPropagation(SparkSession spark, String possibleUpdatesPath, String inputPath, - String outputPath, Class resultClazz ) { + private static void execPropagation( + SparkSession spark, + String possibleUpdatesPath, + String inputPath, + String outputPath, + Class resultClazz) { - //read possible updates (resultId and list of possible orcid to add - Dataset possible_updates = readAssocResultOrcidList(spark, possibleUpdatesPath); - //read the result we have been considering + // read possible updates (resultId and list of possible orcid to add + Dataset possible_updates = + readAssocResultOrcidList(spark, possibleUpdatesPath); + // read the result we have been considering Dataset result = readPathEntity(spark, inputPath, resultClazz); - //make join result left_outer with possible updates + // make join result left_outer with possible updates - result.joinWith(possible_updates, result.col("id").equalTo(possible_updates.col("resultId")), - "left_outer") - .map(value -> { - R ret = value._1(); - Optional rol = Optional.ofNullable(value._2()); - if(rol.isPresent()) { - List toenrich_author = ret.getAuthor(); - List autoritativeAuthors = rol.get().getAuthorList(); - for(Author author: toenrich_author){ - if (!containsAllowedPid(author)){ - enrichAuthor(author, autoritativeAuthors); + result.joinWith( + possible_updates, + result.col("id").equalTo(possible_updates.col("resultId")), + "left_outer") + .map( + value -> { + R ret = value._1(); + Optional rol = Optional.ofNullable(value._2()); + if (rol.isPresent()) { + List toenrich_author = ret.getAuthor(); + List autoritativeAuthors = + rol.get().getAuthorList(); + for (Author author : toenrich_author) { + if (!containsAllowedPid(author)) { + enrichAuthor(author, autoritativeAuthors); + } } } - } - - return ret; - }, Encoders.bean(resultClazz)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(outputPath); - + return ret; + }, + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); } - private static Dataset readAssocResultOrcidList(SparkSession spark, String relationPath) { - return spark - .read() + private static Dataset readAssocResultOrcidList( + SparkSession spark, String relationPath) { + return spark.read() .textFile(relationPath) - .map(value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), Encoders.bean(ResultOrcidList.class)); + .map( + value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), + Encoders.bean(ResultOrcidList.class)); } - private static void enrichAuthor(Author a, List au){ - for (AutoritativeAuthor aa: au){ - if(enrichAuthor(aa, a)){ + private static void enrichAuthor(Author a, List au) { + for (AutoritativeAuthor aa : au) { + if (enrichAuthor(aa, a)) { return; } } - } - - private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, Author author) { boolean toaddpid = false; if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { if (StringUtils.isNoneEmpty(author.getSurname())) { - if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) { + if (autoritative_author + .getSurname() + .trim() + .equalsIgnoreCase(author.getSurname().trim())) { - //have the same surname. Check the name + // have the same surname. Check the name if (StringUtils.isNoneEmpty(autoritative_author.getName())) { if (StringUtils.isNoneEmpty(author.getName())) { - if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) { + if (autoritative_author + .getName() + .trim() + .equalsIgnoreCase(author.getName().trim())) { toaddpid = true; } - //they could be differently written (i.e. only the initials of the name in one of the two - if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) { + // they could be differently written (i.e. only the initials of the name + // in one of the two + if (autoritative_author + .getName() + .trim() + .substring(0, 0) + .equalsIgnoreCase(author.getName().trim().substring(0, 0))) { toaddpid = true; } } @@ -148,21 +168,20 @@ public class SparkOrcidToResultFromSemRelJob3 { } } } - if (toaddpid){ + if (toaddpid) { StructuredProperty p = new StructuredProperty(); p.setValue(autoritative_author.getOrcid()); - p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID )); - p.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID)); + p.setDataInfo( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, + PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); author.addPid(p); - } return toaddpid; - } - - - private static boolean containsAllowedPid(Author a) { for (StructuredProperty pid : a.getPid()) { if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { @@ -171,5 +190,4 @@ public class SparkOrcidToResultFromSemRelJob3 { } return false; } - } From 769aa8178a49809fab3669146d2c27c33518bbee Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:40:44 +0200 Subject: [PATCH 159/259] refactoring --- .../PrepareProjectResultsAssociation.java | 136 +++++++++------- ...SparkResultToProjectThroughSemRelJob3.java | 149 ++++++++++-------- 2 files changed, 163 insertions(+), 122 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index f94d67734..2e9255ed5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -1,37 +1,36 @@ package eu.dnetlib.dhp.projecttoresult; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; import eu.dnetlib.dhp.schema.oaf.Relation; +import java.util.Arrays; +import java.util.List; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class PrepareProjectResultsAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception{ + public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareProjectResultsAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareProjectResultsAssociation.class.getResourceAsStream( + "/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -53,69 +52,88 @@ public class PrepareProjectResultsAssociation { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { -// removeOutputDir(spark, potentialUpdatePath); -// removeOutputDir(spark, alreadyLinkedPath); - prepareResultProjProjectResults(spark, inputPath, potentialUpdatePath, alreadyLinkedPath, allowedsemrel); - + // removeOutputDir(spark, potentialUpdatePath); + // removeOutputDir(spark, alreadyLinkedPath); + prepareResultProjProjectResults( + spark, + inputPath, + potentialUpdatePath, + alreadyLinkedPath, + allowedsemrel); }); } - private static void prepareResultProjProjectResults(SparkSession spark, String inputPath, String potentialUpdatePath, - String alreadyLinkedPath, List allowedsemrel) { + private static void prepareResultProjProjectResults( + SparkSession spark, + String inputPath, + String potentialUpdatePath, + String alreadyLinkedPath, + List allowedsemrel) { JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset relation = spark.createDataset(sc.textFile(inputPath ) - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + Dataset relation = + spark.createDataset( + sc.textFile(inputPath) + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); relation.createOrReplaceTempView("relation"); - String query = "SELECT source, target " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; + String query = + "SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_RESULT_PROJECT_REL_CLASS + + "'"; Dataset resproj_relation = spark.sql(query); resproj_relation.createOrReplaceTempView("resproj_relation"); - query ="SELECT projectId, collect_set(resId) resultSet " + - "FROM (" + - " SELECT r1.target resId, r2.target projectId " + - " FROM (SELECT source, target " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - getConstraintList(" relClass = '", allowedsemrel ) + ") r1" + - " JOIN resproj_relation r2 " + - " ON r1.source = r2.source " + - " ) tmp " + - "GROUP BY projectId "; - - spark.sql(query).as(Encoders.bean(ProjectResultSet.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(potentialUpdatePath); -// .toJavaRDD() -// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) -// .saveAsTextFile(potentialUpdatePath, GzipCodec.class); - - - query = "SELECT target projectId, collect_set(source) resultSet " + - "FROM resproj_relation " + - "GROUP BY target"; + query = + "SELECT projectId, collect_set(resId) resultSet " + + "FROM (" + + " SELECT r1.target resId, r2.target projectId " + + " FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + getConstraintList(" relClass = '", allowedsemrel) + + ") r1" + + " JOIN resproj_relation r2 " + + " ON r1.source = r2.source " + + " ) tmp " + + "GROUP BY projectId "; spark.sql(query) .as(Encoders.bean(ProjectResultSet.class)) .toJSON() .write() .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(alreadyLinkedPath); -// .toJavaRDD() -// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) -// .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + .option("compression", "gzip") + .text(potentialUpdatePath); + // .toJavaRDD() + // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + // .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + query = + "SELECT target projectId, collect_set(source) resultSet " + + "FROM resproj_relation " + + "GROUP BY target"; + + spark.sql(query) + .as(Encoders.bean(ProjectResultSet.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(alreadyLinkedPath); + // .toJavaRDD() + // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + // .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index 2125f351f..45d697651 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -1,31 +1,34 @@ package eu.dnetlib.dhp.projecttoresult; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; import eu.dnetlib.dhp.schema.oaf.Relation; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; public class SparkResultToProjectThroughSemRelJob3 { - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkResultToProjectThroughSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkResultToProjectThroughSemRelJob3.class.getResourceAsStream( + "/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -49,84 +52,104 @@ public class SparkResultToProjectThroughSemRelJob3 { SparkConf conf = new SparkConf(); - - runWithSparkSession(conf, isSparkSessionManaged, + runWithSparkSession( + conf, + isSparkSessionManaged, spark -> { - if(isTest(parser)) { + if (isTest(parser)) { removeOutputDir(spark, outputPath); } - execPropagation(spark, outputPath, alreadyLinkedPath, potentialUpdatePath, writeUpdates, saveGraph); + execPropagation( + spark, + outputPath, + alreadyLinkedPath, + potentialUpdatePath, + writeUpdates, + saveGraph); }); } + private static void execPropagation( + SparkSession spark, + String outputPath, + String alreadyLinkedPath, + String potentialUpdatePath, + Boolean writeUpdate, + Boolean saveGraph) { - private static void execPropagation(SparkSession spark, String outputPath, String alreadyLinkedPath, String potentialUpdatePath, - Boolean writeUpdate, Boolean saveGraph){ + Dataset toaddrelations = + readAssocProjectResults(spark, potentialUpdatePath); + Dataset alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath); - Dataset toaddrelations = readAssocProjectResults(spark, potentialUpdatePath); - Dataset alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath); - - if(writeUpdate){ + if (writeUpdate) { toaddrelations .toJSON() .write() .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(outputPath +"/potential_updates"); + .option("compression", "gzip") + .text(outputPath + "/potential_updates"); } - if (saveGraph){ + if (saveGraph) { getNewRelations(alreadyLinked, toaddrelations) .toJSON() .write() .mode(SaveMode.Append) .option("compression", "gzip") .text(outputPath); - } - } - private static Dataset getNewRelations(Dataset alreadyLinked, - Dataset toaddrelations){ - + private static Dataset getNewRelations( + Dataset alreadyLinked, Dataset toaddrelations) { return toaddrelations - .joinWith(alreadyLinked, toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")), "left") - .flatMap(value -> { - List new_relations = new ArrayList<>(); - ProjectResultSet potential_update = value._1(); - ProjectResultSet already_linked = value._2(); - String projId = already_linked.getProjectId(); - potential_update - .getResultSet() - .stream() - .forEach(rId -> { - if (!already_linked.getResultSet().contains(rId)){ - new_relations.add(getRelation(rId, projId, RELATION_RESULT_PROJECT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - new_relations.add(getRelation(projId, rId, RELATION_PROJECT_RESULT_REL_CLASS, RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - } - }); - return new_relations.iterator(); - - } - ,Encoders.bean(Relation.class)); - - - - - } - - private static Dataset readAssocProjectResults(SparkSession spark, String potentialUpdatePath) { - return spark - .read() - .textFile(potentialUpdatePath) - .map(value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class), Encoders.bean(ProjectResultSet.class)); + .joinWith( + alreadyLinked, + toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")), + "left") + .flatMap( + value -> { + List new_relations = new ArrayList<>(); + ProjectResultSet potential_update = value._1(); + ProjectResultSet already_linked = value._2(); + String projId = already_linked.getProjectId(); + potential_update.getResultSet().stream() + .forEach( + rId -> { + if (!already_linked.getResultSet().contains(rId)) { + new_relations.add( + getRelation( + rId, + projId, + RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + new_relations.add( + getRelation( + projId, + rId, + RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + } + }); + return new_relations.iterator(); + }, + Encoders.bean(Relation.class)); } + private static Dataset readAssocProjectResults( + SparkSession spark, String potentialUpdatePath) { + return spark.read() + .textFile(potentialUpdatePath) + .map( + value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class), + Encoders.bean(ProjectResultSet.class)); + } } From 44fab140de0cb06323385be07eb99e5983e00000 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:42:07 +0200 Subject: [PATCH 160/259] - --- ...parkResultToCommunityThroughSemRelJob.java | 895 ++++++++++------- ...arkResultToCommunityThroughSemRelJob2.java | 943 ++++++++++-------- ...arkResultToCommunityThroughSemRelJob3.java | 869 +++++++++------- 3 files changed, 1626 insertions(+), 1081 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java index f5c859280..cc90feac2 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -1,10 +1,14 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; @@ -16,88 +20,102 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; - public class SparkResultToCommunityThroughSemRelJob { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + final ArgumentApplicationParser parser = + new ArgumentApplicationParser( + IOUtils.toString( + SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); parser.parseArgument(args); - for(String key : parser.getObjectMap().keySet()){ + for (String key : parser.getObjectMap().keySet()) { System.out.println(key + " = " + parser.get(key)); } - - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); + final SparkSession spark = + SparkSession.builder() + .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; - //final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + // final List allowedsemrel = + // Arrays.asList(parser.get("allowedsemrels").split(";")); final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); - //final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); + // final List communityIdList = + // QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + final List communityIdList = + QueryInformationSystem.getCommunityList( + "http://beta.services.openaire.eu:8280/is/services/isLookUp"); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + JavaRDD all_publication_rdd = + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .cache(); + JavaRDD publication_rdd = + all_publication_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaRDD all_publication_rdd = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD publication_rdd = all_publication_rdd - .filter(p -> relatedToCommunities(p, communityIdList)).cache(); + JavaRDD all_dataset_rdd = + sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .cache(); + JavaRDD dataset_rdd = + all_dataset_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaRDD all_dataset_rdd = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD dataset_rdd = all_dataset_rdd - .filter(p -> relatedToCommunities(p, communityIdList)).cache(); + JavaRDD all_orp_rdd = + sc.textFile(inputPath + "/otherresearchproduct") + .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .cache(); + JavaRDD orp_rdd = + all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaRDD all_orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD orp_rdd = all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + JavaRDD all_software_rdd = + sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)) + .filter(p -> !p.getDataInfo().getDeletedbyinference()) + .cache(); + JavaRDD software_rdd = + all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - JavaRDD all_software_rdd = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()).cache(); - JavaRDD software_rdd = all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); + JavaRDD relation_rdd = + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .filter(r -> !r.getDataInfo().getDeletedbyinference()) + .filter( + r -> + allowedsemrel.contains(r.getRelClass()) + && RELATION_RESULTRESULT_REL_TYPE.equals( + r.getRelType())) + .cache(); - JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + org.apache.spark.sql.Dataset publication = + spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); + org.apache.spark.sql.Dataset dataset = + spark.createDataset(dataset_rdd.rdd(), Encoders.bean(Dataset.class)); - org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), - Encoders.bean(Publication.class)); + org.apache.spark.sql.Dataset other = + spark.createDataset(orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); - org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset software = + spark.createDataset(software_rdd.rdd(), Encoders.bean(Software.class)); - org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), - Encoders.bean(Software.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), - Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); publication.createOrReplaceTempView("publication"); relation.createOrReplaceTempView("relation"); @@ -105,14 +123,17 @@ public class SparkResultToCommunityThroughSemRelJob { software.createOrReplaceTempView("software"); other.createOrReplaceTempView("other"); -// org.apache.spark.sql.Dataset publication_context = getContext(spark, "publication"); -// publication_context.createOrReplaceTempView("publication_context"); + // org.apache.spark.sql.Dataset publication_context = getContext(spark, + // "publication"); + // publication_context.createOrReplaceTempView("publication_context"); - org.apache.spark.sql.Dataset publication_context = spark.sql( "SELECT relation.source, " + - "publication.context , relation.target " + - "FROM publication " + - " JOIN relation " + - "ON id = source"); + org.apache.spark.sql.Dataset publication_context = + spark.sql( + "SELECT relation.source, " + + "publication.context , relation.target " + + "FROM publication " + + " JOIN relation " + + "ON id = source"); org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); dataset_context.createOrReplaceTempView("dataset_context"); @@ -123,316 +144,473 @@ public class SparkResultToCommunityThroughSemRelJob { org.apache.spark.sql.Dataset other_context = getContext(spark, "other"); other_context.createOrReplaceTempView("other_context"); - publication = spark.createDataset(all_publication_rdd.rdd(), - Encoders.bean(Publication.class)); + publication = + spark.createDataset(all_publication_rdd.rdd(), Encoders.bean(Publication.class)); publication.createOrReplaceTempView("publication"); - dataset = spark.createDataset(all_dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); + dataset = spark.createDataset(all_dataset_rdd.rdd(), Encoders.bean(Dataset.class)); dataset.createOrReplaceTempView("dataset"); - other = spark.createDataset(all_orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); + other = spark.createDataset(all_orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); other.createOrReplaceTempView("other"); - software = spark.createDataset(all_software_rdd.rdd(), - Encoders.bean(Software.class)); + software = spark.createDataset(all_software_rdd.rdd(), Encoders.bean(Software.class)); software.createOrReplaceTempView("software"); + org.apache.spark.sql.Dataset toupdatesoftwareresult = + getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = + getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdatepublicationreresult = + getUpdateCommunitiesForTable(spark, "publication"); + org.apache.spark.sql.Dataset toupdateotherresult = + getUpdateCommunitiesForTable(spark, "other"); - org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); - org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + createUpdateForResultDatasetWrite( + toupdatesoftwareresult.toJavaRDD(), + outputPath, + "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdatedatasetresult.toJavaRDD(), + outputPath, + "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdatepublicationreresult.toJavaRDD(), + outputPath, + "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdateotherresult.toJavaRDD(), + outputPath, + "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForDatasetDataset( + toupdatedatasetresult.toJavaRDD(), + dataset.toJavaRDD(), + outputPath, + "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); + updateForOtherDataset( + toupdateotherresult.toJavaRDD(), + other.toJavaRDD(), + outputPath, + "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForSoftwareDataset( + toupdatesoftwareresult.toJavaRDD(), + software.toJavaRDD(), + outputPath, + "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForPublicationDataset( + toupdatepublicationreresult.toJavaRDD(), + publication.toJavaRDD(), + outputPath, + "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - -/* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + /* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) .filter(p -> !(p == null)) .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla -*/ + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla + */ } - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ - String query = "SELECT target_id, collect_set(co.id) context_id " + - " FROM (SELECT t.id target_id, s.context source_context " + - " FROM context_software s " + - " JOIN " + table + " t " + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, d.context source_context " + - " FROM dataset_context d " + - " JOIN " + table + " t" + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, p.context source_context " + - " FROM publication_context p" + - " JOIN " + table +" t " + - " on p.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, o.context source_context " + - " FROM other_context o " + - " JOIN " + table + " t " + - " ON o.target = t.id) TMP " + - " LATERAL VIEW EXPLODE(source_context) MyT as co " + - " GROUP BY target_id" ; + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( + SparkSession spark, String table) { + String query = + "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + + table + + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + + table + + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + + table + + " t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + + table + + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id"; return spark.sql(query); } - private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return toupdateresult.map(r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - contextList.add(newContext); - } + private static JavaRDD createUpdateForResultDatasetWrite( + JavaRDD toupdateresult, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return toupdateresult + .map( + r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + contextList.add(newContext); + } + } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }).filter(r -> r != null); + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }) + .filter(r -> r != null); } - private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForSoftwareDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r-> (Dataset)r) + private static void updateForDatasetDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Dataset) r) .map(d -> new ObjectMapper().writeValueAsString(d)) .saveAsTextFile(outputPath + "/" + type); } - private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Publication)r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForPublicationDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Publication) r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r -> (OtherResearchProduct)r) - .map( o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForOtherDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (OtherResearchProduct) r) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); } + private static JavaRDD getUpdateForResultDataset( + JavaRDD toupdateresult, + JavaPairRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return result.leftOuterJoin( + toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map( + c -> { + if (!c._2()._2().isPresent()) { + return c._2()._1(); + } - - private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map(c -> { - if(! c._2()._2().isPresent()){ - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for(Object cId: toAddContext){ - String id = (String)cId; - if (communityIdList.contains(id)){ - context_set.add(id); - } - } - for (Context context: c._2()._1().getContext()){ - if(context_set.contains(context)){ - context_set.remove(context); - } - } - - List contextList = context_set.stream().map(co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - return newContext; - - }).collect(Collectors.toList()); - - if(contextList.size() > 0 ){ - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }).filter(r -> r != null); - - -// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) -// .join(result) -// .map(c -> { -// List toAddContext = c._2()._1(); -// Set context_set = new HashSet<>(); -// for(Object cId: toAddContext){ -// String id = (String)cId; -// if (communityIdList.contains(id)){ -// context_set.add(id); -// } -// } -// for (Context context: c._2()._2().getContext()){ -// if(context_set.contains(context)){ -// context_set.remove(context); -// } -// } -// -// List contextList = context_set.stream().map(co -> { -// Context newContext = new Context(); -// newContext.setId(co); -// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); -// return newContext; -// -// }).collect(Collectors.toList()); -// -// if(contextList.size() > 0 ){ -// Result r = new Result(); -// r.setId(c._1()); -// r.setContext(contextList); -// return r; -// } -// return null; -// }) -// .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, - JavaRDD result, String class_id, String class_name) { - return result - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map(c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())){ - if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); - //community id already in the context of the result. Remove it from the set that has to be added - contexts.remove(context.getId()); + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for (Object cId : toAddContext) { + String id = (String) cId; + if (communityIdList.contains(id)) { + context_set.add(id); + } + } + for (Context context : c._2()._1().getContext()) { + if (context_set.contains(context)) { + context_set.remove(context); } } - } - List cc = oaf.getContext(); - for(String cId: contexts){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - cc.add(context); - } - oaf.setContext(cc); + List contextList = + context_set.stream() + .map( + co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + return newContext; + }) + .collect(Collectors.toList()); + + if (contextList.size() > 0) { + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }) + .filter(r -> r != null); + + // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + // .join(result) + // .map(c -> { + // List toAddContext = c._2()._1(); + // Set context_set = new HashSet<>(); + // for(Object cId: toAddContext){ + // String id = (String)cId; + // if (communityIdList.contains(id)){ + // context_set.add(id); + // } + // } + // for (Context context: c._2()._2().getContext()){ + // if(context_set.contains(context)){ + // context_set.remove(context); + // } + // } + // + // List contextList = context_set.stream().map(co -> { + // Context newContext = new Context(); + // newContext.setId(co); + // + // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, + // class_name))); + // return newContext; + // + // }).collect(Collectors.toList()); + // + // if(contextList.size() > 0 ){ + // Result r = new Result(); + // r.setId(c._1()); + // r.setContext(contextList); + // return r; + // } + // return null; + // }) + // .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset( + JavaRDD toupdateresult, + List communityList, + JavaRDD result, + String class_id, + String class_name) { + return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) + .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map( + c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())) { + if (!context.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + context.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name)); + // community id already in the context of the result. + // Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for (String cId : contexts) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + cc.add(context); + } + oaf.setContext(cc); + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD( + JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair( + c -> { + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } } - return oaf; + + return new Tuple2<>(c.getString(0), contextList); }); } - private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair(c -> { - - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0) ,contextList); - }); - } - - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ - String query = "SELECT relation.source, " + table +".context , relation.target " + - "FROM " + table + - " JOIN relation " + - "ON id = source" ; + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { + String query = + "SELECT relation.source, " + + table + + ".context , relation.target " + + "FROM " + + table + + " JOIN relation " + + "ON id = source"; return spark.sql(query); } private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = r.getContext() - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + Set result_communities = + r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); for (String communityId : result_communities) { if (communityIdList.contains(communityId)) { return true; @@ -441,44 +619,61 @@ public class SparkResultToCommunityThroughSemRelJob { return false; } - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + private static void updateResult( + JavaPairRDD results, + JavaPairRDD toupdateresult, + String outputPath, + String type) { results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - Set communityList = p._2()._2().get().getAccumulator(); - for(Context c: r.getContext()){ - if (communityList.contains(c.getId())){ - //verify if the datainfo for this context contains propagation - if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - //community id already in the context of the result. Remove it from the set that has to be added - communityList.remove(c.getId()); + .map( + p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()) { + Set communityList = p._2()._2().get().getAccumulator(); + for (Context c : r.getContext()) { + if (communityList.contains(c.getId())) { + // verify if the datainfo for this context contains + // propagation + if (!c.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + c.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + // community id already in the context of the result. + // Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } } + List cc = r.getContext(); + for (String cId : communityList) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); } - } - List cc = r.getContext(); - for(String cId: communityList){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) + return r; + }) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); + .saveAsTextFile(outputPath + "/" + type); } - - - private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { - Set result_communities = context - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + private static TypedRow getTypedRow( + List communityIdList, List context, String id, String type) { + Set result_communities = + context.stream().map(c -> c.getId()).collect(Collectors.toSet()); TypedRow tp = new TypedRow(); tp.setSourceId(id); tp.setType(type); diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java index 345bd7905..3a0904404 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java @@ -1,10 +1,14 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; @@ -16,447 +20,589 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; - public class SparkResultToCommunityThroughSemRelJob2 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob2.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + final ArgumentApplicationParser parser = + new ArgumentApplicationParser( + IOUtils.toString( + SparkResultToCommunityThroughSemRelJob2.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); parser.parseArgument(args); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); + final SparkSession spark = + SparkSession.builder() + .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - //final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); - final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - //final List communityIdList = QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); + // final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); + final List communityIdList = + QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + // final List communityIdList = + // QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD publication_rdd = + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); System.out.println(publication_rdd.count()); -// JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") -// .map(item -> new ObjectMapper().readValue(item, Dataset.class)); -// -// JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") -// .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); -// -// JavaRDD software_rdd = sc.textFile(inputPath + "/software") -// .map(item -> new ObjectMapper().readValue(item, Software.class)); + // JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") + // .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + // + // JavaRDD orp_rdd = sc.textFile(inputPath + + // "/otherresearchproduct") + // .map(item -> new ObjectMapper().readValue(item, + // OtherResearchProduct.class)); + // + // JavaRDD software_rdd = sc.textFile(inputPath + "/software") + // .map(item -> new ObjectMapper().readValue(item, Software.class)); - JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); + JavaRDD relation_rdd = + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); System.out.println(relation_rdd.count()); -// .filter(r -> !r.getDataInfo().getDeletedbyinference()) -// .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + // .filter(r -> !r.getDataInfo().getDeletedbyinference()) + // .filter(r -> allowedsemrel.contains(r.getRelClass()) && + // RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); + org.apache.spark.sql.Dataset publication = + spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); - org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), - Encoders.bean(Publication.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); - org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), - Encoders.bean(Relation.class)); - -// org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), -// Encoders.bean(Dataset.class)); -// -// org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), -// Encoders.bean(OtherResearchProduct.class)); -// -// org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), -// Encoders.bean(Software.class)); -// -// org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), -// Encoders.bean(Relation.class)); + // org.apache.spark.sql.Dataset dataset = + // spark.createDataset(dataset_rdd.rdd(), + // Encoders.bean(Dataset.class)); + // + // org.apache.spark.sql.Dataset other = + // spark.createDataset(orp_rdd.rdd(), + // Encoders.bean(OtherResearchProduct.class)); + // + // org.apache.spark.sql.Dataset software = + // spark.createDataset(software_rdd.rdd(), + // Encoders.bean(Software.class)); + // + // org.apache.spark.sql.Dataset relation = + // spark.createDataset(relation_rdd.rdd(), + // Encoders.bean(Relation.class)); publication.createOrReplaceTempView("publication"); relation.createOrReplaceTempView("relation"); -// relation.createOrReplaceTempView("relation"); -// dataset.createOrReplaceTempView("dataset"); -// software.createOrReplaceTempView("software"); -// other.createOrReplaceTempView("other"); + // relation.createOrReplaceTempView("relation"); + // dataset.createOrReplaceTempView("dataset"); + // software.createOrReplaceTempView("software"); + // other.createOrReplaceTempView("other"); String communitylist = getConstraintList(" co.id = '", communityIdList); - String semrellist = getConstraintList(" relClass = '", allowedsemrel ); + String semrellist = getConstraintList(" relClass = '", allowedsemrel); + String query = + "Select source, community_context, target " + + "from (select id, collect_set(co.id) community_context " + + "from publication " + + "lateral view explode (context) c as co " + + "where datainfo.deletedbyinference = false " + + communitylist + + " group by id) p " + + "JOIN " + + "(select * " + + "from relation " + + "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " + + "ON p.id = r.source"; - String query = "Select source, community_context, target " + - "from (select id, collect_set(co.id) community_context " + - "from publication " + - "lateral view explode (context) c as co " + - "where datainfo.deletedbyinference = false "+ communitylist + - " group by id) p " + - "JOIN " + - "(select * " + - "from relation " + - "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " + - "ON p.id = r.source"; - - - org.apache.spark.sql.Dataset publication_context = spark.sql( query); + org.apache.spark.sql.Dataset publication_context = spark.sql(query); publication_context.createOrReplaceTempView("publication_context"); - //( source, (mes, dh-ch-, ni), target ) - query = "select target , collect_set(co) " + - "from (select target, community_context " + - "from publication_context pc join publication p on " + - "p.id = pc.source) tmp " + - "lateral view explode (community_context) c as co " + - "group by target"; - - + // ( source, (mes, dh-ch-, ni), target ) + query = + "select target , collect_set(co) " + + "from (select target, community_context " + + "from publication_context pc join publication p on " + + "p.id = pc.source) tmp " + + "lateral view explode (community_context) c as co " + + "group by target"; org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); System.out.println(toupdatepublicationreresult.count()); - toupdatepublicationreresult.toJavaRDD() - .map(r -> { - TypedRow tp = new TypedRow(); - tp.setSourceId(r.getString(0)); - r.getList(1).stream().forEach(c -> tp.add((String)c)); - return tp; - }) + toupdatepublicationreresult + .toJavaRDD() + .map( + r -> { + TypedRow tp = new TypedRow(); + tp.setSourceId(r.getString(0)); + r.getList(1).stream().forEach(c -> tp.add((String) c)); + return tp; + }) .map(tr -> new ObjectMapper().writeValueAsString(tr)) .saveAsTextFile(outputPath + "/community2semrelonpublication"); -// toupdatepublicationreresult.toJavaRDD().flatMap(c -> { -// -// String source = c.getString(0); -// List relation_list = new ArrayList<>(); -// c.getList(1).stream() -// .forEach(res -> { -// Relation r = new Relation(); -// r.setSource(source); -// r.setTarget((String)res); -// r.setRelClass("produces"); -// relation_list.add(r); -// r = new Relation(); -// r.setSource((String)res); -// r.setTarget(source); -// r.setRelClass("isProducedBy"); -// relation_list.add(r); -// }); -// return relation_list.iterator(); -// }).map(tr -> new ObjectMapper().writeValueAsString(tr)) -// .saveAsTextFile(outputPath + "/community2semrel"); -// + // toupdatepublicationreresult.toJavaRDD().flatMap(c -> { + // + // String source = c.getString(0); + // List relation_list = new ArrayList<>(); + // c.getList(1).stream() + // .forEach(res -> { + // Relation r = new Relation(); + // r.setSource(source); + // r.setTarget((String)res); + // r.setRelClass("produces"); + // relation_list.add(r); + // r = new Relation(); + // r.setSource((String)res); + // r.setTarget(source); + // r.setRelClass("isProducedBy"); + // relation_list.add(r); + // }); + // return relation_list.iterator(); + // }).map(tr -> new ObjectMapper().writeValueAsString(tr)) + // .saveAsTextFile(outputPath + "/community2semrel"); + // -// org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); -// org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); -// org.apache.spark.sql.Dataset toupdatepublicationreresult = getUpdateCommunitiesForTable(spark, "publication"); -// org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + // org.apache.spark.sql.Dataset toupdatesoftwareresult = + // getUpdateCommunitiesForTable(spark, "software"); + // org.apache.spark.sql.Dataset toupdatedatasetresult = + // getUpdateCommunitiesForTable(spark, "dataset"); + // org.apache.spark.sql.Dataset toupdatepublicationreresult = + // getUpdateCommunitiesForTable(spark, "publication"); + // org.apache.spark.sql.Dataset toupdateotherresult = + // getUpdateCommunitiesForTable(spark, "other"); -// createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, + // "software_update", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // + // createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, + // "dataset_update", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), + // outputPath, "publication_update", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// -// updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// -// updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", -// PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// + // createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, + // "other_update", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // + // + // updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), + // outputPath, "dataset", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // + // updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), + // outputPath, "otherresearchproduct", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // + // updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), + // outputPath, "software", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // + // updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), + // publication.toJavaRDD(), outputPath, "publication", + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + // -/* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + /* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) .filter(p -> !(p == null)) .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla -*/ + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla + */ } - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ - String query = "SELECT target_id, collect_set(co.id) context_id " + - " FROM (SELECT t.id target_id, s.context source_context " + - " FROM context_software s " + - " JOIN " + table + " t " + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, d.context source_context " + - " FROM dataset_context d " + - " JOIN " + table + " t" + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, p.context source_context " + - " FROM publication_context p" + - " JOIN " + table +" t " + - " on p.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, o.context source_context " + - " FROM other_context o " + - " JOIN " + table + " t " + - " ON o.target = t.id) TMP " + - " LATERAL VIEW EXPLODE(source_context) MyT as co " + - " GROUP BY target_id" ; + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( + SparkSession spark, String table) { + String query = + "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + + table + + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + + table + + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + + table + + " t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + + table + + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id"; return spark.sql(query); } - private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return toupdateresult.map(r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - contextList.add(newContext); - } + private static JavaRDD createUpdateForResultDatasetWrite( + JavaRDD toupdateresult, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return toupdateresult + .map( + r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + contextList.add(newContext); + } + } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }).filter(r -> r != null); + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }) + .filter(r -> r != null); } - private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForSoftwareDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r-> (Dataset)r) + private static void updateForDatasetDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Dataset) r) .map(d -> new ObjectMapper().writeValueAsString(d)) .saveAsTextFile(outputPath + "/" + type); } - private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Publication)r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForPublicationDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Publication) r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r -> (OtherResearchProduct)r) - .map( o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForOtherDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (OtherResearchProduct) r) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); } + private static JavaRDD getUpdateForResultDataset( + JavaRDD toupdateresult, + JavaPairRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return result.leftOuterJoin( + toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map( + c -> { + if (!c._2()._2().isPresent()) { + return c._2()._1(); + } - - private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map(c -> { - if(! c._2()._2().isPresent()){ - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for(Object cId: toAddContext){ - String id = (String)cId; - if (communityIdList.contains(id)){ - context_set.add(id); - } - } - for (Context context: c._2()._1().getContext()){ - if(context_set.contains(context)){ - context_set.remove(context); - } - } - - List contextList = context_set.stream().map(co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - return newContext; - - }).collect(Collectors.toList()); - - if(contextList.size() > 0 ){ - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }).filter(r -> r != null); - - -// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) -// .join(result) -// .map(c -> { -// List toAddContext = c._2()._1(); -// Set context_set = new HashSet<>(); -// for(Object cId: toAddContext){ -// String id = (String)cId; -// if (communityIdList.contains(id)){ -// context_set.add(id); -// } -// } -// for (Context context: c._2()._2().getContext()){ -// if(context_set.contains(context)){ -// context_set.remove(context); -// } -// } -// -// List contextList = context_set.stream().map(co -> { -// Context newContext = new Context(); -// newContext.setId(co); -// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); -// return newContext; -// -// }).collect(Collectors.toList()); -// -// if(contextList.size() > 0 ){ -// Result r = new Result(); -// r.setId(c._1()); -// r.setContext(contextList); -// return r; -// } -// return null; -// }) -// .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, - JavaRDD result, String class_id, String class_name) { - return result - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map(c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())){ - if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); - //community id already in the context of the result. Remove it from the set that has to be added - contexts.remove(context.getId()); + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for (Object cId : toAddContext) { + String id = (String) cId; + if (communityIdList.contains(id)) { + context_set.add(id); + } + } + for (Context context : c._2()._1().getContext()) { + if (context_set.contains(context)) { + context_set.remove(context); } } - } - List cc = oaf.getContext(); - for(String cId: contexts){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - cc.add(context); - } - oaf.setContext(cc); + List contextList = + context_set.stream() + .map( + co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + return newContext; + }) + .collect(Collectors.toList()); + + if (contextList.size() > 0) { + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }) + .filter(r -> r != null); + + // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + // .join(result) + // .map(c -> { + // List toAddContext = c._2()._1(); + // Set context_set = new HashSet<>(); + // for(Object cId: toAddContext){ + // String id = (String)cId; + // if (communityIdList.contains(id)){ + // context_set.add(id); + // } + // } + // for (Context context: c._2()._2().getContext()){ + // if(context_set.contains(context)){ + // context_set.remove(context); + // } + // } + // + // List contextList = context_set.stream().map(co -> { + // Context newContext = new Context(); + // newContext.setId(co); + // + // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, + // class_name))); + // return newContext; + // + // }).collect(Collectors.toList()); + // + // if(contextList.size() > 0 ){ + // Result r = new Result(); + // r.setId(c._1()); + // r.setContext(contextList); + // return r; + // } + // return null; + // }) + // .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset( + JavaRDD toupdateresult, + List communityList, + JavaRDD result, + String class_id, + String class_name) { + return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) + .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map( + c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())) { + if (!context.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + context.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name)); + // community id already in the context of the result. + // Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for (String cId : contexts) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + cc.add(context); + } + oaf.setContext(cc); + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD( + JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair( + c -> { + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } } - return oaf; + + return new Tuple2<>(c.getString(0), contextList); }); } - private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair(c -> { - - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0) ,contextList); - }); - } - - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ - String query = "SELECT relation.source, " + table +".context , relation.target " + - "FROM " + table + - " JOIN relation " + - "ON id = source" ; + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { + String query = + "SELECT relation.source, " + + table + + ".context , relation.target " + + "FROM " + + table + + " JOIN relation " + + "ON id = source"; return spark.sql(query); } private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = r.getContext() - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + Set result_communities = + r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); for (String communityId : result_communities) { if (communityIdList.contains(communityId)) { return true; @@ -465,44 +611,61 @@ public class SparkResultToCommunityThroughSemRelJob2 { return false; } - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + private static void updateResult( + JavaPairRDD results, + JavaPairRDD toupdateresult, + String outputPath, + String type) { results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - Set communityList = p._2()._2().get().getAccumulator(); - for(Context c: r.getContext()){ - if (communityList.contains(c.getId())){ - //verify if the datainfo for this context contains propagation - if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - //community id already in the context of the result. Remove it from the set that has to be added - communityList.remove(c.getId()); + .map( + p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()) { + Set communityList = p._2()._2().get().getAccumulator(); + for (Context c : r.getContext()) { + if (communityList.contains(c.getId())) { + // verify if the datainfo for this context contains + // propagation + if (!c.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + c.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + // community id already in the context of the result. + // Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } } + List cc = r.getContext(); + for (String cId : communityList) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); } - } - List cc = r.getContext(); - for(String cId: communityList){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) + return r; + }) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); + .saveAsTextFile(outputPath + "/" + type); } - - - private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { - Set result_communities = context - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + private static TypedRow getTypedRow( + List communityIdList, List context, String id, String type) { + Set result_communities = + context.stream().map(c -> c.getId()).collect(Collectors.toSet()); TypedRow tp = new TypedRow(); tp.setSourceId(id); tp.setType(type); diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java index 0e39090dd..9c5e6c3b3 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java @@ -1,10 +1,14 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.TypedRow; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.SparkConf; @@ -16,28 +20,25 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import scala.Tuple2; -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; - public class SparkResultToCommunityThroughSemRelJob3 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + final ArgumentApplicationParser parser = + new ArgumentApplicationParser( + IOUtils.toString( + SparkResultToCommunityThroughSemRelJob3.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); parser.parseArgument(args); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); + final SparkSession spark = + SparkSession.builder() + .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) + .master(parser.get("master")) + .config(conf) + .enableHiveSupport() + .getOrCreate(); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); final String inputPath = parser.get("sourcePath"); @@ -45,42 +46,48 @@ public class SparkResultToCommunityThroughSemRelJob3 { final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + final List communityIdList = + QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); + JavaRDD publication_rdd = + sc.textFile(inputPath + "/publication") + .map(item -> new ObjectMapper().readValue(item, Publication.class)); - JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); + JavaRDD dataset_rdd = + sc.textFile(inputPath + "/dataset") + .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); + JavaRDD orp_rdd = + sc.textFile(inputPath + "/otherresearchproduct") + .map( + item -> + new ObjectMapper() + .readValue(item, OtherResearchProduct.class)); - JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); + JavaRDD software_rdd = + sc.textFile(inputPath + "/software") + .map(item -> new ObjectMapper().readValue(item, Software.class)); - JavaRDD software_rdd = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); + JavaRDD relation_rdd = + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)); - JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); + org.apache.spark.sql.Dataset publication = + spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); - org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), - Encoders.bean(Publication.class)); + org.apache.spark.sql.Dataset dataset = + spark.createDataset(dataset_rdd.rdd(), Encoders.bean(Dataset.class)); - org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), - Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), - Encoders.bean(Software.class)); + org.apache.spark.sql.Dataset other = + spark.createDataset(orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); + org.apache.spark.sql.Dataset software = + spark.createDataset(software_rdd.rdd(), Encoders.bean(Software.class)); publication.createOrReplaceTempView("publication"); relation.createOrReplaceTempView("relation"); @@ -90,327 +97,490 @@ public class SparkResultToCommunityThroughSemRelJob3 { String communitylist = getConstraintList(" co.id = '", communityIdList); - String semrellist = getConstraintList(" relClass = '", allowedsemrel ); + String semrellist = getConstraintList(" relClass = '", allowedsemrel); + String query = + "Select source, community_context, target " + + "from (select id, collect_set(co.id) community_context " + + "from publication " + + "lateral view explode (context) c as co " + + "where datainfo.deletedbyinference = false " + + communitylist + + " group by id) p " + + "JOIN " + + "(select * " + + "from relation " + + "where datainfo.deletedbyinference = false " + + semrellist + + ") r " + + "ON p.id = r.source"; - String query = "Select source, community_context, target " + - "from (select id, collect_set(co.id) community_context " + - "from publication " + - "lateral view explode (context) c as co " + - "where datainfo.deletedbyinference = false "+ communitylist + - " group by id) p " + - "JOIN " + - "(select * " + - "from relation " + - "where datainfo.deletedbyinference = false " + semrellist + ") r " + - "ON p.id = r.source"; - - - org.apache.spark.sql.Dataset publication_context = spark.sql( query); + org.apache.spark.sql.Dataset publication_context = spark.sql(query); publication_context.createOrReplaceTempView("publication_context"); - //( source, (mes, dh-ch-, ni), target ) - query = "select target , collect_set(co) " + - "from (select target, community_context " + - "from publication_context pc join publication p on " + - "p.id = pc.source) tmp " + - "lateral view explode (community_context) c as co " + - "group by target"; - - + // ( source, (mes, dh-ch-, ni), target ) + query = + "select target , collect_set(co) " + + "from (select target, community_context " + + "from publication_context pc join publication p on " + + "p.id = pc.source) tmp " + + "lateral view explode (community_context) c as co " + + "group by target"; org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); - org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); + org.apache.spark.sql.Dataset toupdatesoftwareresult = + getUpdateCommunitiesForTable(spark, "software"); + org.apache.spark.sql.Dataset toupdatedatasetresult = + getUpdateCommunitiesForTable(spark, "dataset"); + org.apache.spark.sql.Dataset toupdateotherresult = + getUpdateCommunitiesForTable(spark, "other"); - createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdatesoftwareresult.toJavaRDD(), + outputPath, + "software_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdatedatasetresult.toJavaRDD(), + outputPath, + "dataset_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdatepublicationreresult.toJavaRDD(), + outputPath, + "publication_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + createUpdateForResultDatasetWrite( + toupdateotherresult.toJavaRDD(), + outputPath, + "other_update", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForDatasetDataset( + toupdatedatasetresult.toJavaRDD(), + dataset.toJavaRDD(), + outputPath, + "dataset", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForOtherDataset( + toupdateotherresult.toJavaRDD(), + other.toJavaRDD(), + outputPath, + "otherresearchproduct", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); + updateForSoftwareDataset( + toupdatesoftwareresult.toJavaRDD(), + software.toJavaRDD(), + outputPath, + "software", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); - updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// + updateForPublicationDataset( + toupdatepublicationreresult.toJavaRDD(), + publication.toJavaRDD(), + outputPath, + "publication", + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + communityIdList); + // -/* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + /* + JavaPairRDD resultLinkedToCommunities = publication + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) .filter(p -> !(p == null)) .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); + .union(datasets + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(software + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ) + .union(other + .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) + .filter(p -> !(p == null)) + .mapToPair(toPair()) + ); - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); + JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) + .mapToPair(toPair()); - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); + JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla -*/ + updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); + //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] + //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla + */ } - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ - String query = "SELECT target_id, collect_set(co.id) context_id " + - " FROM (SELECT t.id target_id, s.context source_context " + - " FROM context_software s " + - " JOIN " + table + " t " + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, d.context source_context " + - " FROM dataset_context d " + - " JOIN " + table + " t" + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, p.context source_context " + - " FROM publication_context p" + - " JOIN " + table +" t " + - " on p.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, o.context source_context " + - " FROM other_context o " + - " JOIN " + table + " t " + - " ON o.target = t.id) TMP " + - " LATERAL VIEW EXPLODE(source_context) MyT as co " + - " GROUP BY target_id" ; + private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( + SparkSession spark, String table) { + String query = + "SELECT target_id, collect_set(co.id) context_id " + + " FROM (SELECT t.id target_id, s.context source_context " + + " FROM context_software s " + + " JOIN " + + table + + " t " + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, d.context source_context " + + " FROM dataset_context d " + + " JOIN " + + table + + " t" + + " ON s.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, p.context source_context " + + " FROM publication_context p" + + " JOIN " + + table + + " t " + + " on p.target = t.id " + + " UNION ALL " + + " SELECT t.id target_id, o.context source_context " + + " FROM other_context o " + + " JOIN " + + table + + " t " + + " ON o.target = t.id) TMP " + + " LATERAL VIEW EXPLODE(source_context) MyT as co " + + " GROUP BY target_id"; return spark.sql(query); } - private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return toupdateresult.map(r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - contextList.add(newContext); - } + private static JavaRDD createUpdateForResultDatasetWrite( + JavaRDD toupdateresult, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return toupdateresult + .map( + r -> { + List contextList = new ArrayList(); + List toAddContext = r.getList(1); + for (String cId : toAddContext) { + if (communityIdList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + contextList.add(newContext); + } + } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }).filter(r -> r != null); + if (contextList.size() > 0) { + Result ret = new Result(); + ret.setId(r.getString(0)); + ret.setContext(contextList); + return ret; + } + return null; + }) + .filter(r -> r != null); } - private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForSoftwareDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Software) r) + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r-> (Dataset)r) + private static void updateForDatasetDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Dataset) r) .map(d -> new ObjectMapper().writeValueAsString(d)) .saveAsTextFile(outputPath + "/" + type); } - private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Publication)r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForPublicationDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (Publication) r) + .map(p -> new ObjectMapper().writeValueAsString(p)) + .saveAsTextFile(outputPath + "/" + type); } - private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r -> (OtherResearchProduct)r) - .map( o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); + private static void updateForOtherDataset( + JavaRDD toupdateresult, + JavaRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); + getUpdateForResultDataset( + toupdateresult, + tmp, + outputPath, + type, + class_id, + class_name, + communityIdList) + .map(r -> (OtherResearchProduct) r) + .map(o -> new ObjectMapper().writeValueAsString(o)) + .saveAsTextFile(outputPath + "/" + type); } + private static JavaRDD getUpdateForResultDataset( + JavaRDD toupdateresult, + JavaPairRDD result, + String outputPath, + String type, + String class_id, + String class_name, + List communityIdList) { + return result.leftOuterJoin( + toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) + .map( + c -> { + if (!c._2()._2().isPresent()) { + return c._2()._1(); + } - - private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map(c -> { - if(! c._2()._2().isPresent()){ - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for(Object cId: toAddContext){ - String id = (String)cId; - if (communityIdList.contains(id)){ - context_set.add(id); - } - } - for (Context context: c._2()._1().getContext()){ - if(context_set.contains(context)){ - context_set.remove(context); - } - } - - List contextList = context_set.stream().map(co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - return newContext; - - }).collect(Collectors.toList()); - - if(contextList.size() > 0 ){ - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }).filter(r -> r != null); - - -// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) -// .join(result) -// .map(c -> { -// List toAddContext = c._2()._1(); -// Set context_set = new HashSet<>(); -// for(Object cId: toAddContext){ -// String id = (String)cId; -// if (communityIdList.contains(id)){ -// context_set.add(id); -// } -// } -// for (Context context: c._2()._2().getContext()){ -// if(context_set.contains(context)){ -// context_set.remove(context); -// } -// } -// -// List contextList = context_set.stream().map(co -> { -// Context newContext = new Context(); -// newContext.setId(co); -// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); -// return newContext; -// -// }).collect(Collectors.toList()); -// -// if(contextList.size() > 0 ){ -// Result r = new Result(); -// r.setId(c._1()); -// r.setContext(contextList); -// return r; -// } -// return null; -// }) -// .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, - JavaRDD result, String class_id, String class_name) { - return result - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map(c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())){ - if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); - //community id already in the context of the result. Remove it from the set that has to be added - contexts.remove(context.getId()); + List toAddContext = c._2()._2().get(); + Set context_set = new HashSet<>(); + for (Object cId : toAddContext) { + String id = (String) cId; + if (communityIdList.contains(id)) { + context_set.add(id); + } + } + for (Context context : c._2()._1().getContext()) { + if (context_set.contains(context)) { + context_set.remove(context); } } - } - List cc = oaf.getContext(); - for(String cId: contexts){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - cc.add(context); - } - oaf.setContext(cc); + List contextList = + context_set.stream() + .map( + co -> { + Context newContext = new Context(); + newContext.setId(co); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + return newContext; + }) + .collect(Collectors.toList()); + + if (contextList.size() > 0) { + Result r = new Result(); + r.setId(c._1()); + r.setContext(contextList); + return r; + } + return null; + }) + .filter(r -> r != null); + + // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) + // .join(result) + // .map(c -> { + // List toAddContext = c._2()._1(); + // Set context_set = new HashSet<>(); + // for(Object cId: toAddContext){ + // String id = (String)cId; + // if (communityIdList.contains(id)){ + // context_set.add(id); + // } + // } + // for (Context context: c._2()._2().getContext()){ + // if(context_set.contains(context)){ + // context_set.remove(context); + // } + // } + // + // List contextList = context_set.stream().map(co -> { + // Context newContext = new Context(); + // newContext.setId(co); + // + // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, + // class_name))); + // return newContext; + // + // }).collect(Collectors.toList()); + // + // if(contextList.size() > 0 ){ + // Result r = new Result(); + // r.setId(c._1()); + // r.setContext(contextList); + // return r; + // } + // return null; + // }) + // .filter(r -> r != null); + } + + private static JavaRDD createUpdateForSoftwareDataset( + JavaRDD toupdateresult, + List communityList, + JavaRDD result, + String class_id, + String class_name) { + return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) + .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) + .map( + c -> { + Software oaf = c._2()._1(); + if (c._2()._2().isPresent()) { + + HashSet contexts = new HashSet<>(c._2()._2().get()); + + for (Context context : oaf.getContext()) { + if (contexts.contains(context.getId())) { + if (!context.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + context.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name)); + // community id already in the context of the result. + // Remove it from the set that has to be added + contexts.remove(context.getId()); + } + } + } + List cc = oaf.getContext(); + for (String cId : contexts) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + class_id, + class_name))); + cc.add(context); + } + oaf.setContext(cc); + } + return oaf; + }); + } + + private static JavaPairRDD> getStringResultJavaPairRDD( + JavaRDD toupdateresult, List communityList) { + return toupdateresult.mapToPair( + c -> { + List contextList = new ArrayList<>(); + List contexts = c.getList(1); + for (String context : contexts) { + if (communityList.contains(context)) { + contextList.add(context); + } } - return oaf; + + return new Tuple2<>(c.getString(0), contextList); }); } - private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair(c -> { - - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0) ,contextList); - }); - } - - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ - String query = "SELECT relation.source, " + table +".context , relation.target " + - "FROM " + table + - " JOIN relation " + - "ON id = source" ; + private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { + String query = + "SELECT relation.source, " + + table + + ".context , relation.target " + + "FROM " + + table + + " JOIN relation " + + "ON id = source"; return spark.sql(query); } private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = r.getContext() - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + Set result_communities = + r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); for (String communityId : result_communities) { if (communityIdList.contains(communityId)) { return true; @@ -419,44 +589,61 @@ public class SparkResultToCommunityThroughSemRelJob3 { return false; } - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { + private static void updateResult( + JavaPairRDD results, + JavaPairRDD toupdateresult, + String outputPath, + String type) { results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - Set communityList = p._2()._2().get().getAccumulator(); - for(Context c: r.getContext()){ - if (communityList.contains(c.getId())){ - //verify if the datainfo for this context contains propagation - if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - //community id already in the context of the result. Remove it from the set that has to be added - communityList.remove(c.getId()); + .map( + p -> { + Result r = p._2()._1(); + if (p._2()._2().isPresent()) { + Set communityList = p._2()._2().get().getAccumulator(); + for (Context c : r.getContext()) { + if (communityList.contains(c.getId())) { + // verify if the datainfo for this context contains + // propagation + if (!c.getDataInfo().stream() + .map(di -> di.getInferenceprovenance()) + .collect(Collectors.toSet()) + .contains(PROPAGATION_DATA_INFO_TYPE)) { + c.getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); + // community id already in the context of the result. + // Remove it from the set that has to be added + communityList.remove(c.getId()); + } + } } + List cc = r.getContext(); + for (String cId : communityList) { + Context context = new Context(); + context.setId(cId); + context.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + cc.add(context); + } + r.setContext(cc); } - } - List cc = r.getContext(); - for(String cId: communityList){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) + return r; + }) .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); + .saveAsTextFile(outputPath + "/" + type); } - - - private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { - Set result_communities = context - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); + private static TypedRow getTypedRow( + List communityIdList, List context, String id, String type) { + Set result_communities = + context.stream().map(c -> c.getId()).collect(Collectors.toSet()); TypedRow tp = new TypedRow(); tp.setSourceId(id); tp.setType(type); From edb00db86a2ef1ee48de8e9e827977821ff36eb0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:57:35 +0200 Subject: [PATCH 161/259] refactoring --- .../OrganizationMap.java | 8 +- .../PrepareResultCommunitySet.java | 137 ++++++++++-------- ...ResultToCommunityFromOrganizationJob2.java | 121 ++++++++-------- 3 files changed, 144 insertions(+), 122 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java index 9e9c2030a..9a42f3f7e 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java @@ -6,15 +6,15 @@ import java.util.List; public class OrganizationMap extends HashMap> { - public OrganizationMap(){ + public OrganizationMap() { super(); } - public List get(String key){ + public List get(String key) { - if (super.get(key) == null){ + if (super.get(key) == null) { return new ArrayList<>(); } return super.get(key); } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index bd2d106fe..9e62f9b4f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -1,20 +1,19 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; +import java.util.*; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class PrepareResultCommunitySet { private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); @@ -22,11 +21,12 @@ public class PrepareResultCommunitySet { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySet.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareResultCommunitySet.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -39,76 +39,91 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final OrganizationMap organizationMap = new Gson().fromJson(parser.get("organizationtoresultcommunitymap"), OrganizationMap.class); + final OrganizationMap organizationMap = + new Gson() + .fromJson( + parser.get("organizationtoresultcommunitymap"), + OrganizationMap.class); log.info("organizationMap: {}", new Gson().toJson(organizationMap)); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - prepareInfo(spark, inputPath, outputPath, organizationMap); + prepareInfo(spark, inputPath, outputPath, organizationMap); }); } - private static void prepareInfo(SparkSession spark, String inputPath, String outputPath, OrganizationMap organizationMap) { + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + OrganizationMap organizationMap) { Dataset relation = readRelations(spark, inputPath); relation.createOrReplaceTempView("relation"); - String query = "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " + - "FROM (SELECT source, target " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS + "') result_organization " + - "LEFT JOIN (SELECT source, collect_set(target) org_set " + - " FROM relation " + - " WHERE datainfo.deletedbyinference = false " + - " AND relClass = '" + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + "' " + - " GROUP BY source) organization_organization " + - "ON result_organization.target = organization_organization.source "; + String query = + "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " + + "FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_RESULT_ORGANIZATION_REL_CLASS + + "') result_organization " + + "LEFT JOIN (SELECT source, collect_set(target) org_set " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + + "' " + + " GROUP BY source) organization_organization " + + "ON result_organization.target = organization_organization.source "; - org.apache.spark.sql.Dataset result_organizationset = spark.sql(query) - .as(Encoders.bean(ResultOrganizations.class)); + org.apache.spark.sql.Dataset result_organizationset = + spark.sql(query).as(Encoders.bean(ResultOrganizations.class)); result_organizationset - .map(value -> { - String rId = value.getResultId(); - List orgs = value.getMerges(); - String oTarget = value.getOrgId(); - Set communitySet = new HashSet<>(); - if (organizationMap.containsKey(oTarget)) { - communitySet.addAll(organizationMap.get(oTarget)); - } - try{ - for (String oId : orgs) { - if (organizationMap.containsKey(oId)) { - communitySet.addAll(organizationMap.get(oId)); + .map( + value -> { + String rId = value.getResultId(); + Optional> orgs = Optional.ofNullable(value.getMerges()); + String oTarget = value.getOrgId(); + Set communitySet = new HashSet<>(); + if (organizationMap.containsKey(oTarget)) { + communitySet.addAll(organizationMap.get(oTarget)); } - } - }catch(Exception e){ - - } - if (communitySet.size() > 0){ - ResultCommunityList rcl = new ResultCommunityList(); - rcl.setResultId(rId); - ArrayList communityList = new ArrayList<>(); - communityList.addAll(communitySet); - rcl.setCommunityList(communityList); - return rcl; - } - return null; - }, Encoders.bean(ResultCommunityList.class)) - .filter(r -> r!= null) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); + if (orgs.isPresent()) + // try{ + for (String oId : orgs.get()) { + if (organizationMap.containsKey(oId)) { + communitySet.addAll(organizationMap.get(oId)); + } + } + // }catch(Exception e){ + // + // } + if (communitySet.size() > 0) { + ResultCommunityList rcl = new ResultCommunityList(); + rcl.setResultId(rId); + ArrayList communityList = new ArrayList<>(); + communityList.addAll(communitySet); + rcl.setCommunityList(communityList); + return rcl; + } + return null; + }, + Encoders.bean(ResultCommunityList.class)) + .filter(r -> r != null) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); } - - - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index c19c24d28..3c5b0a04c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -1,8 +1,13 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Encoders; @@ -10,25 +15,21 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; -import java.util.stream.Collectors; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; public class SparkResultToCommunityFromOrganizationJob2 { - private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); + private static final Logger log = + LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkResultToCommunityFromOrganizationJob2.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkResultToCommunityFromOrganizationJob2.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -47,73 +48,79 @@ public class SparkResultToCommunityFromOrganizationJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("saveGraph: {}", saveGraph); - Class resultClazz = (Class) Class.forName(resultClassName); + Class resultClazz = + (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); }); - } - private static void execPropagation(SparkSession spark, String inputPath, String outputPath, - Class resultClazz, String possibleUpdatesPath) { - org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList(spark, possibleUpdatesPath); + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + Class resultClazz, + String possibleUpdatesPath) { + org.apache.spark.sql.Dataset possibleUpdates = + readResultCommunityList(spark, possibleUpdatesPath); org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - - result - .joinWith(possibleUpdates, result.col("id").equalTo(possibleUpdates.col("resultId")), + result.joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), "left_outer") - .map(value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if(rcl.isPresent()){ - ArrayList communitySet = rcl.get().getCommunityList(); - List contextList = ret.getContext().stream().map(con -> con.getId()).collect(Collectors.toList()); - Result res = new Result(); - res.setId(ret.getId()); - List propagatedContexts = new ArrayList<>(); - for(String cId:communitySet){ - if(!contextList.contains(cId)){ - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - propagatedContexts.add(newContext); + .map( + value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + ArrayList communitySet = rcl.get().getCommunityList(); + List contextList = + ret.getContext().stream() + .map(con -> con.getId()) + .collect(Collectors.toList()); + Result res = new Result(); + res.setId(ret.getId()); + List propagatedContexts = new ArrayList<>(); + for (String cId : communitySet) { + if (!contextList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); + propagatedContexts.add(newContext); + } + } + res.setContext(propagatedContexts); + ret.mergeFrom(res); } - } - res.setContext(propagatedContexts); - ret.mergeFrom(res); - } - return ret; - }, Encoders.bean(resultClazz)) + return ret; + }, + Encoders.bean(resultClazz)) .toJSON() .write() .mode(SaveMode.Overwrite) - .option("compression","gzip") + .option("compression", "gzip") .text(outputPath); - } - - private static org.apache.spark.sql.Dataset readResultCommunityList(SparkSession spark, String possibleUpdatesPath) { - return spark - .read() - .textFile(possibleUpdatesPath) - .map(value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), Encoders.bean(ResultCommunityList.class)); - } - } From 0e447add6634666e3d4d116e7940a3a9c5cb721c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 23 Apr 2020 12:59:43 +0200 Subject: [PATCH 162/259] removed unuseful classes --- ...parkResultToCommunityThroughSemRelJob.java | 690 ------------------ ...arkResultToCommunityThroughSemRelJob2.java | 682 ----------------- ...arkResultToCommunityThroughSemRelJob3.java | 660 ----------------- 3 files changed, 2032 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java deleted file mode 100644 index cc90feac2..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java +++ /dev/null @@ -1,690 +0,0 @@ -package eu.dnetlib.dhp.resulttocommunityfromsemrel; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.QueryInformationSystem; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import java.util.*; -import java.util.stream.Collectors; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -public class SparkResultToCommunityThroughSemRelJob { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkResultToCommunityThroughSemRelJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); - parser.parseArgument(args); - - for (String key : parser.getObjectMap().keySet()) { - System.out.println(key + " = " + parser.get(key)); - } - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = - SparkSession.builder() - .appName(SparkResultToCommunityThroughSemRelJob.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; - - // final List allowedsemrel = - // Arrays.asList(parser.get("allowedsemrels").split(";")); - final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); - // final List communityIdList = - // QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - final List communityIdList = - QueryInformationSystem.getCommunityList( - "http://beta.services.openaire.eu:8280/is/services/isLookUp"); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD all_publication_rdd = - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .cache(); - JavaRDD publication_rdd = - all_publication_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD all_dataset_rdd = - sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .cache(); - JavaRDD dataset_rdd = - all_dataset_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD all_orp_rdd = - sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .cache(); - JavaRDD orp_rdd = - all_orp_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD all_software_rdd = - sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)) - .filter(p -> !p.getDataInfo().getDeletedbyinference()) - .cache(); - JavaRDD software_rdd = - all_software_rdd.filter(p -> relatedToCommunities(p, communityIdList)).cache(); - - JavaRDD relation_rdd = - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter( - r -> - allowedsemrel.contains(r.getRelClass()) - && RELATION_RESULTRESULT_REL_TYPE.equals( - r.getRelType())) - .cache(); - - org.apache.spark.sql.Dataset publication = - spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); - - org.apache.spark.sql.Dataset dataset = - spark.createDataset(dataset_rdd.rdd(), Encoders.bean(Dataset.class)); - - org.apache.spark.sql.Dataset other = - spark.createDataset(orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = - spark.createDataset(software_rdd.rdd(), Encoders.bean(Software.class)); - - org.apache.spark.sql.Dataset relation = - spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); - - publication.createOrReplaceTempView("publication"); - relation.createOrReplaceTempView("relation"); - dataset.createOrReplaceTempView("dataset"); - software.createOrReplaceTempView("software"); - other.createOrReplaceTempView("other"); - - // org.apache.spark.sql.Dataset publication_context = getContext(spark, - // "publication"); - // publication_context.createOrReplaceTempView("publication_context"); - - org.apache.spark.sql.Dataset publication_context = - spark.sql( - "SELECT relation.source, " - + "publication.context , relation.target " - + "FROM publication " - + " JOIN relation " - + "ON id = source"); - - org.apache.spark.sql.Dataset dataset_context = getContext(spark, "dataset"); - dataset_context.createOrReplaceTempView("dataset_context"); - - org.apache.spark.sql.Dataset software_context = getContext(spark, "software"); - software_context.createOrReplaceTempView("software_context"); - - org.apache.spark.sql.Dataset other_context = getContext(spark, "other"); - other_context.createOrReplaceTempView("other_context"); - - publication = - spark.createDataset(all_publication_rdd.rdd(), Encoders.bean(Publication.class)); - publication.createOrReplaceTempView("publication"); - - dataset = spark.createDataset(all_dataset_rdd.rdd(), Encoders.bean(Dataset.class)); - dataset.createOrReplaceTempView("dataset"); - - other = spark.createDataset(all_orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); - other.createOrReplaceTempView("other"); - - software = spark.createDataset(all_software_rdd.rdd(), Encoders.bean(Software.class)); - software.createOrReplaceTempView("software"); - - org.apache.spark.sql.Dataset toupdatesoftwareresult = - getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = - getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdatepublicationreresult = - getUpdateCommunitiesForTable(spark, "publication"); - org.apache.spark.sql.Dataset toupdateotherresult = - getUpdateCommunitiesForTable(spark, "other"); - - createUpdateForResultDatasetWrite( - toupdatesoftwareresult.toJavaRDD(), - outputPath, - "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdatedatasetresult.toJavaRDD(), - outputPath, - "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdatepublicationreresult.toJavaRDD(), - outputPath, - "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdateotherresult.toJavaRDD(), - outputPath, - "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForDatasetDataset( - toupdatedatasetresult.toJavaRDD(), - dataset.toJavaRDD(), - outputPath, - "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForOtherDataset( - toupdateotherresult.toJavaRDD(), - other.toJavaRDD(), - outputPath, - "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForSoftwareDataset( - toupdatesoftwareresult.toJavaRDD(), - software.toJavaRDD(), - outputPath, - "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForPublicationDataset( - toupdatepublicationreresult.toJavaRDD(), - publication.toJavaRDD(), - outputPath, - "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - /* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); - - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla - */ - } - - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( - SparkSession spark, String table) { - String query = - "SELECT target_id, collect_set(co.id) context_id " - + " FROM (SELECT t.id target_id, s.context source_context " - + " FROM context_software s " - + " JOIN " - + table - + " t " - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, d.context source_context " - + " FROM dataset_context d " - + " JOIN " - + table - + " t" - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, p.context source_context " - + " FROM publication_context p" - + " JOIN " - + table - + " t " - + " on p.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, o.context source_context " - + " FROM other_context o " - + " JOIN " - + table - + " t " - + " ON o.target = t.id) TMP " - + " LATERAL VIEW EXPLODE(source_context) MyT as co " - + " GROUP BY target_id"; - - return spark.sql(query); - } - - private static JavaRDD createUpdateForResultDatasetWrite( - JavaRDD toupdateresult, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return toupdateresult - .map( - r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - contextList.add(newContext); - } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }) - .filter(r -> r != null); - } - - private static void updateForSoftwareDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForDatasetDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Dataset) r) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForPublicationDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Publication) r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForOtherDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (OtherResearchProduct) r) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static JavaRDD getUpdateForResultDataset( - JavaRDD toupdateresult, - JavaPairRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return result.leftOuterJoin( - toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map( - c -> { - if (!c._2()._2().isPresent()) { - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for (Object cId : toAddContext) { - String id = (String) cId; - if (communityIdList.contains(id)) { - context_set.add(id); - } - } - for (Context context : c._2()._1().getContext()) { - if (context_set.contains(context)) { - context_set.remove(context); - } - } - - List contextList = - context_set.stream() - .map( - co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - return newContext; - }) - .collect(Collectors.toList()); - - if (contextList.size() > 0) { - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }) - .filter(r -> r != null); - - // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - // .join(result) - // .map(c -> { - // List toAddContext = c._2()._1(); - // Set context_set = new HashSet<>(); - // for(Object cId: toAddContext){ - // String id = (String)cId; - // if (communityIdList.contains(id)){ - // context_set.add(id); - // } - // } - // for (Context context: c._2()._2().getContext()){ - // if(context_set.contains(context)){ - // context_set.remove(context); - // } - // } - // - // List contextList = context_set.stream().map(co -> { - // Context newContext = new Context(); - // newContext.setId(co); - // - // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, - // class_name))); - // return newContext; - // - // }).collect(Collectors.toList()); - // - // if(contextList.size() > 0 ){ - // Result r = new Result(); - // r.setId(c._1()); - // r.setContext(contextList); - // return r; - // } - // return null; - // }) - // .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset( - JavaRDD toupdateresult, - List communityList, - JavaRDD result, - String class_id, - String class_name) { - return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) - .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map( - c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())) { - if (!context.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - context.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name)); - // community id already in the context of the result. - // Remove it from the set that has to be added - contexts.remove(context.getId()); - } - } - } - List cc = oaf.getContext(); - for (String cId : contexts) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - cc.add(context); - } - oaf.setContext(cc); - } - return oaf; - }); - } - - private static JavaPairRDD> getStringResultJavaPairRDD( - JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair( - c -> { - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0), contextList); - }); - } - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { - String query = - "SELECT relation.source, " - + table - + ".context , relation.target " - + "FROM " - + table - + " JOIN relation " - + "ON id = source"; - - return spark.sql(query); - } - - private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = - r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - return true; - } - } - return false; - } - - private static void updateResult( - JavaPairRDD results, - JavaPairRDD toupdateresult, - String outputPath, - String type) { - results.leftOuterJoin(toupdateresult) - .map( - p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()) { - Set communityList = p._2()._2().get().getAccumulator(); - for (Context c : r.getContext()) { - if (communityList.contains(c.getId())) { - // verify if the datainfo for this context contains - // propagation - if (!c.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - c.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - // community id already in the context of the result. - // Remove it from the set that has to be added - communityList.remove(c.getId()); - } - } - } - List cc = r.getContext(); - for (String cId : communityList) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static TypedRow getTypedRow( - List communityIdList, List context, String id, String type) { - Set result_communities = - context.stream().map(c -> c.getId()).collect(Collectors.toSet()); - TypedRow tp = new TypedRow(); - tp.setSourceId(id); - tp.setType(type); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - tp.add(communityId); - } - } - if (tp.getAccumulator() != null) { - return tp; - } - return null; - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java deleted file mode 100644 index 3a0904404..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob2.java +++ /dev/null @@ -1,682 +0,0 @@ -package eu.dnetlib.dhp.resulttocommunityfromsemrel; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.QueryInformationSystem; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import java.util.*; -import java.util.stream.Collectors; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -public class SparkResultToCommunityThroughSemRelJob2 { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkResultToCommunityThroughSemRelJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); - parser.parseArgument(args); - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = - SparkSession.builder() - .appName(SparkResultToCommunityThroughSemRelJob2.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - // final List allowedsemrel = Arrays.asList("isSupplementedBy", "isSupplementTo"); - final List communityIdList = - QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - // final List communityIdList = - // QueryInformationSystem.getCommunityList("http://beta.services.openaire.eu:8280/is/services/isLookUp"); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD publication_rdd = - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - - System.out.println(publication_rdd.count()); - // JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") - // .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - // - // JavaRDD orp_rdd = sc.textFile(inputPath + - // "/otherresearchproduct") - // .map(item -> new ObjectMapper().readValue(item, - // OtherResearchProduct.class)); - // - // JavaRDD software_rdd = sc.textFile(inputPath + "/software") - // .map(item -> new ObjectMapper().readValue(item, Software.class)); - - JavaRDD relation_rdd = - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); - - System.out.println(relation_rdd.count()); - - // .filter(r -> !r.getDataInfo().getDeletedbyinference()) - // .filter(r -> allowedsemrel.contains(r.getRelClass()) && - // RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())).cache(); - - org.apache.spark.sql.Dataset publication = - spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); - - org.apache.spark.sql.Dataset relation = - spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); - - // org.apache.spark.sql.Dataset dataset = - // spark.createDataset(dataset_rdd.rdd(), - // Encoders.bean(Dataset.class)); - // - // org.apache.spark.sql.Dataset other = - // spark.createDataset(orp_rdd.rdd(), - // Encoders.bean(OtherResearchProduct.class)); - // - // org.apache.spark.sql.Dataset software = - // spark.createDataset(software_rdd.rdd(), - // Encoders.bean(Software.class)); - // - // org.apache.spark.sql.Dataset relation = - // spark.createDataset(relation_rdd.rdd(), - // Encoders.bean(Relation.class)); - - publication.createOrReplaceTempView("publication"); - relation.createOrReplaceTempView("relation"); - // relation.createOrReplaceTempView("relation"); - // dataset.createOrReplaceTempView("dataset"); - // software.createOrReplaceTempView("software"); - // other.createOrReplaceTempView("other"); - - String communitylist = getConstraintList(" co.id = '", communityIdList); - - String semrellist = getConstraintList(" relClass = '", allowedsemrel); - - String query = - "Select source, community_context, target " - + "from (select id, collect_set(co.id) community_context " - + "from publication " - + "lateral view explode (context) c as co " - + "where datainfo.deletedbyinference = false " - + communitylist - + " group by id) p " - + "JOIN " - + "(select * " - + "from relation " - + "where datainfo.deletedbyinference = false and (relClass = 'isSupplementedBy' OR relClass = 'isSupplementTo')) r " - + "ON p.id = r.source"; - - org.apache.spark.sql.Dataset publication_context = spark.sql(query); - publication_context.createOrReplaceTempView("publication_context"); - - // ( source, (mes, dh-ch-, ni), target ) - query = - "select target , collect_set(co) " - + "from (select target, community_context " - + "from publication_context pc join publication p on " - + "p.id = pc.source) tmp " - + "lateral view explode (community_context) c as co " - + "group by target"; - - org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); - - System.out.println(toupdatepublicationreresult.count()); - - toupdatepublicationreresult - .toJavaRDD() - .map( - r -> { - TypedRow tp = new TypedRow(); - tp.setSourceId(r.getString(0)); - r.getList(1).stream().forEach(c -> tp.add((String) c)); - return tp; - }) - .map(tr -> new ObjectMapper().writeValueAsString(tr)) - .saveAsTextFile(outputPath + "/community2semrelonpublication"); - // toupdatepublicationreresult.toJavaRDD().flatMap(c -> { - // - // String source = c.getString(0); - // List relation_list = new ArrayList<>(); - // c.getList(1).stream() - // .forEach(res -> { - // Relation r = new Relation(); - // r.setSource(source); - // r.setTarget((String)res); - // r.setRelClass("produces"); - // relation_list.add(r); - // r = new Relation(); - // r.setSource((String)res); - // r.setTarget(source); - // r.setRelClass("isProducedBy"); - // relation_list.add(r); - // }); - // return relation_list.iterator(); - // }).map(tr -> new ObjectMapper().writeValueAsString(tr)) - // .saveAsTextFile(outputPath + "/community2semrel"); - // - - // org.apache.spark.sql.Dataset toupdatesoftwareresult = - // getUpdateCommunitiesForTable(spark, "software"); - // org.apache.spark.sql.Dataset toupdatedatasetresult = - // getUpdateCommunitiesForTable(spark, "dataset"); - // org.apache.spark.sql.Dataset toupdatepublicationreresult = - // getUpdateCommunitiesForTable(spark, "publication"); - // org.apache.spark.sql.Dataset toupdateotherresult = - // getUpdateCommunitiesForTable(spark, "other"); - - // createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, - // "software_update", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - // createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, - // "dataset_update", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - // createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), - // outputPath, "publication_update", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - // createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, - // "other_update", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - // - // updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), - // outputPath, "dataset", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - // updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), - // outputPath, "otherresearchproduct", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - // updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), - // outputPath, "software", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - // updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), - // publication.toJavaRDD(), outputPath, "publication", - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - // PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - // - - /* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); - - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla - */ - } - - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( - SparkSession spark, String table) { - String query = - "SELECT target_id, collect_set(co.id) context_id " - + " FROM (SELECT t.id target_id, s.context source_context " - + " FROM context_software s " - + " JOIN " - + table - + " t " - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, d.context source_context " - + " FROM dataset_context d " - + " JOIN " - + table - + " t" - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, p.context source_context " - + " FROM publication_context p" - + " JOIN " - + table - + " t " - + " on p.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, o.context source_context " - + " FROM other_context o " - + " JOIN " - + table - + " t " - + " ON o.target = t.id) TMP " - + " LATERAL VIEW EXPLODE(source_context) MyT as co " - + " GROUP BY target_id"; - - return spark.sql(query); - } - - private static JavaRDD createUpdateForResultDatasetWrite( - JavaRDD toupdateresult, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return toupdateresult - .map( - r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - contextList.add(newContext); - } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }) - .filter(r -> r != null); - } - - private static void updateForSoftwareDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForDatasetDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Dataset) r) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForPublicationDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Publication) r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForOtherDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (OtherResearchProduct) r) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static JavaRDD getUpdateForResultDataset( - JavaRDD toupdateresult, - JavaPairRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return result.leftOuterJoin( - toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map( - c -> { - if (!c._2()._2().isPresent()) { - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for (Object cId : toAddContext) { - String id = (String) cId; - if (communityIdList.contains(id)) { - context_set.add(id); - } - } - for (Context context : c._2()._1().getContext()) { - if (context_set.contains(context)) { - context_set.remove(context); - } - } - - List contextList = - context_set.stream() - .map( - co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - return newContext; - }) - .collect(Collectors.toList()); - - if (contextList.size() > 0) { - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }) - .filter(r -> r != null); - - // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - // .join(result) - // .map(c -> { - // List toAddContext = c._2()._1(); - // Set context_set = new HashSet<>(); - // for(Object cId: toAddContext){ - // String id = (String)cId; - // if (communityIdList.contains(id)){ - // context_set.add(id); - // } - // } - // for (Context context: c._2()._2().getContext()){ - // if(context_set.contains(context)){ - // context_set.remove(context); - // } - // } - // - // List contextList = context_set.stream().map(co -> { - // Context newContext = new Context(); - // newContext.setId(co); - // - // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, - // class_name))); - // return newContext; - // - // }).collect(Collectors.toList()); - // - // if(contextList.size() > 0 ){ - // Result r = new Result(); - // r.setId(c._1()); - // r.setContext(contextList); - // return r; - // } - // return null; - // }) - // .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset( - JavaRDD toupdateresult, - List communityList, - JavaRDD result, - String class_id, - String class_name) { - return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) - .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map( - c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())) { - if (!context.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - context.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name)); - // community id already in the context of the result. - // Remove it from the set that has to be added - contexts.remove(context.getId()); - } - } - } - List cc = oaf.getContext(); - for (String cId : contexts) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - cc.add(context); - } - oaf.setContext(cc); - } - return oaf; - }); - } - - private static JavaPairRDD> getStringResultJavaPairRDD( - JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair( - c -> { - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0), contextList); - }); - } - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { - String query = - "SELECT relation.source, " - + table - + ".context , relation.target " - + "FROM " - + table - + " JOIN relation " - + "ON id = source"; - - return spark.sql(query); - } - - private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = - r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - return true; - } - } - return false; - } - - private static void updateResult( - JavaPairRDD results, - JavaPairRDD toupdateresult, - String outputPath, - String type) { - results.leftOuterJoin(toupdateresult) - .map( - p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()) { - Set communityList = p._2()._2().get().getAccumulator(); - for (Context c : r.getContext()) { - if (communityList.contains(c.getId())) { - // verify if the datainfo for this context contains - // propagation - if (!c.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - c.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - // community id already in the context of the result. - // Remove it from the set that has to be added - communityList.remove(c.getId()); - } - } - } - List cc = r.getContext(); - for (String cId : communityList) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static TypedRow getTypedRow( - List communityIdList, List context, String id, String type) { - Set result_communities = - context.stream().map(c -> c.getId()).collect(Collectors.toSet()); - TypedRow tp = new TypedRow(); - tp.setSourceId(id); - tp.setType(type); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - tp.add(communityId); - } - } - if (tp.getAccumulator() != null) { - return tp; - } - return null; - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java deleted file mode 100644 index 9c5e6c3b3..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob3.java +++ /dev/null @@ -1,660 +0,0 @@ -package eu.dnetlib.dhp.resulttocommunityfromsemrel; - -import static eu.dnetlib.dhp.PropagationConstant.*; - -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.QueryInformationSystem; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import java.util.*; -import java.util.stream.Collectors; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; - -public class SparkResultToCommunityThroughSemRelJob3 { - public static void main(String[] args) throws Exception { - - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkResultToCommunityThroughSemRelJob3.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json"))); - parser.parseArgument(args); - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = - SparkSession.builder() - .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; - - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - - final List communityIdList = - QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); - - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - JavaRDD publication_rdd = - sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - - JavaRDD dataset_rdd = - sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - - JavaRDD orp_rdd = - sc.textFile(inputPath + "/otherresearchproduct") - .map( - item -> - new ObjectMapper() - .readValue(item, OtherResearchProduct.class)); - - JavaRDD software_rdd = - sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - - JavaRDD relation_rdd = - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); - - org.apache.spark.sql.Dataset publication = - spark.createDataset(publication_rdd.rdd(), Encoders.bean(Publication.class)); - - org.apache.spark.sql.Dataset relation = - spark.createDataset(relation_rdd.rdd(), Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset dataset = - spark.createDataset(dataset_rdd.rdd(), Encoders.bean(Dataset.class)); - - org.apache.spark.sql.Dataset other = - spark.createDataset(orp_rdd.rdd(), Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = - spark.createDataset(software_rdd.rdd(), Encoders.bean(Software.class)); - - publication.createOrReplaceTempView("publication"); - relation.createOrReplaceTempView("relation"); - dataset.createOrReplaceTempView("dataset"); - software.createOrReplaceTempView("software"); - other.createOrReplaceTempView("other"); - - String communitylist = getConstraintList(" co.id = '", communityIdList); - - String semrellist = getConstraintList(" relClass = '", allowedsemrel); - - String query = - "Select source, community_context, target " - + "from (select id, collect_set(co.id) community_context " - + "from publication " - + "lateral view explode (context) c as co " - + "where datainfo.deletedbyinference = false " - + communitylist - + " group by id) p " - + "JOIN " - + "(select * " - + "from relation " - + "where datainfo.deletedbyinference = false " - + semrellist - + ") r " - + "ON p.id = r.source"; - - org.apache.spark.sql.Dataset publication_context = spark.sql(query); - publication_context.createOrReplaceTempView("publication_context"); - - // ( source, (mes, dh-ch-, ni), target ) - query = - "select target , collect_set(co) " - + "from (select target, community_context " - + "from publication_context pc join publication p on " - + "p.id = pc.source) tmp " - + "lateral view explode (community_context) c as co " - + "group by target"; - - org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); - org.apache.spark.sql.Dataset toupdatesoftwareresult = - getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = - getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdateotherresult = - getUpdateCommunitiesForTable(spark, "other"); - - createUpdateForResultDatasetWrite( - toupdatesoftwareresult.toJavaRDD(), - outputPath, - "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdatedatasetresult.toJavaRDD(), - outputPath, - "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdatepublicationreresult.toJavaRDD(), - outputPath, - "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - createUpdateForResultDatasetWrite( - toupdateotherresult.toJavaRDD(), - outputPath, - "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForDatasetDataset( - toupdatedatasetresult.toJavaRDD(), - dataset.toJavaRDD(), - outputPath, - "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForOtherDataset( - toupdateotherresult.toJavaRDD(), - other.toJavaRDD(), - outputPath, - "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForSoftwareDataset( - toupdatesoftwareresult.toJavaRDD(), - software.toJavaRDD(), - outputPath, - "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - - updateForPublicationDataset( - toupdatepublicationreresult.toJavaRDD(), - publication.toJavaRDD(), - outputPath, - "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - communityIdList); - // - - /* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); - - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla - */ - } - - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable( - SparkSession spark, String table) { - String query = - "SELECT target_id, collect_set(co.id) context_id " - + " FROM (SELECT t.id target_id, s.context source_context " - + " FROM context_software s " - + " JOIN " - + table - + " t " - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, d.context source_context " - + " FROM dataset_context d " - + " JOIN " - + table - + " t" - + " ON s.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, p.context source_context " - + " FROM publication_context p" - + " JOIN " - + table - + " t " - + " on p.target = t.id " - + " UNION ALL " - + " SELECT t.id target_id, o.context source_context " - + " FROM other_context o " - + " JOIN " - + table - + " t " - + " ON o.target = t.id) TMP " - + " LATERAL VIEW EXPLODE(source_context) MyT as co " - + " GROUP BY target_id"; - - return spark.sql(query); - } - - private static JavaRDD createUpdateForResultDatasetWrite( - JavaRDD toupdateresult, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return toupdateresult - .map( - r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - contextList.add(newContext); - } - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }) - .filter(r -> r != null); - } - - private static void updateForSoftwareDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForDatasetDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Dataset) r) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForPublicationDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (Publication) r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForOtherDataset( - JavaRDD toupdateresult, - JavaRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset( - toupdateresult, - tmp, - outputPath, - type, - class_id, - class_name, - communityIdList) - .map(r -> (OtherResearchProduct) r) - .map(o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static JavaRDD getUpdateForResultDataset( - JavaRDD toupdateresult, - JavaPairRDD result, - String outputPath, - String type, - String class_id, - String class_name, - List communityIdList) { - return result.leftOuterJoin( - toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map( - c -> { - if (!c._2()._2().isPresent()) { - return c._2()._1(); - } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for (Object cId : toAddContext) { - String id = (String) cId; - if (communityIdList.contains(id)) { - context_set.add(id); - } - } - for (Context context : c._2()._1().getContext()) { - if (context_set.contains(context)) { - context_set.remove(context); - } - } - - List contextList = - context_set.stream() - .map( - co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - return newContext; - }) - .collect(Collectors.toList()); - - if (contextList.size() > 0) { - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }) - .filter(r -> r != null); - - // return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) - // .join(result) - // .map(c -> { - // List toAddContext = c._2()._1(); - // Set context_set = new HashSet<>(); - // for(Object cId: toAddContext){ - // String id = (String)cId; - // if (communityIdList.contains(id)){ - // context_set.add(id); - // } - // } - // for (Context context: c._2()._2().getContext()){ - // if(context_set.contains(context)){ - // context_set.remove(context); - // } - // } - // - // List contextList = context_set.stream().map(co -> { - // Context newContext = new Context(); - // newContext.setId(co); - // - // newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, - // class_name))); - // return newContext; - // - // }).collect(Collectors.toList()); - // - // if(contextList.size() > 0 ){ - // Result r = new Result(); - // r.setId(c._1()); - // r.setContext(contextList); - // return r; - // } - // return null; - // }) - // .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset( - JavaRDD toupdateresult, - List communityList, - JavaRDD result, - String class_id, - String class_name) { - return result.mapToPair(s -> new Tuple2<>(s.getId(), s)) - .leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map( - c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())) { - if (!context.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - context.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name)); - // community id already in the context of the result. - // Remove it from the set that has to be added - contexts.remove(context.getId()); - } - } - } - List cc = oaf.getContext(); - for (String cId : contexts) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - class_id, - class_name))); - cc.add(context); - } - oaf.setContext(cc); - } - return oaf; - }); - } - - private static JavaPairRDD> getStringResultJavaPairRDD( - JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair( - c -> { - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } - - return new Tuple2<>(c.getString(0), contextList); - }); - } - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table) { - String query = - "SELECT relation.source, " - + table - + ".context , relation.target " - + "FROM " - + table - + " JOIN relation " - + "ON id = source"; - - return spark.sql(query); - } - - private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = - r.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet()); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - return true; - } - } - return false; - } - - private static void updateResult( - JavaPairRDD results, - JavaPairRDD toupdateresult, - String outputPath, - String type) { - results.leftOuterJoin(toupdateresult) - .map( - p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()) { - Set communityList = p._2()._2().get().getAccumulator(); - for (Context c : r.getContext()) { - if (communityList.contains(c.getId())) { - // verify if the datainfo for this context contains - // propagation - if (!c.getDataInfo().stream() - .map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()) - .contains(PROPAGATION_DATA_INFO_TYPE)) { - c.getDataInfo() - .add( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - // community id already in the context of the result. - // Remove it from the set that has to be added - communityList.remove(c.getId()); - } - } - } - List cc = r.getContext(); - for (String cId : communityList) { - Context context = new Context(); - context.setId(cId); - context.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static TypedRow getTypedRow( - List communityIdList, List context, String id, String type) { - Set result_communities = - context.stream().map(c -> c.getId()).collect(Collectors.toSet()); - TypedRow tp = new TypedRow(); - tp.setSourceId(id); - tp.setType(type); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - tp.add(communityId); - } - } - if (tp.getAccumulator() != null) { - return tp; - } - return null; - } -} From adcbf0e29a56cf29acb34e22fa3f0fdc5826fd56 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 24 Apr 2020 10:47:43 +0200 Subject: [PATCH 163/259] refactoring --- .../dhp-bulktag/project-default.properties | 7 + .../dnetlib/dhp/bulktag/SparkBulkTagJob2.java | 4 +- .../dnetlib/dhp/community/ResultTagger.java | 68 +- .../dhp/bulktag/oozie_app/workflow.xml | 16 +- .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 1 + .../CommunityConfigurationFactoryTest.java | 11 + .../communityconfiguration/tagging_conf.json | 26 +- .../communityconfiguration/tagging_conf.xml | 2624 +++++++++-------- 8 files changed, 1474 insertions(+), 1283 deletions(-) create mode 100644 dhp-workflows/dhp-bulktag/project-default.properties diff --git a/dhp-workflows/dhp-bulktag/project-default.properties b/dhp-workflows/dhp-bulktag/project-default.properties new file mode 100644 index 000000000..84a56f19f --- /dev/null +++ b/dhp-workflows/dhp-bulktag/project-default.properties @@ -0,0 +1,7 @@ +#sandboxName when not provided explicitly will be generated +sandboxName=${sandboxName} +sandboxDir=/user/${dhp.hadoop.frontend.user.name}/${sandboxName} +workingDir=${sandboxDir}/working_dir +oozie.wf.application.path = ${nameNode}${sandboxDir}/${oozieAppDir} +oozieTopWfApplicationPath = ${oozie.wf.application.path} + diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java index 3ca22de7c..89106560c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java @@ -1,4 +1,4 @@ -package eu.dnetlib.dhp; +package eu.dnetlib.dhp.bulktag; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; @@ -28,7 +28,7 @@ public class SparkBulkTagJob2 { String jsonConfiguration = IOUtils.toString( SparkBulkTagJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/bulktag/input_bulktag_parameters.json")); + "/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java index 4e58bb1a2..8752a4c57 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java @@ -65,13 +65,16 @@ public class ResultTagger implements Serializable { // tagging for Subject final Set subjects = new HashSet<>(); - result.getSubject().stream() - .map(subject -> subject.getValue()) - .filter(StringUtils::isNotBlank) - .map(String::toLowerCase) - .map(String::trim) - .collect(Collectors.toCollection(HashSet::new)) - .forEach(s -> subjects.addAll(conf.getCommunityForSubjectValue(s))); + Optional> oresultsubj = Optional.ofNullable(result.getSubject()); + if (oresultsubj.isPresent()) { + oresultsubj.get().stream() + .map(subject -> subject.getValue()) + .filter(StringUtils::isNotBlank) + .map(String::toLowerCase) + .map(String::trim) + .collect(Collectors.toCollection(HashSet::new)) + .forEach(s -> subjects.addAll(conf.getCommunityForSubjectValue(s))); + } communities.addAll(subjects); @@ -79,32 +82,43 @@ public class ResultTagger implements Serializable { final Set datasources = new HashSet<>(); final Set tmp = new HashSet<>(); - for (Instance i : result.getInstance()) { - tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); - tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); - } + Optional> oresultinstance = Optional.ofNullable(result.getInstance()); + if (oresultinstance.isPresent()) { + for (Instance i : oresultinstance.get()) { + tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); + tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); + } - result.getInstance().stream() - .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) - .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) - .map(s -> StringUtils.substringAfter(s, "|")) - .collect(Collectors.toCollection(HashSet::new)) - .forEach(dsId -> datasources.addAll(conf.getCommunityForDatasource(dsId, param))); + oresultinstance.get().stream() + .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) + .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) + .map(s -> StringUtils.substringAfter(s, "|")) + .collect(Collectors.toCollection(HashSet::new)) + .forEach( + dsId -> + datasources.addAll( + conf.getCommunityForDatasource(dsId, param))); + } communities.addAll(datasources); /*Tagging for Zenodo Communities*/ final Set czenodo = new HashSet<>(); - result.getContext().stream() - .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) - .collect(Collectors.toList()) - .forEach( - c -> - czenodo.addAll( - conf.getCommunityForZenodoCommunityValue( - c.getId() - .substring(c.getId().lastIndexOf("/") + 1) - .trim()))); + + Optional> oresultcontext = Optional.ofNullable(result.getContext()); + if (oresultcontext.isPresent()) { + oresultcontext.get().stream() + .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) + .collect(Collectors.toList()) + .forEach( + c -> + czenodo.addAll( + conf.getCommunityForZenodoCommunityValue( + c.getId() + .substring( + c.getId().lastIndexOf("/") + 1) + .trim()))); + } communities.addAll(czenodo); diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 4015843f3..02efeb7ae 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -67,8 +67,8 @@ ${nameNode} yarn-cluster cluster - bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob2 + bulkTagging-publication + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -96,8 +96,8 @@ ${nameNode} yarn-cluster cluster - bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob2 + bulkTagging-dataset + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -125,8 +125,8 @@ ${nameNode} yarn-cluster cluster - bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob2 + bulkTagging-orp + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -154,8 +154,8 @@ ${nameNode} yarn-cluster cluster - bulkTagging - eu.dnetlib.dhp.SparkBulkTagJob2 + bulkTagging-software + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index d04504253..e20d87280 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp; import static eu.dnetlib.dhp.community.TagginConstants.ZENODO_COMMUNITY_INDICATOR; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.bulktag.SparkBulkTagJob2; import eu.dnetlib.dhp.schema.oaf.Dataset; import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; import eu.dnetlib.dhp.schema.oaf.Publication; diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java index 4a855bd3a..a04395337 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java @@ -141,4 +141,15 @@ public class CommunityConfigurationFactoryTest { System.out.println(cc.toJson()); } + + @Test + public void temporaneo() throws Exception { + String xml = + IOUtils.toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + System.out.println(cc.toJson()); + } } diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json index c150e4386..411a64fed 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json @@ -17,4 +17,28 @@ ]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"driver______::bee53aa31dc2cbb538c10c2b65fa5824","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::437f4b072b1aa198adcbc35910ff3b98","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::081b82f96300b6a6e3d282bad31cb6e2","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"openaire____::9e3be59865b2c1c335d32dae2fe7b254","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]} ,{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"opendoar____::8b6dd7db9af49e67306feb59a8bdc52c","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"share_______::4719356ec8d7d55d3feb384ce879ad6c","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"share_______::bbd802baad85d1fd440f32a7a3a2c2b1","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}},{"openaireId":"opendoar____::6f4922f45568161a8cdf4ad2299f6d23","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]}}],"zenodoCommunities":[{"zenodoCommunityId":"chicago-covid-19"},{"zenodoCommunityId":"covid-19-senacyt-panama-sample"},{"zenodoCommunityId":"covid-19-tx-rct-stats-review"},{"zenodoCommunityId":"covid_19_senacyt_abc_panama"}],"organizationCommunity":[]}, - "dariah":{"id":"dariah","subjects":[],"datasources":[{"openaireId":"opendoar____::7e7757b1e12abcb736ab9a754ffb617a","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}},{"openaireId":"opendoar____::96da2f590cd7246bbde0051047b0d6f7","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}}],"zenodoCommunities":[{"zenodoCommunityId":"dimpo"}],"organizationCommunity":[]},"rda":{"id":"rda","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"rda"}],"organizationCommunity":[]},"clarin":{"id":"clarin","subjects":[],"datasources":[{"openaireId":"re3data_____::a507cdacc5bbcc08761c92185dee5cab"}],"zenodoCommunities":[],"organizationCommunity":[]},"aginfra":{"id":"aginfra","subjects":["animal production and health","fisheries and aquaculture","food safety and human nutrition","information management","food technology","agri-food education and extension","natural resources and environment","food system","engineering technology and Research","agriculture","food safety risk assessment","food security","farming practices and systems","plant production and protection","agri-food economics and policy","Agri-food","food distribution","forestry"],"datasources":[{"openaireId":"opendoar____::1a551829d50f1400b0dab21fdd969c04"},{"openaireId":"opendoar____::49af6c4e558a7569d80eee2e035e2bd7"},{"openaireId":"opendoar____::0266e33d3f546cb5436a10798e657d97"},{"openaireId":"opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06"},{"openaireId":"opendoar____::41bfd20a38bb1b0bec75acf0845530a7"},{"openaireId":"opendoar____::87ae6fb631f7c8a627e8e28785d9992d"}],"zenodoCommunities":[{"zenodoCommunityId":"edenis"},{"zenodoCommunityId":"efsa-pilot"},{"zenodoCommunityId":"egene3"},{"zenodoCommunityId":"efsa-kj"},{"zenodoCommunityId":"euromixproject"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"sedinstcjfst"},{"zenodoCommunityId":"afinet-kc"},{"zenodoCommunityId":"2231-4784"},{"zenodoCommunityId":"2231-0606"},{"zenodoCommunityId":"solace"},{"zenodoCommunityId":"pa17"},{"zenodoCommunityId":"smartakis"},{"zenodoCommunityId":"sedinstcjae"},{"zenodoCommunityId":"phenology_camera"},{"zenodoCommunityId":"aginfra"},{"zenodoCommunityId":"erosa"},{"zenodoCommunityId":"bigdatagrapes"}],"organizationCommunity":[]},"fam":{"id":"fam","subjects":["Stock Assessment","pelagic","Acoustic","Fish farming","Fisheries","Fishermen","maximum sustainable yield","trawler","Fishing vessel","Fisherman","Fishing gear","mackerel","RFMO","Fish Aggregating Device","Bycatch","Fishery","common fisheries policy","Fishing fleet","Aquaculture"],"datasources":[{"openaireId":"doajarticles::8cec81178926caaca531afbd8eb5d64c"},{"openaireId":"doajarticles::0f7a7f30b5400615cae1829f3e743982"},{"openaireId":"doajarticles::9740f7f5af3e506d2ad2c215cdccd51a"},{"openaireId":"doajarticles::9f3fbaae044fa33cb7069b72935a3254"},{"openaireId":"doajarticles::cb67f33eb9819f5c624ce0313957f6b3"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::554cde3be9e5c4588b4c4f9f503120cb"},{"openaireId":"tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b"},{"openaireId":"doajarticles::57c5d3837da943e93b28ec4db82ec7a5"},{"openaireId":"doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::dca64612dfe0963fffc119098a319957"},{"openaireId":"doajarticles::dd70e44479f0ade25aa106aef3e87a0a"}],"zenodoCommunities":[{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"h2020_vicinaqua"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"rlsadb"},{"zenodoCommunityId":"iotc_ctoi"}],"organizationCommunity":[]},"beopen":{"id":"beopen","subjects":["Green Transport","City mobility systems","Vulnerable road users","Traffic engineering","Transport electrification","Mobility","Intermodal freight transport","Clean vehicle fleets","Intelligent mobility","Inflight refueling","District mobility systems","Navigation and control systems for optimised planning and routing","European Space Technology Platform","European Transport networks","Green cars","Inter-modality infrastructures","Advanced Take Off and Landing Ideas","Sustainable urban systems","port-area railway networks","Innovative forms of urban transport","Alliance for Logistics Innovation through Collaboration in Europe","Advisory Council for Aeronautics Research in Europe","Mobility services for people and goods","Guidance and traffic management","Passenger mobility","Smart mobility and services","transport innovation","high-speed railway","Vehicle design","Inland shipping","public transportation","aviation’s climate impact","Road transport","On-demand public transport","Personal Air Transport","Transport","transport vulnerability","Pipeline transport","European Association of Aviation Training and Education Organisations","Defrosting of railway infrastructure","Inclusive and affordable transport","River Information Services","jel:L92","Increased use of public transport","Seamless mobility","STRIA","trolleybus transport","Intelligent Transport System","Low-emission alternative energy for transport","Shared mobility for people and goods","Business model for urban mobility","Interoperability of transport systems","Cross-border train slot booking","Air transport","Transport pricing","Sustainable transport","European Rail Transport Research Advisory Council","Alternative aircraft configurations","Transport and Mobility","Railways applications","urban transport","Environmental impact of transport","urban freight delivery systems","Automated Road Transport","Alternative fuels in public transport","Active LIDAR-sensor for GHG-measurements","Autonomous logistics operations","Rational use of motorised transport","Network and traffic management systems","electrification of railway wagons","Single European Sky","Electrified road systems","transportation planning","Railway dynamics","Motorway of the Sea","smart railway communications","Maritime transport","Environmental- friendly transport","Combined transport","Connected automated driving technology","Innovative freight logistics services","automated and shared vehicles","Alternative Aircraft Systems","Land-use and transport interaction","Public transport system","Business plan for shared mobility","Shared mobility","Growing of mobility demand","European Road Transport Research Advisory Council","WATERBORNE ETP","Effective transport management system","Short Sea Shipping","air traffic management","Sea hubs and the motorways of the sea","Urban mobility solutions","Smart city planning","Maritime spatial planning","EUropean rail Research Network of Excellence","Transport governance","ENERGY CONSUMPTION BY THE TRANSPORT SECTOR","Integrated urban plan","inland waterway services","European Conference of Transport Research Institutes","air vehicles","E-freight","Automated Driving","Automated ships","pricing for cross-border passenger transport","Vehicle efficiency","Railway transport","Electric vehicles","Road traffic monitoring","Deep sea shipping","Circular economy in transport","Traffic congestion","air transport system","Urban logistics","Rail transport","OpenStreetMap","high speed rail","Transportation engineering","Intermodal travel information","Flight Data Recorders","Advanced driver assistance systems","long distance freight transport","Inland waterway transport","Smart mobility","Mobility integration","Personal Rapid Transit system","Safety measures \\u0026 requirements for roads","Green rail transport","Electrical","Vehicle manufacturing","Future Airport Layout","Rail technologies","European Intermodal Research Advisory Council","inland navigation","Automated urban vehicles","ECSS-standards","Traveller services","Polluting transport","Air Traffic Control","Cooperative and connected and automated transport","Innovative powertrains","Quality of transport system and services","door-to- door logistics chain","Inter-modal aspects of urban mobility","travel (and mobility)","Innovative freight delivery systems","urban freight delivery infrastructures"],"datasources":[{"openaireId":"doajarticles::1c5bdf8fca58937894ad1441cca99b76"},{"openaireId":"doajarticles::b37a634324a45c821687e6e80e6f53b4"},{"openaireId":"doajarticles::4bf64f2a104040e4e055cd9594b2d77c"},{"openaireId":"doajarticles::479ca537c12755d1868bbf02938a900c"},{"openaireId":"doajarticles::55f31df96a60e2309f45b7c265fcf7a2"},{"openaireId":"doajarticles::c52a09891a5301f9986ebbfe3761810c"},{"openaireId":"doajarticles::379807bc7f6c71a227ef1651462c414c"},{"openaireId":"doajarticles::36069db531a00b85a2e8fb301f4bdc19"},{"openaireId":"doajarticles::b6a898da311ded96fabf49c520b80d5d"},{"openaireId":"doajarticles::d0753d9180b35a271d8b4a31f449749f"},{"openaireId":"doajarticles::172050a92511838393a3fe237ae47e31"},{"openaireId":"doajarticles::301ed96c62abb160a3e29796efe5c95c"},{"openaireId":"doajarticles::0f4f805b3d842f2c7f1b077c3426fa59"},{"openaireId":"doajarticles::ba73728b84437b8d48ae287b867c7215"},{"openaireId":"doajarticles::86faef424d804309ccf45f692523aa48"},{"openaireId":"doajarticles::73bd758fa41671de70964c3ecba013af"},{"openaireId":"doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4"},{"openaireId":"doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7"},{"openaireId":"doajarticles::ca61df07089acc53a1569bde6673d82a"},{"openaireId":"doajarticles::237dd6f1606600459d0297abd8ed9976"},{"openaireId":"doajarticles::fba6191177ede7c51ea1cdf58eae7f8b"}],"zenodoCommunities":[{"zenodoCommunityId":"jsdtl"},{"zenodoCommunityId":"utc-martrec"},{"zenodoCommunityId":"utc-uti"},{"zenodoCommunityId":"stp"},{"zenodoCommunityId":"c2smart"},{"zenodoCommunityId":"stride-utc"},{"zenodoCommunityId":"crowd4roads"},{"zenodoCommunityId":"lemo"},{"zenodoCommunityId":"imov3d"},{"zenodoCommunityId":"tra2018"},{"zenodoCommunityId":"optimum"},{"zenodoCommunityId":"stars"},{"zenodoCommunityId":"iecteim"},{"zenodoCommunityId":"iccpt2019"}],"organizationCommunity":[]},"science-innovation-policy":{"id":"science-innovation-policy","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"risis"}],"organizationCommunity":[]},"mes":{"id":"mes","subjects":["marine","ocean","fish","aqua","sea"],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"adriplan"},{"zenodoCommunityId":"devotes-project"},{"zenodoCommunityId":"euro-basin"},{"zenodoCommunityId":"naclim"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"assisibf"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"aquatrace"},{"zenodoCommunityId":"myfish"},{"zenodoCommunityId":"atlas"},{"zenodoCommunityId":"blue-actionh2020"},{"zenodoCommunityId":"sponges"},{"zenodoCommunityId":"merces_project"},{"zenodoCommunityId":"bigdataocean"},{"zenodoCommunityId":"columbus"},{"zenodoCommunityId":"h2020-aquainvad-ed"},{"zenodoCommunityId":"aquarius"},{"zenodoCommunityId":"southern-ocean-observing-system"},{"zenodoCommunityId":"eawag"},{"zenodoCommunityId":"mossco"},{"zenodoCommunityId":"onc"},{"zenodoCommunityId":"oceanbiogeochemistry"},{"zenodoCommunityId":"oceanliteracy"},{"zenodoCommunityId":"openearth"},{"zenodoCommunityId":"ocean"},{"zenodoCommunityId":"calcifierraman"},{"zenodoCommunityId":"bermudabream"},{"zenodoCommunityId":"brcorp1"},{"zenodoCommunityId":"mce"},{"zenodoCommunityId":"biogeochem"},{"zenodoCommunityId":"ecc2014"},{"zenodoCommunityId":"fisheries"},{"zenodoCommunityId":"sedinstcjfas"},{"zenodoCommunityId":"narmada"},{"zenodoCommunityId":"umr-entropie"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"zf-ilcs"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"afrimed_eu"},{"zenodoCommunityId":"spi-ace"},{"zenodoCommunityId":"cice-consortium"},{"zenodoCommunityId":"nemo-ocean"},{"zenodoCommunityId":"mesopp-h2020"},{"zenodoCommunityId":"marxiv"}],"organizationCommunity":[]},"ni":{"id":"ni","subjects":["brain mapping","brain imaging","electroencephalography","arterial spin labelling","brain fingerprinting","brain","neuroimaging","Multimodal Brain Image Analysis","fMRI","neuroinformatics","fetal brain","brain ultrasonic imaging","topographic brain mapping","diffusion tensor imaging","computerized knowledge assessment","connectome mapping","brain magnetic resonance imaging","brain abnormalities"],"datasources":[{"openaireId":"re3data_____::5b9bf9171d92df854cf3c520692e9122"},{"openaireId":"doajarticles::c7d3de67dc77af72f6747157441252ec"},{"openaireId":"re3data_____::8515794670370f49c1d176c399c714f5"},{"openaireId":"doajarticles::d640648c84b10d425f96f11c3de468f3"},{"openaireId":"doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a"},{"openaireId":"rest________::fb1a3d4523c95e63496e3bc7ba36244b"}],"zenodoCommunities":[{"zenodoCommunityId":"neuroinformatics"},{"zenodoCommunityId":"hbp"},{"zenodoCommunityId":"from_neuroscience_to_machine_learning"},{"zenodoCommunityId":"ci2c"},{"zenodoCommunityId":"opensourcebrain"},{"zenodoCommunityId":"brainspeak"},{"zenodoCommunityId":"braincom"},{"zenodoCommunityId":"nextgenvis"},{"zenodoCommunityId":"meso-brain"},{"zenodoCommunityId":"neuroplasticity-workshop"},{"zenodoCommunityId":"bionics"},{"zenodoCommunityId":"brainmattrain-676408"},{"zenodoCommunityId":"repronim"},{"zenodoCommunityId":"affectiveneuro"},{"zenodoCommunityId":"con"},{"zenodoCommunityId":"lab_neurol_sperim_irfmn_irccs_milano_it"}],"organizationCommunity":[]},"dh-ch":{"id":"dh-ch","subjects":["modern art","monuments","europeana data model","sites","field walking","frescoes","LIDO metadata schema","art history","excavation","Arts and Humanities General","cities","coins","temples","numismatics","lithics","roads","environmental archaeology","digital cultural heritage","archaeological reports","history","CRMba","churches","cultural heritage","archaeological stratigraphy","religious art","buidings","digital humanities","survey","archaeological sites","linguistic studies","bioarchaeology","architectural orders","palaeoanthropology","fine arts","europeana","CIDOC CRM","decorations","classic art","stratigraphy","digital archaeology","intangible cultural heritage","walls","humanities","chapels","CRMtex","Language and Literature","paintings","archaeology","fair data","mosaics","burials","architecture","medieval art","castles","CARARE metadata schema","statues","natural language processing","inscriptions","CRMsci","vaults","contemporary art","Arts and Humanities","CRMarchaeo","pottery","site","architectural","vessels"],"datasources":[{"openaireId":"re3data_____::9ebe127e5f3a0bf401875690f3bb6b81"},{"openaireId":"doajarticles::c6cd4b532e12868c1d760a8d7cda6815"},{"openaireId":"doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b"},{"openaireId":"doajarticles::6eb31d13b12bc06bbac06aef63cf33c9"},{"openaireId":"doajarticles::0da84e9dfdc8419576169e027baa8028"},{"openaireId":"re3data_____::84e123776089ce3c7a33db98d9cd15a8"},{"openaireId":"openaire____::c5502a43e76feab55dd00cf50f519125"},{"openaireId":"re3data_____::a48f09c562b247a9919acfe195549b47"},{"openaireId":"opendoar____::97275a23ca44226c9964043c8462be96"}],"zenodoCommunities":[{"zenodoCommunityId":"storm"},{"zenodoCommunityId":"crosscult"},{"zenodoCommunityId":"wholodance_eu"},{"zenodoCommunityId":"digcur2013"},{"zenodoCommunityId":"gravitate"},{"zenodoCommunityId":"dipp2014"},{"zenodoCommunityId":"digitalhumanities"},{"zenodoCommunityId":"dimpo"},{"zenodoCommunityId":"adho"},{"zenodoCommunityId":"chc"},{"zenodoCommunityId":"wahr"},{"zenodoCommunityId":"ibe"},{"zenodoCommunityId":"ariadne"},{"zenodoCommunityId":"parthenos-hub"},{"zenodoCommunityId":"parthenos-training"},{"zenodoCommunityId":"gandhara"},{"zenodoCommunityId":"cmsouthasia"},{"zenodoCommunityId":"nilgirihills"},{"zenodoCommunityId":"shamsa_mustecio"},{"zenodoCommunityId":"bodhgaya"}],"organizationCommunity":[]}}} \ No newline at end of file + "dariah":{"id":"dariah","subjects":[],"datasources":[{"openaireId":"opendoar____::7e7757b1e12abcb736ab9a754ffb617a","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}},{"openaireId":"opendoar____::96da2f590cd7246bbde0051047b0d6f7","selectionConstraints":{"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]}}],"zenodoCommunities":[{"zenodoCommunityId":"dimpo"}],"organizationCommunity":[]},"rda":{"id":"rda","subjects":[],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"rda"}],"organizationCommunity":[]},"clarin":{"id":"clarin","subjects":[],"datasources":[{"openaireId":"re3data_____::a507cdacc5bbcc08761c92185dee5cab"}],"zenodoCommunities":[],"organizationCommunity":[]},"aginfra":{"id":"aginfra","subjects":["animal production and health","fisheries and aquaculture","food safety and human nutrition","information management","food technology","agri-food education and extension","natural resources and environment","food system","engineering technology and Research","agriculture","food safety risk assessment","food security","farming practices and systems","plant production and protection","agri-food economics and policy","Agri-food","food distribution","forestry"],"datasources":[{"openaireId":"opendoar____::1a551829d50f1400b0dab21fdd969c04"},{"openaireId":"opendoar____::49af6c4e558a7569d80eee2e035e2bd7"},{"openaireId":"opendoar____::0266e33d3f546cb5436a10798e657d97"},{"openaireId":"opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06"},{"openaireId":"opendoar____::41bfd20a38bb1b0bec75acf0845530a7"},{"openaireId":"opendoar____::87ae6fb631f7c8a627e8e28785d9992d"}],"zenodoCommunities":[{"zenodoCommunityId":"edenis"},{"zenodoCommunityId":"efsa-pilot"},{"zenodoCommunityId":"egene3"},{"zenodoCommunityId":"efsa-kj"},{"zenodoCommunityId":"euromixproject"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"sedinstcjfst"},{"zenodoCommunityId":"afinet-kc"},{"zenodoCommunityId":"2231-4784"},{"zenodoCommunityId":"2231-0606"},{"zenodoCommunityId":"solace"},{"zenodoCommunityId":"pa17"},{"zenodoCommunityId":"smartakis"},{"zenodoCommunityId":"sedinstcjae"},{"zenodoCommunityId":"phenology_camera"},{"zenodoCommunityId":"aginfra"},{"zenodoCommunityId":"erosa"},{"zenodoCommunityId":"bigdatagrapes"}],"organizationCommunity":[]},"fam":{"id":"fam","subjects":["Stock Assessment","pelagic","Acoustic","Fish farming","Fisheries","Fishermen","maximum sustainable yield","trawler","Fishing vessel","Fisherman","Fishing gear","mackerel","RFMO","Fish Aggregating Device","Bycatch","Fishery","common fisheries policy","Fishing fleet","Aquaculture"],"datasources":[{"openaireId":"doajarticles::8cec81178926caaca531afbd8eb5d64c"},{"openaireId":"doajarticles::0f7a7f30b5400615cae1829f3e743982"},{"openaireId":"doajarticles::9740f7f5af3e506d2ad2c215cdccd51a"},{"openaireId":"doajarticles::9f3fbaae044fa33cb7069b72935a3254"},{"openaireId":"doajarticles::cb67f33eb9819f5c624ce0313957f6b3"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::554cde3be9e5c4588b4c4f9f503120cb"},{"openaireId":"tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b"},{"openaireId":"doajarticles::57c5d3837da943e93b28ec4db82ec7a5"},{"openaireId":"doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1"},{"openaireId":"doajarticles::e21c97cbb7a209afc75703681c462906"},{"openaireId":"doajarticles::dca64612dfe0963fffc119098a319957"},{"openaireId":"doajarticles::dd70e44479f0ade25aa106aef3e87a0a"}],"zenodoCommunities":[{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"h2020_vicinaqua"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"rlsadb"},{"zenodoCommunityId":"iotc_ctoi"}],"organizationCommunity":[]},"beopen":{"id":"beopen","subjects":["Green Transport","City mobility systems","Vulnerable road users","Traffic engineering","Transport electrification","Mobility","Intermodal freight transport","Clean vehicle fleets","Intelligent mobility","Inflight refueling","District mobility systems","Navigation and control systems for optimised planning and routing","European Space Technology Platform","European Transport networks","Green cars","Inter-modality infrastructures","Advanced Take Off and Landing Ideas","Sustainable urban systems","port-area railway networks","Innovative forms of urban transport","Alliance for Logistics Innovation through Collaboration in Europe","Advisory Council for Aeronautics Research in Europe","Mobility services for people and goods","Guidance and traffic management","Passenger mobility","Smart mobility and services","transport innovation","high-speed railway","Vehicle design","Inland shipping","public transportation","aviation’s climate impact","Road transport","On-demand public transport","Personal Air Transport","Transport","transport vulnerability","Pipeline transport","European Association of Aviation Training and Education Organisations","Defrosting of railway infrastructure","Inclusive and affordable transport","River Information Services","jel:L92","Increased use of public transport","Seamless mobility","STRIA","trolleybus transport","Intelligent Transport System","Low-emission alternative energy for transport","Shared mobility for people and goods","Business model for urban mobility","Interoperability of transport systems","Cross-border train slot booking","Air transport","Transport pricing","Sustainable transport","European Rail Transport Research Advisory Council","Alternative aircraft configurations","Transport and Mobility","Railways applications","urban transport","Environmental impact of transport","urban freight delivery systems","Automated Road Transport","Alternative fuels in public transport","Active LIDAR-sensor for GHG-measurements","Autonomous logistics operations","Rational use of motorised transport","Network and traffic management systems","electrification of railway wagons","Single European Sky","Electrified road systems","transportation planning","Railway dynamics","Motorway of the Sea","smart railway communications","Maritime transport","Environmental- friendly transport","Combined transport","Connected automated driving technology","Innovative freight logistics services","automated and shared vehicles","Alternative Aircraft Systems","Land-use and transport interaction","Public transport system","Business plan for shared mobility","Shared mobility","Growing of mobility demand","European Road Transport Research Advisory Council","WATERBORNE ETP","Effective transport management system","Short Sea Shipping","air traffic management","Sea hubs and the motorways of the sea","Urban mobility solutions","Smart city planning","Maritime spatial planning","EUropean rail Research Network of Excellence","Transport governance","ENERGY CONSUMPTION BY THE TRANSPORT SECTOR","Integrated urban plan","inland waterway services","European Conference of Transport Research Institutes","air vehicles","E-freight","Automated Driving","Automated ships","pricing for cross-border passenger transport","Vehicle efficiency","Railway transport","Electric vehicles","Road traffic monitoring","Deep sea shipping","Circular economy in transport","Traffic congestion","air transport system","Urban logistics","Rail transport","OpenStreetMap","high speed rail","Transportation engineering","Intermodal travel information","Flight Data Recorders","Advanced driver assistance systems","long distance freight transport","Inland waterway transport","Smart mobility","Mobility integration","Personal Rapid Transit system","Safety measures \\u0026 requirements for roads","Green rail transport","Electrical","Vehicle manufacturing","Future Airport Layout","Rail technologies","European Intermodal Research Advisory Council","inland navigation","Automated urban vehicles","ECSS-standards","Traveller services","Polluting transport","Air Traffic Control","Cooperative and connected and automated transport","Innovative powertrains","Quality of transport system and services","door-to- door logistics chain","Inter-modal aspects of urban mobility","travel (and mobility)","Innovative freight delivery systems","urban freight delivery infrastructures"],"datasources":[{"openaireId":"doajarticles::1c5bdf8fca58937894ad1441cca99b76"},{"openaireId":"doajarticles::b37a634324a45c821687e6e80e6f53b4"},{"openaireId":"doajarticles::4bf64f2a104040e4e055cd9594b2d77c"},{"openaireId":"doajarticles::479ca537c12755d1868bbf02938a900c"},{"openaireId":"doajarticles::55f31df96a60e2309f45b7c265fcf7a2"},{"openaireId":"doajarticles::c52a09891a5301f9986ebbfe3761810c"},{"openaireId":"doajarticles::379807bc7f6c71a227ef1651462c414c"},{"openaireId":"doajarticles::36069db531a00b85a2e8fb301f4bdc19"},{"openaireId":"doajarticles::b6a898da311ded96fabf49c520b80d5d"},{"openaireId":"doajarticles::d0753d9180b35a271d8b4a31f449749f"},{"openaireId":"doajarticles::172050a92511838393a3fe237ae47e31"},{"openaireId":"doajarticles::301ed96c62abb160a3e29796efe5c95c"},{"openaireId":"doajarticles::0f4f805b3d842f2c7f1b077c3426fa59"},{"openaireId":"doajarticles::ba73728b84437b8d48ae287b867c7215"},{"openaireId":"doajarticles::86faef424d804309ccf45f692523aa48"},{"openaireId":"doajarticles::73bd758fa41671de70964c3ecba013af"},{"openaireId":"doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4"},{"openaireId":"doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7"},{"openaireId":"doajarticles::ca61df07089acc53a1569bde6673d82a"},{"openaireId":"doajarticles::237dd6f1606600459d0297abd8ed9976"},{"openaireId":"doajarticles::fba6191177ede7c51ea1cdf58eae7f8b"}],"zenodoCommunities":[{"zenodoCommunityId":"jsdtl"},{"zenodoCommunityId":"utc-martrec"},{"zenodoCommunityId":"utc-uti"},{"zenodoCommunityId":"stp"},{"zenodoCommunityId":"c2smart"},{"zenodoCommunityId":"stride-utc"},{"zenodoCommunityId":"crowd4roads"},{"zenodoCommunityId":"lemo"},{"zenodoCommunityId":"imov3d"},{"zenodoCommunityId":"tra2018"},{"zenodoCommunityId":"optimum"},{"zenodoCommunityId":"stars"},{"zenodoCommunityId":"iecteim"},{"zenodoCommunityId":"iccpt2019"}],"organizationCommunity":[]},"science-innovation-policy": + {"id":"science-innovation-policy","subjects":["Sustainability-oriented science policy", "STI policies", "science—society relations", + "Science & Technology Policy", "Innovation policy", "science policy", "Policy and Law"], + "datasources":[{"openaireId":"doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d"}, + {"openaireId":"doajarticles::ae4c7286c79590f19fdca670156ce816"}, + {"openaireId":"doajarticles::0f664bce92ce953e0c7a92068c46bfb3"}, + {"openaireId":"doajarticles::00017183dc4c858fb77541985323a4ef"}, + {"openaireId":"doajarticles::93b306f458cce3d7aaaf58c0a725f4f9"}, + {"openaireId":"doajarticles::9dbf8fbf3e9fe0fe1fc01e55fbd90bfc"}, + {"openaireId":"doajarticles::a2bda8785c863279bba4b8f34827b4c9"}, + {"openaireId":"doajarticles::019a1fcb42c3fea1c1b689df76330b58"}, + {"openaireId":"doajarticles::0daa8281938831e9c82bfed8b55a2975"}, + {"openaireId":"doajarticles::f67ad6d268162079b3abd51a24468744"}, + {"openaireId":"doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d"}, + {"openaireId":"doajarticles::ad114356e196a4a3d84dda59c720dacd"}, + {"openaireId":"doajarticles::01e8a54fdecaaf354c67a2dd74ae7d4f"}, + {"openaireId":"doajarticles::449305f096b10a9464449ff2d0e10e06"}, + {"openaireId":"doajarticles::982c0c0ac378256254cce2fa6572bb6c"}, + {"openaireId":"doajarticles::49d6ed47138884566ce93cf0ccb12c02"}, + {"openaireId":"doajarticles::a98e820dbc2e8ee0fc84ab66f263267c"}, + {"openaireId":"doajarticles::50b1ce37427b36368f8f0f1317e47f83"}, + {"openaireId":"doajarticles::f0ec29b7450b2ac5d0ad45327eeb531a"}, + {"openaireId":"doajarticles::d8d421d3b0349a7aaa93758b27a54e84"}, + {"openaireId":"doajarticles::7ffc35ac5133da01d421ccf8af5b70bc"} + ],"zenodoCommunities":[{"zenodoCommunityId":"risis"}],"organizationCommunity":[]},"mes":{"id":"mes","subjects":["marine","ocean","fish","aqua","sea"],"datasources":[],"zenodoCommunities":[{"zenodoCommunityId":"adriplan"},{"zenodoCommunityId":"devotes-project"},{"zenodoCommunityId":"euro-basin"},{"zenodoCommunityId":"naclim"},{"zenodoCommunityId":"discardless"},{"zenodoCommunityId":"assisibf"},{"zenodoCommunityId":"meece"},{"zenodoCommunityId":"facts"},{"zenodoCommunityId":"proeel"},{"zenodoCommunityId":"aquatrace"},{"zenodoCommunityId":"myfish"},{"zenodoCommunityId":"atlas"},{"zenodoCommunityId":"blue-actionh2020"},{"zenodoCommunityId":"sponges"},{"zenodoCommunityId":"merces_project"},{"zenodoCommunityId":"bigdataocean"},{"zenodoCommunityId":"columbus"},{"zenodoCommunityId":"h2020-aquainvad-ed"},{"zenodoCommunityId":"aquarius"},{"zenodoCommunityId":"southern-ocean-observing-system"},{"zenodoCommunityId":"eawag"},{"zenodoCommunityId":"mossco"},{"zenodoCommunityId":"onc"},{"zenodoCommunityId":"oceanbiogeochemistry"},{"zenodoCommunityId":"oceanliteracy"},{"zenodoCommunityId":"openearth"},{"zenodoCommunityId":"ocean"},{"zenodoCommunityId":"calcifierraman"},{"zenodoCommunityId":"bermudabream"},{"zenodoCommunityId":"brcorp1"},{"zenodoCommunityId":"mce"},{"zenodoCommunityId":"biogeochem"},{"zenodoCommunityId":"ecc2014"},{"zenodoCommunityId":"fisheries"},{"zenodoCommunityId":"sedinstcjfas"},{"zenodoCommunityId":"narmada"},{"zenodoCommunityId":"umr-entropie"},{"zenodoCommunityId":"farfish2020"},{"zenodoCommunityId":"primefish"},{"zenodoCommunityId":"zf-ilcs"},{"zenodoCommunityId":"climefish"},{"zenodoCommunityId":"afrimed_eu"},{"zenodoCommunityId":"spi-ace"},{"zenodoCommunityId":"cice-consortium"},{"zenodoCommunityId":"nemo-ocean"},{"zenodoCommunityId":"mesopp-h2020"},{"zenodoCommunityId":"marxiv"}],"organizationCommunity":[]},"ni":{"id":"ni","subjects":["brain mapping","brain imaging","electroencephalography","arterial spin labelling","brain fingerprinting","brain","neuroimaging","Multimodal Brain Image Analysis","fMRI","neuroinformatics","fetal brain","brain ultrasonic imaging","topographic brain mapping","diffusion tensor imaging","computerized knowledge assessment","connectome mapping","brain magnetic resonance imaging","brain abnormalities"],"datasources":[{"openaireId":"re3data_____::5b9bf9171d92df854cf3c520692e9122"},{"openaireId":"doajarticles::c7d3de67dc77af72f6747157441252ec"},{"openaireId":"re3data_____::8515794670370f49c1d176c399c714f5"},{"openaireId":"doajarticles::d640648c84b10d425f96f11c3de468f3"},{"openaireId":"doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a"},{"openaireId":"rest________::fb1a3d4523c95e63496e3bc7ba36244b"}],"zenodoCommunities":[{"zenodoCommunityId":"neuroinformatics"},{"zenodoCommunityId":"hbp"},{"zenodoCommunityId":"from_neuroscience_to_machine_learning"},{"zenodoCommunityId":"ci2c"},{"zenodoCommunityId":"opensourcebrain"},{"zenodoCommunityId":"brainspeak"},{"zenodoCommunityId":"braincom"},{"zenodoCommunityId":"nextgenvis"},{"zenodoCommunityId":"meso-brain"},{"zenodoCommunityId":"neuroplasticity-workshop"},{"zenodoCommunityId":"bionics"},{"zenodoCommunityId":"brainmattrain-676408"},{"zenodoCommunityId":"repronim"},{"zenodoCommunityId":"affectiveneuro"},{"zenodoCommunityId":"con"},{"zenodoCommunityId":"lab_neurol_sperim_irfmn_irccs_milano_it"}],"organizationCommunity":[]},"dh-ch":{"id":"dh-ch","subjects":["modern art","monuments","europeana data model","sites","field walking","frescoes","LIDO metadata schema","art history","excavation","Arts and Humanities General","cities","coins","temples","numismatics","lithics","roads","environmental archaeology","digital cultural heritage","archaeological reports","history","CRMba","churches","cultural heritage","archaeological stratigraphy","religious art","buidings","digital humanities","survey","archaeological sites","linguistic studies","bioarchaeology","architectural orders","palaeoanthropology","fine arts","europeana","CIDOC CRM","decorations","classic art","stratigraphy","digital archaeology","intangible cultural heritage","walls","humanities","chapels","CRMtex","Language and Literature","paintings","archaeology","fair data","mosaics","burials","architecture","medieval art","castles","CARARE metadata schema","statues","natural language processing","inscriptions","CRMsci","vaults","contemporary art","Arts and Humanities","CRMarchaeo","pottery","site","architectural","vessels"],"datasources":[{"openaireId":"re3data_____::9ebe127e5f3a0bf401875690f3bb6b81"},{"openaireId":"doajarticles::c6cd4b532e12868c1d760a8d7cda6815"},{"openaireId":"doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b"},{"openaireId":"doajarticles::6eb31d13b12bc06bbac06aef63cf33c9"},{"openaireId":"doajarticles::0da84e9dfdc8419576169e027baa8028"},{"openaireId":"re3data_____::84e123776089ce3c7a33db98d9cd15a8"},{"openaireId":"openaire____::c5502a43e76feab55dd00cf50f519125"},{"openaireId":"re3data_____::a48f09c562b247a9919acfe195549b47"},{"openaireId":"opendoar____::97275a23ca44226c9964043c8462be96"}],"zenodoCommunities":[{"zenodoCommunityId":"storm"},{"zenodoCommunityId":"crosscult"},{"zenodoCommunityId":"wholodance_eu"},{"zenodoCommunityId":"digcur2013"},{"zenodoCommunityId":"gravitate"},{"zenodoCommunityId":"dipp2014"},{"zenodoCommunityId":"digitalhumanities"},{"zenodoCommunityId":"dimpo"},{"zenodoCommunityId":"adho"},{"zenodoCommunityId":"chc"},{"zenodoCommunityId":"wahr"},{"zenodoCommunityId":"ibe"},{"zenodoCommunityId":"ariadne"},{"zenodoCommunityId":"parthenos-hub"},{"zenodoCommunityId":"parthenos-training"},{"zenodoCommunityId":"gandhara"},{"zenodoCommunityId":"cmsouthasia"},{"zenodoCommunityId":"nilgirihills"},{"zenodoCommunityId":"shamsa_mustecio"},{"zenodoCommunityId":"bodhgaya"}],"organizationCommunity":[]}}} \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml index dfea7b588..98114ef81 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml @@ -9,1248 +9,1382 @@ - - - - - - re3data_____::a507cdacc5bbcc08761c92185dee5cab - - - - - - - - - SDG13 - Climate action - SDG8 - Decent work and economic - growth - SDG15 - Life on land - SDG2 - Zero hunger - SDG17 - Partnerships for the - goals - SDG10 - Reduced inequalities - SDG5 - Gender equality - SDG12 - Responsible - consumption and production - SDG14 - Life below water - SDG6 - Clean water and - sanitation - SDG11 - Sustainable cities and communities - SDG1 - No poverty - SDG3 - - Good health and well being - SDG7 - Affordable and clean energy - SDG4 - Quality - education - SDG9 - Industry innovation and infrastructure - SDG16 - Peace justice - and strong institutions - - - - - - - - modern art - monuments - europeana data model - sites - field walking - frescoes - LIDO metadata schema - art history - excavation - Arts and Humanities General - cities - coins - temples - numismatics - lithics - roads - environmental archaeology - digital cultural heritage - archaeological reports - history - CRMba - churches - cultural heritage - archaeological stratigraphy - religious art - buidings - digital humanities - survey - archaeological sites - linguistic studies - bioarchaeology - architectural orders - palaeoanthropology - fine arts - europeana - CIDOC CRM - decorations - classic art - stratigraphy - digital archaeology - intangible cultural heritage - walls - humanities - chapels - CRMtex - Language and Literature - paintings - archaeology - fair data - mosaics - burials - architecture - medieval art - castles - CARARE metadata schema - statues - natural language processing - inscriptions - CRMsci - vaults - contemporary art - Arts and Humanities - CRMarchaeo - pottery - site - architectural - vessels - - - - re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 - - - - doajarticles::c6cd4b532e12868c1d760a8d7cda6815 - - - - doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b - - - - doajarticles::6eb31d13b12bc06bbac06aef63cf33c9 - - - - doajarticles::0da84e9dfdc8419576169e027baa8028 - - - - re3data_____::84e123776089ce3c7a33db98d9cd15a8 - - - - openaire____::c5502a43e76feab55dd00cf50f519125 - - - - re3data_____::a48f09c562b247a9919acfe195549b47 - - - - opendoar____::97275a23ca44226c9964043c8462be96 - - - - - - storm - - - - crosscult - - - - wholodance_eu - - - - digcur2013 - - - - gravitate - - - - dipp2014 - - - - digitalhumanities - - - - dimpo - - - - adho - - - - chc - - - - wahr - - - - ibe - - - - ariadne - - - - parthenos-hub - - - - parthenos-training - - - - gandhara - - - - cmsouthasia - - - - nilgirihills - - - - shamsa_mustecio - - - - bodhgaya - - - - - - - - Stock Assessment - pelagic - Acoustic - Fish farming - Fisheries - Fishermen - maximum sustainable yield - trawler - Fishing vessel - Fisherman - Fishing gear - mackerel - RFMO - Fish Aggregating Device - Bycatch - Fishery - common fisheries policy - Fishing fleet - Aquaculture - - - - doajarticles::8cec81178926caaca531afbd8eb5d64c - - - - doajarticles::0f7a7f30b5400615cae1829f3e743982 - - - - doajarticles::9740f7f5af3e506d2ad2c215cdccd51a - - - - doajarticles::9f3fbaae044fa33cb7069b72935a3254 - - - - doajarticles::cb67f33eb9819f5c624ce0313957f6b3 - - - - doajarticles::e21c97cbb7a209afc75703681c462906 - - - - doajarticles::554cde3be9e5c4588b4c4f9f503120cb - - - - tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b - - - - doajarticles::57c5d3837da943e93b28ec4db82ec7a5 - - - - doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1 - - - - doajarticles::e21c97cbb7a209afc75703681c462906 - - - - doajarticles::dca64612dfe0963fffc119098a319957 - - - - doajarticles::dd70e44479f0ade25aa106aef3e87a0a - - - - - - discardless - - - - farfish2020 - - - - facts - - - - climefish - - - - proeel - - - - primefish - - - - h2020_vicinaqua - - - - meece - - - - rlsadb - - - - iotc_ctoi - - - - - - - - brain mapping - brain imaging - electroencephalography - arterial spin labelling - brain fingerprinting - brain - neuroimaging - Multimodal Brain Image Analysis - fMRI - neuroinformatics - fetal brain - brain ultrasonic imaging - topographic brain mapping - diffusion tensor imaging - computerized knowledge assessment - connectome mapping - brain magnetic resonance imaging - brain abnormalities - - - - re3data_____::5b9bf9171d92df854cf3c520692e9122 - - - - doajarticles::c7d3de67dc77af72f6747157441252ec - - - - re3data_____::8515794670370f49c1d176c399c714f5 - - - - doajarticles::d640648c84b10d425f96f11c3de468f3 - - - - doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a - - - - rest________::fb1a3d4523c95e63496e3bc7ba36244b - - - - - - neuroinformatics - - - - hbp - - - - from_neuroscience_to_machine_learning - - - - ci2c - - - - opensourcebrain - - - - brainspeak - - - - braincom - - - - nextgenvis - - - - meso-brain - - - - neuroplasticity-workshop - - - - bionics - - - - brainmattrain-676408 - - - - repronim - - - - affectiveneuro - - - - con - - - - lab_neurol_sperim_irfmn_irccs_milano_it - - - - - - - - marine - ocean - fish - aqua - sea - - - - - adriplan - - - - devotes-project - - - - euro-basin - - - - naclim - - - - discardless - - - - assisibf - - - - meece - - - - facts - - - - proeel - - - - aquatrace - - - - myfish - - - - atlas - - - - blue-actionh2020 - - - - sponges - - - - merces_project - - - - bigdataocean - - - - columbus - - - - h2020-aquainvad-ed - - - - aquarius - - - - southern-ocean-observing-system - - - - eawag - - - - mossco - - - - onc - - - - oceanbiogeochemistry - - - - oceanliteracy - - - - openearth - - - - ocean - - - - calcifierraman - - - - bermudabream - - - - brcorp1 - - - - mce - - - - biogeochem - - - - ecc2014 - - - - fisheries - - - - sedinstcjfas - - - - narmada - - - - umr-entropie - - - - farfish2020 - - - - primefish - - - - zf-ilcs - - - - climefish - - - - afrimed_eu - - - - spi-ace - - - - cice-consortium - - - - nemo-ocean - - - - mesopp-h2020 - - - - marxiv - - - - - - - - - - - instruct - - - - west-life - - - - - - - - animal production and health - fisheries and aquaculture - food safety and human nutrition - information management - food technology - agri-food education and extension - natural resources and environment - food system - engineering technology and Research - agriculture - food safety risk assessment - food security - farming practices and systems - plant production and protection - agri-food economics and policy - Agri-food - food distribution - forestry - - - - opendoar____::1a551829d50f1400b0dab21fdd969c04 - - - - opendoar____::49af6c4e558a7569d80eee2e035e2bd7 - - - - opendoar____::0266e33d3f546cb5436a10798e657d97 - - - - opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 - - - - opendoar____::41bfd20a38bb1b0bec75acf0845530a7 - - - - opendoar____::87ae6fb631f7c8a627e8e28785d9992d - - - - - - edenis - - - - efsa-pilot - - - - egene3 - - - - efsa-kj - - - - euromixproject - - - - discardless - - - - sedinstcjfst - - - - afinet-kc - - - - 2231-4784 - - - - 2231-0606 - - - - solace - - - - pa17 - - - - smartakis - - - - sedinstcjae - - - - phenology_camera - - - - aginfra - - - - erosa - - - - bigdatagrapes - - - - - - - - - - opendoar____::7e7757b1e12abcb736ab9a754ffb617a - {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - - - opendoar____::96da2f590cd7246bbde0051047b0d6f7 - {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - - - - - dimpo - - - - - - - - Green Transport - City mobility systems - Vulnerable road users - Traffic engineering - Transport electrification - Mobility - Intermodal freight transport - Clean vehicle fleets - Intelligent mobility - Inflight refueling - District mobility systems - Navigation and control systems for optimised planning and routing - European Space Technology Platform - European Transport networks - Green cars - Inter-modality infrastructures - Advanced Take Off and Landing Ideas - Sustainable urban systems - port-area railway networks - Innovative forms of urban transport - Alliance for Logistics Innovation through Collaboration in Europe - Advisory Council for Aeronautics Research in Europe - Mobility services for people and goods - Guidance and traffic management - Passenger mobility - Smart mobility and services - transport innovation - high-speed railway - Vehicle design - Inland shipping - public transportation - aviation’s climate impact - Road transport - On-demand public transport - Personal Air Transport - Transport - transport vulnerability - Pipeline transport - European Association of Aviation Training and Education Organisations - Defrosting of railway infrastructure - Inclusive and affordable transport - River Information Services - jel:L92 - Increased use of public transport - Seamless mobility - STRIA - trolleybus transport - Intelligent Transport System - Low-emission alternative energy for transport - Shared mobility for people and goods - Business model for urban mobility - Interoperability of transport systems - Cross-border train slot booking - Air transport - Transport pricing - Sustainable transport - European Rail Transport Research Advisory Council - Alternative aircraft configurations - Transport and Mobility - Railways applications - urban transport - Environmental impact of transport - urban freight delivery systems - Automated Road Transport - Alternative fuels in public transport - Active LIDAR-sensor for GHG-measurements - Autonomous logistics operations - Rational use of motorised transport - Network and traffic management systems - electrification of railway wagons - Single European Sky - Electrified road systems - transportation planning - Railway dynamics - Motorway of the Sea - smart railway communications - Maritime transport - Environmental- friendly transport - Combined transport - Connected automated driving technology - Innovative freight logistics services - automated and shared vehicles - Alternative Aircraft Systems - Land-use and transport interaction - Public transport system - Business plan for shared mobility - Shared mobility - Growing of mobility demand - European Road Transport Research Advisory Council - WATERBORNE ETP - Effective transport management system - Short Sea Shipping - air traffic management - Sea hubs and the motorways of the sea - Urban mobility solutions - Smart city planning - Maritime spatial planning - EUropean rail Research Network of Excellence - Transport governance - ENERGY CONSUMPTION BY THE TRANSPORT SECTOR - Integrated urban plan - inland waterway services - European Conference of Transport Research Institutes - air vehicles - E-freight - Automated Driving - Automated ships - pricing for cross-border passenger transport - Vehicle efficiency - Railway transport - Electric vehicles - Road traffic monitoring - Deep sea shipping - Circular economy in transport - Traffic congestion - air transport system - Urban logistics - Rail transport - OpenStreetMap - high speed rail - Transportation engineering - Intermodal travel information - Flight Data Recorders - Advanced driver assistance systems - long distance freight transport - Inland waterway transport - Smart mobility - Mobility integration - Personal Rapid Transit system - Safety measures & requirements for roads - Green rail transport - Electrical - Vehicle manufacturing - Future Airport Layout - Rail technologies - European Intermodal Research Advisory Council - inland navigation - Automated urban vehicles - ECSS-standards - Traveller services - Polluting transport - Air Traffic Control - Cooperative and connected and automated transport - Innovative powertrains - Quality of transport system and services - door-to- door logistics chain - Inter-modal aspects of urban mobility - travel (and mobility) - Innovative freight delivery systems - urban freight delivery infrastructures - - - - doajarticles::1c5bdf8fca58937894ad1441cca99b76 - - - - doajarticles::b37a634324a45c821687e6e80e6f53b4 - - - - doajarticles::4bf64f2a104040e4e055cd9594b2d77c - - - - doajarticles::479ca537c12755d1868bbf02938a900c - - - - doajarticles::55f31df96a60e2309f45b7c265fcf7a2 - - - - doajarticles::c52a09891a5301f9986ebbfe3761810c - - - - doajarticles::379807bc7f6c71a227ef1651462c414c - - - - doajarticles::36069db531a00b85a2e8fb301f4bdc19 - - - - doajarticles::b6a898da311ded96fabf49c520b80d5d - - - - doajarticles::d0753d9180b35a271d8b4a31f449749f - - - - doajarticles::172050a92511838393a3fe237ae47e31 - - - - doajarticles::301ed96c62abb160a3e29796efe5c95c - - - - doajarticles::0f4f805b3d842f2c7f1b077c3426fa59 - - - - doajarticles::ba73728b84437b8d48ae287b867c7215 - - - - doajarticles::86faef424d804309ccf45f692523aa48 - - - - doajarticles::73bd758fa41671de70964c3ecba013af - - - - doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4 - - - - doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7 - - - - doajarticles::ca61df07089acc53a1569bde6673d82a - - - - doajarticles::237dd6f1606600459d0297abd8ed9976 - - - - doajarticles::fba6191177ede7c51ea1cdf58eae7f8b - - - - - - jsdtl - - - - utc-martrec - - - - utc-uti - - - - stp - - - - c2smart - - - - stride-utc - - - - crowd4roads - - - - lemo - - - - imov3d - - - - tra2018 - - - - optimum - - - - stars - - - - iecteim - - - - iccpt2019 - - - - - - - - COVID-19 - Severe acute respiratory syndrome coronavirus 2 - SARS-CoV-2 - COVID19 - 2019 novel coronavirus - coronavirus disease 2019 - HCoV-19 - mesh:C000657245 - 2019-nCoV - coronavirus disease-19 - mesh:COVID-19 - COVID2019 - - - - opendoar____::358aee4cc897452c00244351e4d91f69 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"sars-cov-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - driver______::bee53aa31dc2cbb538c10c2b65fa5824 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - openaire____::437f4b072b1aa198adcbc35910ff3b98 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - openaire____::081b82f96300b6a6e3d282bad31cb6e2 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - openaire____::9e3be59865b2c1c335d32dae2fe7b254 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - opendoar____::8b6dd7db9af49e67306feb59a8bdc52c - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - share_______::4719356ec8d7d55d3feb384ce879ad6c - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - share_______::bbd802baad85d1fd440f32a7a3a2c2b1 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - opendoar____::6f4922f45568161a8cdf4ad2299f6d23 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} - - - - re3data_____::7980778c78fb4cf0fab13ce2159030dc - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} - - - re3data_____::978378def740bbf2bfb420de868c460b - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} - - - - - chicago-covid-19 - - - - covid-19-senacyt-panama-sample - - - - covid-19-tx-rct-stats-review - - - - covid_19_senacyt_abc_panama - - - - - - \ No newline at end of file + Result: 2 + + + + + Result: 3 + + + + + Result: 4 + + + + re3data_____::a507cdacc5bbcc08761c92185dee5cab + + + + + + Result: 5 + + + + + rda + + + + + Result: 6 + + SDG13 - Climate action + SDG8 - Decent work and economic growth + SDG15 - Life on land + SDG2 - Zero hunger + SDG17 - Partnerships for the goals + SDG10 - Reduced inequalities + SDG5 - Gender equality + SDG12 - Responsible consumption and production + SDG14 - Life below water + SDG6 - Clean water and sanitation + SDG11 - Sustainable cities and communities + SDG1 - No poverty + SDG3 - Good health and well being + SDG7 - Affordable and clean energy + SDG4 - Quality education + SDG9 - Industry innovation and infrastructure + SDG16 - Peace justice and strong institutions + + + + + Result: 7 + + modern art + monuments + europeana data model + sites + field walking + frescoes + LIDO metadata schema + art history + excavation + Arts and Humanities General + cities + coins + temples + numismatics + lithics + roads + environmental archaeology + digital cultural heritage + archaeological reports + history + CRMba + churches + cultural heritage + archaeological stratigraphy + religious art + buidings + digital humanities + survey + archaeological sites + linguistic studies + bioarchaeology + architectural orders + palaeoanthropology + fine arts + europeana + CIDOC CRM + decorations + classic art + stratigraphy + digital archaeology + intangible cultural heritage + walls + humanities + chapels + CRMtex + Language and Literature + paintings + archaeology + fair data + mosaics + burials + architecture + medieval art + castles + CARARE metadata schema + statues + natural language processing + inscriptions + CRMsci + vaults + contemporary art + Arts and Humanities + CRMarchaeo + pottery + site + architectural + vessels + + + + re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 + + + + doajarticles::c6cd4b532e12868c1d760a8d7cda6815 + + + + doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b + + + + doajarticles::6eb31d13b12bc06bbac06aef63cf33c9 + + + + doajarticles::0da84e9dfdc8419576169e027baa8028 + + + + re3data_____::84e123776089ce3c7a33db98d9cd15a8 + + + + openaire____::c5502a43e76feab55dd00cf50f519125 + + + + re3data_____::a48f09c562b247a9919acfe195549b47 + + + + opendoar____::97275a23ca44226c9964043c8462be96 + + + + + + storm + + + + crosscult + + + + wholodance_eu + + + + digcur2013 + + + + gravitate + + + + dipp2014 + + + + digitalhumanities + + + + dimpo + + + + adho + + + + chc + + + + wahr + + + + ibe + + + + ariadne + + + + parthenos-hub + + + + parthenos-training + + + + gandhara + + + + cmsouthasia + + + + nilgirihills + + + + shamsa_mustecio + + + + bodhgaya + + + + + Result: 8 + + Stock Assessment + pelagic + Acoustic + Fish farming + Fisheries + Fishermen + maximum sustainable yield + trawler + Fishing vessel + Fisherman + Fishing gear + mackerel + RFMO + Fish Aggregating Device + Bycatch + Fishery + common fisheries policy + Fishing fleet + Aquaculture + + + + doajarticles::8cec81178926caaca531afbd8eb5d64c + + + + doajarticles::0f7a7f30b5400615cae1829f3e743982 + + + + doajarticles::9740f7f5af3e506d2ad2c215cdccd51a + + + + doajarticles::9f3fbaae044fa33cb7069b72935a3254 + + + + doajarticles::cb67f33eb9819f5c624ce0313957f6b3 + + + + doajarticles::e21c97cbb7a209afc75703681c462906 + + + + doajarticles::554cde3be9e5c4588b4c4f9f503120cb + + + + tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b + + + + doajarticles::57c5d3837da943e93b28ec4db82ec7a5 + + + + doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1 + + + + doajarticles::e21c97cbb7a209afc75703681c462906 + + + + doajarticles::dca64612dfe0963fffc119098a319957 + + + + doajarticles::dd70e44479f0ade25aa106aef3e87a0a + + + + + + discardless + + + + farfish2020 + + + + facts + + + + climefish + + + + proeel + + + + primefish + + + + h2020_vicinaqua + + + + meece + + + + rlsadb + + + + iotc_ctoi + + + + + Result: 9 + + brain mapping + brain imaging + electroencephalography + arterial spin labelling + brain fingerprinting + brain + neuroimaging + Multimodal Brain Image Analysis + fMRI + neuroinformatics + fetal brain + brain ultrasonic imaging + topographic brain mapping + diffusion tensor imaging + computerized knowledge assessment + connectome mapping + brain magnetic resonance imaging + brain abnormalities + + + + re3data_____::5b9bf9171d92df854cf3c520692e9122 + + + + doajarticles::c7d3de67dc77af72f6747157441252ec + + + + re3data_____::8515794670370f49c1d176c399c714f5 + + + + doajarticles::d640648c84b10d425f96f11c3de468f3 + + + + doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a + + + + rest________::fb1a3d4523c95e63496e3bc7ba36244b + + + + + + neuroinformatics + + + + hbp + + + + from_neuroscience_to_machine_learning + + + + ci2c + + + + opensourcebrain + + + + brainspeak + + + + braincom + + + + nextgenvis + + + + meso-brain + + + + neuroplasticity-workshop + + + + bionics + + + + brainmattrain-676408 + + + + repronim + + + + affectiveneuro + + + + con + + + + lab_neurol_sperim_irfmn_irccs_milano_it + + + + + Result: 10 + + marine + ocean + fish + aqua + sea + + + + + adriplan + + + + devotes-project + + + + euro-basin + + + + naclim + + + + discardless + + + + assisibf + + + + meece + + + + facts + + + + proeel + + + + aquatrace + + + + myfish + + + + atlas + + + + blue-actionh2020 + + + + sponges + + + + merces_project + + + + bigdataocean + + + + columbus + + + + h2020-aquainvad-ed + + + + aquarius + + + + southern-ocean-observing-system + + + + eawag + + + + mossco + + + + onc + + + + oceanbiogeochemistry + + + + oceanliteracy + + + + openearth + + + + ocean + + + + calcifierraman + + + + bermudabream + + + + brcorp1 + + + + mce + + + + biogeochem + + + + ecc2014 + + + + fisheries + + + + sedinstcjfas + + + + narmada + + + + umr-entropie + + + + farfish2020 + + + + primefish + + + + zf-ilcs + + + + climefish + + + + afrimed_eu + + + + spi-ace + + + + cice-consortium + + + + nemo-ocean + + + + mesopp-h2020 + + + + marxiv + + + + + Result: 11 + + + + + instruct + + + + west-life + + + + + Result: 12 + + + + + Result: 13 + + animal production and health + fisheries and aquaculture + food safety and human nutrition + information management + food technology + agri-food education and extension + natural resources and environment + food system + engineering technology and Research + agriculture + food safety risk assessment + food security + farming practices and systems + plant production and protection + agri-food economics and policy + Agri-food + food distribution + forestry + + + + opendoar____::1a551829d50f1400b0dab21fdd969c04 + + + + opendoar____::49af6c4e558a7569d80eee2e035e2bd7 + + + + opendoar____::0266e33d3f546cb5436a10798e657d97 + + + + opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 + + + + opendoar____::41bfd20a38bb1b0bec75acf0845530a7 + + + + opendoar____::87ae6fb631f7c8a627e8e28785d9992d + + + + + + edenis + + + + efsa-pilot + + + + egene3 + + + + efsa-kj + + + + euromixproject + + + + discardless + + + + sedinstcjfst + + + + afinet-kc + + + + 2231-4784 + + + + 2231-0606 + + + + solace + + + + pa17 + + + + smartakis + + + + sedinstcjae + + + + phenology_camera + + + + aginfra + + + + erosa + + + + bigdatagrapes + + + + + Result: 14 + + + + opendoar____::7e7757b1e12abcb736ab9a754ffb617a + {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} + + + opendoar____::96da2f590cd7246bbde0051047b0d6f7 + {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} + + + + + dimpo + + + + + Result: 15 + + + + + Result: 16 + + + + + Result: 17 + + Green Transport + City mobility systems + Vulnerable road users + Traffic engineering + Transport electrification + Mobility + Intermodal freight transport + Clean vehicle fleets + Intelligent mobility + Inflight refueling + District mobility systems + Navigation and control systems for optimised planning and routing + European Space Technology Platform + European Transport networks + Green cars + Inter-modality infrastructures + Advanced Take Off and Landing Ideas + Sustainable urban systems + port-area railway networks + Innovative forms of urban transport + Alliance for Logistics Innovation through Collaboration in Europe + Advisory Council for Aeronautics Research in Europe + Mobility services for people and goods + Guidance and traffic management + Passenger mobility + Smart mobility and services + transport innovation + high-speed railway + Vehicle design + Inland shipping + public transportation + aviation’s climate impact + Road transport + On-demand public transport + Personal Air Transport + Transport + transport vulnerability + Pipeline transport + European Association of Aviation Training and Education Organisations + Defrosting of railway infrastructure + Inclusive and affordable transport + River Information Services + jel:L92 + Increased use of public transport + Seamless mobility + STRIA + trolleybus transport + Intelligent Transport System + Low-emission alternative energy for transport + Shared mobility for people and goods + Business model for urban mobility + Interoperability of transport systems + Cross-border train slot booking + Air transport + Transport pricing + Sustainable transport + European Rail Transport Research Advisory Council + Alternative aircraft configurations + Transport and Mobility + Railways applications + urban transport + Environmental impact of transport + urban freight delivery systems + Automated Road Transport + Alternative fuels in public transport + Active LIDAR-sensor for GHG-measurements + Autonomous logistics operations + Rational use of motorised transport + Network and traffic management systems + electrification of railway wagons + Single European Sky + Electrified road systems + transportation planning + Railway dynamics + Motorway of the Sea + smart railway communications + Maritime transport + Environmental- friendly transport + Combined transport + Connected automated driving technology + Innovative freight logistics services + automated and shared vehicles + Alternative Aircraft Systems + Land-use and transport interaction + Public transport system + Business plan for shared mobility + Shared mobility + Growing of mobility demand + European Road Transport Research Advisory Council + WATERBORNE ETP + Effective transport management system + Short Sea Shipping + air traffic management + Sea hubs and the motorways of the sea + Urban mobility solutions + Smart city planning + Maritime spatial planning + EUropean rail Research Network of Excellence + Transport governance + ENERGY CONSUMPTION BY THE TRANSPORT SECTOR + Integrated urban plan + inland waterway services + European Conference of Transport Research Institutes + air vehicles + E-freight + Automated Driving + Automated ships + pricing for cross-border passenger transport + Vehicle efficiency + Railway transport + Electric vehicles + Road traffic monitoring + Deep sea shipping + Circular economy in transport + Traffic congestion + air transport system + Urban logistics + Rail transport + OpenStreetMap + high speed rail + Transportation engineering + Intermodal travel information + Flight Data Recorders + Advanced driver assistance systems + long distance freight transport + Inland waterway transport + Smart mobility + Mobility integration + Personal Rapid Transit system + Safety measures & requirements for roads + Green rail transport + Electrical + Vehicle manufacturing + Future Airport Layout + Rail technologies + European Intermodal Research Advisory Council + inland navigation + Automated urban vehicles + ECSS-standards + Traveller services + Polluting transport + Air Traffic Control + Cooperative and connected and automated transport + Innovative powertrains + Quality of transport system and services + door-to- door logistics chain + Inter-modal aspects of urban mobility + travel (and mobility) + Innovative freight delivery systems + urban freight delivery infrastructures + + + + doajarticles::1c5bdf8fca58937894ad1441cca99b76 + + + + doajarticles::b37a634324a45c821687e6e80e6f53b4 + + + + doajarticles::4bf64f2a104040e4e055cd9594b2d77c + + + + doajarticles::479ca537c12755d1868bbf02938a900c + + + + doajarticles::55f31df96a60e2309f45b7c265fcf7a2 + + + + doajarticles::c52a09891a5301f9986ebbfe3761810c + + + + doajarticles::379807bc7f6c71a227ef1651462c414c + + + + doajarticles::36069db531a00b85a2e8fb301f4bdc19 + + + + doajarticles::b6a898da311ded96fabf49c520b80d5d + + + + doajarticles::d0753d9180b35a271d8b4a31f449749f + + + + doajarticles::172050a92511838393a3fe237ae47e31 + + + + doajarticles::301ed96c62abb160a3e29796efe5c95c + + + + doajarticles::0f4f805b3d842f2c7f1b077c3426fa59 + + + + doajarticles::ba73728b84437b8d48ae287b867c7215 + + + + doajarticles::86faef424d804309ccf45f692523aa48 + + + + doajarticles::73bd758fa41671de70964c3ecba013af + + + + doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4 + + + + doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7 + + + + doajarticles::ca61df07089acc53a1569bde6673d82a + + + + doajarticles::237dd6f1606600459d0297abd8ed9976 + + + + doajarticles::fba6191177ede7c51ea1cdf58eae7f8b + + + + + + jsdtl + + + + utc-martrec + + + + utc-uti + + + + stp + + + + c2smart + + + + stride-utc + + + + crowd4roads + + + + lemo + + + + imov3d + + + + tra2018 + + + + optimum + + + + stars + + + + iecteim + + + + iccpt2019 + + + + + Result: 18 + + + + + Result: 19 + + + + + Result: 20 + + + + + Result: 21 + + Sustainability-oriented science policy + STI policies + science—society relations + Science & Technology Policy + Innovation policy + science policy + Policy and Law + + + + doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d + + + + doajarticles::ae4c7286c79590f19fdca670156ce816 + + + + doajarticles::0f664bce92ce953e0c7a92068c46bfb3 + + + + doajarticles::00017183dc4c858fb77541985323a4ef + + + + doajarticles::93b306f458cce3d7aaaf58c0a725f4f9 + + + + doajarticles::9dbf8fbf3e9fe0fe1fc01e55fbd90bfc + + + + doajarticles::a2bda8785c863279bba4b8f34827b4c9 + + + + doajarticles::019a1fcb42c3fea1c1b689df76330b58 + + + + doajarticles::0daa8281938831e9c82bfed8b55a2975 + + + + doajarticles::f67ad6d268162079b3abd51a24468744 + + + + doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d + + + + doajarticles::ad114356e196a4a3d84dda59c720dacd + + + + doajarticles::01e8a54fdecaaf354c67a2dd74ae7d4f + + + + doajarticles::449305f096b10a9464449ff2d0e10e06 + + + + doajarticles::982c0c0ac378256254cce2fa6572bb6c + + + + doajarticles::49d6ed47138884566ce93cf0ccb12c02 + + + + doajarticles::a98e820dbc2e8ee0fc84ab66f263267c + + + + doajarticles::50b1ce37427b36368f8f0f1317e47f83 + + + + doajarticles::f0ec29b7450b2ac5d0ad45327eeb531a + + + + doajarticles::d8d421d3b0349a7aaa93758b27a54e84 + + + + doajarticles::7ffc35ac5133da01d421ccf8af5b70bc + + + + + + risis + + + + + Result: 22 + + COVID-19 + Severe acute respiratory syndrome coronavirus 2 + SARS-CoV-2 + COVID19 + 2019 novel coronavirus + coronavirus disease 2019 + HCoV-19 + mesh:C000657245 + 2019-nCoV + coronavirus disease-19 + mesh:COVID-19 + COVID2019 + + + + opendoar____::358aee4cc897452c00244351e4d91f69 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + driver______::bee53aa31dc2cbb538c10c2b65fa5824 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::437f4b072b1aa198adcbc35910ff3b98 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::081b82f96300b6a6e3d282bad31cb6e2 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + openaire____::9e3be59865b2c1c335d32dae2fe7b254 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + opendoar____::8b6dd7db9af49e67306feb59a8bdc52c + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + share_______::4719356ec8d7d55d3feb384ce879ad6c + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + share_______::bbd802baad85d1fd440f32a7a3a2c2b1 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + opendoar____::6f4922f45568161a8cdf4ad2299f6d23 + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + + + + re3data_____::7980778c78fb4cf0fab13ce2159030dc + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + + + re3data_____::978378def740bbf2bfb420de868c460b + {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + + + + + chicago-covid-19 + + + + covid-19-senacyt-panama-sample + + + + covid-19-tx-rct-stats-review + + + + covid_19_senacyt_abc_panama + + + + + + From 8802e4126baeaafe78df05111c7c8aca9f1f1d1f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:26:55 +0200 Subject: [PATCH 164/259] re-implemented inverting the couple: from (projectId, relatedResultList) to (resultId, relatedProjectList) --- .../PrepareProjectResultsAssociation.java | 65 ++++++----- .../dhp/projecttoresult/ProjectResultSet.java | 25 ----- .../dhp/projecttoresult/ResultProjectSet.java | 25 +++++ ...SparkResultToProjectThroughSemRelJob3.java | 102 +++++++++--------- .../potentialUpdates_1.json.gz | Bin 357 -> 0 bytes 5 files changed, 112 insertions(+), 105 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates_1.json.gz diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index 2e9255ed5..cf970048d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -12,6 +12,7 @@ import eu.dnetlib.dhp.schema.oaf.Relation; import java.util.Arrays; import java.util.List; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; @@ -95,45 +96,57 @@ public class PrepareProjectResultsAssociation { resproj_relation.createOrReplaceTempView("resproj_relation"); query = - "SELECT projectId, collect_set(resId) resultSet " - + "FROM (" - + " SELECT r1.target resId, r2.target projectId " + "SELECT resultId, collect_set(projectId) projectSet " + + "FROM ( " + + "SELECT r1.target resultId, r2.target projectId " + " FROM (SELECT source, target " + " FROM relation " + " WHERE datainfo.deletedbyinference = false " + getConstraintList(" relClass = '", allowedsemrel) - + ") r1" + + " ) r1" + " JOIN resproj_relation r2 " + " ON r1.source = r2.source " + " ) tmp " - + "GROUP BY projectId "; + + "GROUP BY resultId "; + // query = + // "SELECT projectId, collect_set(resId) resultSet " + // + "FROM (" + // + " SELECT r1.target resId, r2.target projectId " + // + " FROM (SELECT source, target " + // + " FROM relation " + // + " WHERE datainfo.deletedbyinference = false " + // + getConstraintList(" relClass = '", allowedsemrel) + // + ") r1" + // + " JOIN resproj_relation r2 " + // + " ON r1.source = r2.source " + // + " ) tmp " + // + "GROUP BY projectId "; spark.sql(query) - .as(Encoders.bean(ProjectResultSet.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(potentialUpdatePath); - // .toJavaRDD() - // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - // .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + .as(Encoders.bean(ResultProjectSet.class)) + // .toJSON() + // .write() + // .mode(SaveMode.Overwrite) + // .option("compression", "gzip") + // .text(potentialUpdatePath); + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(potentialUpdatePath, GzipCodec.class); query = - "SELECT target projectId, collect_set(source) resultSet " + "SELECT source resultId, collect_set(target) projectSet " + "FROM resproj_relation " - + "GROUP BY target"; + + "GROUP BY source"; spark.sql(query) - .as(Encoders.bean(ProjectResultSet.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(alreadyLinkedPath); - // .toJavaRDD() - // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - // .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); - + .as(Encoders.bean(ResultProjectSet.class)) + // .toJSON() + // .write() + // .mode(SaveMode.Overwrite) + // .option("compression", "gzip") + // .text(alreadyLinkedPath); + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java deleted file mode 100644 index da1be3b69..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ProjectResultSet.java +++ /dev/null @@ -1,25 +0,0 @@ -package eu.dnetlib.dhp.projecttoresult; - -import java.io.Serializable; -import java.util.ArrayList; - -public class ProjectResultSet implements Serializable { - private String projectId; - private ArrayList resultSet; - - public String getProjectId() { - return projectId; - } - - public void setProjectId(String projectId) { - this.projectId = projectId; - } - - public ArrayList getResultSet() { - return resultSet; - } - - public void setResultSet(ArrayList resultSet) { - this.resultSet = resultSet; - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java new file mode 100644 index 000000000..183ae1489 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java @@ -0,0 +1,25 @@ +package eu.dnetlib.dhp.projecttoresult; + +import java.io.Serializable; +import java.util.ArrayList; + +public class ResultProjectSet implements Serializable { + private String resultId; + private ArrayList projectSet; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public ArrayList getProjectSet() { + return projectSet; + } + + public void setProjectSet(ArrayList project) { + this.projectSet = project; + } +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index 45d697651..e32242a90 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -9,6 +9,7 @@ import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; import eu.dnetlib.dhp.schema.oaf.Relation; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; @@ -44,9 +45,6 @@ public class SparkResultToProjectThroughSemRelJob3 { final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); - final Boolean writeUpdates = Boolean.valueOf(parser.get("writeUpdate")); - log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); log.info("saveGraph: {}", saveGraph); @@ -60,12 +58,7 @@ public class SparkResultToProjectThroughSemRelJob3 { removeOutputDir(spark, outputPath); } execPropagation( - spark, - outputPath, - alreadyLinkedPath, - potentialUpdatePath, - writeUpdates, - saveGraph); + spark, outputPath, alreadyLinkedPath, potentialUpdatePath, saveGraph); }); } @@ -74,21 +67,12 @@ public class SparkResultToProjectThroughSemRelJob3 { String outputPath, String alreadyLinkedPath, String potentialUpdatePath, - Boolean writeUpdate, Boolean saveGraph) { - Dataset toaddrelations = - readAssocProjectResults(spark, potentialUpdatePath); - Dataset alreadyLinked = readAssocProjectResults(spark, alreadyLinkedPath); + Dataset toaddrelations = + readAssocResultProjects(spark, potentialUpdatePath); + Dataset alreadyLinked = readAssocResultProjects(spark, alreadyLinkedPath); - if (writeUpdate) { - toaddrelations - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath + "/potential_updates"); - } if (saveGraph) { getNewRelations(alreadyLinked, toaddrelations) .toJSON() @@ -100,56 +84,66 @@ public class SparkResultToProjectThroughSemRelJob3 { } private static Dataset getNewRelations( - Dataset alreadyLinked, Dataset toaddrelations) { + Dataset alreadyLinked, Dataset toaddrelations) { return toaddrelations .joinWith( alreadyLinked, - toaddrelations.col("projectId").equalTo(alreadyLinked.col("projectId")), - "left") + toaddrelations.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") .flatMap( value -> { List new_relations = new ArrayList<>(); - ProjectResultSet potential_update = value._1(); - ProjectResultSet already_linked = value._2(); - String projId = already_linked.getProjectId(); - potential_update.getResultSet().stream() + ResultProjectSet potential_update = value._1(); + Optional already_linked = + Optional.ofNullable(value._2()); + if (already_linked.isPresent()) { + already_linked.get().getProjectSet().stream() + .forEach( + (p -> { + if (potential_update + .getProjectSet() + .contains(p)) { + potential_update.getProjectSet().remove(p); + } + })); + } + String resId = potential_update.getResultId(); + potential_update.getProjectSet().stream() .forEach( - rId -> { - if (!already_linked.getResultSet().contains(rId)) { - new_relations.add( - getRelation( - rId, - projId, - RELATION_RESULT_PROJECT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - new_relations.add( - getRelation( - projId, - rId, - RELATION_PROJECT_RESULT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - } + pId -> { + new_relations.add( + getRelation( + resId, + pId, + RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + new_relations.add( + getRelation( + pId, + resId, + RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); }); return new_relations.iterator(); }, Encoders.bean(Relation.class)); } - private static Dataset readAssocProjectResults( + private static Dataset readAssocResultProjects( SparkSession spark, String potentialUpdatePath) { return spark.read() .textFile(potentialUpdatePath) .map( - value -> OBJECT_MAPPER.readValue(value, ProjectResultSet.class), - Encoders.bean(ProjectResultSet.class)); + value -> OBJECT_MAPPER.readValue(value, ResultProjectSet.class), + Encoders.bean(ResultProjectSet.class)); } } diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates_1.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates_1.json.gz deleted file mode 100644 index 0280131de51a3d4d3923e3208eaae0b6ee356f2c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 357 zcmV-r0h<0FiwFoMewAJT18{G2Wo~q7VQf`!WMOn=b6+toYIARH0Bw;yZX7WTM(;9p z3W}6SX>Q=!rC}_@KLp0YyI8N`0I~0G1wpEe3O3^7Cq4e&uD8oe*!{t`eXH;}UvH;> zXWvPqNvByv7-u7|1&52v7$Zm0_G7!n+xz+c824>|+8TU%`1|$rktcw%@0Es50Km%5 z0Xo#Yw;bX!#+4I%W18G=nKN^$gp;#Mma%Gt946WbA! zT!~eYEu*m__SGJhEV@-`{V(%oKQHI_bv<8hvG2VGMUp}hjtbHXHZ%19aH?(1Dv%JdTBh!G~770m%u6jvrEtTAKhENF*Jfgorw!q5dGaB?O|*Pb+N z9(u!UgV|b1#gu{!>z;+7hyHPYXRJ%4=zT@(#rR{{V4 Di3+4$ From 95a54d54605d83e905c75007a9636b3b589c6502 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:30:32 +0200 Subject: [PATCH 165/259] removed the writeUpdate option. The update is available in the preparedInfo path --- .../SparkCountryPropagationJob2.java | 56 +------------------ 1 file changed, 1 insertion(+), 55 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 37d4a5271..cd37e79eb 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -50,16 +50,6 @@ public class SparkCountryPropagationJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = - resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); - - final Boolean writeUpdates = - Optional.ofNullable(parser.get("writeUpdate")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = Optional.ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) @@ -76,17 +66,12 @@ public class SparkCountryPropagationJob2 { conf, isSparkSessionManaged, spark -> { - // createOutputDirs(outputPath, - // FileSystem.get(spark.sparkContext().hadoopConfiguration())); - removeOutputDir(spark, outputPath); execPropagation( spark, datasourcecountrypath, inputPath, outputPath, resultClazz, - resultType, - writeUpdates, saveGraph); }); } @@ -97,12 +82,10 @@ public class SparkCountryPropagationJob2 { String inputPath, String outputPath, Class resultClazz, - String resultType, - boolean writeUpdates, boolean saveGraph) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - // Load parque file with preprocessed association datasource - country + // Load file with preprocessed association datasource - country Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); // broadcasting the result of the preparation step @@ -114,10 +97,6 @@ public class SparkCountryPropagationJob2 { spark, inputPath, resultClazz, broadcast_datasourcecountryassoc) .as(Encoders.bean(ResultCountrySet.class)); - if (writeUpdates) { - writeUpdates(potentialUpdates, outputPath + "/update_" + resultType); - } - if (saveGraph) { updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); } @@ -138,11 +117,6 @@ public class SparkCountryPropagationJob2 { r -> new Tuple2<>(r.getId(), r), Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); - // Dataset> potential_update_pair = - // potentialUpdates.map(pu -> new Tuple2<>(pu.getResultId(), - // pu), - // Encoders.tuple(Encoders.STRING(), Encoders.bean(ResultCountrySet.class))); - Dataset new_table = result_pair .joinWith( @@ -184,10 +158,6 @@ public class SparkCountryPropagationJob2 { log.info("Saving graph table to path: {}", outputPath); // log.info("number of saved recordsa: {}", new_table.count()); new_table.toJSON().write().option("compression", "gzip").text(outputPath); - // .toJavaRDD() - // .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - // .saveAsTextFile(outputPath , GzipCodec.class); - } private static Dataset getPotentialResultToUpdate( @@ -203,18 +173,6 @@ public class SparkCountryPropagationJob2 { return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); } - // private static void createCfHbforresult(SparkSession spark) { - // String query; - // query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + - // "FROM ( SELECT id, instance " + - // "FROM result " + - // " WHERE datainfo.deletedbyinference = false) ds " + - // "LATERAL VIEW EXPLODE(instance) i AS inst"; - // Dataset cfhb = spark.sql(query); - // cfhb.createOrReplaceTempView("cfhb"); - // //log.info("cfhb_number : {}", cfhb.count()); - // } - private static Dataset countryPropagationAssoc( SparkSession spark, Broadcast> broadcast_datasourcecountryassoc) { @@ -248,16 +206,4 @@ public class SparkCountryPropagationJob2 { value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), Encoders.bean(DatasourceCountry.class)); } - - private static void writeUpdates( - Dataset potentialUpdates, String outputPath) { - potentialUpdates - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - // map(u -> OBJECT_MAPPER.writeValueAsString(u)) - // .saveAsTextFile(outputPath, GzipCodec.class); - } } From e000754c92cb679518bb086b189b2306559881d0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:34:03 +0200 Subject: [PATCH 166/259] refactoring --- .../PrepareResultCommunitySetStep1.java | 118 ++-- .../PrepareResultCommunitySetStep2.java | 93 ++- ...arkResultToCommunityThroughSemRelJob4.java | 555 ++++-------------- 3 files changed, 226 insertions(+), 540 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index d334e518d..cbb9b580e 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -1,5 +1,8 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; import eu.dnetlib.dhp.QueryInformationSystem; @@ -7,6 +10,8 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; +import java.util.Arrays; +import java.util.List; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -18,23 +23,18 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class PrepareResultCommunitySetStep1 { private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep1.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySetStep1.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareResultCommunitySetStep1.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -62,30 +62,47 @@ public class PrepareResultCommunitySetStep1 { final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); log.info("communityIdList: {}", new Gson().toJson(communityIdList)); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + final String resultType = + resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); + Class resultClazz = + (Class) Class.forName(resultClassName); - Class resultClazz = (Class) Class.forName(resultClassName); - - - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - prepareInfo(spark, inputPath, outputPath, allowedsemrel, resultClazz, resultType, + prepareInfo( + spark, + inputPath, + outputPath, + allowedsemrel, + resultClazz, + resultType, communityIdList); }); } - private static void prepareInfo(SparkSession spark, String inputPath, String outputPath, - List allowedsemrel, Class resultClazz, String resultType, - List communityIdList) { - //read the relation table and the table related to the result it is using + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + List allowedsemrel, + Class resultClazz, + String resultType, + List communityIdList) { + // read the relation table and the table related to the result it is using final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset( + sc.textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); relation.createOrReplaceTempView("relation"); log.info("Reading Graph table from: {}", inputPath + "/" + resultType); @@ -93,16 +110,18 @@ public class PrepareResultCommunitySetStep1 { result.createOrReplaceTempView("result"); - getPossibleResultcommunityAssociation(spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); - + getPossibleResultcommunityAssociation( + spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); } - private static void getPossibleResultcommunityAssociation(SparkSession spark, List allowedsemrel, String outputPath, - List communityIdList) { + private static void getPossibleResultcommunityAssociation( + SparkSession spark, + List allowedsemrel, + String outputPath, + List communityIdList) { String communitylist = getConstraintList(" co.id = '", communityIdList); - String semrellist = getConstraintList(" relClass = '", allowedsemrel ); - + String semrellist = getConstraintList(" relClass = '", allowedsemrel); /* associates to each result the set of community contexts they are associated to @@ -115,33 +134,38 @@ public class PrepareResultCommunitySetStep1 { associates to each target of a relation with allowed semantics the set of community context it could possibly inherit from the source of the relation */ - String query = "Select target resultId, community_context " + - "from (select id, collect_set(co.id) community_context " + - " from result " + - " lateral view explode (context) c as co " + - " where datainfo.deletedbyinference = false "+ communitylist + - " group by id) p " + - "JOIN " + - "(select source, target " + - "from relation " + - "where datainfo.deletedbyinference = false " + semrellist + ") r " + - "ON p.id = r.source"; + String query = + "Select target resultId, community_context " + + "from (select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false " + + communitylist + + " group by id) p " + + "JOIN " + + "(select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + + semrellist + + ") r " + + "ON p.id = r.source"; - org.apache.spark.sql.Dataset result_context = spark.sql( query); + org.apache.spark.sql.Dataset result_context = spark.sql(query); result_context.createOrReplaceTempView("result_context"); - //( target, (mes, dh-ch-, ni)) + // ( target, (mes, dh-ch-, ni)) /* a dataset for example could be linked to more than one publication. For each publication linked to that dataset - the previous query will produce a row: targetId set of community context the te=arget could possibly inherit + the previous query will produce a row: targetId set of community context the target could possibly inherit with the following query there will be a single row for each result linked to more than one result of the result type currently being used */ - query = "select resultId , collect_set(co) communityList " + - "from result_context " + - "lateral view explode (community_context) c as co " + - "where length(co) > 0 " + - "group by resultId"; + query = + "select resultId , collect_set(co) communityList " + + "from result_context " + + "lateral view explode (community_context) c as co " + + "where length(co) > 0 " + + "group by resultId"; spark.sql(query) .as(Encoders.bean(ResultCommunityList.class)) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java index 6efb2205b..3579db9e6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -1,8 +1,13 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.ResultOrcidList; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import java.util.HashSet; +import java.util.Set; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -11,27 +16,20 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; import scala.Tuple2; -import java.util.HashSet; -import java.util.Set; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - - public class PrepareResultCommunitySetStep2 { private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep2.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultCommunitySetStep2.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_prepareresulttocommunity2_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareResultCommunitySetStep2.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -46,60 +44,61 @@ public class PrepareResultCommunitySetStep2 { SparkConf conf = new SparkConf(); - runWithSparkSession(conf, isSparkSessionManaged, + runWithSparkSession( + conf, + isSparkSessionManaged, spark -> { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - mergeInfo(spark, inputPath, outputPath); + mergeInfo(spark, inputPath, outputPath); }); - } private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssocCommunityList = readResultCommunityList(spark, inputPath + "/publication") - .union(readResultCommunityList(spark, inputPath + "/dataset")) - .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) - .union(readResultCommunityList(spark, inputPath + "/software")); + Dataset resultOrcidAssocCommunityList = + readResultCommunityList(spark, inputPath + "/publication") + .union(readResultCommunityList(spark, inputPath + "/dataset")) + .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) + .union(readResultCommunityList(spark, inputPath + "/software")); resultOrcidAssocCommunityList .toJavaRDD() .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) - .reduceByKey((a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } - Set community_set = new HashSet<>(); + .reduceByKey( + (a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set community_set = new HashSet<>(); - a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); + a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); - b.getCommunityList().stream().forEach(aa -> { - if (!community_set.contains(aa)) { - a.getCommunityList().add(aa); - community_set.add(aa); - } - }); - return a; - }) + b.getCommunityList().stream() + .forEach( + aa -> { + if (!community_set.contains(aa)) { + a.getCommunityList().add(aa); + community_set.add(aa); + } + }); + return a; + }) .map(c -> c._2()) .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(outputPath, GzipCodec.class); } - private static Dataset readResultCommunityList(SparkSession spark, String relationPath) { - return spark - .read() + private static Dataset readResultCommunityList( + SparkSession spark, String relationPath) { + return spark.read() .textFile(relationPath) - .map(value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), Encoders.bean(ResultCommunityList.class)); + .map( + value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), + Encoders.bean(ResultCommunityList.class)); } - - - - - } - diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java index 0e39090dd..4e72fac27 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java @@ -1,473 +1,136 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.QueryInformationSystem; -import eu.dnetlib.dhp.TypedRow; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import scala.Tuple2; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ximpleware.extended.xpath.parser; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.oaf.*; import java.util.*; import java.util.stream.Collectors; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static eu.dnetlib.dhp.PropagationConstant.*; +public class SparkResultToCommunityThroughSemRelJob4 { + + private static final Logger log = + LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob4.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); -public class SparkResultToCommunityThroughSemRelJob3 { public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = new ArgumentApplicationParser(IOUtils - .toString(SparkResultToCommunityThroughSemRelJob3.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_propagationresultcommunityfromsemrel_parameters.json"))); + String jsonConfiguration = + IOUtils.toString( + SparkResultToCommunityThroughSemRelJob4.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String preparedInfoPath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", preparedInfoPath); + + SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final SparkSession spark = SparkSession - .builder() - .appName(SparkResultToCommunityThroughSemRelJob3.class.getSimpleName()) - .master(parser.get("master")) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String outputPath = "/tmp/provision/propagation/resulttocommunityfromsemrel"; + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - final List communityIdList = QueryInformationSystem.getCommunityList(parser.get("isLookupUrl")); + Class resultClazz = + (Class) Class.forName(resultClassName); - createOutputDirs(outputPath, FileSystem.get(spark.sparkContext().hadoopConfiguration())); - - - JavaRDD publication_rdd = sc.textFile(inputPath + "/publication") - .map(item -> new ObjectMapper().readValue(item, Publication.class)); - - JavaRDD dataset_rdd = sc.textFile(inputPath + "/dataset") - .map(item -> new ObjectMapper().readValue(item, Dataset.class)); - - JavaRDD orp_rdd = sc.textFile(inputPath + "/otherresearchproduct") - .map(item -> new ObjectMapper().readValue(item, OtherResearchProduct.class)); - - JavaRDD software_rdd = sc.textFile(inputPath + "/software") - .map(item -> new ObjectMapper().readValue(item, Software.class)); - - JavaRDD relation_rdd = sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)); - - - org.apache.spark.sql.Dataset publication = spark.createDataset(publication_rdd.rdd(), - Encoders.bean(Publication.class)); - - org.apache.spark.sql.Dataset relation = spark.createDataset(relation_rdd.rdd(), - Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset dataset = spark.createDataset(dataset_rdd.rdd(), - Encoders.bean(Dataset.class)); - - org.apache.spark.sql.Dataset other = spark.createDataset(orp_rdd.rdd(), - Encoders.bean(OtherResearchProduct.class)); - - org.apache.spark.sql.Dataset software = spark.createDataset(software_rdd.rdd(), - Encoders.bean(Software.class)); - - - publication.createOrReplaceTempView("publication"); - relation.createOrReplaceTempView("relation"); - dataset.createOrReplaceTempView("dataset"); - software.createOrReplaceTempView("software"); - other.createOrReplaceTempView("other"); - - String communitylist = getConstraintList(" co.id = '", communityIdList); - - String semrellist = getConstraintList(" relClass = '", allowedsemrel ); - - - String query = "Select source, community_context, target " + - "from (select id, collect_set(co.id) community_context " + - "from publication " + - "lateral view explode (context) c as co " + - "where datainfo.deletedbyinference = false "+ communitylist + - " group by id) p " + - "JOIN " + - "(select * " + - "from relation " + - "where datainfo.deletedbyinference = false " + semrellist + ") r " + - "ON p.id = r.source"; - - - org.apache.spark.sql.Dataset publication_context = spark.sql( query); - publication_context.createOrReplaceTempView("publication_context"); - - //( source, (mes, dh-ch-, ni), target ) - query = "select target , collect_set(co) " + - "from (select target, community_context " + - "from publication_context pc join publication p on " + - "p.id = pc.source) tmp " + - "lateral view explode (community_context) c as co " + - "group by target"; - - - - org.apache.spark.sql.Dataset toupdatepublicationreresult = spark.sql(query); - org.apache.spark.sql.Dataset toupdatesoftwareresult = getUpdateCommunitiesForTable(spark, "software"); - org.apache.spark.sql.Dataset toupdatedatasetresult = getUpdateCommunitiesForTable(spark, "dataset"); - org.apache.spark.sql.Dataset toupdateotherresult = getUpdateCommunitiesForTable(spark, "other"); - - createUpdateForResultDatasetWrite(toupdatesoftwareresult.toJavaRDD(), outputPath, "software_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdatedatasetresult.toJavaRDD(), outputPath, "dataset_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdatepublicationreresult.toJavaRDD(), outputPath, "publication_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - createUpdateForResultDatasetWrite(toupdateotherresult.toJavaRDD(), outputPath, "other_update", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - updateForDatasetDataset(toupdatedatasetresult.toJavaRDD(), dataset.toJavaRDD(), outputPath, "dataset", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - updateForOtherDataset(toupdateotherresult.toJavaRDD(), other.toJavaRDD(), outputPath, "otherresearchproduct", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - updateForSoftwareDataset(toupdatesoftwareresult.toJavaRDD(), software.toJavaRDD(), outputPath, "software", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); - - updateForPublicationDataset(toupdatepublicationreresult.toJavaRDD(), publication.toJavaRDD(), outputPath, "publication", - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, communityIdList); -// - -/* - JavaPairRDD resultLinkedToCommunities = publication - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"publication")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - .union(datasets - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"dataset")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(software - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"software")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ) - .union(other - .map(p -> getTypedRow(communityIdList, p.getContext(), p.getId(),"otherresearchproduct")) - .filter(p -> !(p == null)) - .mapToPair(toPair()) - ); - - JavaPairRDD to_add_result_communities = resultLinkedToCommunities.join(result_result).map(r -> r._2()._1().setSourceId(r._2()._2().getTargetId())) - .mapToPair(toPair()); - - JavaPairRDD pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p)); - JavaPairRDD orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p)); - - updateResultForCommunity(pubs, to_add_result_communities, outputPath, "publication", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(dss, to_add_result_communities, outputPath, "dataset", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(sfw, to_add_result_communities, outputPath, "software", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - updateResultForCommunity(orp, to_add_result_communities, outputPath, "otherresearchproduct", PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME); - //leftouterjoin result.to_add_result_communities (result = java pair rdd result) [left outer join perche' li voglio tutti anche quelli che non ho aggiornato] - //per quelli che matchano cercare nel risultato se i context da aggiungere sono gia' presenti. Se non ci sono aggiungerli, altrimenti nulla -*/ - } - - private static org.apache.spark.sql.Dataset getUpdateCommunitiesForTable(SparkSession spark, String table){ - String query = "SELECT target_id, collect_set(co.id) context_id " + - " FROM (SELECT t.id target_id, s.context source_context " + - " FROM context_software s " + - " JOIN " + table + " t " + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, d.context source_context " + - " FROM dataset_context d " + - " JOIN " + table + " t" + - " ON s.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, p.context source_context " + - " FROM publication_context p" + - " JOIN " + table +" t " + - " on p.target = t.id " + - " UNION ALL " + - " SELECT t.id target_id, o.context source_context " + - " FROM other_context o " + - " JOIN " + table + " t " + - " ON o.target = t.id) TMP " + - " LATERAL VIEW EXPLODE(source_context) MyT as co " + - " GROUP BY target_id" ; - - return spark.sql(query); - } - - private static JavaRDD createUpdateForResultDatasetWrite(JavaRDD toupdateresult, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return toupdateresult.map(r -> { - List contextList = new ArrayList(); - List toAddContext = r.getList(1); - for (String cId : toAddContext) { - if (communityIdList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - contextList.add(newContext); - } - - } - - if (contextList.size() > 0) { - Result ret = new Result(); - ret.setId(r.getString(0)); - ret.setContext(contextList); - return ret; - } - return null; - }).filter(r -> r != null); - } - - private static void updateForSoftwareDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Software) r) - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForDatasetDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r-> (Dataset)r) - .map(d -> new ObjectMapper().writeValueAsString(d)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForPublicationDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map(r -> (Publication)r) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath + "/" + type); - } - - private static void updateForOtherDataset(JavaRDD toupdateresult, JavaRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - JavaPairRDD tmp = result.mapToPair(r -> new Tuple2(r.getId(), r)); - getUpdateForResultDataset(toupdateresult, tmp, outputPath, type, class_id, class_name, communityIdList) - .map( r -> (OtherResearchProduct)r) - .map( o -> new ObjectMapper().writeValueAsString(o)) - .saveAsTextFile(outputPath + "/" + type); - } - - - - private static JavaRDD getUpdateForResultDataset(JavaRDD toupdateresult, JavaPairRDD result, String outputPath, String type, String class_id, String class_name, List communityIdList){ - return result.leftOuterJoin(toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1)))) - .map(c -> { - if(! c._2()._2().isPresent()){ - return c._2()._1(); + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); } - - List toAddContext = c._2()._2().get(); - Set context_set = new HashSet<>(); - for(Object cId: toAddContext){ - String id = (String)cId; - if (communityIdList.contains(id)){ - context_set.add(id); - } + if (saveGraph) { + execPropagation( + spark, inputPath, outputPath, preparedInfoPath, resultClazz); } - for (Context context: c._2()._1().getContext()){ - if(context_set.contains(context)){ - context_set.remove(context); - } - } - - List contextList = context_set.stream().map(co -> { - Context newContext = new Context(); - newContext.setId(co); - newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - return newContext; - - }).collect(Collectors.toList()); - - if(contextList.size() > 0 ){ - Result r = new Result(); - r.setId(c._1()); - r.setContext(contextList); - return r; - } - return null; - }).filter(r -> r != null); - - -// return toupdateresult.mapToPair(r -> new Tuple2<>(r.getString(0), r.getList(1))) -// .join(result) -// .map(c -> { -// List toAddContext = c._2()._1(); -// Set context_set = new HashSet<>(); -// for(Object cId: toAddContext){ -// String id = (String)cId; -// if (communityIdList.contains(id)){ -// context_set.add(id); -// } -// } -// for (Context context: c._2()._2().getContext()){ -// if(context_set.contains(context)){ -// context_set.remove(context); -// } -// } -// -// List contextList = context_set.stream().map(co -> { -// Context newContext = new Context(); -// newContext.setId(co); -// newContext.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); -// return newContext; -// -// }).collect(Collectors.toList()); -// -// if(contextList.size() > 0 ){ -// Result r = new Result(); -// r.setId(c._1()); -// r.setContext(contextList); -// return r; -// } -// return null; -// }) -// .filter(r -> r != null); - } - - private static JavaRDD createUpdateForSoftwareDataset(JavaRDD toupdateresult, List communityList, - JavaRDD result, String class_id, String class_name) { - return result - .mapToPair(s -> new Tuple2<>(s.getId(), s)).leftOuterJoin(getStringResultJavaPairRDD(toupdateresult, communityList)) - .map(c -> { - Software oaf = c._2()._1(); - if (c._2()._2().isPresent()) { - - HashSet contexts = new HashSet<>(c._2()._2().get()); - - for (Context context : oaf.getContext()) { - if (contexts.contains(context.getId())){ - if (!context.getDataInfo().stream().map(di -> di.getInferenceprovenance()) - .collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - context.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); - //community id already in the context of the result. Remove it from the set that has to be added - contexts.remove(context.getId()); - } - } - } - List cc = oaf.getContext(); - for(String cId: contexts){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - cc.add(context); - } - oaf.setContext(cc); - - } - return oaf; }); } - private static JavaPairRDD> getStringResultJavaPairRDD(JavaRDD toupdateresult, List communityList) { - return toupdateresult.mapToPair(c -> { + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + String preparedInfoPath, + Class resultClazz) { - List contextList = new ArrayList<>(); - List contexts = c.getList(1); - for (String context : contexts) { - if (communityList.contains(context)) { - contextList.add(context); - } - } + org.apache.spark.sql.Dataset possibleUpdates = + readResultCommunityList(spark, preparedInfoPath); + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - return new Tuple2<>(c.getString(0) ,contextList); - }); - } - - - private static org.apache.spark.sql.Dataset getContext(SparkSession spark, String table){ - String query = "SELECT relation.source, " + table +".context , relation.target " + - "FROM " + table + - " JOIN relation " + - "ON id = source" ; - - return spark.sql(query); - } - - private static Boolean relatedToCommunities(Result r, List communityIdList) { - Set result_communities = r.getContext() - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - return true; - } - } - return false; - } - - private static void updateResult(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type) { - results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - Set communityList = p._2()._2().get().getAccumulator(); - for(Context c: r.getContext()){ - if (communityList.contains(c.getId())){ - //verify if the datainfo for this context contains propagation - if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME)); - //community id already in the context of the result. Remove it from the set that has to be added - communityList.remove(c.getId()); - } + result.joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map( + value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + Set context_set = new HashSet<>(); + ret.getContext().stream().forEach(c -> context_set.add(c.getId())); + List contextList = + rcl.get().getCommunityList().stream() + .map( + c -> { + if (!context_set.contains(c)) { + Context newContext = new Context(); + newContext.setId(c); + newContext.setDataInfo( + Arrays.asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + return newContext; + } + return null; + }) + .filter(c -> c != null) + .collect(Collectors.toList()); + Result r = new Result(); + r.setId(ret.getId()); + r.setContext(contextList); + ret.mergeFrom(r); } - } - List cc = r.getContext(); - for(String cId: communityList){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - - - private static TypedRow getTypedRow(List communityIdList, List context, String id, String type) { - Set result_communities = context - .stream() - .map(c -> c.getId()) - .collect(Collectors.toSet()); - TypedRow tp = new TypedRow(); - tp.setSourceId(id); - tp.setType(type); - for (String communityId : result_communities) { - if (communityIdList.contains(communityId)) { - tp.add(communityId); - } - } - if (tp.getAccumulator() != null) { - return tp; - } - return null; + return ret; + }, + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); } } From e5a177f0a766572c0e1ba118137b559e0bee2d22 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:36:21 +0200 Subject: [PATCH 167/259] refactoring --- .../PrepareResultInstRepoAssociation.java | 121 ++++--- ...rkResultToOrganizationFromIstRepoJob2.java | 330 ++++++++++-------- 2 files changed, 258 insertions(+), 193 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 51c46e2be..095f476cf 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -1,5 +1,8 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Datasource; @@ -9,28 +12,26 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class PrepareResultInstRepoAssociation { - private static final Logger log = LoggerFactory.getLogger(PrepareResultInstRepoAssociation.class); + private static final Logger log = + LoggerFactory.getLogger(PrepareResultInstRepoAssociation.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception{ + public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(PrepareResultInstRepoAssociation.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + PrepareResultInstRepoAssociation.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -49,77 +50,89 @@ public class PrepareResultInstRepoAssociation { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { readNeededResources(spark, inputPath); - prepareDatasourceOrganizationAssociations(spark, datasourceOrganizationPath, alreadyLinkedPath); + prepareDatasourceOrganizationAssociations( + spark, datasourceOrganizationPath, alreadyLinkedPath); prepareAlreadyLinkedAssociation(spark, alreadyLinkedPath); }); } - private static void prepareAlreadyLinkedAssociation(SparkSession spark, String alreadyLinkedPath) { - String query = "Select source resultId, collect_set(target) organizationSet " + - "from relation " + - "where datainfo.deletedbyinference = false " + - "and relClass = '" + RELATION_RESULT_ORGANIZATION_REL_CLASS +"' " + - "group by source"; - + private static void prepareAlreadyLinkedAssociation( + SparkSession spark, String alreadyLinkedPath) { + String query = + "Select source resultId, collect_set(target) organizationSet " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + + RELATION_RESULT_ORGANIZATION_REL_CLASS + + "' " + + "group by source"; spark.sql(query) .as(Encoders.bean(ResultOrganizationSet.class)) .toJavaRDD() .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); -// .as(Encoders.bean(ResultOrganizationSet.class)) -// .toJSON() -// .write() -// .mode(SaveMode.Overwrite) -// .option("compression","gzip") -// .text(alreadyLinkedPath); } private static void readNeededResources(SparkSession spark, String inputPath) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset datasource = spark.createDataset(sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)).rdd(), Encoders.bean(Datasource.class)); + org.apache.spark.sql.Dataset datasource = + spark.createDataset( + sc.textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)) + .rdd(), + Encoders.bean(Datasource.class)); - org.apache.spark.sql.Dataset relation = spark.createDataset(sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)).rdd(), Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset relation = + spark.createDataset( + sc.textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); - org.apache.spark.sql.Dataset organization = spark.createDataset(sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)).rdd(), Encoders.bean(Organization.class)); + org.apache.spark.sql.Dataset organization = + spark.createDataset( + sc.textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)) + .rdd(), + Encoders.bean(Organization.class)); datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); organization.createOrReplaceTempView("organization"); } - private static void prepareDatasourceOrganizationAssociations(SparkSession spark, String datasourceOrganizationPath, - String alreadyLinkedPath){ + private static void prepareDatasourceOrganizationAssociations( + SparkSession spark, String datasourceOrganizationPath, String alreadyLinkedPath) { - - String query = "SELECT source datasourceId, target organizationId " + - "FROM ( SELECT id " + - "FROM datasource " + - "WHERE datasourcetype.classid = 'pubsrepository::institutional' " + - "AND datainfo.deletedbyinference = false ) d " + - "JOIN ( SELECT source, target " + - "FROM relation " + - "WHERE relclass = '" + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + "' " + - "AND datainfo.deletedbyinference = false ) rel " + - "ON d.id = rel.source "; + String query = + "SELECT source datasourceId, target organizationId " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = '" + + INSTITUTIONAL_REPO_TYPE + + "' " + + "AND datainfo.deletedbyinference = false ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = '" + + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + + "' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source "; spark.sql(query) - .as(Encoders.bean(DatasourceOrganization.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(datasourceOrganizationPath); - - - + .as(Encoders.bean(DatasourceOrganization.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(datasourceOrganizationPath); } - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index 82c69e927..db8b99ac7 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -1,17 +1,14 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.DatasourceCountry; -import eu.dnetlib.dhp.countrypropagation.ResultCountrySet; -import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2; import eu.dnetlib.dhp.schema.oaf.*; +import java.util.*; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.broadcast.Broadcast; @@ -19,27 +16,23 @@ import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Option; import scala.Tuple2; -import java.util.*; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - public class SparkResultToOrganizationFromIstRepoJob2 { - private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); + private static final Logger log = + LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils.toString(SparkResultToOrganizationFromIstRepoJob2.class - .getResourceAsStream("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); + String jsonConfiguration = + IOUtils.toString( + SparkResultToOrganizationFromIstRepoJob2.class.getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser( - jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); @@ -49,7 +42,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); @@ -62,119 +54,163 @@ public class SparkResultToOrganizationFromIstRepoJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + final String resultType = + resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); - final Boolean writeUpdates = Optional - .ofNullable(parser.get("writeUpdate")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean writeUpdates = + Optional.ofNullable(parser.get("writeUpdate")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); + final Boolean saveGraph = + Optional.ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); log.info("saveGraph: {}", saveGraph); - Class resultClazz = (Class) Class.forName(resultClassName); + Class resultClazz = + (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession(conf, isSparkSessionManaged, + runWithSparkHiveSession( + conf, + isSparkSessionManaged, spark -> { - if(isTest(parser)) { + if (isTest(parser)) { removeOutputDir(spark, outputPath); } - execPropagation(spark, datasourceorganization, alreadylinked, inputPath, outputPath, resultClazz, resultType, - writeUpdates, saveGraph); + execPropagation( + spark, + datasourceorganization, + alreadylinked, + inputPath, + outputPath, + resultClazz, + resultType, + writeUpdates, + saveGraph); }); - } - private static void execPropagation(SparkSession spark, String datasourceorganization, String alreadylinked, String inputPath, - String outputPath, Class resultClazz, String resultType, - Boolean writeUpdates, Boolean saveGraph) { + private static void execPropagation( + SparkSession spark, + String datasourceorganization, + String alreadylinked, + String inputPath, + String outputPath, + Class resultClazz, + String resultType, + Boolean writeUpdates, + Boolean saveGraph) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); org.apache.spark.sql.Dataset datasourceorganizationassoc = readAssocDatasourceOrganization(spark, datasourceorganization); - //broadcasting the result of the preparation step - Broadcast> broadcast_datasourceorganizationassoc = - sc.broadcast(datasourceorganizationassoc); + // broadcasting the result of the preparation step + Broadcast> + broadcast_datasourceorganizationassoc = sc.broadcast(datasourceorganizationassoc); - org.apache.spark.sql.Dataset potentialUpdates = getPotentialRelations(spark, inputPath, resultClazz, - broadcast_datasourceorganizationassoc).as(Encoders.bean(ResultOrganizationSet.class)); + org.apache.spark.sql.Dataset potentialUpdates = + getPotentialRelations( + spark, + inputPath, + resultClazz, + broadcast_datasourceorganizationassoc) + .as(Encoders.bean(ResultOrganizationSet.class)); - if(writeUpdates){ + if (writeUpdates) { createUpdateForRelationWrite(potentialUpdates, outputPath + "/" + resultType); } - if(saveGraph){ - getNewRelations(spark - .read() - .textFile(alreadylinked) - .map(value -> OBJECT_MAPPER.readValue(value, ResultOrganizationSet.class), - Encoders.bean(ResultOrganizationSet.class)), potentialUpdates) + if (saveGraph) { + getNewRelations( + spark.read() + .textFile(alreadylinked) + .map( + value -> + OBJECT_MAPPER.readValue( + value, ResultOrganizationSet.class), + Encoders.bean(ResultOrganizationSet.class)), + potentialUpdates) .toJSON() .write() .mode(SaveMode.Append) .option("compression", "gzip") .text(outputPath); -// .toJavaRDD() -// .map(r -> OBJECT_MAPPER.writeValueAsString(r)) -// .saveAsTextFile(outputPath , GzipCodec.class); } - - } - private static Dataset getNewRelations(Dataset alreadyLinked, Dataset potentialUpdates) { - - return potentialUpdates - .joinWith(alreadyLinked, potentialUpdates.col("resultId") - .equalTo(alreadyLinked.col("resultId")), "left_outer").flatMap((FlatMapFunction, Relation>) value -> { - List new_relations = new ArrayList<>(); - ResultOrganizationSet potential_update = value._1(); - Optional already_linked = Optional.ofNullable(value._2()); - List organization_list = potential_update.getOrganizationSet(); - if(already_linked.isPresent()){ - already_linked.get().getOrganizationSet() - .stream() - .forEach(rId -> { - if (organization_list.contains(rId)) { - organization_list.remove(rId); - } - }); - } - String resultId = potential_update.getResultId(); - organization_list - .stream() - .forEach(orgId -> { - new_relations.add(getRelation(orgId, resultId, RELATION_ORGANIZATION_RESULT_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - new_relations.add(getRelation(resultId, orgId, RELATION_RESULT_ORGANIZATION_REL_CLASS, RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - } - ); - return new_relations.iterator(); - - } - ,Encoders.bean(Relation.class)); + private static Dataset getNewRelations( + Dataset alreadyLinked, + Dataset potentialUpdates) { + return potentialUpdates + .joinWith( + alreadyLinked, + potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") + .flatMap( + (FlatMapFunction< + Tuple2, + Relation>) + value -> { + List new_relations = new ArrayList<>(); + ResultOrganizationSet potential_update = value._1(); + Optional already_linked = + Optional.ofNullable(value._2()); + List organization_list = + potential_update.getOrganizationSet(); + if (already_linked.isPresent()) { + already_linked.get().getOrganizationSet().stream() + .forEach( + rId -> { + if (organization_list.contains(rId)) { + organization_list.remove(rId); + } + }); + } + String resultId = potential_update.getResultId(); + organization_list.stream() + .forEach( + orgId -> { + new_relations.add( + getRelation( + orgId, + resultId, + RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + new_relations.add( + getRelation( + resultId, + orgId, + RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + }); + return new_relations.iterator(); + }, + Encoders.bean(Relation.class)); } - - private static org.apache.spark.sql.Dataset getPotentialRelations(SparkSession spark, - String inputPath, - Class resultClazz, - Broadcast> broadcast_datasourceorganizationassoc) { + private static + org.apache.spark.sql.Dataset getPotentialRelations( + SparkSession spark, + String inputPath, + Class resultClazz, + Broadcast> + broadcast_datasourceorganizationassoc) { org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); result.createOrReplaceTempView("result"); createCfHbforresult(spark); @@ -182,59 +218,75 @@ public class SparkResultToOrganizationFromIstRepoJob2 { return organizationPropagationAssoc(spark, broadcast_datasourceorganizationassoc); } - - - - private static org.apache.spark.sql.Dataset readAssocDatasourceOrganization(SparkSession spark, - String datasourcecountryorganization) { - return spark - .read() + private static org.apache.spark.sql.Dataset + readAssocDatasourceOrganization( + SparkSession spark, String datasourcecountryorganization) { + return spark.read() .textFile(datasourcecountryorganization) - .map(value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), Encoders.bean(DatasourceOrganization.class)); + .map( + value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), + Encoders.bean(DatasourceOrganization.class)); } - - - private static void createUpdateForRelationWrite(Dataset toupdaterelation, String outputPath) { - toupdaterelation.flatMap(s -> { - List relationList = new ArrayList<>(); - List orgs = s.getOrganizationSet(); - String resId = s.getResultId(); - for (String org : orgs) { - relationList.add(getRelation(org, resId, RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList.add(getRelation(resId, org, RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, RELATION_RESULTORGANIZATION_SUBREL_TYPE, PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - - } - return relationList.iterator(); - }, Encoders.bean(Relation.class)) + private static void createUpdateForRelationWrite( + Dataset toupdaterelation, String outputPath) { + toupdaterelation + .flatMap( + s -> { + List relationList = new ArrayList<>(); + List orgs = s.getOrganizationSet(); + String resId = s.getResultId(); + for (String org : orgs) { + relationList.add( + getRelation( + org, + resId, + RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList.add( + getRelation( + resId, + org, + RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + } + return relationList.iterator(); + }, + Encoders.bean(Relation.class)) .toJSON() .write() .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(outputPath) - ; + .option("compression", "gzip") + .text(outputPath); } - - private static org.apache.spark.sql.Dataset organizationPropagationAssoc(SparkSession spark, Broadcast> broadcast_datasourceorganizationassoc){ - org.apache.spark.sql.Dataset datasourceorganization = broadcast_datasourceorganizationassoc.value(); + private static org.apache.spark.sql.Dataset organizationPropagationAssoc( + SparkSession spark, + Broadcast> + broadcast_datasourceorganizationassoc) { + org.apache.spark.sql.Dataset datasourceorganization = + broadcast_datasourceorganizationassoc.value(); datasourceorganization.createOrReplaceTempView("rels"); - String query = "SELECT id resultId, collect_set(organizationId) organizationSet "+ - "FROM ( SELECT id, organizationId " + - "FROM rels " + - "JOIN cfhb " + - " ON cf = datasourceId " + - "UNION ALL " + - "SELECT id , organizationId " + - "FROM rels " + - "JOIN cfhb " + - " ON hb = datasourceId ) tmp " + - "GROUP BY id"; + String query = + "SELECT id resultId, collect_set(organizationId) organizationSet " + + "FROM ( SELECT id, organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON cf = datasourceId " + + "UNION ALL " + + "SELECT id , organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON hb = datasourceId ) tmp " + + "GROUP BY id"; return spark.sql(query).as(Encoders.bean(ResultOrganizationSet.class)); } - } From 1b0e0bd1b588dcf79977bb8df6e270a3176be4d3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:40:26 +0200 Subject: [PATCH 168/259] refactoring --- .../eu/dnetlib/dhp/PropagationConstant.java | 263 ++++++++---------- .../dnetlib/dhp/QueryInformationSystem.java | 14 +- .../main/java/eu/dnetlib/dhp/TypedRow.java | 94 ------- 3 files changed, 127 insertions(+), 244 deletions(-) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index fa0ab94ee..1a0d98182 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -1,88 +1,95 @@ package eu.dnetlib.dhp; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.PairFunction; -import scala.Tuple2; - import java.io.IOException; import java.util.*; -import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; public class PropagationConstant { public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional"; - public final static String PROPAGATION_DATA_INFO_TYPE = "propagation"; + public static final String PROPAGATION_DATA_INFO_TYPE = "propagation"; public static final String TRUE = "true"; + public static final String DNET_COUNTRY_SCHEMA = "dnet:countries"; + public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; - public final static String DNET_COUNTRY_SCHEMA = "dnet:countries"; - public final static String DNET_SCHEMA_NAME = "dnet:provenanceActions"; - public final static String DNET_SCHEMA_ID = "dnet:provenanceActions"; + public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; + public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = + "Propagation of country to result collected from datasources of type institutional repositories"; - public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; - public final static String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = + "result:organization:instrepo"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = + "Propagation of affiliation to result collected from datasources of type institutional repository"; - public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; - public final static String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; + public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = + "result:project:semrel"; + public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = + "Propagation of result to project through semantic relation"; - public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; - public final static String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; + public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = + "result:community:semrel"; + public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = + " Propagation of result belonging to community through semantic relation"; - public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = "result:community:semrel"; - public final static String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = " Propagation of result belonging to community through semantic relation"; + public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = + "result:community:organization"; + public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = + " Propagation of result belonging to community through organization"; - public final static String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = "result:community:organization"; - public final static String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = " Propagation of result belonging to community through organization"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = + "authorpid:result"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = + "Propagation of authors pid to result through semantic relations"; - public final static String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "propagation:orcid:result"; - public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of ORCID through result linked by isSupplementedBy or isSupplementTo semantic relations"; + public static final String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; - public final static String RELATION_DATASOURCEORGANIZATION_REL_TYPE = "datasourceOrganization"; - public final static String RELATION_DATASOURCEORGANIZATION_SUBREL_TYPE = "provision"; - public final static String RELATION_ORGANIZATION_DATASOURCE_REL_CLASS = "isProvidedBy"; - public final static String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; - - public final static String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; - public final static String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; - public final static String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; - public final static String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; + public static final String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; + public static final String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; + public static final String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; + public static final String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; public static final String RELATION_RESULTRESULT_REL_TYPE = "resultResult"; - public static final String RELATION_RESULTRESULT_SUBREL_TYPE = "supplement"; public static final String RELATION_RESULTPROJECT_REL_TYPE = "resultProject"; public static final String RELATION_RESULTPROJECT_SUBREL_TYPE = "outcome"; public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; - - public static final String RELATION_RESULT_REPRESENTATIVERESULT_REL_CLASS = "isMergedIn"; public static final String RELATION_REPRESENTATIVERESULT_RESULT_CLASS = "merges"; - - public static final String RELATION_ORGANIZATIONORGANIZATION_REL_TYPE = "organizationOrganization"; - - public static final String RELATION_DEDUPORGANIZATION_SUBREL_TYPE = "dedup"; - public static final String PROPAGATION_AUTHOR_PID = "ORCID"; - public static Country getCountry(String country){ + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static Country getCountry(String classid, String classname) { Country nc = new Country(); - nc.setClassid(country); - nc.setClassname(country); + nc.setClassid(classid); + nc.setClassname(classname); nc.setSchemename(DNET_COUNTRY_SCHEMA); nc.setSchemeid(DNET_COUNTRY_SCHEMA); - nc.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME)); + nc.setDataInfo( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, + PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME)); return nc; } - public static DataInfo getDataInfo(String inference_provenance, String inference_class_id, String inference_class_name){ + public static DataInfo getDataInfo( + String inference_provenance, String inference_class_id, String inference_class_name) { DataInfo di = new DataInfo(); di.setInferred(true); di.setDeletedbyinference(false); @@ -101,8 +108,15 @@ public class PropagationConstant { return pa; } - - public static Relation getRelation(String source, String target, String rel_class, String rel_type, String subrel_type, String inference_provenance, String inference_class_id, String inference_class_name){ + public static Relation getRelation( + String source, + String target, + String rel_class, + String rel_type, + String subrel_type, + String inference_provenance, + String inference_class_id, + String inference_class_name) { Relation r = new Relation(); r.setSource(source); r.setTarget(target); @@ -111,107 +125,17 @@ public class PropagationConstant { r.setSubRelType(subrel_type); r.setDataInfo(getDataInfo(inference_provenance, inference_class_id, inference_class_name)); return r; -} - - public static PairFunction toPair() { - return e -> new Tuple2<>( e.getSourceId(), e); - } - public static JavaPairRDD getResultResultSemRel(List allowedsemrel, JavaRDD relations) { - return relations - .filter(r -> !r.getDataInfo().getDeletedbyinference()) - .filter(r -> allowedsemrel.contains(r.getRelClass()) && RELATION_RESULTRESULT_REL_TYPE.equals(r.getRelType())) - .map(r -> { - TypedRow tr = new TypedRow(); - tr.setSourceId(r.getSource()); - tr.setTargetId(r.getTarget()); - return tr; - }) - .mapToPair(toPair()); - } - - - public static String getConstraintList(String text, List constraints){ + public static String getConstraintList(String text, List constraints) { String ret = " and (" + text + constraints.get(0) + "'"; - for (int i =1; i < constraints.size(); i++){ - ret += " OR " + text + constraints.get(i) + "'"; + for (int i = 1; i < constraints.size(); i++) { + ret += " OR " + text + constraints.get(i) + "'"; } ret += ")"; return ret; } - - public static List getTypedRowsDatasourceResult(OafEntity oaf) { - List lst = new ArrayList<>(); - Set datasources_provenance = new HashSet<>(); - List instanceList = null; - String type = ""; - if (oaf.getClass() == Publication.class) { - instanceList = ((Publication) oaf).getInstance(); - type = "publication"; - } - if (oaf.getClass() == Dataset.class){ - instanceList = ((Dataset)oaf).getInstance(); - type = "dataset"; - } - - if (oaf.getClass() == Software.class){ - instanceList = ((Software)oaf).getInstance(); - type = "software"; - } - - if (oaf.getClass() == OtherResearchProduct.class){ - instanceList = ((OtherResearchProduct)oaf).getInstance(); - type = "otherresearchproduct"; - } - - - for (Instance i : instanceList) { - datasources_provenance.add(i.getCollectedfrom().getKey()); - datasources_provenance.add(i.getHostedby().getKey()); - } - for (String dsId : datasources_provenance) { - TypedRow tr = new TypedRow(); - tr.setSourceId(dsId); - tr.setTargetId(oaf.getId()); - tr.setType(type); - lst.add(tr); - } - return lst; - } - - public static void updateResultForCommunity(JavaPairRDD results, JavaPairRDD toupdateresult, String outputPath, String type, String class_id, String class_name) { - results.leftOuterJoin(toupdateresult) - .map(p -> { - Result r = p._2()._1(); - if (p._2()._2().isPresent()){ - Set communityList = p._2()._2().get().getAccumulator(); - for(Context c: r.getContext()){ - if (communityList.contains(c.getId())){ - //verify if the datainfo for this context contains propagation - if (!c.getDataInfo().stream().map(di -> di.getInferenceprovenance()).collect(Collectors.toSet()).contains(PROPAGATION_DATA_INFO_TYPE)){ - c.getDataInfo().add(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name)); - //community id already in the context of the result. Remove it from the set that has to be added - communityList.remove(c.getId()); - } - } - } - List cc = r.getContext(); - for(String cId: communityList){ - Context context = new Context(); - context.setId(cId); - context.setDataInfo(Arrays.asList(getDataInfo(PROPAGATION_DATA_INFO_TYPE, class_id, class_name))); - cc.add(context); - } - r.setContext(cc); - } - return r; - }) - .map(p -> new ObjectMapper().writeValueAsString(p)) - .saveAsTextFile(outputPath+"/"+type); - } - public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { if (fs.exists(new Path(outputPath))) { fs.delete(new Path(outputPath), true); @@ -219,4 +143,61 @@ public class PropagationConstant { fs.mkdirs(new Path(outputPath)); } + public static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + + public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) { + return Optional.ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + } + + public static Boolean isTest(ArgumentApplicationParser parser) { + return Optional.ofNullable(parser.get("isTest")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); + } + + public static void createCfHbforresult(SparkSession spark) { + String query; + query = + "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM result " + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + org.apache.spark.sql.Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + } + + public static org.apache.spark.sql.Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class resultClazz) { + + return spark.read() + .textFile(inputEntityPath) + .map( + (MapFunction) + value -> OBJECT_MAPPER.readValue(value, resultClazz), + Encoders.bean(resultClazz)); + } + + public static org.apache.spark.sql.Dataset readRelations( + SparkSession spark, String inputPath) { + return spark.read() + .textFile(inputPath) + .map( + (MapFunction) + value -> OBJECT_MAPPER.readValue(value, Relation.class), + Encoders.bean(Relation.class)); + } + + public static org.apache.spark.sql.Dataset readResultCommunityList( + SparkSession spark, String possibleUpdatesPath) { + return spark.read() + .textFile(possibleUpdatesPath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), + Encoders.bean(ResultCommunityList.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java index b1ec7726e..a33919d19 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java @@ -3,21 +3,17 @@ package eu.dnetlib.dhp; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; - import java.util.List; - public class QueryInformationSystem { - private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" + - " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" + - " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" + - " return $x//CONFIGURATION/context/@id/string()"; + private static final String XQUERY = + "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" + + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" + + " return $x//CONFIGURATION/context/@id/string()"; public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); return isLookUp.quickSearchProfile(XQUERY); - } - - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java deleted file mode 100644 index 56d519509..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/TypedRow.java +++ /dev/null @@ -1,94 +0,0 @@ -package eu.dnetlib.dhp; - - -import eu.dnetlib.dhp.schema.oaf.Author; - -import java.io.Serializable; -import java.util.*; - -public class TypedRow implements Serializable { - private String sourceId; - private String targetId; - private String type; - private String value; - private Set accumulator; - private List authors; - - public List getAuthors() { - return authors; - } - - public TypedRow setAuthors(List authors) { - this.authors = authors; - return this; - } - - public void addAuthor(Author a){ - if(authors == null){ - authors = new ArrayList<>(); - } - authors.add(a); - } - - public Set getAccumulator() { - return accumulator; - } - - public TypedRow setAccumulator(Set accumulator) { - this.accumulator = accumulator; - return this; - } - - - public void addAll(Collection toadd){ - if(accumulator == null){ - accumulator = new HashSet<>(); - } - this.accumulator.addAll(toadd); - } - - - public void add(String a){ - if (accumulator == null){ - accumulator = new HashSet<>(); - } - accumulator.add(a); - } - - public Iterator getAccumulatorIterator(){ - return accumulator.iterator(); - } - - public String getValue() { - return value; - } - - public TypedRow setValue(String value) { - this.value = value; - return this; - } - - public String getSourceId() { - return sourceId; - } - public TypedRow setSourceId(String sourceId) { - this.sourceId = sourceId; - return this; - } - public String getTargetId() { - return targetId; - } - public TypedRow setTargetId(String targetId) { - this.targetId = targetId; - return this; - } - - public String getType() { - return type; - } - public TypedRow setType(String type) { - this.type = type; - return this; - } - -} From 7b6505ec6913d9aba61a5f15be519c4e22ce3978 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:42:16 +0200 Subject: [PATCH 169/259] new resuorces for testing propagation of project to result after the re-implementation --- .../alreadyLinked/alreadyLinked.json.gz | Bin 0 -> 1685 bytes .../alreadyLinked/alreadyLinked_57.json.gz | Bin 6170 -> 0 bytes .../potentialUpdates/potentialUpdates.json.gz | Bin 0 -> 302 bytes .../potentialUpdates/potentialUpdates_1.json.gz | Bin 831 -> 0 bytes .../potentialUpdates/potentialUpdates.json.gz | Bin 0 -> 302 bytes .../potentialUpdates/potentialUpdates.json.gz | Bin 0 -> 302 bytes .../potentialUpdates/potentialUpdates_1.json.gz | Bin 352 -> 0 bytes 7 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked_57.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz delete mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates_1.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..9dc35adae6a28d3c6a2718286ce26a042f1f2236 GIT binary patch literal 1685 zcmV;G25R{qiwFp=o26a=17U1(WnpA_OlfXwWn?aDb8l_{tyjs8CASg0`!o9NLz20W zoImi{N5e3fnM_DtMncaFe31O_jE-pj`|$Vof5?|N zFG@e}_{U%V`}O7xH)dlUG{&6S+a!-uv<|hsh~wq=FF(Hi@YfUX|9IZNy!q3M(9iM1 zYwll9$Z|a9W7s)57WEv|hWZkc+GFE{|9M8M%|l1GEGlWkI#WL`PU-z*y!`oJzx(M6 z-0{PY=k@LP@!{Qd0nxWV9C(*U zxNVxbC@&S?wkfs}_j9aE$+GhtH$bYy_38>z; zhxbvln4V*B@6>&z4V$}8R3kWRo%? z+7YiwGuOI-!96Xga7;rn_WH+b(_Y(mNRD7I1KwP688h z)+B)p?#5AMWWQ+2Ykc_I$4}i|S6j6#7dW8F*?|(IcOO=j7!#XR*CcBavm}tkewO(O2I3##M@hKN@ut-M>H861Tj>Wxh z>=Ia4P2nV!Oadn3-k_D9dIF<-j=_wxPwF;UvC$hhs0k!C$H_%E?oVZAQPE(`b6b{p zUvrLVp!A$y(bmr}3`qNk?3*p)40gx?ck4<`=`n}C-~;>FsX8OpY;a?PxUTSpmm5(0 zhrU$OGv>u2kO@i_5EXoT$r+vdbKE0l6?C*gJ)1GM7p8zM7WWvq&*vNhUmHWxHQ;fL z_=3`YBrM-ls!%JquSjMzu ze~w`t2yh=G#|(_S<-tWt!78S}E+6{R+wZ^obPf#)2qQQROpb98fu$8Je@n}K2Sa66 zHAi}i;d??0eK>`7cw;wtJ_igBTL(|r&`!c2J>7V=4a0|0-`MAkTLTM^NY{j?9&5nF z8xnEXBf(pScym~M3zO5^cWBkM;K{f>(XIVGr;RJJvR11A3bxK)f`J;Uly7>M#t`6AMJUYTu@h?qTTQr!DL>r5x9`qteB}Ob)^HeyV=9K}z&De7 zsuNPlGRNO@+~8fV6%90*gGakQ-WvpPBLe8$IN>RVU%%#Ga|{p+Mw3(AiRh3P9i9YxEEzNH;r@N0g@h=4JZFOf7h|lk z!KEye#oToRLz0e7@FKYHgeak5pu8X}pJfZ`{+tgBF&Nzl3D*Jv5$nt_JJf|75+*l% z36C?Tjwr|zFvy^xsJMIdX<&BV$mobSNiU!7>W?D0^Z7~ zEF0|)vcenDxT+&zKh%fM=Wx&~20}hc1u-f^qBk)krny(4d)>g$M!Q<-gp;DXqwhc} z1?@Z7Xg|)GpJMt7rm(qeo^$_38P0|e1zfM+jsR5gNj~4sR2!@(O6sW8)-i} z#SxWXJm&LfI(*2XKWEFq|F3{~YGs7?D}M*k8+V1Yn@%HA#H5pyl4oZH-WW~jE(V~Zt)l$fzG$0=Gg$I5~|kGqG;dgU@=>@`*y z(e~NG%LsO$1-!|e_KG};<@%w zOnvh&O;?w=&%#dT;q4}^fgHZmN4-hLjMh;9vKB=J0dY-`GMLTBbvUOT{*GB;dTgO3 zVg(&X%DkAzbxv~*GE%84)E+G)mT+p7L*LD{-qCLX{`EGo=8h#t4q;9v?J`QXI?cU&nF;L?S%ZqXhTb({lI6^w_%xUYd`KUC` zP@$XqfN)S|beXPw)>J(SN$E7eHs&OtbbMrT-qAF!<7Qe~#kgi>Dl1=Oh_KMX4Lexz z9X(^B5;PR&LJ)HlzG<|Xd$v84nSAymuRJ{mmI23@G;7k!5+wClJ@2sH=q(|@jQcDm z;qcNSj&v9h#kJdI3_s>RgRUTane*j1R7q?=L|i2v%e%$Rs`oxO_4qsng=^2;&>gei z$Ltx0uWJ`9ht4jK^MKNhl$ipm^sLmvRBES%Bx4q7@65wKyt=;hm#!L>DG3dg(73nc z`RvOjomgs&85YTOPRMaqx(DEsVJ|S_nV>|VN?dbiWVFC?Knatu z-kCeHNM;(+Y<-tF<4sR0{5oc( z(3$N7H}-Ox8L9-$B5Sv2Cg9D`upBM)7Yve-hR)I#0!}Pf@0B^0L25N*>TtijOso9zcLe0(}%yLTn9a4U;#sB>AZ)!0V zo@RAKb_}WvbCOoQ6|9nD<++C_f2=AVX;WcxK4o@M{F49sNV<(1k&bDsU_ml1FhZEC zEl&q<8hUo+CdhMGVd1d0NFHdj4>)p0qEQT2ZaOgfz&U%@6I}d^?8Kk%hWE65<-pTA zMrN>bIV6O_V-y(=N{$V|e`iiDAn4O0`>IP)#M}5}P6Uk>L|BREdA#@s5R45kFpPqOWY zcf|~nNv;3C%Dg!`Dkq|vt; zB2n{Vy**Q=x5E$?bt{i-U*EQg$ykl*uUYFS``7RPwEz2uKYjn>roPuWz5x*i>#FpJ z#H9y!!opjO*L+3=wM7)86vK%oR8xtu#|ORwdOEGMF~)U+bjEKegb`O%_v{~K3VG|-9`|GclCFc`KJHnZJ_zG(Ak8+5Dfng* z1r(d@lK8CaTz7Bv+~bzQ(oCU9N~I@?ul3$ny-2>dD@4;>BV0dC0@j@qhMkEc;!>qlPdaiR#ksYd}DN7>(m)K~G@+Qou0Et)qQ*wMaGG`h?j&BJT z1lBik6#KodkBj9&I0>7CtG-|vUh0)tQrLRk)0*WRq!DDQJK5Q3w{EN_VNp&=_c_Q6 z8v%s+x_}&1>dCnhwVf*Kl>rxtnQ;-Bl7fs44z1*kG~kB}eO&h)RxNn!4x4o)>8hu% zUSWlTedv9EBcdq8O7I;cu2Osp&9@J8&2!py4(JvNIFLw~rM3#34-(g^4SOPO-}eu- zGPQU-@crh`GBfcD>P~FjvCi`qYf3{ z84z4+>r9v`4jLXfV9E4a2Z8XFjt4nI{(-K5t@m0xCX7Pf_i>;cwE?S5aixKFuwB@m zMwEK*Q!UM?HSns-@^ZF?OoBQG0dD&`2N<%QI#&Ha`JoX>4{5un z%#z>?WvU8zt+DQVPGO!38XbH%t$-05k$mHSmkjWjaiNMl!v zz4*anz$K8k<9oSRJ;0s_0@+3llANfh>*zofL(}`3!pLbJ9mTgA2AU9ieukGa;FLEWzLC zL0%ukFy9`z*1>?rI@LJQoP2g%YXDnk`PnH$yyjvdrNF2wHoP76Tm)PCx}_F;PxgAA zz+^SVDf=b?&ep)qNq-WH>!`6^=K$bNLr^GGy@}|q)Henfm^{MYb$tejU5h{yp%+gi zM&o3}uG6P>eTD!*wU8%I%6YZDC51}krGNBF5>L+}1p7DV@p|**q zrf7a7-F*Wx^yNM+(0_g2Z$ZO%KKLaph!ql<+x8lc-qE>Vd+PvpkZP|I4Ru9d4{`6X zB9uYA_oI!06%}N7FXxPq5ScdU#T;1F>zw-DJ@7-KjAazj&Vj;ept~QD6JD~H|_t`!LE=1s+qMbM)r2@3l zi@ef1dmYrjPhISMMjLn`wNl8)zHy!WBc9f-q3H+>V%3qjN*C1SyCmrhotoPlW2p2| zD%d)W zk$;v41wNI69vIm3x$+@5w(OyV_0|$`xxPlTMshgoo&h>#D1~I&W2-8OB5Rq&rgU=3 zSGMrxI%KOfUfFo}0IIu;@6kxg_cJutkE#e;EU_98L#q@_T}Q6ze(r{-Lz0v%|H&xn zqLD{If2*Tszk2>4R*@5M;%d3u40B8+LB&Z}*WY^b_dR0X`IWsDNj1_2eT$I`d;(m8 z&vTab$$9f@f7CB|1oQ22D~No{37d$Y?BNUgi>x2Qd5oeB+QO9V^gIX!_kGj6g`}9< zl;IbBQaFB8&v%F+q#P%AQVj!+J00FTm?!={=@9tjbt?l90C|I4vGb)tf*q)&w z&t>=b*~<*Y%w_DIU6gc^h6(J;4@B-9XD{0EQx*>@09p21PZZC_ZO9>S^a(8QDfP5p zPst+jDH1(dD?;KMd+X+651tVF;KNAMPT%oc@XM?K_8Vi6nDc833xI>g3Tr{S&6D-a zBJ7+MmoL!0+T=WPXD~MR|82(A!KJSOwLdL~W zn~a{?LJP=WEiwR0t6u$ZcQ)(3jv+-ZR+BvsAQsvN6)ze z?Mm9cY9o-WEPo-vtP$U z=BJc+or7A!tbLGHP!PO&ln8x#1^Z>FdG)TT(W{%RDu5~9-3s5de1`^#$0@!nqPS-<~%cZ9Plz6#*tEML z!~LS6Ogt;C;`jOP8DXBb3oQm7;Jl5BAWGv@3aK&d`~G6~@adT@tWxOdNG2YHNdqy) zyz35qMnO{kJtHD%5L?75RZ8W>e&-QHM~@2$TtPy`Rg+^~_QTmZ?|a}tUl-DTI7^#) zr~2D3AtCN~=fxCCt|+}HrBUl^ACLS_M#c=z{oHi}%>*eL8WPVTkCJzqezZ-Fk$Bz9 zC3u7liutJg?oG+Q;pyY}fuVeoi4P zBu8v|=tRKbyu*I4^NIk>_;sEPs1sYt3PnjHL)Y!@;xzIQu{N1dNw%-11?eg{CM?QvD%8e;%cAWWB)J)=a&wP z?2!E(mYX=RGQtsPHr?*aeRDsEcZ~xB*bwZY8W9|_sy#toq@HF?_ZO#1RdvpkRVASNWvkr-}i9V^uA>vI|d<}B&Z5a0Bf1H_J=In zUiWgzUeHwaUvqSnpE-UVG#RUPKYMocq<)K~vtD~Sgg^1~uT#b%OuVrU%mTEWPND`Q zYUqaz47uTQy|4IuS0p2h)PM*%B+l5n?@tP$!b2Vz`;9T2X-MOIUEDx7(S^cm1+)(f z9Q!#J^5z6&@jJ!f$zCOEBIWauOk+#;vzg^~gp}#1t|PI90mxUxo^;bc$tgeD-;A5j znK^U{9?Qo=AX69k1<(0O@bldHMt=5Fwd$L#sU%{HVm5$^S=!gyf(cZqi;(KZHK!&z zQB#V-BJvLJjGJs!3I2JZ$(sN}N zaLuo?dDoH*lVcThH2oHv-!{DWUf+TANU~0uW>81Tjs-hl7~g&EiBx40lK>DuwuM#v zQUpm|lviz^+{gGZKh%Fl9xCSTk%JUkM~JB$LQzR?+yNT3lS^zeqf~JZs=&rBIC}4X z?DozWhz>m89oGGNAG}{XZ-VuWjg~xT;EFzFFRQJ_j4bz^nfwI8Z#LaCqa}uD4PRto zL0Du!$QT+;w}emDHF@_Aw97!tr4*(3rA0Ea=ioQqmCO(%o@9e2Vx=5ZQ(4*XdHIpy zvoHH^i2U+O@%xZfKwIeE@}v}i%4?kv|8$!vRav&7VI%d%im~ z7Dh~F6y}l91cExL!8}f8UG)>D`7nmI$oT;$)a2J%{I?L9i6we~c}t$xVmbq!9g`lA zwEC}VjMebp=xrwEoiV)ja~8|rG|~0D3t=eMI++##<9Q4KYTuj2Wf-C{)3?d*ndk}G zPQ2e0$LfM#70VvBQb2z(h331exKaLg-~5-m?=@~LYWr2`O^g;2exMunBGzsG9`=$K~*<`!R`dffvD0HKKHKe8K%(r$A@ zK+F8MU=7E)^Oitm%{3@F&8LCT9iWy{@?V5r=dGx;*SUw7{}g}%B2Vzul{m#JzO&xb zPyRcjN^SB_7D=E*;j>OC=lQpo<+y{lz2GJ5mi*Lf(!r6cKkJ^~JnyhZ>X%7NXT)9o z2MKEtZt>Mma?poy588TE3>)bI;OmNK;I73Z

}6f7uDl@|!~fZv)#+?RDZ>dG|m5 z*5LIU?l?~h9`Hw~KpsZOFbq3gK3nG&taAo={Qx{CXM5j-n!-DzPZ;~-xMk#B3uM@H!jz(7 svj!VvG-|wYM!DKpX_W-{&3gQX=&I3wA_-(v!UALXKT0hqc!NIx09_0wDgXcg diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..bd0c4f29bf3e73c8dfe3259402cb7648a1ef2510 GIT binary patch literal 302 zcmV+}0nz>+iwFq(o~2#@18{G2Wo~q7VQf`!WMOn=b1rIgZ*BmEkGoC-K@3HEK4WVN z&#XO<{R3)BgjRkE0fK0gR0;piNJyk$C0x1iJwEp`ZGJqzpB`*lCW5QEoe%5JEDHcj zGgtBu^QgU!Em@EhUw(g)3iKI2v_~wjMw!gLxyF+>;dCgxDjRURxssi zHfIG1j_LX9?(&a=0+iM<;he|g(l~1fN>_*ueb3?eag6nsP+7GEmlD+lyXPq5MYOpL z0D|o~j23<)uw_(EP$Ypv0-_4Rs9+*F`L4sJ=eHZ7#0)%&qX^{@GOsnz)gWev4ukL~ zLyn<+?&3_H>acA(H}EDp8X@gvumrW*QBo)8fW~3fDp?hBqddBP16imC7xMuC0H;8V A0{{R3 literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates_1.json.gz deleted file mode 100644 index 7dfd477ad4581f78a7e0bc7d65efc12dc2594e60..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 831 zcmV-F1Hk+riwFpjbCq5I18{G2Wo~q7VQf`!WMOn=b6+toYIARH0Bx1aj%GCsgx}>~ zvxqN=oqhw>ED>rsz6hZfJ?iNZNDS|ee<4=nY@^Dh>MNhW9$!A*e@p-TF&-}u9^dWv zkJtacmlrxYMae*9p-l<~+#w-*@7hI=A08k1_1oL$Px<+H`S~%&_sID6<@Jwx9Rpro zN+ErmfN_wxmdat}8c?+P=;;mqr$H1)j3gzrNDo^q&wi4(Fdc$ z@?;!x?53F--fI-w97!aHpDngUD|d|fujZ@&`u>)GetCQU$miYsnCg<8KvRzEuoS`p zRDyLqM;B1k;$VX|ajYR@Z16K>@vL#vuVKSO@j>k3h? zoBK>>htd8yaBfu4n`4}&!zyuPFXZ#*}9#F3?-pXkS7{9VI7g>h*%@A`I%r{xp8HOwhWa{;&czF5+ZgQSJ0W*$K5nb7w^glSjxVq z$0Bb&7p~zU*K?pBzE0ON1hJ>n+FMSuxv&d9VO`&o-JGr6v@%_)mXl`okQ!~o9@n`u z_I5T_cNxM`-LyocF0C}ze$~|GPElnS5}p2+aA$e;BJH*h;2Q^^Wj*1|<)8;=9)*|y zPdH*BT)U0WAuD;d_C)m#M`kZn_aV9rSKm+iwFpxo26a=18{G2Wo~q7VQf`!WMOn=b1rIgZ*BmEkg-lfF$_g_KI6>{ zPHe}?`vYc1gr00CRzZkXrHFy@??hr?pn%t_Yn^-Em*aSOeLCIrWnCD~ee~DE_I2wD zAXPc5&6os5gZ{|qIYn_&hpi3XiLFttR8trfuf zJHuOS0BL$i6TdPxl^%mxeJ+Vw46S&jiww^XyKVb6sVqf2OASdZ?#9`J)NAwJOUw^H zIyBe^Ox4B!(J`1Jgsg#6151}WkjISkVA7_cHU}Cc@3~r5Wtdw(0WWD#t@8l@0GM8m AWB>pF literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..400e2dc93e994577f3e2e64fe07e36547c54d201 GIT binary patch literal 302 zcmV+}0nz>+iwFp*ouys?18{G2Wo~q7VQf`!WMOn=b1rIgZ*BmEkh@MpK@3EDKI7ID zUa!56`v=sN2wnNHRzQeEA&7$T@2o^YK>$~lJY$`i_vJWVUY|}kzN`zw*@wR#wy#-N z0MX{dc!c_h+Uwks2Wv42vgLX?9QTjIPPgN6fyD9d3_?VV)6!>`tA1|x$L(wS^J8rL*5Td0+2!Bt zJIDl{MG(D_kVNn4iN(+*(AM^0d*tie_4z5E+y1gKyhp~{{qnso0J!fhdnhkYn1E#^ z5YwkEku}P}j|KlRMCwG5Z8Wk-50hwzlC5DHo-(r*r(|flBLPPZBuDY=1iI{u;Sqjl zEv7j&)dXKtZR)6;0sNPF^)I(;{S@jLG&k(R0#wyw36HJ=Q+u< Date: Mon, 27 Apr 2020 10:45:06 +0200 Subject: [PATCH 170/259] removed the writeUpdate option --- .../input_countrypropagation_parameters.json | 8 +------ .../countrypropagation/oozie_app/workflow.xml | 24 ++++++++----------- 2 files changed, 11 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 7545f30c4..912c88743 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -11,17 +11,11 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be written. No double check if information is already present", - "paramRequired": true - }, { "paramName":"sg", "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true + "paramRequired": false }, { "paramName":"tn", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 3a8aa949a..bd2473308 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -28,10 +28,6 @@ sparkExecutorNumber number of executors used - - writeUpdate - writes the information found for the update. No double check done if the information is already present - saveGraph writes new version of the graph after the propagation step @@ -46,8 +42,12 @@ - + + + + + @@ -58,7 +58,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/country_propagation/relation + ${nameNode}/${workingDir}/relation @@ -115,10 +115,9 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/country_propagation/publication + --outputPath${workingDir}/publication --preparedInfoPath${workingDir}/preparedInfo @@ -145,10 +144,9 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/country_propagation/dataset + --outputPath${workingDir}/dataset --preparedInfoPath${workingDir}/preparedInfo @@ -175,10 +173,9 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/country_propagation/otherresearchproduct + --outputPath${workingDir}/otherresearchproduct --preparedInfoPath${workingDir}/preparedInfo @@ -205,10 +202,9 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/country_propagation/software + --outputPath${workingDir}/software --preparedInfoPath${workingDir}/preparedInfo From e2093644dce521fff4f96d871fb3d9c29ab00ab6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:46:44 +0200 Subject: [PATCH 171/259] changed in the workflow the directory where to store the preparedInfo and the graph genearated at this step --- .../input_projecttoresult_parameters.json | 8 +---- .../projecttoresult/oozie_app/workflow.xml | 34 +++++++++---------- 2 files changed, 18 insertions(+), 24 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json index af52c4e5b..7f44ba03c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json @@ -5,17 +5,11 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", - "paramRequired": true - }, { "paramName":"sg", "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true + "paramRequired": false }, { "paramName":"pu", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 011ff70e3..e4f791dbc 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -20,10 +20,6 @@ sparkExecutorCores number of cores used by single executor - - writeUpdate - writes the information found for the update. No double check done if the information is already present - saveGraph writes new version of the graph after the propagation step @@ -31,13 +27,19 @@ + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + + + + + @@ -55,7 +57,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/projecttoresult_propagation/relation + ${nameNode}/${workingDir}/relation @@ -66,7 +68,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/publication - ${nameNode}/${workingDir}/projecttoresult_propagation/publication + ${nameNode}/${workingDir}/publication @@ -77,7 +79,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/dataset - ${nameNode}/${workingDir}/projecttoresult_propagation/dataset + ${nameNode}/${workingDir}/dataset @@ -88,7 +90,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/otherresearchproduct - ${nameNode}/${workingDir}/projecttoresult_propagation/otherresearchproduct + ${nameNode}/${workingDir}/otherresearchproduct @@ -99,7 +101,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/software - ${nameNode}/${workingDir}/projecttoresult_propagation/software + ${nameNode}/${workingDir}/software @@ -127,8 +129,8 @@ --sourcePath${sourcePath}/relation --allowedsemrels${allowedsemrels} --hive_metastore_uris${hive_metastore_uris} - --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates - --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked + --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked @@ -152,13 +154,11 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --hive_metastore_uris${hive_metastore_uris} - --outputPath${workingDir}/projecttoresult_propagation/relation - --potentialUpdatePath${workingDir}/projecttoresult_propagation/preparedInfo/potentialUpdates - --alreadyLinkedPath${workingDir}/projecttoresult_propagation/preparedInfo/alreadyLinked + --outputPath${workingDir}/relation + --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked From f9ee343fc081ab8715d39305ac44ba79e2038ec7 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:48:31 +0200 Subject: [PATCH 172/259] new parametrized workflow with preparation steps and new parameter input files --- .../input_communitytoresult_parameters.json | 6 + ..._preparecommunitytoresult2_parameters.json | 32 +- ...t_preparecommunitytoresult_parameters.json | 29 +- .../oozie_app/config-default.xml | 36 ++ .../oozie_app/workflow.xml | 315 ++++++++++++++++-- 5 files changed, 338 insertions(+), 80 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json index 27bb097f9..2baec0e68 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json @@ -35,6 +35,12 @@ "paramLongName": "preparedInfoPath", "paramDescription": "the path where prepared info have been stored", "paramRequired": true + }, + { + "paramName":"test", + "paramLongName":"isTest", + "paramDescription": "true if it is executing a test", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json index f61db6f34..3ba3c8e9c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json @@ -1,34 +1,10 @@ [ - { - "paramName":"is", - "paramLongName":"isLookupUrl", - "paramDescription": "URL of the isLookUp Service", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"as", - "paramLongName":"allowedsemrels", - "paramDescription": "the allowed semantic relations for propagation", - "paramRequired": true - }, - { - "paramName":"h", - "paramLongName":"hive_metastore_uris", - "paramDescription": "the hive metastore uris", - "paramRequired": true - }, - { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": false - }, { "paramName": "ssm", "paramLongName": "isSparkSessionManaged", @@ -40,11 +16,5 @@ "paramLongName": "outputPath", "paramDescription": "the path used to store temporary output files", "paramRequired": true - }, - { - "paramName":"tn", - "paramLongName":"resultTableName", - "paramDescription": "the name of the result table we are currently working on", - "paramRequired": true - } + } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json index 90b5974ea..a5dfefc0a 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json @@ -5,12 +5,6 @@ "paramDescription": "URL of the isLookUp Service", "paramRequired": true }, - { - "paramName":"mt", - "paramLongName":"master", - "paramDescription": "should be local or yarn", - "paramRequired": true - }, { "paramName":"s", "paramLongName":"sourcePath", @@ -30,16 +24,21 @@ "paramRequired": true }, { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", "paramRequired": true }, { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true - } - + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml index ea3a4d922..2744ea92b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml @@ -19,4 +19,40 @@ hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index 4312ec068..d320bc9eb 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -8,54 +8,301 @@ allowedsemrels the semantic relationships allowed for propagation - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - isLookupUrl the isLookup service endpoint - + - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${workingDir}/projecttoresult_propagation/relation + + + + - - - ${jobTracker} - ${nameNode} - yarn-cluster + + + + + + + + + + yarn + cluster + ResultToCommunitySemRel-PreparePhase1-Publications + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/preparedInfo/targetCommunityAssoc + --allowedsemrels${allowedsemrels} + --isLookupUrl${isLookupUrl} + + + + + + + yarn + cluster + ResultToCommunitySemRel-PreparePhase1-Dataset + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/preparedInfo/targetCommunityAssoc + --allowedsemrels${allowedsemrels} + --isLookupUrl${isLookupUrl} + + + + + + + yarn + cluster + ResultToCommunitySemRel-PreparePhase1-ORP + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/preparedInfo/targetCommunityAssoc + --allowedsemrels${allowedsemrels} + --isLookupUrl${isLookupUrl} + + + + + + + yarn + cluster + ResultToCommunitySemRel-PreparePhase1-Software + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/preparedInfo/targetCommunityAssoc + --allowedsemrels${allowedsemrels} + --isLookupUrl${isLookupUrl} + + + + + + + + + + yarn cluster - ResultToCommunitySemRelPropagation - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob + ResultToCommunityEmRelPropagation-PreparePhase2 + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep2 dhp-propagation-${projectVersion}.jar - --executor-memory ${sparkExecutorMemory} - --executor-cores ${sparkExecutorCores} + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners="com.cloudera.spark.lineage.NavigatorAppListener" - --conf spark.sql.queryExecutionListeners="com.cloudera.spark.lineage.NavigatorQueryListener" + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - -mt yarn-cluster - --sourcePath${sourcePath} - --allowedsemrels${allowedsemrels} - --hive_metastore_uris${hive_metastore_uris} - --isLookupUrl${isLookupUrl} + --sourcePath${workingDir}/preparedInfo/targetCommunityAssoc + --outputPath${workingDir}/preparedInfo/mergedCommunityAssoc - + + - + + + + + + + + + + + + + yarn + cluster + Result2CommunitySemRelPropagation-Publication + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/mergedCommunityAssoc + --sourcePath${sourcePath}/publication + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/publication + + + + + + + yarn + cluster + Result2CommunitySemRelPropagation-Dataset + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/mergedCommunityAssoc + --sourcePath${sourcePath}/dataset + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/dataset + + + + + + + yarn + cluster + Result2CommunitySemRelPropagation-ORP + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/mergedCommunityAssoc + --sourcePath${sourcePath}/otherresearchproduct + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/otherresearchproduct + + + + + + + yarn + cluster + Result2CommunitySemRelPropagation-Software + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/mergedCommunityAssoc + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/software + + + + + + \ No newline at end of file From d30e71016550f20ac838fc284e1a330d0e3a0b58 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:52:30 +0200 Subject: [PATCH 173/259] fixed duplicates action name in the workflow --- .../oozie_app/workflow.xml | 41 +++---------------- 1 file changed, 5 insertions(+), 36 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 13c3a2fee..245c4f556 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -149,7 +149,7 @@ --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked - + @@ -180,7 +180,7 @@ --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked - + @@ -211,7 +211,7 @@ --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked - + @@ -242,42 +242,11 @@ --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked - + - + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file From 6135096ef119c9fc399f38f143a41ca863442c3c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:57:50 +0200 Subject: [PATCH 174/259] refactoring --- .../CountryPropagationJobTest.java | 270 +++++++++++------ .../OrcidPropagationJobTest.java | 233 +++++++++------ .../Result2OrganizationJobTest.java | 274 +++++++++++++----- 3 files changed, 517 insertions(+), 260 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index db8e3fd99..675bb3917 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -1,10 +1,13 @@ package eu.dnetlib.dhp.countrypropagation; -import com.google.gson.Gson; -import eu.dnetlib.dhp.schema.common.ModelSupport; +import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.Country; -import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.Software; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -16,18 +19,8 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; -import scala.Array; import scala.Tuple2; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - public class CountryPropagationJobTest { private static final Logger log = LoggerFactory.getLogger(CountryPropagationJobTest.class); @@ -54,13 +47,11 @@ public class CountryPropagationJobTest { conf.set("spark.sql.warehouse.dir", workingDir.toString()); conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = SparkSession - .builder() - .appName(CountryPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - - + spark = + SparkSession.builder() + .appName(CountryPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); } @AfterAll @@ -69,95 +60,190 @@ public class CountryPropagationJobTest { spark.stop(); } - @Test + @Test public void testCountryPropagationSoftware() throws Exception { - SparkCountryPropagationJob2.main(new String[]{ - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/countrypropagation/sample/software").getPath(), - "-hive_metastore_uris", "", - "-writeUpdate","false", - "-saveGraph","true", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath",workingDir.toString() + "/software", - "-preparedInfoPath", getClass().getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo").getPath(), - }); + SparkCountryPropagationJob2.main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/sample/software") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath", + workingDir.toString() + "/software", + "-preparedInfoPath", + getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/software") - .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/software") + .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); - //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); Assertions.assertEquals(10, tmp.count()); - Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); + Dataset verificationDs = + spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); - Assertions.assertEquals(6, verificationDs.filter("size(country) > 0").count()); - Assertions.assertEquals(3, verificationDs.filter("size(country) = 1").count()); - Assertions.assertEquals(3, verificationDs.filter("size(country) = 2").count()); - Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); + Assertions.assertEquals(6, verificationDs.filter("size(country) > 0").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 1").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 2").count()); + Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); + Dataset countryExploded = + verificationDs + .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) + .map(c -> c.getClassid(), Encoders.STRING()); - Dataset countryExploded = verificationDs - .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) - .map(c -> c.getClassid(), Encoders.STRING()); + Assertions.assertEquals(9, countryExploded.count()); - Assertions.assertEquals(9, countryExploded.count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'FR'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'TR'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'IT'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'US'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'MX'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'CH'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'FR'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'TR'").count()); - Assertions.assertEquals(2, countryExploded.filter("value = 'IT'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'US'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'MX'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'CH'").count()); - Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); + Dataset> countryExplodedWithCountryclassid = + verificationDs.flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream() + .forEach( + c -> + prova.add( + new Tuple2<>( + row.getId(), c.getClassid()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - Dataset> countryExplodedWithCountryclassid = verificationDs - .flatMap(row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getClassid()))); - return prova.iterator(); - }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); - Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); + countryExplodedWithCountryclassid.show(false); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'FR' ") + .count()); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'TR' ") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'IT' or _2 = 'MX') ") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'IT' or _2 = 'US') ") + .count()); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'JP'") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'CH' or _2 = 'JP') ") + .count()); - countryExplodedWithCountryclassid.show(false); - Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'FR' ").count()); - Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'TR' ").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'IT' or _2 = 'MX') ").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'IT' or _2 = 'US') ").count()); - Assertions.assertEquals(1, countryExplodedWithCountryclassid.filter("_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'JP'").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassid.filter("_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'CH' or _2 = 'JP') ").count()); + Dataset> countryExplodedWithCountryclassname = + verificationDs.flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream() + .forEach( + c -> + prova.add( + new Tuple2<>( + row.getId(), + c.getClassname()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - Dataset> countryExplodedWithCountryclassname = verificationDs - .flatMap(row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getClassname()))); - return prova.iterator(); - }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + countryExplodedWithCountryclassname.show(false); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'France' ") + .count()); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'Turkey' ") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'Italy' or _2 = 'Mexico') ") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'Italy' or _2 = 'United States') ") + .count()); + Assertions.assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'Japan' ") + .count()); + Assertions.assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'Switzerland' or _2 = 'Japan') ") + .count()); - countryExplodedWithCountryclassname.show(false); - Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'France' ").count()); - Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'Turkey' ").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'Italy' or _2 = 'Mexico') ").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'Italy' or _2 = 'United States') ").count()); - Assertions.assertEquals(1, countryExplodedWithCountryclassname.filter("_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'Japan' ").count()); - Assertions.assertEquals(2, countryExplodedWithCountryclassname.filter("_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'Switzerland' or _2 = 'Japan') ").count()); - - Dataset> countryExplodedWithCountryProvenance = verificationDs - .flatMap(row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream().forEach(c -> prova.add(new Tuple2<>(row.getId(), c.getDataInfo().getInferenceprovenance()))); - return prova.iterator(); - }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - - Assertions.assertEquals(7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); - } + Dataset> countryExplodedWithCountryProvenance = + verificationDs.flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list.stream() + .forEach( + c -> + prova.add( + new Tuple2<>( + row.getId(), + c.getDataInfo() + .getInferenceprovenance()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + Assertions.assertEquals( + 7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); + } } - - diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index b666846ef..4292f3b05 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -1,8 +1,10 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.resulttoorganizationfrominstrepo.Result2OrganizationJobTest; import eu.dnetlib.dhp.schema.oaf.Dataset; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -17,23 +19,18 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - public class OrcidPropagationJobTest { - private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(OrcidPropagationJobTest.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); + private static final ClassLoader cl = OrcidPropagationJobTest.class.getClassLoader(); private static SparkSession spark; private static Path workingDir; - @BeforeAll public static void beforeAll() throws IOException { workingDir = Files.createTempDirectory(OrcidPropagationJobTest.class.getSimpleName()); @@ -49,13 +46,11 @@ public class OrcidPropagationJobTest { conf.set("spark.sql.warehouse.dir", workingDir.toString()); conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - - spark = SparkSession - .builder() - .appName(OrcidPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - + spark = + SparkSession.builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); } @AfterAll @@ -64,132 +59,194 @@ public class OrcidPropagationJobTest { spark.stop(); } - @Test - public void noUpdateTest()throws Exception{ - SparkOrcidToResultFromSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate").getPath(), - "-hive_metastore_uris", "", - "-saveGraph","true", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath",workingDir.toString() + "/dataset", - "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() - }); + public void noUpdateTest() throws Exception { + SparkOrcidToResultFromSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); verificationDataset.createOrReplaceTempView("dataset"); - String query = "select id " + - "from dataset " + - "lateral view explode(author) a as MyT " + - "lateral view explode(MyT.pid) p as MyP " + - "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = + "select id " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; Assertions.assertEquals(0, spark.sql(query).count()); } @Test - public void oneUpdateTest() throws Exception{ - SparkOrcidToResultFromSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate").getPath(), - "-hive_metastore_uris", "", - "-saveGraph","true", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath",workingDir.toString() + "/dataset", - "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() - }); + public void oneUpdateTest() throws Exception { + SparkOrcidToResultFromSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - //tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); verificationDataset.createOrReplaceTempView("dataset"); - String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + - "from dataset " + - "lateral view explode(author) a as MyT " + - "lateral view explode(MyT.pid) p as MyP " + - "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = + "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); Assertions.assertEquals(1, propagatedAuthors.count()); - Assertions.assertEquals(1, propagatedAuthors.filter("id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " + - "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'").count()); + Assertions.assertEquals( + 1, + propagatedAuthors + .filter( + "id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " + + "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'") + .count()); Assertions.assertEquals(1, propagatedAuthors.filter("pid = '0000-0002-8825-3517'").count()); - } @Test - public void twoUpdatesTest() throws Exception{ - SparkOrcidToResultFromSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates").getPath(), - "-hive_metastore_uris", "", - "-saveGraph","true", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath",workingDir.toString() + "/dataset", - "-possibleUpdatesPath", getClass().getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc").getPath() - }); + public void twoUpdatesTest() throws Exception { + SparkOrcidToResultFromSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); verificationDataset.createOrReplaceTempView("dataset"); - String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + - "from dataset " + - "lateral view explode(author) a as MyT " + - "lateral view explode(MyT.pid) p as MyP " + - "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = + "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); Assertions.assertEquals(2, propagatedAuthors.count()); - Assertions.assertEquals(1, propagatedAuthors.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); - Assertions.assertEquals(1, propagatedAuthors.filter("name = 'Ruediger' and surname = 'Beckhaus'").count()); + Assertions.assertEquals( + 1, propagatedAuthors.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions.assertEquals( + 1, propagatedAuthors.filter("name = 'Ruediger' and surname = 'Beckhaus'").count()); - query = "select id, MyT.name name, MyT.surname surname, MyP.value pid ,MyP.qualifier.classid pidType " + - "from dataset " + - "lateral view explode(author) a as MyT " + - "lateral view explode(MyT.pid) p as MyP "; + query = + "select id, MyT.name name, MyT.surname surname, MyP.value pid ,MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP "; org.apache.spark.sql.Dataset authorsExplodedPids = spark.sql(query); - Assertions.assertEquals(2, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); - Assertions.assertEquals(1, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann' and pidType = 'MAG Identifier'").count()); - + Assertions.assertEquals( + 2, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions.assertEquals( + 1, + authorsExplodedPids + .filter( + "name = 'Marc' and surname = 'Schmidtmann' and pidType = 'MAG Identifier'") + .count()); } - } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index 61366b6d7..2b2f64a33 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -2,6 +2,9 @@ package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.Relation; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -16,10 +19,6 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - public class Result2OrganizationJobTest { private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); @@ -32,10 +31,11 @@ public class Result2OrganizationJobTest { private static Path workingDir; - @BeforeAll public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + workingDir = + Files.createTempDirectory( + SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); log.info("using work dir {}", workingDir); SparkConf conf = new SparkConf(); @@ -48,13 +48,11 @@ public class Result2OrganizationJobTest { conf.set("spark.sql.warehouse.dir", workingDir.toString()); conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - - spark = SparkSession - .builder() - .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) - .config(conf) - .getOrCreate(); - + spark = + SparkSession.builder() + .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) + .config(conf) + .getOrCreate(); } @AfterAll @@ -65,109 +63,225 @@ public class Result2OrganizationJobTest { /** * No modifications done to the sample sets, so that no possible updates are created + * * @throws Exception */ @Test public void NoUpdateTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix").getPath(), - "-hive_metastore_uris", "", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked").getPath(), - }); + SparkResultToOrganizationFromIstRepoJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); Assertions.assertEquals(0, tmp.count()); - - } /** - * Testing set with modified association between datasource and organization. Copied some hostedby collectedfrom - * from the software sample set. No intersection with the already linked (all the possible new relations, will became - * new relations) + * Testing set with modified association between datasource and organization. Copied some + * hostedby collectedfrom from the software sample set. No intersection with the already linked + * (all the possible new relations, will became new relations) + * * @throws Exception */ @Test public void UpdateNoMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix").getPath(), - "-hive_metastore_uris", "", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked").getPath(), - }); + SparkResultToOrganizationFromIstRepoJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); Assertions.assertEquals(20, tmp.count()); - Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals(8, verificationDs.filter("target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2'").count()); - Assertions.assertEquals(1, verificationDs.filter("target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091'").count()); - Assertions.assertEquals(1, verificationDs.filter("target = '20|opendoar____::4429502fa1936b0941f4647b69b844c8'").count()); + Dataset verificationDs = + spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Assertions.assertEquals( + 8, + verificationDs + .filter("target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2'") + .count()); + Assertions.assertEquals( + 1, + verificationDs + .filter("target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091'") + .count()); + Assertions.assertEquals( + 1, + verificationDs + .filter("target = '20|opendoar____::4429502fa1936b0941f4647b69b844c8'") + .count()); - Assertions.assertEquals(2, verificationDs.filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and " + - "(target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091' " + - "or target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2')").count()); + Assertions.assertEquals( + 2, + verificationDs + .filter( + "source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and " + + "(target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091' " + + "or target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2')") + .count()); } @Test public void UpdateMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix").getPath(), - "-hive_metastore_uris", "", - "-resultTableName","eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-datasourceOrganizationPath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked").getPath(), - }); + SparkResultToOrganizationFromIstRepoJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Dataset verificationDs = + spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); Assertions.assertEquals(8, verificationDs.count()); - Assertions.assertEquals(2, verificationDs.filter("source = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6'").count()); - Assertions.assertEquals(1, verificationDs.filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218'").count()); - Assertions.assertEquals(1, verificationDs.filter("source = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523'").count()); + Assertions.assertEquals( + 2, + verificationDs + .filter("source = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6'") + .count()); + Assertions.assertEquals( + 1, + verificationDs + .filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218'") + .count()); + Assertions.assertEquals( + 1, + verificationDs + .filter("source = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523'") + .count()); - Assertions.assertEquals(1, verificationDs.filter("source = '20|wt__________::a72760363ca885e6bef165804770e00c'").count()); + Assertions.assertEquals( + 1, + verificationDs + .filter("source = '20|wt__________::a72760363ca885e6bef165804770e00c'") + .count()); - Assertions.assertEquals(4, verificationDs.filter("relclass = 'hasAuthorInstitution' and substring(source, 1,2) = '50'").count()); - Assertions.assertEquals(4, verificationDs.filter("relclass = 'isAuthorInstitutionOf' and substring(source, 1,2) = '20'").count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + "relclass = 'hasAuthorInstitution' and substring(source, 1,2) = '50'") + .count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + "relclass = 'isAuthorInstitutionOf' and substring(source, 1,2) = '20'") + .count()); - Assertions.assertEquals(4, verificationDs.filter("relclass = 'hasAuthorInstitution' and " + - "substring(source, 1,2) = '50' and substring(target, 1, 2) = '20'").count()); - Assertions.assertEquals(4, verificationDs.filter("relclass = 'isAuthorInstitutionOf' and " + - "substring(source, 1,2) = '20' and substring(target, 1, 2) = '50'").count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + "relclass = 'hasAuthorInstitution' and " + + "substring(source, 1,2) = '50' and substring(target, 1, 2) = '20'") + .count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + "relclass = 'isAuthorInstitutionOf' and " + + "substring(source, 1,2) = '20' and substring(target, 1, 2) = '50'") + .count()); } - - } From ec7f166690751dde6b605e492003a863f6830735 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:58:41 +0200 Subject: [PATCH 175/259] changed the bl because of changed of the examples for the re implementation of the propagation step --- .../ProjectPropagationJobTest.java | 245 ++++++++++++------ 1 file changed, 161 insertions(+), 84 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index 521e12c11..7a742e4db 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -2,6 +2,9 @@ package eu.dnetlib.dhp.projecttoresult; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.Relation; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -15,9 +18,6 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; public class ProjectPropagationJobTest { @@ -31,7 +31,6 @@ public class ProjectPropagationJobTest { private static Path workingDir; - @BeforeAll public static void beforeAll() throws IOException { workingDir = Files.createTempDirectory(ProjectPropagationJobTest.class.getSimpleName()); @@ -47,13 +46,11 @@ public class ProjectPropagationJobTest { conf.set("spark.sql.warehouse.dir", workingDir.toString()); conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - - spark = SparkSession - .builder() - .appName(ProjectPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - + spark = + SparkSession.builder() + .appName(ProjectPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); } @AfterAll @@ -63,123 +60,203 @@ public class ProjectPropagationJobTest { } /** - * There are no new relations to be added. All the possible relations have already been linked with the project in the graph + * There are no new relations to be added. All the possible relations have already been linked + * with the project in the graph + * * @throws Exception */ @Test public void NoUpdateTest() throws Exception { - SparkResultToProjectThroughSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), -// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", "", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), - }); + SparkResultToProjectThroughSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); Assertions.assertEquals(0, tmp.count()); - - } /** - * All the possible updates will produce a new relation. No relations are already linked in the grpha + * All the possible updates will produce a new relation. No relations are already linked in the + * grpha + * * @throws Exception */ @Test - public void UpdateTenTest() throws Exception{ - SparkResultToProjectThroughSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), -// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", "", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), - }); + public void UpdateTenTest() throws Exception { + SparkResultToProjectThroughSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - //got 20 new relations because "produces" and "isProducedBy" are added - Assertions.assertEquals(20, tmp.count()); + // got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(10, tmp.count()); - Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Dataset verificationDs = + spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals(10, verificationDs.filter("relClass = 'produces'").count()); - Assertions.assertEquals(10, verificationDs.filter("relClass = 'isProducedBy'").count()); + Assertions.assertEquals(5, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(5, verificationDs.filter("relClass = 'isProducedBy'").count()); - Assertions.assertEquals(10, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("50") && - r.getTarget().substring(0,2).equals("40") && - r.getRelClass().equals("isProducedBy")).count()); - Assertions.assertEquals(10, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("40") && - r.getTarget().substring(0,2).equals("50") && - r.getRelClass().equals("produces")).count()); + Assertions.assertEquals( + 5, + verificationDs + .filter( + r -> + r.getSource().substring(0, 2).equals("50") + && r.getTarget().substring(0, 2).equals("40") + && r.getRelClass().equals("isProducedBy")) + .count()); + Assertions.assertEquals( + 5, + verificationDs + .filter( + r -> + r.getSource().substring(0, 2).equals("40") + && r.getTarget().substring(0, 2).equals("50") + && r.getRelClass().equals("produces")) + .count()); verificationDs.createOrReplaceTempView("temporary"); - Assertions.assertEquals(20, spark.sql("Select * from temporary where datainfo.inferenceprovenance = 'propagation'").count()); + Assertions.assertEquals( + 10, + spark.sql( + "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") + .count()); } /** * One of the relations in the possible updates is already linked to the project in the graph. * All the others are not. There will be 9 new associations leading to 18 new relations + * * @throws Exception */ @Test - public void UpdateMixTest() throws Exception{ - SparkResultToProjectThroughSemRelJob3.main(new String[]{ - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), -// "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", "", - "-writeUpdate", "false", - "-saveGraph", "true", - "-outputPath", workingDir.toString() + "/relation", - "-potentialUpdatePath", getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates").getPath(), - "-alreadyLinkedPath",getClass().getResource("/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked").getPath(), - }); + public void UpdateMixTest() throws Exception { + SparkResultToProjectThroughSemRelJob3.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = sc.textFile(workingDir.toString()+"/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); -// JavaRDD tmp = sc.textFile("/tmp/relation") -// .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + // JavaRDD tmp = sc.textFile("/tmp/relation") + // .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + // got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(8, tmp.count()); - //got 20 new relations because "produces" and "isProducedBy" are added - Assertions.assertEquals(18, tmp.count()); + Dataset verificationDs = + spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Assertions.assertEquals(4, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(4, verificationDs.filter("relClass = 'isProducedBy'").count()); - Assertions.assertEquals(9, verificationDs.filter("relClass = 'produces'").count()); - Assertions.assertEquals(9, verificationDs.filter("relClass = 'isProducedBy'").count()); - - Assertions.assertEquals(9, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("50") && - r.getTarget().substring(0,2).equals("40") && - r.getRelClass().equals("isProducedBy")).count()); - Assertions.assertEquals(9, verificationDs.filter(r -> r.getSource().substring(0, 2). equals("40") && - r.getTarget().substring(0,2).equals("50") && - r.getRelClass().equals("produces")).count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + r -> + r.getSource().substring(0, 2).equals("50") + && r.getTarget().substring(0, 2).equals("40") + && r.getRelClass().equals("isProducedBy")) + .count()); + Assertions.assertEquals( + 4, + verificationDs + .filter( + r -> + r.getSource().substring(0, 2).equals("40") + && r.getTarget().substring(0, 2).equals("50") + && r.getRelClass().equals("produces")) + .count()); verificationDs.createOrReplaceTempView("temporary"); - Assertions.assertEquals(18, spark.sql("Select * from temporary where datainfo.inferenceprovenance = 'propagation'").count()); - + Assertions.assertEquals( + 8, + spark.sql( + "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") + .count()); } } From c925e2be165abe793bdb228f2d955882a179623a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 10:59:53 +0200 Subject: [PATCH 176/259] test for propagation of result to community from organization and result to community from semrel --- .../ResultToCommunityJobTest.java | 312 ++++++++++++++++++ .../ResultToCommunityJobTest.java | 276 ++++++++++++++++ 2 files changed, 588 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index 0051ac01b..78b311bc1 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -1,4 +1,316 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; +import static org.apache.spark.sql.functions.*; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class ResultToCommunityJobTest { + + private static final Logger log = LoggerFactory.getLogger(ResultToCommunityJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = ResultToCommunityJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(ResultToCommunityJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(ResultToCommunityJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = + SparkSession.builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void test1() throws Exception { + SparkResultToCommunityFromOrganizationJob2.main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-preparedInfoPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") + .getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = + "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'propagation'"; + + org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); + Assertions.assertEquals(5, resultExplodedProvenance.count()); + Assertions.assertEquals( + 0, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") + .count()); + Assertions.assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") + .count()); + Assertions.assertEquals( + "beopen", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b")) + .collectAsList() + .get(0) + .getString(0)); + + Assertions.assertEquals( + 2, + resultExplodedProvenance + .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") + .count()); + Assertions.assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + Assertions.assertEquals( + "euromarine", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(1) + .getString(0)); + + Assertions.assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions.assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + + Assertions.assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") + .count()); + Assertions.assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + + /* + {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8"} "context" ["id": euromarine] updates = 1 + {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6"} context = [ni, euromarine] updates = 1 + + */ + + query = + "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; + + org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); + + Assertions.assertEquals(10, resultCommunityId.count()); + + Assertions.assertEquals( + 1, + resultCommunityId + .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") + .count()); + Assertions.assertEquals( + "beopen", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe")) + .collectAsList() + .get(0) + .getString(0)); + + Assertions.assertEquals( + 1, + resultCommunityId + .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") + .count()); + + Assertions.assertEquals( + 3, + resultCommunityId + .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") + .count()); + Assertions.assertEquals( + "beopen", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(2) + .getString(0)); + + Assertions.assertEquals( + 2, + resultCommunityId + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions.assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .sort(desc("community")) + .collectAsList() + .get(1) + .getString(0)); + + Assertions.assertEquals( + 3, + resultCommunityId + .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") + .count()); + Assertions.assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(2) + .getString(0)); + Assertions.assertEquals( + "ni", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index 9c4d37fb5..f8806d8bb 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -1,4 +1,280 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel; +import static org.apache.spark.sql.functions.desc; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class ResultToCommunityJobTest { + + private static final Logger log = + LoggerFactory.getLogger( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = + Files.createTempDirectory( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = + SparkSession.builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void test1() throws Exception { + SparkResultToCommunityThroughSemRelJob4.main( + new String[] { + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample") + .getPath(), + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-preparedInfoPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo") + .getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = + sc.textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = + spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = + "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'propagation'"; + + org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); + Assertions.assertEquals(5, resultExplodedProvenance.count()); + + Assertions.assertEquals( + 0, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") + .count()); + + Assertions.assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") + .count()); + Assertions.assertEquals( + "dh-ch", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .collectAsList() + .get(0) + .getString(0)); + + Assertions.assertEquals( + 3, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") + .count()); + List rowList = + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); + + Assertions.assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") + .count()); + Assertions.assertEquals( + "aginfra", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .collectAsList() + .get(0) + .getString(0)); + + query = + "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; + + org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); + + Assertions.assertEquals(10, resultCommunityId.count()); + + Assertions.assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") + .count()); + rowList = + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("dh-ch", rowList.get(0).getString(0)); + Assertions.assertEquals("beopen", rowList.get(1).getString(0)); + + Assertions.assertEquals( + 3, + resultCommunityId + .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") + .count()); + rowList = + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); + + Assertions.assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") + .count()); + rowList = + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("beopen", rowList.get(0).getString(0)); + Assertions.assertEquals("aginfra", rowList.get(1).getString(0)); + + Assertions.assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") + .count()); + rowList = + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("euromarine", rowList.get(1).getString(0)); + Assertions.assertEquals("ni", rowList.get(0).getString(0)); + + Assertions.assertEquals( + 1, + resultCommunityId + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions.assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .collectAsList() + .get(0) + .getString(0)); + } } From c093d764a354407caf5f5a85904bc10f1d997da7 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 11:12:38 +0200 Subject: [PATCH 177/259] - --- .../graph/sample/software/software_10.json.gz | Bin 6410 -> 6410 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/software/software_10.json.gz b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/sample/software/software_10.json.gz index c2389b7677ca13978ff3a9b6a41cba4cbec20c0e..a5b8c8774c5650641feb3a3c257b4bc182777b38 100644 GIT binary patch delta 16 XcmeA&>M~-N@8;k*)-hotJF_GJE42jd delta 16 XcmeA&>M~-N@8;lWORd?+&MXN4D8B@C From a303fc9f73ff1b4bd1aa5587fe42283f255f9e8c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 11:14:16 +0200 Subject: [PATCH 178/259] resources for testing propagation of result to comminuty from organization and from semrel --- .../preparedInfo/resultCommunityList.json.gz | Bin 0 -> 939 bytes .../sample/dataset_10.json.gz | Bin 0 -> 7128 bytes .../mergedResultCommunityList.json.gz | Bin 0 -> 2066 bytes .../sample/dataset_10.json.gz | Bin 0 -> 7113 bytes 4 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz create mode 100644 dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..8b452d0e17e06baf8f3ac6d62277c2a08d88a13c GIT binary patch literal 939 zcmV;c162GUiwFqE4x3&819D|^b!>D)Z*6ULZfSIROlfm;E^2dcZUC)UNs44g4Bh(} zx$cD`LZP8@l!t}WKNm@oi~mx3dP9D_`Sa7$m-mmK zfBoU{)92g6kGGT0AM*X}r#H8c`RU8M&)>)G;kMzo$oTT%?JxBfU_3lnS&b<$XG|5I zj$wf?&x`|DdHeR=*L$D6il&!OhKaEW(}bA7QSq3r^Uu%DFCU+u?BnD6yuG~3&%s|TnnNK`39&|9gmQR^^)cy&=s6Bd#i*H5QLtIB+1TkZ{EX71BWqR2%q`4|Ej2kV zLaE`$Xbu`XC&wA2Q)t_+wGHmSqYzhtV#169)z=f?4T2SrFGW$MVAci z#cRb&=If|3Y@6Y=v(gpw{#pKlLaMD5z?D&!2rLu=<}o4q`)eGAYpO*Oa}u@F)L3Jd z$>QU@{Qg;9qjXTxWOLryKfW1p80T2`*F7Zf?_;kvCf4TVfQ)M)3`GtHkAtS7p7*EN zE0oqDLv^;uehEwIGY-fOHYCG57pbymXd_8k?Iti*9Lp|GzHD@<-FQ*EKpT17X4im?zDv9FZ`3?^$o{xt=^b%UqvwSXlvmbgQzYv9pQ{J1+XnkNMcuw`5J-E<(r>qn*$BJ1>i$Xvx zy<%O6Y$a7WV@ejqcQpNGrua%0qLgY4#r3T@&G=nzF4&T1Tt7I;c{yw9ex&Ozw)fa4 zUUS#mxBVW}TUIMj7cWJ|i;`xm-pr3Ckrzdi(PZjPq0v}V^Jg&MG zLDLEwd&*avpHNri6lyPNO8C*APoA8>Esa&=Rhqtq8dlggcy3+T_Xb0M5*@#Idh+rp z@chY<9}Gh%`fWv1F&A9z0-H#&ho1T65=0_-Q?6J18fdmx-}as9-pzCoJv3W6w^pJh z?*(4q9eE-AdF6*QFPa6Tfj9O4tykY&Dw-E_sWN(XiP0I2d@xAjX@7 z9cw>Bt8yuo?rU^D7pbu7N4?IWGg`QxRQaM$z7&8dIuQHo_Iur3cRTX6z4NOVCugU7 zCR+d2SfY7E+v>D;XgrDs{~W=8{?Rn@!=qq0n)J~8z>>ZJX9;C1)si1N@y(6=LT&zc z^FJx?lb0fsd8?t@@1~*KI)Zi`Unu&^e_?qe1TU(zMDMZZnLtGaFZaFE_SgNMXr6;^ zNTzZjuJKft@6uyPzNqM8PXJz|$6zgc0=!(&TolVI1gM{Kb;Zrqw>U0Tc?teRYpAH= zbLjp%Zu~@mqZ70s2`4EfCo-#Y!6*}(pYrRS`JSk|GVV*|T=m^ub-zJoEtWJ9MJ62# zhlJrmD9}Aj*c=Q8g<^)Mvwkkq&HEBwstX6qA%Pi%zBlkg^2$x0 z4!RayiV{`RT7-I>5|JkYoEM@2zz`^A_>IuRv* zvwK5Z`n_8m#J{G?owZRV;0tL^Kn^G+3Z4*)?+tSX2XE)#)hp>Kzo2=U=*SH&Ehp{B zPOE3X-^8TxZtY51vzp;Gm%R2L>8NO=Yc%?Q9B#OeL#HJH1Qfum=~_O(Y(EBB3OjyT})82@?oT zGbD1!Y zWPPRn)fULBEN-Sg3tRzXTVBuGfBC`T|4>}hXm}mhG@OqDHfO;YoL~}-hMZ1Dz6$Z z4|gON>c`gmASo>p`k8S=z2_^9Ac(Hk>=SM}Q$qE%KD+M#AwfU_e@S>nVJa`T*UibQ7 zoIK|y5<16MaNPhVbIG{EcHL8m0eQ)Z;uv4u{6wl8g{TsNZM3StkYtJUqirY^Gy!jv zq-ZHYN&FdM#)=rsO`#Df2eqTxxPp0jEsesZrUE`JE10MwG9Yx*^7=MtpeR8#@C2A+ zaw0)h#lS59K45eBHOw`iOoyW(8&Wo8 z@Y9@oZ0rxiF!sk0HSzQghPg)f4Rb}q4;keebt1|&@vcnHfl;m@Cg%vf+3-Uo=V-+v zZhRf>lY6hE+sAIpJ>4lJe)+pD_4NRXw2L*-jjILXzx=(X$!-H%t0sK26LG0+)P+V| zXyl_CS=#HsTX1-@h~jo)9e7(v2JRQe$hqz=ZX4D?!FX;LMvKcUqYjuu0uu~E?aT=@p*84iYUXMxqL; z6ug|;_#4M?(26Q}+rp5MHC}iGTJH=mi{0_Pt}PK7e0WhK^vYQoGz-H=+2&m-t&|vX zT5)Bi$~Hej_nKpZIw*)yC9ICu4zaFq@H)4;1{q_qd8ZpuQuqP%WQ>ENf^Bm)Kf+z5 zm6H?zS7L&1CY<&wdNfGz@EE$K#Rm+ajc_?5aJv#Z5}DY@XzYT<4;eHL{HV1zQqpsHQ57uZ#cVc7CjQt9r*ssK zeQ(a@(=iXdxj!6@0ybikTY|>^IGAFvc;Zi|@ia>OBpQt-kx%F2BxF(4#h`IC7)9Yd zgT~|0hl?6-`!Om#6^u=)YwIhe=kGJZcq$~_bv!_a3KCwBnq_VkT^IJ z&M$Qa1bCZu6qW<;xW-wxlBsYTeLK-5xQ!MT&^Pon%Yov!D-NDpxhqe1tM?LQXb&NrTV=p}3s%9{_ElsuyqC4KnN2Jr=h`{(t(h;hO@2Do9SjTcylixA^w^B`plU9C$U8+gQrjxlh3WK>n z8AtJOI-c?<3St^k>iK-syP0Yd@Ngau8J~Jl;LRxyBae=`?++)*H~}qokZKYR2U9<| z7u94q2>i*1qM9`F9G+@olfqm{H3?_Ka2EInrJDQ&JP>qwhTci1%OhubE-$uGSNoK` z+v=5w88z=3ekY$&N-NXiqTv(maS z>|q^KJFe=WI3yIG@f`P+3+JF5am2xM2ja-tZYchU*zuic8FZrZHOV=~IWBZ8JSA08 zr&wTiM%JX0xOtZBs3To8in~)sp5M&jC^)9t*!p;`Dm1I&b*?!ur&|t8BFu7#Ht(l2mWt|? zAZ+Kg9?bTETh?T9-Dzyf7Pk2b4^KxOkwW15o=#+PS(A$3?^|WeOSY{6(9wd&MQ4>ApCpH0Wt(23KFOL3?R)4 zqNFMXnxDNB#8ueaN@y}Drj5<^#0NY$f`7ll$k*2NG(cZtyS8K_imECA^sHJFFo4W> z-We&C;VCf+jfZ|<0K3^RwoQ5PKZ@_>EV@}T?NqJT2C z^o3%tDxMhpmC?p_C_upmcrJ%A5xsCKTlS%yLt(oGA`4yJFc|yRrtHI)6v5IhQzYZ( zQuS>qbDg4zf$r@u>JGz=7NzklT5XEr~jbsc(j8>9f! zVQEzu5~av-QMxcHU6akvpfLEP)v#|lhFS;Z%&8o&d%Jqrj-OW|xsr+QUfWYf;B&m4 z>>_cm0cCG+)|#vh-)ahaG<*_^A^B`%dk@mD+RR_?F#tV2*Fu5x%9@X}+bK=Q+Nm z<6Ax`-*P&+d>)h~OwNb%kWB)Q4I?j3yh%bC{N;@q{FtzL?-su0ZN)pux14yR`|>S= z=?COnT6qr5w;WH$z3cgw!3_Qc;fLj0o>Y2W@Py{FPoC0BA@|5$tG_mU$=2OHM~-|@ za^#7`IX-jwgAO0re@4|e;Mem7NsKFWa2ym4u1_}5@3g67A051JFqhHk!2Hw3|^R=5cNaOA_wDH)6Xuv6VH~Og2L`<0&b`A{Thtnu&LVCGh5r z0qMPP`;x|ZW}2p7T8^)#q?vSC$Tq5qieh#xP5QZcP^mLqrJ5MSC~pXvSorCkwTDy} zE$Eioaw1c{Ky5u&;)3Goau~o*IIi9ab12>a{pWw5Y~CwY8P3iLC?BhNoK#Gc!^j`C ztbf8Aw^2fkDe{_fq!uP(TcK9;Lg5Xz!HH=w1$}PbWq3{t*e1X^ z-MlOM00Fd-Dnq%#s=oRj3M8lp3GgK{*Q76~9$Iw1`3X$dFz^(wazoZK^$KM{DtWeD z9?_&MM)O3|-k?WTw#{p@0)1uBzL;v#qv4yDt<-5+Zl$@Pq=cKm`jHtywMmH79PT%; zuw8|@UI?wJnYBfd&Ci;cxh29upUMmu7DURqtq_R-6sc4w+7g@;)pR$g97Uvibqd8&@oh^`WL~Vm(nMLcd+C5&qbckiH`+ZWr!$8unk0_MBrOcl3>uzubA+Mh zTuPIvF59NH1FOwMjs{@Jk9G|iHS)+x++_FD=^vSh;b~1`I-4T0QEI=faclLfHpA8! zY|XU=kHb}IWVCnVdhF&O5<%293Y8*%Q6%OrXfpU-@Y04ynd_5RX{xR5l$Kj6C{o)ewS!I4CT$Sfx3vTV*_=94TM48F0+B$5a`%Ijt%s1Y@o3h zO@iy#K;tNwCte)HF=aldK4X!`@wefK$Dt2_hJJ@gRwvQfNUTu z&*9lXlip2iplCb`MlSoBBN*M0VD!8Y%x=PSF_*)pGGcNm9gFCoAe`#;b9dmyy_v8@ZCFwW)BIH<)Pf ziOlO%DA!mYvS41wf|4f{zC`bkb9N52qxGS2S7!%n5;B=&LeMx6?z~!P+6--m{F*wk z-%l{P(KE5HC(Oi!X{1abyw;M^JcM%dj%j9uR&TpE;tq8~xhPxuHVLyd0|Ix<+jXZc zeL?5>S;765*C6txh%I@gImC|YoK1zyX)6m?ew}uaOpu$$+Kf{ec9htHqN=^TZU6<5 zz85upU`g>oHfdH%c>Hu>5-08DDYF&YL_#&IX;+(l0vhMYNYUJb+cU0N^IYju9K;Ik zsgL!Z&G!(<<=o6Z7&6r3`4k9is5U9HTHcgpMX+hMOmWbmz1>nSIz#pJW2>K-#*f&Hb3XvzVro+ zl>o6{RIvbA&-q-FMPRA`O{b-*^B!HTWlW5vEjQ6U25>B-CXe)uL2FCWy%oe^nlOU5 z71cQ}b<5xjCcDxk7&N_HbFws-hNd+JW-2on1@VZARKJQ}l&JetmTqtVkX__>Z(``1qHmz(+LIis+ zxUvTz^g+}NmoYJ0BUJyS0#~OKxk;p@p*P$72yhjbd}trRpfPl++;jc0&ef z-KpRQOmeu^&w>Wg&3k-u0;q_*T60Vd=R$lBo-8YJI(5vU-zA!kIrQmb>zG3wVGd0? z%^Y$=pkod>=FmejhsKe2EpuqjVjfa23?|-a6tX1Y@jMAxJPo|iA7f&f8<|7574IH% zD2N85=zixmMuTbep@>6`IEN<=+1zcHh(lh4!iJbp$q|PfamZyR`UnS!-N-H2oR1WK`a6JqDaGbw7EV?pqVAxd1Wb?i# zVN`dx91@q$={KC@{5v=E8l zCnarP#bjPDtGIRcs7@AkF3I!F&lqUa@;t(1O_YS>ONn$gdTj= zP%UXrnw4U69BD@Jq(7?*Loh0~d4V!nCk(T1GVp_u=-21y>2yl4k}Iaw;ZlORlwd9; zm`e%vI~!nKK9rA%*yng=pR(aNo>?b&W}{B?%$!GdJTu2LdqAF9;CcRaJhNmx^+WhL z8poWCl90!N?+p{q;~^hTM}F^Sp4qL%yT>!bj8K#NomV#=j068eW~iEU;=H=i(2{$1 ziDusL$PZq5<5@JF1!G4vb40T{63t%ep4GhhlN0@(Y*!`rv2!7Y1I9@m5FG)+u@#Z6q?) zLz)DmVI1+8CleY*aWL`1;cX{-+*-Voag(Wkzy0~+!FV|SfN>Km&!OWcxgyhRgqBKzNq&;md(*HcYJ|y z?1?U#!`;~vCo*rdV0W!1wN}$77$(3?_QLFV#+|aysY+?4>9WR<@Z=P8G7HUCm39%2 z(u^*Yybn<#rsonLQo=hMZYY1<`!xoOXRS1Q?3Z}@5m$P2H~W;$c|54NtIM0h*oN)g zjI3>0v$-EPRpVVNo(gPYs-2=8vFrw2c6q8jAg}QJl+8Ol(Wf~^Cy}N-b*=(S3+a=W zG^^msGq`c{K0o@5s+cS3=py&(4tLqq<@-*jjxN%QdN{g>qllPrv2=76ENAT5E!sF~d(hAr>qy;{ z)~+XW)2eaUPO|6X6&!nuFS;phv*j4h)wa&`*h-s)s)sH#=2hvk@A2#J^Jn~<`OY&U z&~rIFWR5vjiq~Y`XZPsSUOnTf{zlpc`YoE}bGjYe1*SXc{ch>SOqC?Qy+fPws4r=r8dR&@m;%P4dRH;l z2JOQX8+N-3^c<4dt}f9;+gQ8P8iUt@H)Qbh<3zaMm;#;E$)(dXfM@i>AtxtvJNh~6 z#>%MnYF(Gg-UaK`b_60?8$O9CkBid1m2?YU zjlj{*oCz&C={Ks^N0W^sV(K8z26MC;9&xOce7!V*PjARamwC)>%`jAj0V}%P3+-qt zq2gb>;~3qg1^w&*62q!*oLd=4f_)i&Gtzx`z!_(pK+(PX#(b=HT9?GwT(#K^kZ`KX z74`);+>Y4}oq<2V$5rWC-D>BNmT3!`u|Yc{L}g;zf3MSDz5M{VmU+mERzZSx9ElyF zK(?TD)$v?4S2<7&Tf^8)j{#g=22GUyQu;pc?75}PTAqD;^=``=IhoDCF5UGV051zB zcD#>aaI%SE^kUpF_i;5Xpgdo^u{0PzhqF(ILY3O&-229CKu>PYK~t7_GXN{F9wY*D z5Z0HM`PN1G8SL0#3g;Gd1L+fC?s+%`8C4+CV-4I9FVi9)yuA zSR=v!=1*1;L{JrfNiVx0voWF&kcwQ7HmV2xVz5gCU7st19MGuj*z-K?Y&0j#!62M9 z@hiQW4{49#Vv;2k^2E?R^o3|Hp!}7*4jn*$fKJ5DQ3t2C7;?^(8nkZ=5_Y>BGmUt? zmJfq9VB2wthy^;&RYP1G>>THVbi>}2AGhgiFo1{W)is_P3`~G=>1xP$zQB_+XGfsf z?M9qNX6I1^c(!=1PKOWA@CH8y20j4ZPk%jvpZNg8k^s*H@-)f-LIa*@82vijzC5n;%93_K z+HPErV9lZv2!9L$;;UKFn7}l!4mF}INAifm$t9|-aU)a=R8^^ohQJ?TyfDnd#)GYl zy}x|p@@{EAZO-O~F{ia2r<5&@(<7VtzS_$mpdbR|nSNF}jlL)8;xU^>yiC<^mX_9f zQ*XZJfdpoZ#g3Rz+9`S5yeWEeZXKDe+cVtYv;|u)S_po~@L&FROF*i#7AQ(57$*hR zyP^*1^ZJ!<+(yWBCs5vWY^(_Kj7jx4XFoK56CwpWNPAmO5b$}3lAAHm8^NT`Yw~-q z9m9;qqF{tEB!f5;fHEmM@>uxG_eAfW8GL|gu0`7^hE}uShZTtE;o_UpNE`fQ4rMsH z)IMt=^b(#hG2A!$A#)#OQbF}+kR3mw5_>Y>W@F!&uPj-}M^z_qRL5h)n7BP&i3PoR z4JFEAN03Vq2N3tlA2L2R`|5Mlm%lm#m@Np>l3e+`NCX06@EQ^|tbNTJITdS@4FRJk z=%y`j2?W|ZL8u*p^`^8Va}+NnK0HU4wzNVXcE#H3)h2=sgsCZwxh4`B;u3N%b(W28 zUzI+2mI@pON|K{voHw)|K{)n2^Qtr|Kt&z3CNWIC6QWJBC>&?#l~U)L8?;gG3}DBU z@blt%3FiUJUrqJ%7JRxl0&=(^B~a@cdo)2#-8XAKA@w$B8TiBOq_mDPksOiB^mX2S zw=@SV0G>h&o;;H6`DzjXz@u4rrE%;oIRoIU&<#i`7QQ(^YI)I1SA6HK+?>hZygaoFJCPD{{!9|#giFoYER}zvRSx3%|`R&28LYq zS~nU@yhjfRhLIxewFGEK8C`uwp1?e$jqY1e-TC>v6kf>ax@*}DMb2H$=d|QFAnDkY wk24?|1ft&S?;M{MW3Vm_u@unjzV|q(Aosqpmko~o$DgeJ3&=jJt+*cm0A;WM0RR91 literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..bf77a6eba9232f71c38c683c7b5f31fdd30f8586 GIT binary patch literal 7113 zcmV;)8#d%0iwFoVpq*X-17u-zVRL14UokK)YIARH0PS7PuG}`3-p^AI1vJnN)UB(Z zQvL0md!5WR?8LC$4P@E_gQ8efyphBTCEcVtV_pT17X4im?zDv9FZ`3?^$o{xt=^b%UqvwSXlvmbgQzYv9pQ{J1+XnkNMcuw`5J-E<(r>qn*$BJ1>i$Xvx zy<%O6Y$a7WV@ejqcQpNGrua%0qLgY4#r3T@&G=nzF4&T1Tt7I;c{yw9ex&Ozw)fa4 zUUS#mxBVW}TUIMj7cWJ|i;`xm-pr3Ckrzdi(PZjPq0v}V^Jg<} zIGIu&1;fDeIZr4ZkE4VJlOSO{jQALYkV-5>PSZ1Zz+ZbZd87Xcr;{lZ50WHG0u}^; zKlNiTp2o>=9)@hfr=uvC_Wm8TBGZ&J0A{Xa29Nlm_YGfzEON#3L{f!Mo6YcfP}v1d zD{Sm3Uu}LuU6E6$y`(AOM}IzfasszBR*_d}`W9+fVcX!jbz$Ed4E;%T{Nm}!%Ohym zGt8Piv1$3lSsO@HXXj*cOB;W*S#kAmme(@B_EX#8Bhw#xKg3sa4R((Pa zDt68W&baosJykijKO~Q_~ zpP^N`luGwCI-iSFSoNb`=g=7~+)t`}(I;ODz?4JH_SfzAy1VXni2z3@Xh9NAQc6x_R^@_GCN@9i*E{n)QFmqBm&&>7ySwUsgUnhiX(EbDIv5TK z!-Y_w`-CPAhJ(Ved7p6C6%L94Twl=z%@gWyI3x}*8CEb`bwfi}aBwgj5{7d-QF168 zex6jW9SVmoUqWVL$^1gIkr3%Awk04Dv#!C<-MO%d)vCw=lJXs`%x4ZCf@F2-- zu=W9=X&c@o($QM|)R<$)Tz?1Ma|J`m3%=Ddq9yXeIxm1C6$}QE9Yk`XVV?9RlgTw< z?VTImHo#ww=X+i-+^hR@k!Dx59o;q4*8HY+3|BY*svALwr7Se^D8PdfuKK-7rAP&{ z_Pb2d!II`l$_F6N@g;T&4(E0@g+iv1)uf%?C34t0 zF%2fYTezlCGGXJ$AC7|gG>#`p5PPF(G)m?%i`Y2+ExD%mpe>H4_oOWb;q*h&7L7cI zrY#1+xOa)RI6m_IBR_iOk7mJi7Dc}?ZSiw~eL(feGt^FR%jc`V9+tNl90e0^bmWI# za2;>aHsDt3;%8LFRoW+~FpcJE)6nhrn~001@U3;cWXnt>Wk&NH6{=WUpZeQ9)%onv z7Z|#z4Q*GimoDBz!C;TWU^Ml1^cuFC+-l41y5;PfC9R4+c~Qa3@Ip&tueE;J)9`Rd z!}bx58G1)%C=O2-dfDys&HDl-!({CmokxWGaxNI%+!Qb7~&MoEg6 z5|qTB5oWB2!Q2!Yk#bNws*Nj{hu6|5Txu%d!?J>jIwAu?H!ZJkg9eHcR0B_dIVL9( zR8lM zq9&f+!7$h8zG1Ft_#vZQqfSJ*Cf=3FIWWpK#N-^IHyeJ4rM@0Ok#?~rx^cBY{FlGiG}&!nYt@8rb|Nmdjk?f?3ypkqBTIW7cnc11 z7E#7&+J7#cjhnC>YP}!f0`MWz+$4NMM4Y@53Yu27GwyFq=vBu=4nv zD<+t5{iZ9it)|n+oa+Vp`^^x@$;qS7Pn-Q5F7Wjc*~8rfU!TD&6QF+~Gp=O6@wJIm z+8`u@mmiKMb5HZRLr|=5V?r@JU)tvuB;CBvIlaR3*FmBs#Yj{^m4cU38-L>%4q8zK zZ(A5Lvc?OKK@=gAXrigkCu-gJxm)DBHX%rIivRPAjggRN3Z7=w5S7 zPzMDus)W_?+9B2z4qoS0*C1moHt%#JN(w)Ko{Vu&RIqK%=0~`zv~rRH;7UyJ&4kl_ zMUMsv9v(xtwD^Dlv=J_61a4PCM*?UKHV?g+;^6&gH%FQkMYlK7?T!5CdXozpcOqzv z!GSIZjomE71&v+M_#uPFfgiPk#*&`Hi>hELFJ`kzGV#Y=IHjX-?0a)IpN@Ix&HdqM z6tEGyZEvJM4yG6^p7_&gJdF}RiAJMIIxA_+FD?5}MbW z)RU_R9RhVzeS340o~#)e8I&%nhWvu1tDKUI<{Zjtc1yt*mBKyj&+r=<9In=Bd1s`x z?o_X89b*vT*b9%hsu>J!OOx$_=#F>t5$W_d;+_069?PNWXeOU1&4gDkrsJE}=H zdz7P^bcAZ+JF3Yi*0CJbnA3R}CkdnTi64*S z-py2#fQR#N$oSNY0&h-v72hb9u3iy4t7g-B#ZtfBCzfi8>yMiFBGqNU~#Lcs0M;+;+ zQQVz6^898FN5L`G#^%R%u95t1$^OY7arx>`q>!=NHLbPIz|XB!vPwU(sPpC1+UN$F zRZ(0`tC$?8^eB$mQK4BCuXD|TIo)z#5@D7@w0S?Ju~byA1YtX`^rP`+ zwy@1lcz8PMh!g_X_jDqY%bINThsO%+Fi?=&vN!wFJQuvNs4^{9tvqvtLhnml^ir29 zLb&pbDuxZqOJxpNU=&Qxwr#%G_9jD%u2IJ4T%S#VB9}1oW;)qQ>qr8Ve#K&^@6snH z?75CTk`3~PM9##DrkT#9Q_z?zP#8WNc{X%iB~VZ;K(co^E|nMy%jg;~MBWguk1(hS zK7Bj^K{k10>cbkotpF*wbYAI61L5Cm4v--LQII&@X8>td5G7S9(ERM3Ag;pRRzj0O zF>P$NCqCf85&ZiVM!vSDrvdsJ+qESdQB+j{pl8*ZfB|I2^Ug@A3{QzsXgu@-1K7=m zv6Zr|ahhAkleR89bwz_%L!E31kGkNPmj~>_mIvME76p{Cr7skFRq@2&uZ%XfLjejl zz;ii_iRgt>*|HDq917bl5LxKzhQZjkHf0~aqzIO7nIaiKm#S||nd=ly40LY?Dr)S; zI*XF4Ou)mjf_8gf_q6(-qhX+E`~4#aKePEMt?SUM+aLv)4oj=TkSIlti_(Qr>6&bQ z28F>Vt%iNeG1NLJXHMmK-P_f}cKo~&$(2lW_u8H^0-xjUWEY8h4Jdnqv({v7_*PTU zqv3NKCxVv7E93wgZCCkLAm|Vagck}l{L>9lJEz<)AG z=FoHmz)zh3*mr9Gt<;_i$G7aJ19N=Kj_@tLPV+6@JkRkh9pCan`Igg3!?)ZcOwNb% zkWB)Q4I?j3yh%bC{N;@q{FtzL?-su0ZN)pux14yR`|>S==?COnT6qr5w;WH$z3cgw z!3_Qc;fLj0o>Y2W@Py{FPoC0BA@|5$tG_mU$=2OHM~-|@a^#7`IX-jwgAO0re@4|e z;Mem7NsKFWa2ym4u1_}5@3g67A051JFqhHk!2Hw3|^R z=5cNaOA_wDH)6Xuv6VH~Og2L`<0&b`A{Thtnu&LVCGh5r0qMPP`;x|ZW}2p7T8^)# zq?vSC$Tq5qieh#xP5QZcP^mLqrJ5MSC~pXvSorCkwTDy}E$Eioaw1c{Ky5u&;)3Go zau~o*IIi9ab12>a{pWw5Y~CwY8P3iLC?BhNoK#Gc!^j`Ctbf8Aw^2fkDe{_fq!uP( zTcK9;Lg5Xz!HH=w1$}PbWq3{t*e1X^-MlOM00Fd-Dnq%#s=oRj z3M8lp3GgK{*Q76~9$Iw1`3X$dFz^(wazoZK^$KM{DtWeD9?_&MM)O3|-k?WTw#{p@ z0)1uBzL;v#qv4yDt<-5+Zl$@Pq=cKm`jHtywMmH79PT%;uw8|@UI?wJnYBfd&Ci;c zxh29upUMmu7DURqtq_R-6sc4w+7g@;)pR$g97Uvibqd8&@oh^`WL~Vm z(nMLcd+C5&qbckiH`+ZWr!$8unk0_MBrOcl3>uzubA+MhTuPIvF59NH1FOwMjs{@J zk9G|iHS)+x++_FD=^vSh;b~1`I-4T0QEI=faclLfHpA8!Y|XU=kHb}IWVCnVdhF&O z5<%293Y8*%Q6%OrXfpU-@Y04ynd_5RX{xR5l$Kj6C{o) zewS!I4CT$Sfx3vTV*_=94TM48F0+B$5a`%Ijt%s1Y@o3hO@iy#K;tNwCte)HF=ald zK4X!`@wefK$Dt2_hJJ@gRwvQfNUTu&*9lXlip2iplCb`MlSoB zBN*M0VD!8Y%x=PSF_*)pGGcNm9gFCoAe`#;b9dmyy_v8@ZCFwW)BIH<)PfiOlO%DA!mYvS41wf|4f{ zzC`bkb9N52qxGS2S7!%n5;B=&LeMx6?z~!P+6--m{F*wk-%l{P(KE5HC(Oi!X{1ab zyw;M^JcM%dj%j9uR&TpE;tq8~xhPxuHVLyd0|Ix<+jXZceL?5>S;765*C6txh%I@g zImC|YoK1zyX)6m?ew}uaOpu$$+Kf{ec9htHqN=^TZU6<5z85upU`g>oHfdH%c>Hu> z5-08DDYF&YL_#&IX;+(l0vhMYNYUJb+cU0N^F)$@z$9Wh8VJ;Tmcv8KsDMhi;imvG+6Xi)*Gbr* zqkLP5)m6RA&`=DYqfcnUHDe+%If%HlK%naS+caf5r4_Y&5pXZY;CC#-f@dwmBq8Ft z$z)VoNowQ36lIByUsBTQUp4#Z@+cv|eEUe7+Weev`_dOQRszI+QN;peJ?C>x7J;b( zG@X{J&UT^*dhEVVen5zv;Zl-!pAPjt|!KH$y^WLfQ7&{_{09 z6`+fu6D~LJ?V+2{hoxpD%_+3HZc$1~JQ~)ro-`S2E=?jmTPe0k>+p4Nr(#Xv0ZXog zb_aN5!RsEO_`HkFJ45hNa-pcDr)knuPPf_c+O)DQ2@&kQ;L09=&<9a7T*ky~jZpoQ z3S6B|TCVHu2i+6@_`b*F+KFv;OsKMNW}H}CPu z37{hKYRxe>C`cYewS!E=Fq2$tz!;#ggG?nG;_!efsQ%km_rZA92!U7 zwalS8i+M=BFqn9wQOJ^n$MYm)@ig#4e~gJ`Ze$MKR=j)6p&%NJqWhiO7!9V;hawI& z;vAkhWOKJ&A`W>G3L9cZB}W``#37fN=p!5?b|b^>B_)T&A+d1XqBBsSy6P>S4vE9E z?BHd^fzb^xuG>@WP&ur#!u2fp!*TxVu;|Lbfnif2lg<00gi+n$a!6c0r{8dr^Y7dg z_n>|u}=QZxa z-%tjP1t^Z>kh_x@`{Y#W6GNfMZnu9_fja=qDwEplvqyMKos_hF6_a_rtm4+$qdHmG zxg^gwKVzUx%kv16HBl0hFD261=(QQZ_RPDT&&#k}cKJ4jLbaqhX;zBOaikf=lm4tO z48f?_<^{@RoiNP4$-oarqF;ZXZf#>l$$=#?MMhT+WjT-?OPP22|k zk8qS|FGqsma4770it&?oZ%) ztdMRK<$(u_OfrS(-5n4I1>!NhBAGXk#B5n~a2&Q@Xb=jrtTJ?04h^><-=TC@D#Qz# zAuLZTQINk-#Wp`G##7vwuOrs2S4Bc``=Z|aST;w)-0=m*u_wA{4tHlyoXEV*g59;6 z)LKoSV3+_m*$cDd8F$J)rz)kHrpp>b!jn_X$t*NmRoX>7N;A4p@;*d~n4U{~ND1$3 zxS{-Y@7EYCp0(2Kv0vipM_lRA-Rx5~=kcK8t}bs1V;i<}GqSd2&E|gGRE>A7cq*`o zsdkEX#IhT7+2yJBfV{%Fg(yW3j&)~+*`~2uLs$#CB zql?_DJKSYcm+w2BI=Vfz`jjxO@4(nT;T*loIq^T>`a^7j7$GTny4d07Df+eqC9 literal 0 HcmV?d00001 From 986c97348d674de4f1d27b6eff7e16b954d3aae6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 12:31:05 +0200 Subject: [PATCH 179/259] added the ignorecase option to each selection verb --- .../ContainsVerbIgnoreCase.java | 28 +++++++++++++++++++ .../dhp/selectioncriteria/EqualVerb.java | 2 +- .../EqualVerbIgnoreCase.java | 28 +++++++++++++++++++ .../NotContainsVerbIgnoreCase.java | 28 +++++++++++++++++++ .../dhp/selectioncriteria/NotEqualVerb.java | 2 +- .../NotEqualVerbIgnoreCase.java | 28 +++++++++++++++++++ 6 files changed, 114 insertions(+), 2 deletions(-) create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java create mode 100644 dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java new file mode 100644 index 000000000..d5651e5b8 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java @@ -0,0 +1,28 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import java.io.Serializable; + +@VerbClass("contains_ignorecase") +public class ContainsVerbIgnoreCase implements Selection, Serializable { + + private String param; + + public ContainsVerbIgnoreCase() {} + + public ContainsVerbIgnoreCase(final String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return value.toLowerCase().contains(param.toLowerCase()); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java index 32171001e..cc793f8f8 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java @@ -15,7 +15,7 @@ public class EqualVerb implements Selection, Serializable { @Override public boolean apply(String value) { - return value.equalsIgnoreCase(param); + return value.equals(param); } public String getParam() { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java new file mode 100644 index 000000000..26c636090 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java @@ -0,0 +1,28 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import java.io.Serializable; + +@VerbClass("equals_ignorecase") +public class EqualVerbIgnoreCase implements Selection, Serializable { + + private String param; + + public EqualVerbIgnoreCase() {} + + public EqualVerbIgnoreCase(final String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return value.equalsIgnoreCase(param); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java new file mode 100644 index 000000000..e26b4a1a3 --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java @@ -0,0 +1,28 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import java.io.Serializable; + +@VerbClass("not_contains_ignorecase") +public class NotContainsVerbIgnoreCase implements Selection, Serializable { + + private String param; + + public NotContainsVerbIgnoreCase() {} + + public NotContainsVerbIgnoreCase(final String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return !(value.toLowerCase().contains(param.toLowerCase())); + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java index d1716588e..1946812f5 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java @@ -23,6 +23,6 @@ public class NotEqualVerb implements Selection, Serializable { @Override public boolean apply(String value) { - return !value.equalsIgnoreCase(param); + return !value.equals(param); } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java new file mode 100644 index 000000000..d669fa6ac --- /dev/null +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java @@ -0,0 +1,28 @@ +package eu.dnetlib.dhp.selectioncriteria; + +import java.io.Serializable; + +@VerbClass("not_equals_ignorecase") +public class NotEqualVerbIgnoreCase implements Selection, Serializable { + + private String param; + + public NotEqualVerbIgnoreCase(final String param) { + this.param = param; + } + + public NotEqualVerbIgnoreCase() {} + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } + + @Override + public boolean apply(String value) { + return !value.equalsIgnoreCase(param); + } +} From 7a59324ccfbc67266ae8c08c9f3848a7937c3dab Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 12:31:46 +0200 Subject: [PATCH 180/259] changed the test to check for the new ignorecase option --- .../src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index e20d87280..04c8816df 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -795,9 +795,9 @@ public class BulkTagJobTest { org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); idExplodeCommunity.show(false); - Assertions.assertEquals(2, idExplodeCommunity.count()); + Assertions.assertEquals(3, idExplodeCommunity.count()); Assertions.assertEquals( - 2, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); } } From df34a4ebcc52baac2e2df259d8da7632ad18f228 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Apr 2020 12:32:56 +0200 Subject: [PATCH 181/259] changed the configuration to add ignorecase option to each verb related to covid-19 community --- .../communityconfiguration/tagging_conf.xml | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml index 98114ef81..4f0d25f34 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml +++ b/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml @@ -1290,81 +1290,81 @@ opendoar____::358aee4cc897452c00244351e4d91f69 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} driver______::bee53aa31dc2cbb538c10c2b65fa5824 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} openaire____::437f4b072b1aa198adcbc35910ff3b98 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} openaire____::081b82f96300b6a6e3d282bad31cb6e2 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} openaire____::9e3be59865b2c1c335d32dae2fe7b254 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} opendoar____::8b6dd7db9af49e67306feb59a8bdc52c - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} share_______::4719356ec8d7d55d3feb384ce879ad6c - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} share_______::bbd802baad85d1fd440f32a7a3a2c2b1 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} opendoar____::6f4922f45568161a8cdf4ad2299f6d23 - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]}, - {"constraint":[{"verb":"contains","field":"title","value":"2019-nCoV"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]}, + {"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCoV"}]}]} re3data_____::7980778c78fb4cf0fab13ce2159030dc - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCov"}]}]} re3data_____::978378def740bbf2bfb420de868c460b - {"criteria":[{"constraint":[{"verb":"contains","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains","field":"title","value":"2019-nCov"}]}]} + {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCov"}]}]} From b85ad7012a14584a8ce9e64fb07257c1d2a8a5e7 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 29 Apr 2020 17:29:49 +0200 Subject: [PATCH 182/259] reads the blacklist from the blacklist db and writes it as a set of relations on hdfs --- .../dhp/schema/common/RelationInverse.java | 4 + dhp-workflows/dhp-blacklist/pom.xml | 15 ++ .../dhp/blacklist/ReadBlacklistFromDB.java | 138 ++++++++++++++++++ .../dhp/blacklist/blacklist_parameters.json | 32 ++++ 4 files changed, 189 insertions(+) create mode 100644 dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java create mode 100644 dhp-workflows/dhp-blacklist/pom.xml create mode 100644 dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java create mode 100644 dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java new file mode 100644 index 000000000..b726a8c16 --- /dev/null +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.schema.common; + +public class RelationInverse { +} diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml new file mode 100644 index 000000000..69f26b961 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -0,0 +1,15 @@ + + + + dhp-workflows + eu.dnetlib.dhp + 1.1.7-SNAPSHOT + + 4.0.0 + + dhp-blacklist + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java new file mode 100644 index 000000000..a8d427761 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -0,0 +1,138 @@ +package eu.dnetlib.dhp.blacklist; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.common.RelationInverse; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.io.BufferedWriter; +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.sql.ResultSet; +import java.util.Arrays; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.hadoop.conf.Configuration; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + + +public class ReadBlacklistFromDB implements Closeable { + + private final DbClient dbClient; + private static final Log log = LogFactory.getLog(ReadBlacklistFromDB.class); + private final Configuration conf; + private final BufferedWriter writer; + + private final static String query = "SELECT source_type, unnest(original_source_objects) as source, " + + "target_type, unnest(original_target_objects) as target, " + + "relationship FROM blacklist WHERE status = 'ACCEPTED'"; + + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = + new ArgumentApplicationParser( + IOUtils.toString( + ReadBlacklistFromDB.class.getResourceAsStream( + "/eu/dnetlib/dhp/blacklist/blacklist_parameters.json"))); + + parser.parseArgument(args); + + final String dbUrl = parser.get("postgresUrl"); + final String dbUser = parser.get("postgresUser"); + final String dbPassword = parser.get("postgresPassword"); + final String hdfsPath = parser.get("hdfsPath"); + final String hdfsNameNode = parser.get("hdfsNameNode"); + + + + try (final ReadBlacklistFromDB rbl = + new ReadBlacklistFromDB(hdfsPath, hdfsNameNode, dbUrl, dbUser, dbPassword)) { + + log.info("Processing blacklist..."); + rbl.execute(query, rbl::processBlacklistEntry); + + } + } + + + public void execute(final String sql, final Function> producer) + throws Exception { + + final Consumer consumer = rs -> producer.apply(rs).forEach(r -> writeRelation(r)); + + dbClient.processResults(sql, consumer); + } + + public List processBlacklistEntry(ResultSet rs){ + try { + Relation direct = new Relation(); + Relation inverse = new Relation(); + + String source_prefix = ModelSupport.entityIdPrefix.get(rs.getString("source_type")); + String target_prefix = ModelSupport.entityIdPrefix.get(rs.getString("target_type")); + + String source_direct = source_prefix + "|" + rs.getString("source"); + direct.setSource(source_direct); + inverse.setTarget(source_direct); + + String target_direct = target_prefix + "|" + rs.getString("target"); + direct.setTarget(target_direct); + inverse.setSource(target_direct); + + String encoding = rs.getString("relationship"); + RelationInverse ri = ModelSupport.relationInverseMap.get(encoding); + direct.setRelClass(ri.getRelation()); + inverse.setRelClass(ri.getInverse()); + direct.setRelType(ri.getRelType()); + inverse.setRelType(ri.getRelType()); + direct.setSubRelType(ri.getSubReltype()); + inverse.setSubRelType(ri.getSubReltype()); + + return Arrays.asList(direct, inverse); + + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException { + dbClient.close(); + writer.close(); + } + + public ReadBlacklistFromDB( + final String hdfsPath, String hdfsNameNode, final String dbUrl, final String dbUser, final String dbPassword) + throws Exception { + + this.dbClient = new DbClient(dbUrl, dbUser, dbPassword); + this.conf = new Configuration(); + this.conf.set("fs.defaultFS", hdfsNameNode); + FileSystem fileSystem = FileSystem.get(this.conf); + Path hdfsWritePath = new Path(hdfsPath); + FSDataOutputStream fsDataOutputStream = fileSystem.append(hdfsWritePath); + this.writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); + } + + protected void writeRelation(final Relation r) { + try { + writer.write(new ObjectMapper().writeValueAsString(r)); + writer.newLine(); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + + + +} diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json new file mode 100644 index 000000000..cb13ff024 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "p", + "paramLongName": "hdfsPath", + "paramDescription": "the path where storing the sequential file", + "paramRequired": true + }, + { + "paramName": "pgurl", + "paramLongName": "postgresUrl", + "paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb", + "paramRequired": true + }, + { + "paramName": "pguser", + "paramLongName": "postgresUser", + "paramDescription": "postgres user", + "paramRequired": false + }, + { + "paramName": "pgpasswd", + "paramLongName": "postgresPassword", + "paramDescription": "postgres password", + "paramRequired": false + }, + { + "paramName": "a", + "paramLongName": "action", + "paramDescription": "process claims", + "paramRequired": false + } +] \ No newline at end of file From 869f5762738fc029f7674d27be554a92cee96183 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 29 Apr 2020 18:14:52 +0200 Subject: [PATCH 183/259] added hash map for relationship entityType id prefix, and relation inverse --- .../dhp/schema/common/ModelSupport.java | 166 +++++++++++++ .../dhp/schema/common/RelationInverse.java | 44 +++- .../blacklist/oozie_app/config-default.xml | 54 +++++ .../dhp/blacklist/oozie_app/workflow.xml | 227 ++++++++++++++++++ 4 files changed, 490 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java index 0054e6d6f..8587dfca5 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java @@ -50,6 +50,172 @@ public class ModelSupport { oafTypes.put("relation", Relation.class); } + public static final Map entityIdPrefix = Maps.newHashMap(); + + static { + entityIdPrefix.put("datasource", "10"); + entityIdPrefix.put("organization", "20"); + entityIdPrefix.put("project", "40"); + entityIdPrefix.put("result","50"); + } + + public static final Map relationInverseMap = Maps.newHashMap(); + + static { + relationInverseMap.put("personResult_authorship_isAuthorOf", new RelationInverse() + .setRelation("isAuthorOf") + .setInverse("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap.put("personResult_authorship_hasAuthor", new RelationInverse() + .setInverse("isAuthorOf") + .setRelation("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap.put("projectOrganization_participation_isParticipant", new RelationInverse() + .setRelation("isParticipant") + .setInverse("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap.put("projectOrganization_participation_hasParticipant", new RelationInverse() + .setInverse("isParticipant") + .setRelation("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap.put("resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() + .setRelation("hasAuthorInstitution") + .setInverse("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap.put("resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() + .setInverse("hasAuthorInstitution") + .setRelation("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap.put("organizationOrganization_dedup_merges", new RelationInverse() + .setRelation("merges") + .setInverse("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap.put("organizationOrganization_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap.put("organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("organizationOrganization") + .setSubReltype("dedupSimilarity")); + + relationInverseMap.put("resultProject_outcome_isProducedBy", new RelationInverse() + .setRelation("isProducedBy") + .setInverse("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap.put("resultProject_outcome_produces", new RelationInverse() + .setInverse("isProducedBy") + .setRelation("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap.put("projectPerson_contactPerson_isContact", new RelationInverse() + .setRelation("isContact") + .setInverse("hasContact") + .setRelType("projectPerson") + .setSubReltype("contactPerson")); + relationInverseMap.put("projectPerson_contactPerson_hasContact", new RelationInverse() + .setInverse("isContact") + .setRelation("hasContact") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap.put("personPerson_coAuthorship_isCoauthorOf", new RelationInverse() + .setInverse("isCoAuthorOf") + .setRelation("isCoAuthorOf") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap.put("personPerson_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap.put("personPerson_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap.put("personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("personPerson") + .setSubReltype("dedupSimilarity")); + relationInverseMap.put("datasourceOrganization_provision_isProvidedBy", new RelationInverse() + .setInverse("provides") + .setRelation("isProvidedBy") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap.put("datasourceOrganization_provision_provides", new RelationInverse() + .setInverse("isProvidedBy") + .setRelation("provides") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap.put("resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("isAmongTopNSimilarDocuments") + .setRelation("hasAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap.put("resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap.put("resultResult_relationship_isRelatedTo", new RelationInverse() + .setInverse("isRelatedTo") + .setRelation("isRelatedTo") + .setRelType("resultResult") + .setSubReltype("relationship")); + relationInverseMap.put("resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap.put("resultResult_supplement_isSupplementTo", new RelationInverse() + .setInverse("isSupplementedBy") + .setRelation("isSupplementTo") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap.put("resultResult_supplement_isSupplementedBy", new RelationInverse() + .setInverse("isSupplementTo") + .setRelation("isSupplementedBy") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap.put("resultResult_part_isPartOf", new RelationInverse() + .setInverse("hasPart") + .setRelation("isPartOf") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap.put("resultResult_part_hasPart", new RelationInverse() + .setInverse("isPartOf") + .setRelation("hasPart") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap.put("resultResult_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap.put("resultResult_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap.put("resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("resultResult") + .setSubReltype("dedupSimilarity")); + + } + private static final String schemeTemplate = "dnet:%s_%s_relations"; private ModelSupport() {} diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java index b726a8c16..d61dce689 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java @@ -1,4 +1,46 @@ package eu.dnetlib.dhp.schema.common; -public class RelationInverse { +public class RelationInverse { + private String relation; + private String inverse; + private String relType; + private String subReltype; + + public String getRelType() { + return relType; + } + + public RelationInverse setRelType(String relType) { + this.relType = relType; + return this; + } + + public String getSubReltype() { + return subReltype; + } + + public RelationInverse setSubReltype(String subReltype) { + this.subReltype = subReltype; + return this; + } + + public String getRelation() { + return relation; + } + + public RelationInverse setRelation(String relation) { + this.relation = relation; + return this; + } + + public String getInverse() { + return inverse; + } + + public RelationInverse setInverse(String inverse) { + this.inverse = inverse; + return this; + } + + } diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/config-default.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/config-default.xml new file mode 100644 index 000000000..fe82ae194 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/config-default.xml @@ -0,0 +1,54 @@ + + + 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 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml new file mode 100644 index 000000000..91be13210 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -0,0 +1,227 @@ + + + + sourcePath + the source path + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + isLookUpUrl + the isLookup service endpoint + + + pathMap + the json path associated to each selection field + + + outputPath + the output path + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/relation + ${nameNode}/${outputPath}/relation + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging-publication + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/publication + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${outputPath}/publication + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging-dataset + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/dataset + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${outputPath}/dataset + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging-orp + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/otherresearchproduct + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${outputPath}/otherresearchproduct + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} + + + + + + + ${jobTracker} + ${nameNode} + yarn-cluster + cluster + bulkTagging-software + eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + dhp-bulktag-${projectVersion}.jar + + --num-executors=${sparkExecutorNumber} + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/software + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${outputPath}/software + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} + + + + + + + \ No newline at end of file From 6a47e6191d845e221dfef10489e4190aa7079526 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 29 Apr 2020 18:16:01 +0200 Subject: [PATCH 184/259] read from blacklist and write the result as relations on hdfs --- dhp-workflows/dhp-blacklist/pom.xml | 18 ++ .../dhp/blacklist/blacklist_parameters.json | 12 +- .../dhp/blacklist/oozie_app/workflow.xml | 220 ++---------------- 3 files changed, 37 insertions(+), 213 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 69f26b961..1c6c00e9a 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -10,6 +10,24 @@ 4.0.0 dhp-blacklist + + + eu.dnetlib.dhp + dhp-graph-mapper + 1.1.7-SNAPSHOT + compile + + + com.fasterxml.jackson.core + jackson-databind + compile + + + org.apache.hadoop + hadoop-common + compile + + \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json index cb13ff024..9a2eadaa7 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/blacklist_parameters.json @@ -5,6 +5,12 @@ "paramDescription": "the path where storing the sequential file", "paramRequired": true }, + { + "paramName": "nn", + "paramLongName": "hdfsNameNode", + "paramDescription": "the name node on hdfs", + "paramRequired": true + }, { "paramName": "pgurl", "paramLongName": "postgresUrl", @@ -22,11 +28,5 @@ "paramLongName": "postgresPassword", "paramDescription": "postgres password", "paramRequired": false - }, - { - "paramName": "a", - "paramLongName": "action", - "paramDescription": "process claims", - "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 91be13210..483c0378a 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -1,34 +1,4 @@ - - - - sourcePath - the source path - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - isLookUpUrl - the isLookup service endpoint - - - pathMap - the json path associated to each selection field - - - outputPath - the output path - - + @@ -38,190 +8,26 @@ - - - - - - - - + - + - - - - - - - - - + + ${jobTracker} ${nameNode} - ${nameNode}/${sourcePath}/relation - ${nameNode}/${outputPath}/relation - - + eu.dnetlib.dhp.blacklist.ReadBlacklistFromDB + --hdfsPath${workingDir}/blacklist + --hdfsNameNode${nameNode} + --postgresUrl${postgresUrl} + --postgresUser${postgresUser} + --postgresPassword${postgresPassword} + + - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/organization - ${nameNode}/${outputPath}/organization - - - - - - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/project - ${nameNode}/${outputPath}/project - - - - - - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/datasource - ${nameNode}/${outputPath}/datasource - - - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - bulkTagging-publication - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 - dhp-bulktag-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/publication - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${outputPath}/publication - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - bulkTagging-dataset - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 - dhp-bulktag-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/dataset - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${outputPath}/dataset - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - bulkTagging-orp - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 - dhp-bulktag-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/otherresearchproduct - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${outputPath}/otherresearchproduct - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - bulkTagging-software - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 - dhp-bulktag-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/software - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${outputPath}/software - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - \ No newline at end of file From 9ab46535e771b6795f17af3f73a62fe31dcc302b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 29 Apr 2020 18:17:15 +0200 Subject: [PATCH 185/259] pom with the new blacklist module added --- dhp-workflows/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index 852b64483..692a6c90f 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -25,6 +25,7 @@ dhp-graph-provision dhp-dedup-scholexplorer dhp-graph-provision-scholexplorer + dhp-blacklist From 564e5d6279f0d69637de82fcff5732961b8852df Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 10:22:58 +0200 Subject: [PATCH 186/259] added new information in support of blacklist reader --- .../dhp/schema/common/ModelSupport.java | 729 ++++++++++-------- .../dhp/schema/common/RelationInverse.java | 68 +- 2 files changed, 432 insertions(+), 365 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java index 8587dfca5..fc85b1ac1 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java @@ -1,364 +1,431 @@ + package eu.dnetlib.dhp.schema.common; -import com.google.common.collect.Maps; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Map; import java.util.Optional; import java.util.function.Function; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.schema.oaf.*; + /** Oaf model utility methods. */ public class ModelSupport { - /** Defines the mapping between the actual entity type and the main entity type */ - private static Map entityMapping = Maps.newHashMap(); + /** Defines the mapping between the actual entity type and the main entity type */ + private static Map entityMapping = Maps.newHashMap(); - static { - entityMapping.put(EntityType.publication, MainEntityType.result); - entityMapping.put(EntityType.dataset, MainEntityType.result); - entityMapping.put(EntityType.otherresearchproduct, MainEntityType.result); - entityMapping.put(EntityType.software, MainEntityType.result); - entityMapping.put(EntityType.datasource, MainEntityType.datasource); - entityMapping.put(EntityType.organization, MainEntityType.organization); - entityMapping.put(EntityType.project, MainEntityType.project); - } + static { + entityMapping.put(EntityType.publication, MainEntityType.result); + entityMapping.put(EntityType.dataset, MainEntityType.result); + entityMapping.put(EntityType.otherresearchproduct, MainEntityType.result); + entityMapping.put(EntityType.software, MainEntityType.result); + entityMapping.put(EntityType.datasource, MainEntityType.datasource); + entityMapping.put(EntityType.organization, MainEntityType.organization); + entityMapping.put(EntityType.project, MainEntityType.project); + } - /** - * Defines the mapping between the actual entity types and the relative classes implementing them - */ - public static final Map entityTypes = Maps.newHashMap(); + /** + * Defines the mapping between the actual entity types and the relative classes implementing them + */ + public static final Map entityTypes = Maps.newHashMap(); - static { - entityTypes.put(EntityType.datasource, Datasource.class); - entityTypes.put(EntityType.organization, Organization.class); - entityTypes.put(EntityType.project, Project.class); - entityTypes.put(EntityType.dataset, Dataset.class); - entityTypes.put(EntityType.otherresearchproduct, OtherResearchProduct.class); - entityTypes.put(EntityType.software, Software.class); - entityTypes.put(EntityType.publication, Publication.class); - } + static { + entityTypes.put(EntityType.datasource, Datasource.class); + entityTypes.put(EntityType.organization, Organization.class); + entityTypes.put(EntityType.project, Project.class); + entityTypes.put(EntityType.dataset, Dataset.class); + entityTypes.put(EntityType.otherresearchproduct, OtherResearchProduct.class); + entityTypes.put(EntityType.software, Software.class); + entityTypes.put(EntityType.publication, Publication.class); + } - public static final Map oafTypes = Maps.newHashMap(); + public static final Map oafTypes = Maps.newHashMap(); - static { - oafTypes.put("datasource", Datasource.class); - oafTypes.put("organization", Organization.class); - oafTypes.put("project", Project.class); - oafTypes.put("dataset", Dataset.class); - oafTypes.put("otherresearchproduct", OtherResearchProduct.class); - oafTypes.put("software", Software.class); - oafTypes.put("publication", Publication.class); - oafTypes.put("relation", Relation.class); - } + static { + oafTypes.put("datasource", Datasource.class); + oafTypes.put("organization", Organization.class); + oafTypes.put("project", Project.class); + oafTypes.put("dataset", Dataset.class); + oafTypes.put("otherresearchproduct", OtherResearchProduct.class); + oafTypes.put("software", Software.class); + oafTypes.put("publication", Publication.class); + oafTypes.put("relation", Relation.class); + } - public static final Map entityIdPrefix = Maps.newHashMap(); + public static final Map entityIdPrefix = Maps.newHashMap(); - static { - entityIdPrefix.put("datasource", "10"); - entityIdPrefix.put("organization", "20"); - entityIdPrefix.put("project", "40"); - entityIdPrefix.put("result","50"); - } + static { + entityIdPrefix.put("datasource", "10"); + entityIdPrefix.put("organization", "20"); + entityIdPrefix.put("project", "40"); + entityIdPrefix.put("result", "50"); + } - public static final Map relationInverseMap = Maps.newHashMap(); + public static final Map relationInverseMap = Maps.newHashMap(); - static { - relationInverseMap.put("personResult_authorship_isAuthorOf", new RelationInverse() - .setRelation("isAuthorOf") - .setInverse("hasAuthor") - .setRelType("personResult") - .setSubReltype("authorship")); - relationInverseMap.put("personResult_authorship_hasAuthor", new RelationInverse() - .setInverse("isAuthorOf") - .setRelation("hasAuthor") - .setRelType("personResult") - .setSubReltype("authorship")); - relationInverseMap.put("projectOrganization_participation_isParticipant", new RelationInverse() - .setRelation("isParticipant") - .setInverse("hasParticipant") - .setRelType("projectOrganization") - .setSubReltype("participation")); - relationInverseMap.put("projectOrganization_participation_hasParticipant", new RelationInverse() - .setInverse("isParticipant") - .setRelation("hasParticipant") - .setRelType("projectOrganization") - .setSubReltype("participation")); - relationInverseMap.put("resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() - .setRelation("hasAuthorInstitution") - .setInverse("isAuthorInstitutionOf") - .setRelType("resultOrganization") - .setSubReltype("affiliation")); - relationInverseMap.put("resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() - .setInverse("hasAuthorInstitution") - .setRelation("isAuthorInstitutionOf") - .setRelType("resultOrganization") - .setSubReltype("affiliation")); - relationInverseMap.put("organizationOrganization_dedup_merges", new RelationInverse() - .setRelation("merges") - .setInverse("isMergedIn") - .setRelType("organizationOrganization") - .setSubReltype("dedup")); - relationInverseMap.put("organizationOrganization_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("organizationOrganization") - .setSubReltype("dedup")); - relationInverseMap.put("organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("organizationOrganization") - .setSubReltype("dedupSimilarity")); + static { + relationInverseMap + .put( + "personResult_authorship_isAuthorOf", new RelationInverse() + .setRelation("isAuthorOf") + .setInverse("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap + .put( + "personResult_authorship_hasAuthor", new RelationInverse() + .setInverse("isAuthorOf") + .setRelation("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap + .put( + "projectOrganization_participation_isParticipant", new RelationInverse() + .setRelation("isParticipant") + .setInverse("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap + .put( + "projectOrganization_participation_hasParticipant", new RelationInverse() + .setInverse("isParticipant") + .setRelation("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap + .put( + "resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() + .setRelation("hasAuthorInstitution") + .setInverse("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap + .put( + "resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() + .setInverse("hasAuthorInstitution") + .setRelation("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap + .put( + "organizationOrganization_dedup_merges", new RelationInverse() + .setRelation("merges") + .setInverse("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap + .put( + "organizationOrganization_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap + .put( + "organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("organizationOrganization") + .setSubReltype("dedupSimilarity")); - relationInverseMap.put("resultProject_outcome_isProducedBy", new RelationInverse() - .setRelation("isProducedBy") - .setInverse("produces") - .setRelType("resultProject") - .setSubReltype("outcome")); - relationInverseMap.put("resultProject_outcome_produces", new RelationInverse() - .setInverse("isProducedBy") - .setRelation("produces") - .setRelType("resultProject") - .setSubReltype("outcome")); - relationInverseMap.put("projectPerson_contactPerson_isContact", new RelationInverse() - .setRelation("isContact") - .setInverse("hasContact") - .setRelType("projectPerson") - .setSubReltype("contactPerson")); - relationInverseMap.put("projectPerson_contactPerson_hasContact", new RelationInverse() - .setInverse("isContact") - .setRelation("hasContact") - .setRelType("personPerson") - .setSubReltype("coAuthorship")); - relationInverseMap.put("personPerson_coAuthorship_isCoauthorOf", new RelationInverse() - .setInverse("isCoAuthorOf") - .setRelation("isCoAuthorOf") - .setRelType("personPerson") - .setSubReltype("coAuthorship")); - relationInverseMap.put("personPerson_dedup_merges", new RelationInverse() - .setInverse("isMergedIn") - .setRelation("merges") - .setRelType("personPerson") - .setSubReltype("dedup")); - relationInverseMap.put("personPerson_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("personPerson") - .setSubReltype("dedup")); - relationInverseMap.put("personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("personPerson") - .setSubReltype("dedupSimilarity")); - relationInverseMap.put("datasourceOrganization_provision_isProvidedBy", new RelationInverse() - .setInverse("provides") - .setRelation("isProvidedBy") - .setRelType("datasourceOrganization") - .setSubReltype("provision")); - relationInverseMap.put("datasourceOrganization_provision_provides", new RelationInverse() - .setInverse("isProvidedBy") - .setRelation("provides") - .setRelType("datasourceOrganization") - .setSubReltype("provision")); - relationInverseMap.put("resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("isAmongTopNSimilarDocuments") - .setRelation("hasAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); - relationInverseMap.put("resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("hasAmongTopNSimilarDocuments") - .setRelation("isAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); - relationInverseMap.put("resultResult_relationship_isRelatedTo", new RelationInverse() - .setInverse("isRelatedTo") - .setRelation("isRelatedTo") - .setRelType("resultResult") - .setSubReltype("relationship")); - relationInverseMap.put("resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("hasAmongTopNSimilarDocuments") - .setRelation("isAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); - relationInverseMap.put("resultResult_supplement_isSupplementTo", new RelationInverse() - .setInverse("isSupplementedBy") - .setRelation("isSupplementTo") - .setRelType("resultResult") - .setSubReltype("supplement")); - relationInverseMap.put("resultResult_supplement_isSupplementedBy", new RelationInverse() - .setInverse("isSupplementTo") - .setRelation("isSupplementedBy") - .setRelType("resultResult") - .setSubReltype("supplement")); - relationInverseMap.put("resultResult_part_isPartOf", new RelationInverse() - .setInverse("hasPart") - .setRelation("isPartOf") - .setRelType("resultResult") - .setSubReltype("part")); - relationInverseMap.put("resultResult_part_hasPart", new RelationInverse() - .setInverse("isPartOf") - .setRelation("hasPart") - .setRelType("resultResult") - .setSubReltype("part")); - relationInverseMap.put("resultResult_dedup_merges", new RelationInverse() - .setInverse("isMergedIn") - .setRelation("merges") - .setRelType("resultResult") - .setSubReltype("dedup")); - relationInverseMap.put("resultResult_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("resultResult") - .setSubReltype("dedup")); - relationInverseMap.put("resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("resultResult") - .setSubReltype("dedupSimilarity")); + relationInverseMap + .put( + "resultProject_outcome_isProducedBy", new RelationInverse() + .setRelation("isProducedBy") + .setInverse("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap + .put( + "resultProject_outcome_produces", new RelationInverse() + .setInverse("isProducedBy") + .setRelation("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap + .put( + "projectPerson_contactPerson_isContact", new RelationInverse() + .setRelation("isContact") + .setInverse("hasContact") + .setRelType("projectPerson") + .setSubReltype("contactPerson")); + relationInverseMap + .put( + "projectPerson_contactPerson_hasContact", new RelationInverse() + .setInverse("isContact") + .setRelation("hasContact") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap + .put( + "personPerson_coAuthorship_isCoauthorOf", new RelationInverse() + .setInverse("isCoAuthorOf") + .setRelation("isCoAuthorOf") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap + .put( + "personPerson_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap + .put( + "personPerson_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap + .put( + "personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("personPerson") + .setSubReltype("dedupSimilarity")); + relationInverseMap + .put( + "datasourceOrganization_provision_isProvidedBy", new RelationInverse() + .setInverse("provides") + .setRelation("isProvidedBy") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap + .put( + "datasourceOrganization_provision_provides", new RelationInverse() + .setInverse("isProvidedBy") + .setRelation("provides") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap + .put( + "resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("isAmongTopNSimilarDocuments") + .setRelation("hasAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_relationship_isRelatedTo", new RelationInverse() + .setInverse("isRelatedTo") + .setRelation("isRelatedTo") + .setRelType("resultResult") + .setSubReltype("relationship")); + relationInverseMap + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_supplement_isSupplementTo", new RelationInverse() + .setInverse("isSupplementedBy") + .setRelation("isSupplementTo") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap + .put( + "resultResult_supplement_isSupplementedBy", new RelationInverse() + .setInverse("isSupplementTo") + .setRelation("isSupplementedBy") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap + .put( + "resultResult_part_isPartOf", new RelationInverse() + .setInverse("hasPart") + .setRelation("isPartOf") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap + .put( + "resultResult_part_hasPart", new RelationInverse() + .setInverse("isPartOf") + .setRelation("hasPart") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap + .put( + "resultResult_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap + .put( + "resultResult_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap + .put( + "resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("resultResult") + .setSubReltype("dedupSimilarity")); - } + } - private static final String schemeTemplate = "dnet:%s_%s_relations"; + private static final String schemeTemplate = "dnet:%s_%s_relations"; - private ModelSupport() {} + private ModelSupport() { + } - /** - * Checks subclass-superclass relationship. - * - * @param subClazzObject Subclass object instance - * @param superClazzObject Superclass object instance - * @param Subclass type - * @param Superclass type - * @return True if X is a subclass of Y - */ - public static Boolean isSubClass( - X subClazzObject, Y superClazzObject) { - return isSubClass(subClazzObject.getClass(), superClazzObject.getClass()); - } + /** + * Checks subclass-superclass relationship. + * + * @param subClazzObject Subclass object instance + * @param superClazzObject Superclass object instance + * @param Subclass type + * @param Superclass type + * @return True if X is a subclass of Y + */ + public static Boolean isSubClass( + X subClazzObject, Y superClazzObject) { + return isSubClass(subClazzObject.getClass(), superClazzObject.getClass()); + } - /** - * Checks subclass-superclass relationship. - * - * @param subClazzObject Subclass object instance - * @param superClazz Superclass class - * @param Subclass type - * @param Superclass type - * @return True if X is a subclass of Y - */ - public static Boolean isSubClass( - X subClazzObject, Class superClazz) { - return isSubClass(subClazzObject.getClass(), superClazz); - } + /** + * Checks subclass-superclass relationship. + * + * @param subClazzObject Subclass object instance + * @param superClazz Superclass class + * @param Subclass type + * @param Superclass type + * @return True if X is a subclass of Y + */ + public static Boolean isSubClass( + X subClazzObject, Class superClazz) { + return isSubClass(subClazzObject.getClass(), superClazz); + } - /** - * Checks subclass-superclass relationship. - * - * @param subClazz Subclass class - * @param superClazz Superclass class - * @param Subclass type - * @param Superclass type - * @return True if X is a subclass of Y - */ - public static Boolean isSubClass( - Class subClazz, Class superClazz) { - return superClazz.isAssignableFrom(subClazz); - } + /** + * Checks subclass-superclass relationship. + * + * @param subClazz Subclass class + * @param superClazz Superclass class + * @param Subclass type + * @param Superclass type + * @return True if X is a subclass of Y + */ + public static Boolean isSubClass( + Class subClazz, Class superClazz) { + return superClazz.isAssignableFrom(subClazz); + } - /** - * Lists all the OAF model classes - * - * @param - * @return - */ - public static Class[] getOafModelClasses() { - return new Class[] { - Author.class, - Context.class, - Country.class, - DataInfo.class, - Dataset.class, - Datasource.class, - ExternalReference.class, - ExtraInfo.class, - Field.class, - GeoLocation.class, - Instance.class, - Journal.class, - KeyValue.class, - Oaf.class, - OafEntity.class, - OAIProvenance.class, - Organization.class, - OriginDescription.class, - OtherResearchProduct.class, - Project.class, - Publication.class, - Qualifier.class, - Relation.class, - Result.class, - Software.class, - StructuredProperty.class - }; - } + /** + * Lists all the OAF model classes + * + * @param + * @return + */ + public static Class[] getOafModelClasses() { + return new Class[] { + Author.class, + Context.class, + Country.class, + DataInfo.class, + Dataset.class, + Datasource.class, + ExternalReference.class, + ExtraInfo.class, + Field.class, + GeoLocation.class, + Instance.class, + Journal.class, + KeyValue.class, + Oaf.class, + OafEntity.class, + OAIProvenance.class, + Organization.class, + OriginDescription.class, + OtherResearchProduct.class, + Project.class, + Publication.class, + Qualifier.class, + Relation.class, + Result.class, + Software.class, + StructuredProperty.class + }; + } - public static String getMainType(final EntityType type) { - return entityMapping.get(type).name(); - } + public static String getMainType(final EntityType type) { + return entityMapping.get(type).name(); + } - public static boolean isResult(EntityType type) { - return MainEntityType.result.name().equals(getMainType(type)); - } + public static boolean isResult(EntityType type) { + return MainEntityType.result.name().equals(getMainType(type)); + } - public static String getScheme(final String sourceType, final String targetType) { - return String.format( - schemeTemplate, - entityMapping.get(EntityType.valueOf(sourceType)).name(), - entityMapping.get(EntityType.valueOf(targetType)).name()); - } + public static String getScheme(final String sourceType, final String targetType) { + return String + .format( + schemeTemplate, + entityMapping.get(EntityType.valueOf(sourceType)).name(), + entityMapping.get(EntityType.valueOf(targetType)).name()); + } - public static Function idFn() { - return x -> { - if (isSubClass(x, Relation.class)) { - return idFnForRelation(x); - } - return idFnForOafEntity(x); - }; - } + public static Function idFn() { + return x -> { + if (isSubClass(x, Relation.class)) { + return idFnForRelation(x); + } + return idFnForOafEntity(x); + }; + } - private static String idFnForRelation(T t) { - Relation r = (Relation) t; - return Optional.ofNullable(r.getSource()) - .map( - source -> - Optional.ofNullable(r.getTarget()) - .map( - target -> - Optional.ofNullable(r.getRelType()) - .map( - relType -> - Optional.ofNullable(r.getSubRelType()) - .map( - subRelType -> - Optional.ofNullable(r.getRelClass()) - .map( - relClass -> - String.join( - source, - target, - relType, - subRelType, - relClass)) - .orElse( - String.join( - source, - target, - relType, - subRelType))) - .orElse(String.join(source, target, relType))) - .orElse(String.join(source, target))) - .orElse(source)) - .orElse(null); - } + private static String idFnForRelation(T t) { + Relation r = (Relation) t; + return Optional + .ofNullable(r.getSource()) + .map( + source -> Optional + .ofNullable(r.getTarget()) + .map( + target -> Optional + .ofNullable(r.getRelType()) + .map( + relType -> Optional + .ofNullable(r.getSubRelType()) + .map( + subRelType -> Optional + .ofNullable(r.getRelClass()) + .map( + relClass -> String + .join( + source, + target, + relType, + subRelType, + relClass)) + .orElse( + String + .join( + source, + target, + relType, + subRelType))) + .orElse(String.join(source, target, relType))) + .orElse(String.join(source, target))) + .orElse(source)) + .orElse(null); + } - private static String idFnForOafEntity(T t) { - return ((OafEntity) t).getId(); - } + private static String idFnForOafEntity(T t) { + return ((OafEntity) t).getId(); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java index d61dce689..4757c637e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/RelationInverse.java @@ -1,46 +1,46 @@ + package eu.dnetlib.dhp.schema.common; -public class RelationInverse { - private String relation; - private String inverse; - private String relType; - private String subReltype; +public class RelationInverse { + private String relation; + private String inverse; + private String relType; + private String subReltype; - public String getRelType() { - return relType; - } + public String getRelType() { + return relType; + } - public RelationInverse setRelType(String relType) { - this.relType = relType; - return this; - } + public RelationInverse setRelType(String relType) { + this.relType = relType; + return this; + } - public String getSubReltype() { - return subReltype; - } + public String getSubReltype() { + return subReltype; + } - public RelationInverse setSubReltype(String subReltype) { - this.subReltype = subReltype; - return this; - } + public RelationInverse setSubReltype(String subReltype) { + this.subReltype = subReltype; + return this; + } - public String getRelation() { - return relation; - } + public String getRelation() { + return relation; + } - public RelationInverse setRelation(String relation) { - this.relation = relation; - return this; - } + public RelationInverse setRelation(String relation) { + this.relation = relation; + return this; + } - public String getInverse() { - return inverse; - } - - public RelationInverse setInverse(String inverse) { - this.inverse = inverse; - return this; - } + public String getInverse() { + return inverse; + } + public RelationInverse setInverse(String inverse) { + this.inverse = inverse; + return this; + } } From 354f0162bed984c9a0ff5714c07f8bc1b6dcf4c3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 10:26:50 +0200 Subject: [PATCH 187/259] changes in the blacklist and workflow definition --- .../dhp/blacklist/ReadBlacklistFromDB.java | 187 +++++++++--------- .../dhp/blacklist/oozie_app/workflow.xml | 21 +- 2 files changed, 109 insertions(+), 99 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index a8d427761..4f66d2f10 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -1,13 +1,5 @@ -package eu.dnetlib.dhp.blacklist; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.common.RelationInverse; -import eu.dnetlib.dhp.schema.oaf.Relation; -import org.apache.commons.io.IOUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +package eu.dnetlib.dhp.blacklist; import java.io.BufferedWriter; import java.io.Closeable; @@ -19,120 +11,125 @@ import java.util.Arrays; import java.util.List; import java.util.function.Consumer; import java.util.function.Function; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.common.RelationInverse; +import eu.dnetlib.dhp.schema.oaf.Relation; public class ReadBlacklistFromDB implements Closeable { - private final DbClient dbClient; - private static final Log log = LogFactory.getLog(ReadBlacklistFromDB.class); - private final Configuration conf; - private final BufferedWriter writer; + private final DbClient dbClient; + private static final Log log = LogFactory.getLog(ReadBlacklistFromDB.class); + private final Configuration conf; + private final BufferedWriter writer; - private final static String query = "SELECT source_type, unnest(original_source_objects) as source, " + - "target_type, unnest(original_target_objects) as target, " + - "relationship FROM blacklist WHERE status = 'ACCEPTED'"; + private final static String query = "SELECT source_type, unnest(original_source_objects) as source, " + + "target_type, unnest(original_target_objects) as target, " + + "relationship FROM blacklist WHERE status = 'ACCEPTED'"; - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - ReadBlacklistFromDB.class.getResourceAsStream( - "/eu/dnetlib/dhp/blacklist/blacklist_parameters.json"))); + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + ReadBlacklistFromDB.class + .getResourceAsStream( + "/eu/dnetlib/dhp/blacklist/blacklist_parameters.json"))); - parser.parseArgument(args); + parser.parseArgument(args); - final String dbUrl = parser.get("postgresUrl"); - final String dbUser = parser.get("postgresUser"); - final String dbPassword = parser.get("postgresPassword"); - final String hdfsPath = parser.get("hdfsPath"); - final String hdfsNameNode = parser.get("hdfsNameNode"); + final String dbUrl = parser.get("postgresUrl"); + final String dbUser = parser.get("postgresUser"); + final String dbPassword = parser.get("postgresPassword"); + final String hdfsPath = parser.get("hdfsPath"); + final String hdfsNameNode = parser.get("hdfsNameNode"); + try (final ReadBlacklistFromDB rbl = new ReadBlacklistFromDB(hdfsPath, hdfsNameNode, dbUrl, dbUser, + dbPassword)) { + log.info("Processing blacklist..."); + rbl.execute(query, rbl::processBlacklistEntry); - try (final ReadBlacklistFromDB rbl = - new ReadBlacklistFromDB(hdfsPath, hdfsNameNode, dbUrl, dbUser, dbPassword)) { + } + } - log.info("Processing blacklist..."); - rbl.execute(query, rbl::processBlacklistEntry); + public void execute(final String sql, final Function> producer) + throws Exception { - } - } + final Consumer consumer = rs -> producer.apply(rs).forEach(r -> writeRelation(r)); + dbClient.processResults(sql, consumer); + } - public void execute(final String sql, final Function> producer) - throws Exception { + public List processBlacklistEntry(ResultSet rs) { + try { + Relation direct = new Relation(); + Relation inverse = new Relation(); - final Consumer consumer = rs -> producer.apply(rs).forEach(r -> writeRelation(r)); + String source_prefix = ModelSupport.entityIdPrefix.get(rs.getString("source_type")); + String target_prefix = ModelSupport.entityIdPrefix.get(rs.getString("target_type")); - dbClient.processResults(sql, consumer); - } + String source_direct = source_prefix + "|" + rs.getString("source"); + direct.setSource(source_direct); + inverse.setTarget(source_direct); - public List processBlacklistEntry(ResultSet rs){ - try { - Relation direct = new Relation(); - Relation inverse = new Relation(); + String target_direct = target_prefix + "|" + rs.getString("target"); + direct.setTarget(target_direct); + inverse.setSource(target_direct); - String source_prefix = ModelSupport.entityIdPrefix.get(rs.getString("source_type")); - String target_prefix = ModelSupport.entityIdPrefix.get(rs.getString("target_type")); + String encoding = rs.getString("relationship"); + RelationInverse ri = ModelSupport.relationInverseMap.get(encoding); + direct.setRelClass(ri.getRelation()); + inverse.setRelClass(ri.getInverse()); + direct.setRelType(ri.getRelType()); + inverse.setRelType(ri.getRelType()); + direct.setSubRelType(ri.getSubReltype()); + inverse.setSubRelType(ri.getSubReltype()); - String source_direct = source_prefix + "|" + rs.getString("source"); - direct.setSource(source_direct); - inverse.setTarget(source_direct); + return Arrays.asList(direct, inverse); - String target_direct = target_prefix + "|" + rs.getString("target"); - direct.setTarget(target_direct); - inverse.setSource(target_direct); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } - String encoding = rs.getString("relationship"); - RelationInverse ri = ModelSupport.relationInverseMap.get(encoding); - direct.setRelClass(ri.getRelation()); - inverse.setRelClass(ri.getInverse()); - direct.setRelType(ri.getRelType()); - inverse.setRelType(ri.getRelType()); - direct.setSubRelType(ri.getSubReltype()); - inverse.setSubRelType(ri.getSubReltype()); - - return Arrays.asList(direct, inverse); - - } catch (final Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public void close() throws IOException { - dbClient.close(); - writer.close(); - } - - public ReadBlacklistFromDB( - final String hdfsPath, String hdfsNameNode, final String dbUrl, final String dbUser, final String dbPassword) - throws Exception { - - this.dbClient = new DbClient(dbUrl, dbUser, dbPassword); - this.conf = new Configuration(); - this.conf.set("fs.defaultFS", hdfsNameNode); - FileSystem fileSystem = FileSystem.get(this.conf); - Path hdfsWritePath = new Path(hdfsPath); - FSDataOutputStream fsDataOutputStream = fileSystem.append(hdfsWritePath); - this.writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); - } - - protected void writeRelation(final Relation r) { - try { - writer.write(new ObjectMapper().writeValueAsString(r)); - writer.newLine(); - } catch (final Exception e) { - throw new RuntimeException(e); - } - } + @Override + public void close() throws IOException { + dbClient.close(); + writer.close(); + } + public ReadBlacklistFromDB( + final String hdfsPath, String hdfsNameNode, final String dbUrl, final String dbUser, final String dbPassword) + throws Exception { + this.dbClient = new DbClient(dbUrl, dbUser, dbPassword); + this.conf = new Configuration(); + this.conf.set("fs.defaultFS", hdfsNameNode); + FileSystem fileSystem = FileSystem.get(this.conf); + Path hdfsWritePath = new Path(hdfsPath); + FSDataOutputStream fsDataOutputStream = fileSystem.append(hdfsWritePath); + this.writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); + } + protected void writeRelation(final Relation r) { + try { + writer.write(new ObjectMapper().writeValueAsString(r)); + writer.newLine(); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 483c0378a..e38d721b9 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -1,5 +1,18 @@ - - + + + + postgresURL + the url of the postgress server to query + + + postgresUser + the username to access the postgres db + + + postgresPassword + the postgres password + + @@ -8,7 +21,7 @@ - + @@ -21,7 +34,7 @@ eu.dnetlib.dhp.blacklist.ReadBlacklistFromDB --hdfsPath${workingDir}/blacklist --hdfsNameNode${nameNode} - --postgresUrl${postgresUrl} + --postgresUrl${postgresURL} --postgresUser${postgresUser} --postgresPassword${postgresPassword} From 638a3c465bca74449b8f55e245e1d758494e40d2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 11:05:17 +0200 Subject: [PATCH 188/259] - --- .../GenerateOoziePropertiesMojo.java | 101 +- .../WritePredefinedProjectProperties.java | 731 ++--- .../GenerateOoziePropertiesMojoTest.java | 131 +- .../WritePredefinedProjectPropertiesTest.java | 576 ++-- .../collector/worker/model/ApiDescriptor.java | 57 +- .../mdstore/manager/common/model/MDStore.java | 162 +- .../common/model/MDStoreCurrentVersion.java | 58 +- .../manager/common/model/MDStoreVersion.java | 126 +- .../manager/common/model/MDStoreWithInfo.java | 186 +- .../ArgumentApplicationParser.java | 136 +- .../dhp/application/OptionsParameter.java | 50 +- .../common/FunctionalInterfaceSupport.java | 79 +- .../eu/dnetlib/dhp/common/HdfsSupport.java | 103 +- .../dhp/common/SparkSessionSupport.java | 121 +- .../dnetlib/dhp/common/ThrowingSupport.java | 124 +- .../dhp/model/mdstore/MetadataRecord.java | 175 +- .../dnetlib/dhp/model/mdstore/Provenance.java | 61 +- .../dhp/parser/utility/VtdException.java | 13 +- .../dhp/parser/utility/VtdUtilityParser.java | 169 +- .../java/eu/dnetlib/dhp/utils/DHPUtils.java | 107 +- .../dhp/utils/ISLookupClientFactory.java | 28 +- .../saxon/AbstractExtensionFunction.java | 34 +- .../dnetlib/dhp/utils/saxon/ExtractYear.java | 94 +- .../dhp/utils/saxon/NormalizeDate.java | 92 +- .../eu/dnetlib/dhp/utils/saxon/PickFirst.java | 82 +- .../utils/saxon/SaxonTransformerFactory.java | 33 +- .../main/java/eu/dnetlib/message/Message.java | 103 +- .../eu/dnetlib/message/MessageConsumer.java | 68 +- .../eu/dnetlib/message/MessageManager.java | 208 +- .../java/eu/dnetlib/message/MessageType.java | 4 +- .../scholexplorer/relation/RelInfo.java | 29 +- .../relation/RelationMapper.java | 16 +- .../ArgumentApplicationParserTest.java | 110 +- .../dnetlib/dhp/common/HdfsSupportTest.java | 99 +- .../dhp/common/SparkSessionSupportTest.java | 75 +- .../dhp/model/mdstore/MetadataRecordTest.java | 11 +- .../java/eu/dnetlib/message/MessageTest.java | 67 +- .../relation/RelationMapperTest.java | 11 +- .../dhp/schema/action/AtomicAction.java | 46 +- .../action/AtomicActionDeserializer.java | 33 +- .../dnetlib/dhp/schema/common/EntityType.java | 29 +- .../dhp/schema/common/MainEntityType.java | 6 +- .../dhp/schema/common/ModelConstants.java | 55 +- .../eu/dnetlib/dhp/schema/oaf/Author.java | 136 +- .../eu/dnetlib/dhp/schema/oaf/Context.java | 56 +- .../eu/dnetlib/dhp/schema/oaf/Country.java | 42 +- .../eu/dnetlib/dhp/schema/oaf/DataInfo.java | 122 +- .../eu/dnetlib/dhp/schema/oaf/Dataset.java | 151 +- .../eu/dnetlib/dhp/schema/oaf/Datasource.java | 786 +++--- .../dhp/schema/oaf/ExternalReference.java | 170 +- .../eu/dnetlib/dhp/schema/oaf/ExtraInfo.java | 105 +- .../java/eu/dnetlib/dhp/schema/oaf/Field.java | 56 +- .../dnetlib/dhp/schema/oaf/GeoLocation.java | 101 +- .../eu/dnetlib/dhp/schema/oaf/Instance.java | 209 +- .../eu/dnetlib/dhp/schema/oaf/Journal.java | 240 +- .../eu/dnetlib/dhp/schema/oaf/KeyValue.java | 97 +- .../dnetlib/dhp/schema/oaf/OAIProvenance.java | 39 +- .../java/eu/dnetlib/dhp/schema/oaf/Oaf.java | 91 +- .../eu/dnetlib/dhp/schema/oaf/OafEntity.java | 168 +- .../dnetlib/dhp/schema/oaf/Organization.java | 323 ++- .../dhp/schema/oaf/OriginDescription.java | 119 +- .../dhp/schema/oaf/OtherResearchProduct.java | 76 +- .../eu/dnetlib/dhp/schema/oaf/Project.java | 534 ++-- .../dnetlib/dhp/schema/oaf/Publication.java | 47 +- .../eu/dnetlib/dhp/schema/oaf/Qualifier.java | 123 +- .../eu/dnetlib/dhp/schema/oaf/Relation.java | 142 +- .../eu/dnetlib/dhp/schema/oaf/Result.java | 400 +-- .../eu/dnetlib/dhp/schema/oaf/Software.java | 102 +- .../dhp/schema/oaf/StructuredProperty.java | 76 +- .../dhp/schema/scholexplorer/DLIDataset.java | 124 +- .../schema/scholexplorer/DLIPublication.java | 126 +- .../dhp/schema/scholexplorer/DLIRelation.java | 15 +- .../dhp/schema/scholexplorer/DLIUnknown.java | 156 +- .../schema/scholexplorer/ProvenaceInfo.java | 57 +- .../dhp/schema/action/AtomicActionTest.java | 40 +- .../dhp/schema/common/ModelSupportTest.java | 42 +- .../eu/dnetlib/dhp/schema/oaf/MergeTest.java | 108 +- .../dhp/schema/scholexplorer/DLItest.java | 119 +- .../dnetlib/dhp/actionmanager/ISClient.java | 219 +- .../migration/LicenseComparator.java | 78 +- .../migration/MigrateActionSet.java | 266 +- .../migration/ProtoConverter.java | 1113 ++++---- .../migration/TransformActions.java | 244 +- .../PartitionActionSetsByPayloadTypeJob.java | 188 +- .../actionmanager/promote/MergeAndGet.java | 135 +- .../PromoteActionPayloadForGraphTableJob.java | 349 ++- .../PromoteActionPayloadFunctions.java | 299 +- ...rtitionActionSetsByPayloadTypeJobTest.java | 335 +-- .../promote/MergeAndGetTest.java | 387 ++- ...moteActionPayloadForGraphTableJobTest.java | 450 +-- .../PromoteActionPayloadFunctionsTest.java | 547 ++-- .../GenerateNativeStoreSparkJob.java | 234 +- .../collection/plugin/CollectorPlugin.java | 6 +- .../plugin/oai/OaiCollectorPlugin.java | 114 +- .../collection/plugin/oai/OaiIterator.java | 279 +- .../plugin/oai/OaiIteratorFactory.java | 31 +- .../worker/DnetCollectorException.java | 43 +- .../worker/DnetCollectorWorker.java | 211 +- .../DnetCollectorWorkerApplication.java | 63 +- .../utils/CollectorPluginErrorLogList.java | 21 +- .../worker/utils/CollectorPluginFactory.java | 20 +- .../worker/utils/HttpConnector.java | 379 +-- .../collection/worker/utils/XmlCleaner.java | 715 +++-- .../dhp/transformation/TransformFunction.java | 119 +- .../transformation/TransformSparkJobNode.java | 151 +- .../dhp/transformation/functions/Cleaner.java | 68 +- .../dhp/transformation/vocabulary/Term.java | 71 +- .../transformation/vocabulary/Vocabulary.java | 71 +- .../vocabulary/VocabularyHelper.java | 21 +- .../dhp/collection/CollectionJobTest.java | 187 +- .../DnetCollectorWorkerApplicationTests.java | 132 +- .../transformation/TransformationJobTest.java | 223 +- .../vocabulary/VocabularyTest.java | 11 +- .../dnetlib/dhp/bulktag/SparkBulkTagJob2.java | 247 +- .../eu/dnetlib/dhp/community/Community.java | 81 +- .../dhp/community/CommunityConfiguration.java | 287 +- .../CommunityConfigurationFactory.java | 183 +- .../eu/dnetlib/dhp/community/Constraint.java | 75 +- .../eu/dnetlib/dhp/community/Constraints.java | 97 +- .../eu/dnetlib/dhp/community/Datasource.java | 76 +- .../java/eu/dnetlib/dhp/community/Pair.java | 50 +- .../eu/dnetlib/dhp/community/ProtoMap.java | 7 +- .../dhp/community/QueryInformationSystem.java | 103 +- .../dnetlib/dhp/community/ResultTagger.java | 394 +-- .../dhp/community/SelectionConstraints.java | 63 +- .../dhp/community/TagginConstants.java | 26 +- .../dhp/community/ZenodoCommunity.java | 55 +- .../dhp/selectioncriteria/ContainsVerb.java | 32 +- .../ContainsVerbIgnoreCase.java | 32 +- .../dhp/selectioncriteria/EqualVerb.java | 32 +- .../EqualVerbIgnoreCase.java | 32 +- .../selectioncriteria/InterfaceAdapter.java | 63 +- .../selectioncriteria/NotContainsVerb.java | 32 +- .../NotContainsVerbIgnoreCase.java | 32 +- .../dhp/selectioncriteria/NotEqualVerb.java | 32 +- .../NotEqualVerbIgnoreCase.java | 32 +- .../dhp/selectioncriteria/Selection.java | 3 +- .../dhp/selectioncriteria/VerbClass.java | 3 +- .../dhp/selectioncriteria/VerbResolver.java | 83 +- .../VerbResolverFactory.java | 7 +- .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 1608 +++++------ .../CommunityConfigurationFactoryTest.java | 265 +- .../dhp/oa/dedup/AbstractSparkAction.java | 116 +- .../eu/dnetlib/dhp/oa/dedup/DatePicker.java | 174 +- .../dhp/oa/dedup/DedupRecordFactory.java | 153 +- .../eu/dnetlib/dhp/oa/dedup/DedupUtility.java | 389 +-- .../java/eu/dnetlib/dhp/oa/dedup/Deduper.java | 81 +- .../dhp/oa/dedup/SparkCreateDedupRecord.java | 147 +- .../dhp/oa/dedup/SparkCreateMergeRels.java | 230 +- .../dhp/oa/dedup/SparkCreateSimRels.java | 190 +- .../dhp/oa/dedup/SparkPropagateRelation.java | 285 +- .../dnetlib/dhp/oa/dedup/SparkReporter.java | 57 +- .../dhp/oa/dedup/SparkUpdateEntity.java | 203 +- .../oa/dedup/graph/ConnectedComponent.java | 120 +- .../eu/dnetlib/dhp/oa/dedup/model/Block.java | 101 +- .../dnetlib/dhp/oa/dedup/MergeAuthorTest.java | 67 +- .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 727 ++--- .../dhp/oa/dedup/jpath/JsonPathTest.java | 559 ++-- .../java/eu/dnetlib/dedup/DatePicker.java | 174 +- .../eu/dnetlib/dedup/DedupRecordFactory.java | 502 ++-- .../java/eu/dnetlib/dedup/DedupUtility.java | 361 +-- .../main/java/eu/dnetlib/dedup/Deduper.java | 298 +- .../java/eu/dnetlib/dedup/OafEntityType.java | 9 +- .../dedup/SparkCreateConnectedComponent.java | 164 +- .../dnetlib/dedup/SparkCreateDedupRecord.java | 78 +- .../eu/dnetlib/dedup/SparkCreateSimRels.java | 117 +- .../java/eu/dnetlib/dedup/SparkReporter.java | 59 +- .../dedup/graph/ConnectedComponent.java | 120 +- .../dedup/sx/SparkPropagateRelationsJob.java | 176 +- .../dedup/sx/SparkUpdateEntityJob.java | 157 +- .../oa/graph/hive/GraphHiveImporterJob.java | 98 +- .../raw/AbstractMdRecordToOafMapper.java | 869 +++--- .../raw/DispatchEntitiesApplication.java | 114 +- .../raw/GenerateEntitiesApplication.java | 294 +- .../oa/graph/raw/MergeClaimsApplication.java | 186 +- .../raw/MigrateDbEntitiesApplication.java | 35 +- .../raw/MigrateMongoMdstoresApplication.java | 95 +- .../dhp/oa/graph/raw/OafToOafMapper.java | 427 +-- .../dhp/oa/graph/raw/OdfToOafMapper.java | 591 ++-- .../common/AbstractMigrationApplication.java | 109 +- .../dhp/oa/graph/raw/common/DbClient.java | 81 +- .../oa/graph/raw/common/MdstoreClient.java | 148 +- .../graph/raw/common/MigrationConstants.java | 40 +- .../oa/graph/raw/common/OafMapperUtils.java | 369 +-- .../dhp/oa/graph/raw/common/PacePerson.java | 287 +- .../dhp/sx/graph/ImportDataFromMongo.java | 240 +- .../dhp/sx/graph/SparkExtractEntitiesJob.java | 193 +- .../sx/graph/SparkSXGeneratePidSimlarity.java | 112 +- .../SparkScholexplorerCreateRawGraphJob.java | 437 ++- .../SparkScholexplorerGraphImporter.java | 100 +- .../parser/AbstractScholexplorerParser.java | 374 +-- .../parser/DatasetScholexplorerParser.java | 474 ++-- .../PublicationScholexplorerParser.java | 410 +-- .../oa/graph/GraphHiveImporterJobTest.java | 122 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 208 +- .../raw/MigrateDbEntitiesApplicationTest.java | 540 ++-- .../dhp/sx/graph/ScholexplorerParserTest.java | 43 +- .../SparkScholexplorerGraphImporterTest.java | 4 +- ...parkScholexplorerMergeEntitiesJobTest.java | 4 +- .../dnetlib/dhp/provision/ProvisionUtil.java | 70 +- .../dhp/provision/RelatedItemInfo.java | 74 +- .../provision/SparkExtractRelationCount.java | 42 +- .../dhp/provision/SparkGenerateScholix.java | 161 +- .../dhp/provision/SparkGenerateSummary.java | 166 +- .../provision/SparkIndexCollectionOnES.java | 95 +- .../dhp/provision/scholix/Scholix.java | 310 +- .../scholix/ScholixCollectedFrom.java | 58 +- .../provision/scholix/ScholixEntityId.java | 40 +- .../provision/scholix/ScholixIdentifier.java | 40 +- .../scholix/ScholixRelationship.java | 56 +- .../provision/scholix/ScholixResource.java | 216 +- .../scholix/summary/CollectedFromType.java | 56 +- .../scholix/summary/SchemeValue.java | 40 +- .../scholix/summary/ScholixSummary.java | 530 ++-- .../scholix/summary/TypedIdentifier.java | 40 +- .../provision/scholix/summary/Typology.java | 5 +- .../provision/update/CrossRefParserJSON.java | 202 +- .../dhp/provision/update/CrossrefClient.java | 133 +- .../provision/update/Datacite2Scholix.java | 389 +-- .../dhp/provision/update/DataciteClient.java | 103 +- .../update/DataciteClientIterator.java | 180 +- .../update/RetrieveUpdateFromDatacite.java | 105 +- .../update/SparkResolveScholixTarget.java | 287 +- .../dhp/provision/DataciteClientTest.java | 70 +- .../dhp/provision/ExtractInfoTest.java | 39 +- .../oa/provision/AdjacencyListBuilderJob.java | 181 +- .../CreateRelatedEntitiesJob_phase1.java | 373 ++- .../CreateRelatedEntitiesJob_phase2.java | 339 ++- .../dhp/oa/provision/PrepareRelationsJob.java | 242 +- .../dhp/oa/provision/XmlConverterJob.java | 335 +-- .../dhp/oa/provision/XmlIndexingJob.java | 342 +-- .../oa/provision/model/EntityRelEntity.java | 93 +- .../dhp/oa/provision/model/JoinedEntity.java | 32 +- .../dhp/oa/provision/model/RelatedEntity.java | 456 +-- .../oa/provision/model/SortableRelation.java | 50 +- .../dhp/oa/provision/model/Tuple2.java | 67 +- .../dhp/oa/provision/model/TypedRow.java | 84 +- .../dhp/oa/provision/utils/ContextDef.java | 71 +- .../dhp/oa/provision/utils/ContextMapper.java | 59 +- .../oa/provision/utils/GraphMappingUtils.java | 26 +- .../oa/provision/utils/LicenseComparator.java | 78 +- .../provision/utils/RelationPartitioner.java | 33 +- .../utils/StreamingInputDocumentFactory.java | 376 ++- .../oa/provision/utils/TemplateFactory.java | 176 +- .../oa/provision/utils/TemplateResources.java | 61 +- .../oa/provision/utils/XmlRecordFactory.java | 2482 +++++++++-------- .../utils/XmlSerializationUtils.java | 261 +- .../dhp/oa/provision/GraphJoinerTest.java | 57 +- .../eu/dnetlib/dhp/PropagationConstant.java | 316 ++- .../dnetlib/dhp/QueryInformationSystem.java | 21 +- .../dhp/countrypropagation/CountrySbs.java | 29 +- .../countrypropagation/DatasourceCountry.java | 29 +- .../PrepareDatasourceCountryAssociation.java | 200 +- .../countrypropagation/ResultCountrySet.java | 29 +- .../SparkCountryPropagationJob2.java | 318 ++- .../AutoritativeAuthor.java | 57 +- .../PrepareResultOrcidAssociationStep1.java | 193 +- .../PrepareResultOrcidAssociationStep2.java | 146 +- .../ResultOrcidList.java | 29 +- .../SparkOrcidToResultFromSemRelJob3.java | 306 +- .../PrepareProjectResultsAssociation.java | 234 +- .../dhp/projecttoresult/ResultProjectSet.java | 29 +- ...SparkResultToProjectThroughSemRelJob3.java | 240 +- .../OrganizationMap.java | 19 +- .../PrepareResultCommunitySet.java | 204 +- .../ResultCommunityList.java | 29 +- .../ResultOrganizations.java | 43 +- ...ResultToCommunityFromOrganizationJob2.java | 193 +- .../PrepareResultCommunitySetStep1.java | 261 +- .../PrepareResultCommunitySetStep2.java | 148 +- ...arkResultToCommunityThroughSemRelJob4.java | 208 +- .../DatasourceOrganization.java | 29 +- .../PrepareResultInstRepoAssociation.java | 206 +- .../ResultOrganizationSet.java | 29 +- ...rkResultToOrganizationFromIstRepoJob2.java | 485 ++-- .../CountryPropagationJobTest.java | 422 +-- .../OrcidPropagationJobTest.java | 382 +-- .../ProjectPropagationJobTest.java | 422 +-- .../ResultToCommunityJobTest.java | 552 ++-- .../ResultToCommunityJobTest.java | 464 +-- .../Result2OrganizationJobTest.java | 491 ++-- 281 files changed, 26243 insertions(+), 24828 deletions(-) diff --git a/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojo.java b/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojo.java index ccc2abef0..10a25fdc3 100644 --- a/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojo.java +++ b/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojo.java @@ -1,8 +1,10 @@ + package eu.dnetlib.maven.plugin.properties; import java.io.File; import java.util.ArrayList; import java.util.List; + import org.apache.commons.lang.ArrayUtils; import org.apache.commons.lang.StringUtils; import org.apache.maven.plugin.AbstractMojo; @@ -17,55 +19,58 @@ import org.apache.maven.plugin.MojoFailureException; */ public class GenerateOoziePropertiesMojo extends AbstractMojo { - public static final String PROPERTY_NAME_WF_SOURCE_DIR = "workflow.source.dir"; - public static final String PROPERTY_NAME_SANDBOX_NAME = "sandboxName"; + public static final String PROPERTY_NAME_WF_SOURCE_DIR = "workflow.source.dir"; + public static final String PROPERTY_NAME_SANDBOX_NAME = "sandboxName"; - private final String[] limiters = {"dhp", "dnetlib", "eu"}; + private final String[] limiters = { + "dhp", "dnetlib", "eu" + }; - @Override - public void execute() throws MojoExecutionException, MojoFailureException { - if (System.getProperties().containsKey(PROPERTY_NAME_WF_SOURCE_DIR) - && !System.getProperties().containsKey(PROPERTY_NAME_SANDBOX_NAME)) { - String generatedSandboxName = - generateSandboxName(System.getProperties().getProperty(PROPERTY_NAME_WF_SOURCE_DIR)); - if (generatedSandboxName != null) { - System.getProperties().setProperty(PROPERTY_NAME_SANDBOX_NAME, generatedSandboxName); - } else { - System.out.println( - "unable to generate sandbox name from path: " - + System.getProperties().getProperty(PROPERTY_NAME_WF_SOURCE_DIR)); - } - } - } + @Override + public void execute() throws MojoExecutionException, MojoFailureException { + if (System.getProperties().containsKey(PROPERTY_NAME_WF_SOURCE_DIR) + && !System.getProperties().containsKey(PROPERTY_NAME_SANDBOX_NAME)) { + String generatedSandboxName = generateSandboxName( + System.getProperties().getProperty(PROPERTY_NAME_WF_SOURCE_DIR)); + if (generatedSandboxName != null) { + System.getProperties().setProperty(PROPERTY_NAME_SANDBOX_NAME, generatedSandboxName); + } else { + System.out + .println( + "unable to generate sandbox name from path: " + + System.getProperties().getProperty(PROPERTY_NAME_WF_SOURCE_DIR)); + } + } + } - /** - * Generates sandbox name from workflow source directory. - * - * @param wfSourceDir - * @return generated sandbox name - */ - private String generateSandboxName(String wfSourceDir) { - // utilize all dir names until finding one of the limiters - List sandboxNameParts = new ArrayList(); - String[] tokens = StringUtils.split(wfSourceDir, File.separatorChar); - ArrayUtils.reverse(tokens); - if (tokens.length > 0) { - for (String token : tokens) { - for (String limiter : limiters) { - if (limiter.equals(token)) { - return sandboxNameParts.size() > 0 - ? StringUtils.join(sandboxNameParts.toArray()) - : null; - } - } - if (sandboxNameParts.size() > 0) { - sandboxNameParts.add(0, File.separator); - } - sandboxNameParts.add(0, token); - } - return StringUtils.join(sandboxNameParts.toArray()); - } else { - return null; - } - } + /** + * Generates sandbox name from workflow source directory. + * + * @param wfSourceDir + * @return generated sandbox name + */ + private String generateSandboxName(String wfSourceDir) { + // utilize all dir names until finding one of the limiters + List sandboxNameParts = new ArrayList(); + String[] tokens = StringUtils.split(wfSourceDir, File.separatorChar); + ArrayUtils.reverse(tokens); + if (tokens.length > 0) { + for (String token : tokens) { + for (String limiter : limiters) { + if (limiter.equals(token)) { + return sandboxNameParts.size() > 0 + ? StringUtils.join(sandboxNameParts.toArray()) + : null; + } + } + if (sandboxNameParts.size() > 0) { + sandboxNameParts.add(0, File.separator); + } + sandboxNameParts.add(0, token); + } + return StringUtils.join(sandboxNameParts.toArray()); + } else { + return null; + } + } } diff --git a/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectProperties.java b/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectProperties.java index 5e0166e4f..c1c567f95 100644 --- a/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectProperties.java +++ b/dhp-build/dhp-build-properties-maven-plugin/src/main/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectProperties.java @@ -9,9 +9,9 @@ * express or implied. See the License for the specific language governing permissions and * limitations under the License. */ + package eu.dnetlib.maven.plugin.properties; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map.Entry; import java.util.Properties; import java.util.Set; + import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; @@ -35,6 +36,8 @@ import org.springframework.core.io.DefaultResourceLoader; import org.springframework.core.io.Resource; import org.springframework.core.io.ResourceLoader; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Writes project properties for the keys listed in specified properties files. Based on: * http://site.kuali.org/maven/plugins/properties-maven-plugin/1.3.2/write-project-properties-mojo.html @@ -44,401 +47,401 @@ import org.springframework.core.io.ResourceLoader; */ public class WritePredefinedProjectProperties extends AbstractMojo { - private static final String CR = "\r"; - private static final String LF = "\n"; - private static final String TAB = "\t"; - protected static final String PROPERTY_PREFIX_ENV = "env."; - private static final String ENCODING_UTF8 = "utf8"; + private static final String CR = "\r"; + private static final String LF = "\n"; + private static final String TAB = "\t"; + protected static final String PROPERTY_PREFIX_ENV = "env."; + private static final String ENCODING_UTF8 = "utf8"; - /** @parameter property="properties.includePropertyKeysFromFiles" */ - private String[] includePropertyKeysFromFiles; + /** @parameter property="properties.includePropertyKeysFromFiles" */ + private String[] includePropertyKeysFromFiles; - /** - * @parameter default-value="${project}" - * @required - * @readonly - */ - protected MavenProject project; + /** + * @parameter default-value="${project}" + * @required + * @readonly + */ + protected MavenProject project; - /** - * The file that properties will be written to - * - * @parameter property="properties.outputFile" - * default-value="${project.build.directory}/properties/project.properties"; - * @required - */ - protected File outputFile; + /** + * The file that properties will be written to + * + * @parameter property="properties.outputFile" + * default-value="${project.build.directory}/properties/project.properties"; + * @required + */ + protected File outputFile; - /** - * If true, the plugin will silently ignore any non-existent properties files, and the build will - * continue - * - * @parameter property="properties.quiet" default-value="true" - */ - private boolean quiet; + /** + * If true, the plugin will silently ignore any non-existent properties files, and the build will continue + * + * @parameter property="properties.quiet" default-value="true" + */ + private boolean quiet; - /** - * Comma separated list of characters to escape when writing property values. cr=carriage return, - * lf=linefeed, tab=tab. Any other values are taken literally. - * - * @parameter default-value="cr,lf,tab" property="properties.escapeChars" - */ - private String escapeChars; + /** + * Comma separated list of characters to escape when writing property values. cr=carriage return, lf=linefeed, + * tab=tab. Any other values are taken literally. + * + * @parameter default-value="cr,lf,tab" property="properties.escapeChars" + */ + private String escapeChars; - /** - * If true, the plugin will include system properties when writing the properties file. System - * properties override both environment variables and project properties. - * - * @parameter default-value="false" property="properties.includeSystemProperties" - */ - private boolean includeSystemProperties; + /** + * If true, the plugin will include system properties when writing the properties file. System properties override + * both environment variables and project properties. + * + * @parameter default-value="false" property="properties.includeSystemProperties" + */ + private boolean includeSystemProperties; - /** - * If true, the plugin will include environment variables when writing the properties file. - * Environment variables are prefixed with "env". Environment variables override project - * properties. - * - * @parameter default-value="false" property="properties.includeEnvironmentVariables" - */ - private boolean includeEnvironmentVariables; + /** + * If true, the plugin will include environment variables when writing the properties file. Environment variables + * are prefixed with "env". Environment variables override project properties. + * + * @parameter default-value="false" property="properties.includeEnvironmentVariables" + */ + private boolean includeEnvironmentVariables; - /** - * Comma separated set of properties to exclude when writing the properties file - * - * @parameter property="properties.exclude" - */ - private String exclude; + /** + * Comma separated set of properties to exclude when writing the properties file + * + * @parameter property="properties.exclude" + */ + private String exclude; - /** - * Comma separated set of properties to write to the properties file. If provided, only the - * properties matching those supplied here will be written to the properties file. - * - * @parameter property="properties.include" - */ - private String include; + /** + * Comma separated set of properties to write to the properties file. If provided, only the properties matching + * those supplied here will be written to the properties file. + * + * @parameter property="properties.include" + */ + private String include; - /* - * (non-Javadoc) - * @see org.apache.maven.plugin.AbstractMojo#execute() - */ - @Override - @SuppressFBWarnings({"NP_UNWRITTEN_FIELD", "UWF_UNWRITTEN_FIELD"}) - public void execute() throws MojoExecutionException, MojoFailureException { - Properties properties = new Properties(); - // Add project properties - properties.putAll(project.getProperties()); - if (includeEnvironmentVariables) { - // Add environment variables, overriding any existing properties with the same key - properties.putAll(getEnvironmentVariables()); - } - if (includeSystemProperties) { - // Add system properties, overriding any existing properties with the same key - properties.putAll(System.getProperties()); - } + /* + * (non-Javadoc) + * @see org.apache.maven.plugin.AbstractMojo#execute() + */ + @Override + @SuppressFBWarnings({ + "NP_UNWRITTEN_FIELD", "UWF_UNWRITTEN_FIELD" + }) + public void execute() throws MojoExecutionException, MojoFailureException { + Properties properties = new Properties(); + // Add project properties + properties.putAll(project.getProperties()); + if (includeEnvironmentVariables) { + // Add environment variables, overriding any existing properties with the same key + properties.putAll(getEnvironmentVariables()); + } + if (includeSystemProperties) { + // Add system properties, overriding any existing properties with the same key + properties.putAll(System.getProperties()); + } - // Remove properties as appropriate - trim(properties, exclude, include); + // Remove properties as appropriate + trim(properties, exclude, include); - String comment = "# " + new Date() + "\n"; - List escapeTokens = getEscapeChars(escapeChars); + String comment = "# " + new Date() + "\n"; + List escapeTokens = getEscapeChars(escapeChars); - getLog().info("Creating " + outputFile); - writeProperties(outputFile, comment, properties, escapeTokens); - } + getLog().info("Creating " + outputFile); + writeProperties(outputFile, comment, properties, escapeTokens); + } - /** - * Provides environment variables. - * - * @return environment variables - */ - protected static Properties getEnvironmentVariables() { - Properties props = new Properties(); - for (Entry entry : System.getenv().entrySet()) { - props.setProperty(PROPERTY_PREFIX_ENV + entry.getKey(), entry.getValue()); - } - return props; - } + /** + * Provides environment variables. + * + * @return environment variables + */ + protected static Properties getEnvironmentVariables() { + Properties props = new Properties(); + for (Entry entry : System.getenv().entrySet()) { + props.setProperty(PROPERTY_PREFIX_ENV + entry.getKey(), entry.getValue()); + } + return props; + } - /** - * Removes properties which should not be written. - * - * @param properties - * @param omitCSV - * @param includeCSV - * @throws MojoExecutionException - */ - protected void trim(Properties properties, String omitCSV, String includeCSV) - throws MojoExecutionException { - List omitKeys = getListFromCSV(omitCSV); - for (String key : omitKeys) { - properties.remove(key); - } + /** + * Removes properties which should not be written. + * + * @param properties + * @param omitCSV + * @param includeCSV + * @throws MojoExecutionException + */ + protected void trim(Properties properties, String omitCSV, String includeCSV) + throws MojoExecutionException { + List omitKeys = getListFromCSV(omitCSV); + for (String key : omitKeys) { + properties.remove(key); + } - List includeKeys = getListFromCSV(includeCSV); - // mh: including keys from predefined properties - if (includePropertyKeysFromFiles != null && includePropertyKeysFromFiles.length > 0) { - for (String currentIncludeLoc : includePropertyKeysFromFiles) { - if (validate(currentIncludeLoc)) { - Properties p = getProperties(currentIncludeLoc); - for (String key : p.stringPropertyNames()) { - includeKeys.add(key); - } - } - } - } - if (includeKeys != null && !includeKeys.isEmpty()) { - // removing only when include keys provided - Set keys = properties.stringPropertyNames(); - for (String key : keys) { - if (!includeKeys.contains(key)) { - properties.remove(key); - } - } - } - } + List includeKeys = getListFromCSV(includeCSV); + // mh: including keys from predefined properties + if (includePropertyKeysFromFiles != null && includePropertyKeysFromFiles.length > 0) { + for (String currentIncludeLoc : includePropertyKeysFromFiles) { + if (validate(currentIncludeLoc)) { + Properties p = getProperties(currentIncludeLoc); + for (String key : p.stringPropertyNames()) { + includeKeys.add(key); + } + } + } + } + if (includeKeys != null && !includeKeys.isEmpty()) { + // removing only when include keys provided + Set keys = properties.stringPropertyNames(); + for (String key : keys) { + if (!includeKeys.contains(key)) { + properties.remove(key); + } + } + } + } - /** - * Checks whether file exists. - * - * @param location - * @return true when exists, false otherwise. - */ - protected boolean exists(String location) { - if (StringUtils.isBlank(location)) { - return false; - } - File file = new File(location); - if (file.exists()) { - return true; - } - ResourceLoader loader = new DefaultResourceLoader(); - Resource resource = loader.getResource(location); - return resource.exists(); - } + /** + * Checks whether file exists. + * + * @param location + * @return true when exists, false otherwise. + */ + protected boolean exists(String location) { + if (StringUtils.isBlank(location)) { + return false; + } + File file = new File(location); + if (file.exists()) { + return true; + } + ResourceLoader loader = new DefaultResourceLoader(); + Resource resource = loader.getResource(location); + return resource.exists(); + } - /** - * Validates resource location. - * - * @param location - * @return true when valid, false otherwise - * @throws MojoExecutionException - */ - protected boolean validate(String location) throws MojoExecutionException { - boolean exists = exists(location); - if (exists) { - return true; - } - if (quiet) { - getLog().info("Ignoring non-existent properties file '" + location + "'"); - return false; - } else { - throw new MojoExecutionException("Non-existent properties file '" + location + "'"); - } - } + /** + * Validates resource location. + * + * @param location + * @return true when valid, false otherwise + * @throws MojoExecutionException + */ + protected boolean validate(String location) throws MojoExecutionException { + boolean exists = exists(location); + if (exists) { + return true; + } + if (quiet) { + getLog().info("Ignoring non-existent properties file '" + location + "'"); + return false; + } else { + throw new MojoExecutionException("Non-existent properties file '" + location + "'"); + } + } - /** - * Provides input stream. - * - * @param location - * @return input stream - * @throws IOException - */ - protected InputStream getInputStream(String location) throws IOException { - File file = new File(location); - if (file.exists()) { - return new FileInputStream(location); - } - ResourceLoader loader = new DefaultResourceLoader(); - Resource resource = loader.getResource(location); - return resource.getInputStream(); - } + /** + * Provides input stream. + * + * @param location + * @return input stream + * @throws IOException + */ + protected InputStream getInputStream(String location) throws IOException { + File file = new File(location); + if (file.exists()) { + return new FileInputStream(location); + } + ResourceLoader loader = new DefaultResourceLoader(); + Resource resource = loader.getResource(location); + return resource.getInputStream(); + } - /** - * Creates properties for given location. - * - * @param location - * @return properties for given location - * @throws MojoExecutionException - */ - protected Properties getProperties(String location) throws MojoExecutionException { - InputStream in = null; - try { - Properties properties = new Properties(); - in = getInputStream(location); - if (location.toLowerCase().endsWith(".xml")) { - properties.loadFromXML(in); - } else { - properties.load(in); - } - return properties; - } catch (IOException e) { - throw new MojoExecutionException("Error reading properties file " + location, e); - } finally { - IOUtils.closeQuietly(in); - } - } + /** + * Creates properties for given location. + * + * @param location + * @return properties for given location + * @throws MojoExecutionException + */ + protected Properties getProperties(String location) throws MojoExecutionException { + InputStream in = null; + try { + Properties properties = new Properties(); + in = getInputStream(location); + if (location.toLowerCase().endsWith(".xml")) { + properties.loadFromXML(in); + } else { + properties.load(in); + } + return properties; + } catch (IOException e) { + throw new MojoExecutionException("Error reading properties file " + location, e); + } finally { + IOUtils.closeQuietly(in); + } + } - /** - * Provides escape characters. - * - * @param escapeChars - * @return escape characters - */ - protected List getEscapeChars(String escapeChars) { - List tokens = getListFromCSV(escapeChars); - List realTokens = new ArrayList(); - for (String token : tokens) { - String realToken = getRealToken(token); - realTokens.add(realToken); - } - return realTokens; - } + /** + * Provides escape characters. + * + * @param escapeChars + * @return escape characters + */ + protected List getEscapeChars(String escapeChars) { + List tokens = getListFromCSV(escapeChars); + List realTokens = new ArrayList(); + for (String token : tokens) { + String realToken = getRealToken(token); + realTokens.add(realToken); + } + return realTokens; + } - /** - * Provides real token. - * - * @param token - * @return real token - */ - protected String getRealToken(String token) { - if (token.equalsIgnoreCase("CR")) { - return CR; - } else if (token.equalsIgnoreCase("LF")) { - return LF; - } else if (token.equalsIgnoreCase("TAB")) { - return TAB; - } else { - return token; - } - } + /** + * Provides real token. + * + * @param token + * @return real token + */ + protected String getRealToken(String token) { + if (token.equalsIgnoreCase("CR")) { + return CR; + } else if (token.equalsIgnoreCase("LF")) { + return LF; + } else if (token.equalsIgnoreCase("TAB")) { + return TAB; + } else { + return token; + } + } - /** - * Returns content. - * - * @param comment - * @param properties - * @param escapeTokens - * @return content - */ - protected String getContent(String comment, Properties properties, List escapeTokens) { - List names = new ArrayList(properties.stringPropertyNames()); - Collections.sort(names); - StringBuilder sb = new StringBuilder(); - if (!StringUtils.isBlank(comment)) { - sb.append(comment); - } - for (String name : names) { - String value = properties.getProperty(name); - String escapedValue = escape(value, escapeTokens); - sb.append(name + "=" + escapedValue + "\n"); - } - return sb.toString(); - } + /** + * Returns content. + * + * @param comment + * @param properties + * @param escapeTokens + * @return content + */ + protected String getContent(String comment, Properties properties, List escapeTokens) { + List names = new ArrayList(properties.stringPropertyNames()); + Collections.sort(names); + StringBuilder sb = new StringBuilder(); + if (!StringUtils.isBlank(comment)) { + sb.append(comment); + } + for (String name : names) { + String value = properties.getProperty(name); + String escapedValue = escape(value, escapeTokens); + sb.append(name + "=" + escapedValue + "\n"); + } + return sb.toString(); + } - /** - * Writes properties to given file. - * - * @param file - * @param comment - * @param properties - * @param escapeTokens - * @throws MojoExecutionException - */ - protected void writeProperties( - File file, String comment, Properties properties, List escapeTokens) - throws MojoExecutionException { - try { - String content = getContent(comment, properties, escapeTokens); - FileUtils.writeStringToFile(file, content, ENCODING_UTF8); - } catch (IOException e) { - throw new MojoExecutionException("Error creating properties file", e); - } - } + /** + * Writes properties to given file. + * + * @param file + * @param comment + * @param properties + * @param escapeTokens + * @throws MojoExecutionException + */ + protected void writeProperties( + File file, String comment, Properties properties, List escapeTokens) + throws MojoExecutionException { + try { + String content = getContent(comment, properties, escapeTokens); + FileUtils.writeStringToFile(file, content, ENCODING_UTF8); + } catch (IOException e) { + throw new MojoExecutionException("Error creating properties file", e); + } + } - /** - * Escapes characters. - * - * @param s - * @param escapeChars - * @return - */ - protected String escape(String s, List escapeChars) { - String result = s; - for (String escapeChar : escapeChars) { - result = result.replace(escapeChar, getReplacementToken(escapeChar)); - } - return result; - } + /** + * Escapes characters. + * + * @param s + * @param escapeChars + * @return + */ + protected String escape(String s, List escapeChars) { + String result = s; + for (String escapeChar : escapeChars) { + result = result.replace(escapeChar, getReplacementToken(escapeChar)); + } + return result; + } - /** - * Provides replacement token. - * - * @param escapeChar - * @return replacement token - */ - protected String getReplacementToken(String escapeChar) { - if (escapeChar.equals(CR)) { - return "\\r"; - } else if (escapeChar.equals(LF)) { - return "\\n"; - } else if (escapeChar.equals(TAB)) { - return "\\t"; - } else { - return "\\" + escapeChar; - } - } + /** + * Provides replacement token. + * + * @param escapeChar + * @return replacement token + */ + protected String getReplacementToken(String escapeChar) { + if (escapeChar.equals(CR)) { + return "\\r"; + } else if (escapeChar.equals(LF)) { + return "\\n"; + } else if (escapeChar.equals(TAB)) { + return "\\t"; + } else { + return "\\" + escapeChar; + } + } - /** - * Returns list from csv. - * - * @param csv - * @return list of values generated from CSV - */ - protected static final List getListFromCSV(String csv) { - if (StringUtils.isBlank(csv)) { - return new ArrayList(); - } - List list = new ArrayList(); - String[] tokens = StringUtils.split(csv, ","); - for (String token : tokens) { - list.add(token.trim()); - } - return list; - } + /** + * Returns list from csv. + * + * @param csv + * @return list of values generated from CSV + */ + protected static final List getListFromCSV(String csv) { + if (StringUtils.isBlank(csv)) { + return new ArrayList(); + } + List list = new ArrayList(); + String[] tokens = StringUtils.split(csv, ","); + for (String token : tokens) { + list.add(token.trim()); + } + return list; + } - public void setIncludeSystemProperties(boolean includeSystemProperties) { - this.includeSystemProperties = includeSystemProperties; - } + public void setIncludeSystemProperties(boolean includeSystemProperties) { + this.includeSystemProperties = includeSystemProperties; + } - public void setEscapeChars(String escapeChars) { - this.escapeChars = escapeChars; - } + public void setEscapeChars(String escapeChars) { + this.escapeChars = escapeChars; + } - public void setIncludeEnvironmentVariables(boolean includeEnvironmentVariables) { - this.includeEnvironmentVariables = includeEnvironmentVariables; - } + public void setIncludeEnvironmentVariables(boolean includeEnvironmentVariables) { + this.includeEnvironmentVariables = includeEnvironmentVariables; + } - public void setExclude(String exclude) { - this.exclude = exclude; - } + public void setExclude(String exclude) { + this.exclude = exclude; + } - public void setInclude(String include) { - this.include = include; - } + public void setInclude(String include) { + this.include = include; + } - public void setQuiet(boolean quiet) { - this.quiet = quiet; - } + public void setQuiet(boolean quiet) { + this.quiet = quiet; + } - /** - * Sets property files for which keys properties should be included. - * - * @param includePropertyKeysFromFiles - */ - public void setIncludePropertyKeysFromFiles(String[] includePropertyKeysFromFiles) { - if (includePropertyKeysFromFiles != null) { - this.includePropertyKeysFromFiles = - Arrays.copyOf(includePropertyKeysFromFiles, includePropertyKeysFromFiles.length); - } - } + /** + * Sets property files for which keys properties should be included. + * + * @param includePropertyKeysFromFiles + */ + public void setIncludePropertyKeysFromFiles(String[] includePropertyKeysFromFiles) { + if (includePropertyKeysFromFiles != null) { + this.includePropertyKeysFromFiles = Arrays + .copyOf(includePropertyKeysFromFiles, includePropertyKeysFromFiles.length); + } + } } diff --git a/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojoTest.java b/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojoTest.java index 3a0d5fcc7..b8075ba5d 100644 --- a/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojoTest.java +++ b/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/GenerateOoziePropertiesMojoTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.maven.plugin.properties; import static eu.dnetlib.maven.plugin.properties.GenerateOoziePropertiesMojo.PROPERTY_NAME_SANDBOX_NAME; @@ -10,87 +11,87 @@ import org.junit.jupiter.api.Test; /** @author mhorst, claudio.atzori */ public class GenerateOoziePropertiesMojoTest { - private GenerateOoziePropertiesMojo mojo = new GenerateOoziePropertiesMojo(); + private GenerateOoziePropertiesMojo mojo = new GenerateOoziePropertiesMojo(); - @BeforeEach - public void clearSystemProperties() { - System.clearProperty(PROPERTY_NAME_SANDBOX_NAME); - System.clearProperty(PROPERTY_NAME_WF_SOURCE_DIR); - } + @BeforeEach + public void clearSystemProperties() { + System.clearProperty(PROPERTY_NAME_SANDBOX_NAME); + System.clearProperty(PROPERTY_NAME_WF_SOURCE_DIR); + } - @Test - public void testExecuteEmpty() throws Exception { - // execute - mojo.execute(); + @Test + public void testExecuteEmpty() throws Exception { + // execute + mojo.execute(); - // assert - assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } - @Test - public void testExecuteSandboxNameAlreadySet() throws Exception { - // given - String workflowSourceDir = "eu/dnetlib/dhp/wf/transformers"; - String sandboxName = "originalSandboxName"; - System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); - System.setProperty(PROPERTY_NAME_SANDBOX_NAME, sandboxName); + @Test + public void testExecuteSandboxNameAlreadySet() throws Exception { + // given + String workflowSourceDir = "eu/dnetlib/dhp/wf/transformers"; + String sandboxName = "originalSandboxName"; + System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); + System.setProperty(PROPERTY_NAME_SANDBOX_NAME, sandboxName); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertEquals(sandboxName, System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertEquals(sandboxName, System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } - @Test - public void testExecuteEmptyWorkflowSourceDir() throws Exception { - // given - String workflowSourceDir = ""; - System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); + @Test + public void testExecuteEmptyWorkflowSourceDir() throws Exception { + // given + String workflowSourceDir = ""; + System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } - @Test - public void testExecuteNullSandboxNameGenerated() throws Exception { - // given - String workflowSourceDir = "eu/dnetlib/dhp/"; - System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); + @Test + public void testExecuteNullSandboxNameGenerated() throws Exception { + // given + String workflowSourceDir = "eu/dnetlib/dhp/"; + System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertNull(System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } - @Test - public void testExecute() throws Exception { - // given - String workflowSourceDir = "eu/dnetlib/dhp/wf/transformers"; - System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); + @Test + public void testExecute() throws Exception { + // given + String workflowSourceDir = "eu/dnetlib/dhp/wf/transformers"; + System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertEquals("wf/transformers", System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertEquals("wf/transformers", System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } - @Test - public void testExecuteWithoutRoot() throws Exception { - // given - String workflowSourceDir = "wf/transformers"; - System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); + @Test + public void testExecuteWithoutRoot() throws Exception { + // given + String workflowSourceDir = "wf/transformers"; + System.setProperty(PROPERTY_NAME_WF_SOURCE_DIR, workflowSourceDir); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertEquals("wf/transformers", System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); - } + // assert + assertEquals("wf/transformers", System.getProperty(PROPERTY_NAME_SANDBOX_NAME)); + } } diff --git a/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectPropertiesTest.java b/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectPropertiesTest.java index 1b247198b..e0b2eff37 100644 --- a/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectPropertiesTest.java +++ b/dhp-build/dhp-build-properties-maven-plugin/src/test/java/eu/dnetlib/maven/plugin/properties/WritePredefinedProjectPropertiesTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.maven.plugin.properties; import static eu.dnetlib.maven.plugin.properties.WritePredefinedProjectProperties.PROPERTY_PREFIX_ENV; @@ -7,6 +8,7 @@ import static org.mockito.Mockito.lenient; import java.io.*; import java.util.Properties; + import org.apache.maven.plugin.MojoExecutionException; import org.apache.maven.project.MavenProject; import org.junit.jupiter.api.*; @@ -20,337 +22,353 @@ import org.mockito.junit.jupiter.MockitoExtension; @ExtendWith(MockitoExtension.class) public class WritePredefinedProjectPropertiesTest { - @Mock private MavenProject mavenProject; + @Mock + private MavenProject mavenProject; - private WritePredefinedProjectProperties mojo; + private WritePredefinedProjectProperties mojo; - @BeforeEach - public void init(@TempDir File testFolder) { - MockitoAnnotations.initMocks(this); - mojo = new WritePredefinedProjectProperties(); - mojo.outputFile = getPropertiesFileLocation(testFolder); - mojo.project = mavenProject; - lenient().doReturn(new Properties()).when(mavenProject).getProperties(); - } + @BeforeEach + public void init(@TempDir File testFolder) { + MockitoAnnotations.initMocks(this); + mojo = new WritePredefinedProjectProperties(); + mojo.outputFile = getPropertiesFileLocation(testFolder); + mojo.project = mavenProject; + lenient().doReturn(new Properties()).when(mavenProject).getProperties(); + } - // ----------------------------------- TESTS --------------------------------------------- + // ----------------------------------- TESTS --------------------------------------------- - @Test - public void testExecuteEmpty() throws Exception { - // execute - mojo.execute(); + @Test + public void testExecuteEmpty() throws Exception { + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(mojo.outputFile.getParentFile()); - assertEquals(0, storedProperties.size()); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(mojo.outputFile.getParentFile()); + assertEquals(0, storedProperties.size()); + } - @Test - public void testExecuteWithProjectProperties() throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteWithProjectProperties() throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + doReturn(projectProperties).when(mavenProject).getProperties(); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(mojo.outputFile.getParentFile()); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(key)); - assertEquals(value, storedProperties.getProperty(key)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(mojo.outputFile.getParentFile()); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(key)); + assertEquals(value, storedProperties.getProperty(key)); + } - @Test() - public void testExecuteWithProjectPropertiesAndInvalidOutputFile(@TempDir File testFolder) { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - doReturn(projectProperties).when(mavenProject).getProperties(); - mojo.outputFile = testFolder; + @Test() + public void testExecuteWithProjectPropertiesAndInvalidOutputFile(@TempDir File testFolder) { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + doReturn(projectProperties).when(mavenProject).getProperties(); + mojo.outputFile = testFolder; - // execute - Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); - } + // execute + Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); + } - @Test - public void testExecuteWithProjectPropertiesExclusion(@TempDir File testFolder) throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String excludedKey = "excludedPropertyKey"; - String excludedValue = "excludedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(excludedKey, excludedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); - mojo.setExclude(excludedKey); + @Test + public void testExecuteWithProjectPropertiesExclusion(@TempDir File testFolder) throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String excludedKey = "excludedPropertyKey"; + String excludedValue = "excludedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(excludedKey, excludedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); + mojo.setExclude(excludedKey); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(key)); - assertEquals(value, storedProperties.getProperty(key)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(key)); + assertEquals(value, storedProperties.getProperty(key)); + } - @Test - public void testExecuteWithProjectPropertiesInclusion(@TempDir File testFolder) throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); - mojo.setInclude(includedKey); + @Test + public void testExecuteWithProjectPropertiesInclusion(@TempDir File testFolder) throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); + mojo.setInclude(includedKey); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(includedKey)); - assertEquals(includedValue, storedProperties.getProperty(includedKey)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(includedKey)); + assertEquals(includedValue, storedProperties.getProperty(includedKey)); + } - @Test - public void testExecuteIncludingPropertyKeysFromFile(@TempDir File testFolder) throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteIncludingPropertyKeysFromFile(@TempDir File testFolder) throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); - File includedPropertiesFile = new File(testFolder, "included.properties"); - Properties includedProperties = new Properties(); - includedProperties.setProperty(includedKey, "irrelevantValue"); - includedProperties.store(new FileWriter(includedPropertiesFile), null); + File includedPropertiesFile = new File(testFolder, "included.properties"); + Properties includedProperties = new Properties(); + includedProperties.setProperty(includedKey, "irrelevantValue"); + includedProperties.store(new FileWriter(includedPropertiesFile), null); - mojo.setIncludePropertyKeysFromFiles(new String[] {includedPropertiesFile.getAbsolutePath()}); + mojo.setIncludePropertyKeysFromFiles(new String[] { + includedPropertiesFile.getAbsolutePath() + }); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(includedKey)); - assertEquals(includedValue, storedProperties.getProperty(includedKey)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(includedKey)); + assertEquals(includedValue, storedProperties.getProperty(includedKey)); + } - @Test - public void testExecuteIncludingPropertyKeysFromClasspathResource(@TempDir File testFolder) - throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteIncludingPropertyKeysFromClasspathResource(@TempDir File testFolder) + throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); - mojo.setIncludePropertyKeysFromFiles( - new String[] {"/eu/dnetlib/maven/plugin/properties/included.properties"}); + mojo + .setIncludePropertyKeysFromFiles( + new String[] { + "/eu/dnetlib/maven/plugin/properties/included.properties" + }); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(includedKey)); - assertEquals(includedValue, storedProperties.getProperty(includedKey)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(includedKey)); + assertEquals(includedValue, storedProperties.getProperty(includedKey)); + } - @Test - public void testExecuteIncludingPropertyKeysFromBlankLocation() { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteIncludingPropertyKeysFromBlankLocation() { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); - mojo.setIncludePropertyKeysFromFiles(new String[] {""}); + mojo.setIncludePropertyKeysFromFiles(new String[] { + "" + }); - // execute - Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); - } + // execute + Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); + } - @Test - public void testExecuteIncludingPropertyKeysFromXmlFile(@TempDir File testFolder) - throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteIncludingPropertyKeysFromXmlFile(@TempDir File testFolder) + throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); - File includedPropertiesFile = new File(testFolder, "included.xml"); - Properties includedProperties = new Properties(); - includedProperties.setProperty(includedKey, "irrelevantValue"); - includedProperties.storeToXML(new FileOutputStream(includedPropertiesFile), null); + File includedPropertiesFile = new File(testFolder, "included.xml"); + Properties includedProperties = new Properties(); + includedProperties.setProperty(includedKey, "irrelevantValue"); + includedProperties.storeToXML(new FileOutputStream(includedPropertiesFile), null); - mojo.setIncludePropertyKeysFromFiles(new String[] {includedPropertiesFile.getAbsolutePath()}); + mojo.setIncludePropertyKeysFromFiles(new String[] { + includedPropertiesFile.getAbsolutePath() + }); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(1, storedProperties.size()); - assertTrue(storedProperties.containsKey(includedKey)); - assertEquals(includedValue, storedProperties.getProperty(includedKey)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(1, storedProperties.size()); + assertTrue(storedProperties.containsKey(includedKey)); + assertEquals(includedValue, storedProperties.getProperty(includedKey)); + } - @Test - public void testExecuteIncludingPropertyKeysFromInvalidXmlFile(@TempDir File testFolder) - throws Exception { - // given - String key = "projectPropertyKey"; - String value = "projectPropertyValue"; - String includedKey = "includedPropertyKey"; - String includedValue = "includedPropertyValue"; - Properties projectProperties = new Properties(); - projectProperties.setProperty(key, value); - projectProperties.setProperty(includedKey, includedValue); - doReturn(projectProperties).when(mavenProject).getProperties(); + @Test + public void testExecuteIncludingPropertyKeysFromInvalidXmlFile(@TempDir File testFolder) + throws Exception { + // given + String key = "projectPropertyKey"; + String value = "projectPropertyValue"; + String includedKey = "includedPropertyKey"; + String includedValue = "includedPropertyValue"; + Properties projectProperties = new Properties(); + projectProperties.setProperty(key, value); + projectProperties.setProperty(includedKey, includedValue); + doReturn(projectProperties).when(mavenProject).getProperties(); - File includedPropertiesFile = new File(testFolder, "included.xml"); - Properties includedProperties = new Properties(); - includedProperties.setProperty(includedKey, "irrelevantValue"); - includedProperties.store(new FileOutputStream(includedPropertiesFile), null); + File includedPropertiesFile = new File(testFolder, "included.xml"); + Properties includedProperties = new Properties(); + includedProperties.setProperty(includedKey, "irrelevantValue"); + includedProperties.store(new FileOutputStream(includedPropertiesFile), null); - mojo.setIncludePropertyKeysFromFiles(new String[] {includedPropertiesFile.getAbsolutePath()}); + mojo.setIncludePropertyKeysFromFiles(new String[] { + includedPropertiesFile.getAbsolutePath() + }); - // execute - Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); - } + // execute + Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); + } - @Test - public void testExecuteWithQuietModeOn(@TempDir File testFolder) throws Exception { - // given - mojo.setQuiet(true); - mojo.setIncludePropertyKeysFromFiles(new String[] {"invalid location"}); + @Test + public void testExecuteWithQuietModeOn(@TempDir File testFolder) throws Exception { + // given + mojo.setQuiet(true); + mojo.setIncludePropertyKeysFromFiles(new String[] { + "invalid location" + }); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertEquals(0, storedProperties.size()); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertEquals(0, storedProperties.size()); + } - @Test - public void testExecuteIncludingPropertyKeysFromInvalidFile() { - // given - mojo.setIncludePropertyKeysFromFiles(new String[] {"invalid location"}); + @Test + public void testExecuteIncludingPropertyKeysFromInvalidFile() { + // given + mojo.setIncludePropertyKeysFromFiles(new String[] { + "invalid location" + }); - // execute - Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); - } + // execute + Assertions.assertThrows(MojoExecutionException.class, () -> mojo.execute()); + } - @Test - public void testExecuteWithEnvironmentProperties(@TempDir File testFolder) throws Exception { - // given - mojo.setIncludeEnvironmentVariables(true); + @Test + public void testExecuteWithEnvironmentProperties(@TempDir File testFolder) throws Exception { + // given + mojo.setIncludeEnvironmentVariables(true); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertTrue(storedProperties.size() > 0); - for (Object currentKey : storedProperties.keySet()) { - assertTrue(((String) currentKey).startsWith(PROPERTY_PREFIX_ENV)); - } - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertTrue(storedProperties.size() > 0); + for (Object currentKey : storedProperties.keySet()) { + assertTrue(((String) currentKey).startsWith(PROPERTY_PREFIX_ENV)); + } + } - @Test - public void testExecuteWithSystemProperties(@TempDir File testFolder) throws Exception { - // given - String key = "systemPropertyKey"; - String value = "systemPropertyValue"; - System.setProperty(key, value); - mojo.setIncludeSystemProperties(true); + @Test + public void testExecuteWithSystemProperties(@TempDir File testFolder) throws Exception { + // given + String key = "systemPropertyKey"; + String value = "systemPropertyValue"; + System.setProperty(key, value); + mojo.setIncludeSystemProperties(true); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertTrue(storedProperties.size() > 0); - assertTrue(storedProperties.containsKey(key)); - assertEquals(value, storedProperties.getProperty(key)); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertTrue(storedProperties.size() > 0); + assertTrue(storedProperties.containsKey(key)); + assertEquals(value, storedProperties.getProperty(key)); + } - @Test - public void testExecuteWithSystemPropertiesAndEscapeChars(@TempDir File testFolder) - throws Exception { - // given - String key = "systemPropertyKey "; - String value = "systemPropertyValue"; - System.setProperty(key, value); - mojo.setIncludeSystemProperties(true); - String escapeChars = "cr,lf,tab,|"; - mojo.setEscapeChars(escapeChars); + @Test + public void testExecuteWithSystemPropertiesAndEscapeChars(@TempDir File testFolder) + throws Exception { + // given + String key = "systemPropertyKey "; + String value = "systemPropertyValue"; + System.setProperty(key, value); + mojo.setIncludeSystemProperties(true); + String escapeChars = "cr,lf,tab,|"; + mojo.setEscapeChars(escapeChars); - // execute - mojo.execute(); + // execute + mojo.execute(); - // assert - assertTrue(mojo.outputFile.exists()); - Properties storedProperties = getStoredProperties(testFolder); - assertTrue(storedProperties.size() > 0); - assertFalse(storedProperties.containsKey(key)); - assertTrue(storedProperties.containsKey(key.trim())); - assertEquals(value, storedProperties.getProperty(key.trim())); - } + // assert + assertTrue(mojo.outputFile.exists()); + Properties storedProperties = getStoredProperties(testFolder); + assertTrue(storedProperties.size() > 0); + assertFalse(storedProperties.containsKey(key)); + assertTrue(storedProperties.containsKey(key.trim())); + assertEquals(value, storedProperties.getProperty(key.trim())); + } - // ----------------------------------- PRIVATE ------------------------------------------- + // ----------------------------------- PRIVATE ------------------------------------------- - private File getPropertiesFileLocation(File testFolder) { - return new File(testFolder, "test.properties"); - } + private File getPropertiesFileLocation(File testFolder) { + return new File(testFolder, "test.properties"); + } - private Properties getStoredProperties(File testFolder) - throws FileNotFoundException, IOException { - Properties properties = new Properties(); - properties.load(new FileInputStream(getPropertiesFileLocation(testFolder))); - return properties; - } + private Properties getStoredProperties(File testFolder) + throws FileNotFoundException, IOException { + Properties properties = new Properties(); + properties.load(new FileInputStream(getPropertiesFileLocation(testFolder))); + return properties; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/collector/worker/model/ApiDescriptor.java b/dhp-common/src/main/java/eu/dnetlib/collector/worker/model/ApiDescriptor.java index 1a7c2a6ef..bfd70e8c6 100644 --- a/dhp-common/src/main/java/eu/dnetlib/collector/worker/model/ApiDescriptor.java +++ b/dhp-common/src/main/java/eu/dnetlib/collector/worker/model/ApiDescriptor.java @@ -1,3 +1,4 @@ + package eu.dnetlib.collector.worker.model; import java.util.HashMap; @@ -5,43 +6,43 @@ import java.util.Map; public class ApiDescriptor { - private String id; + private String id; - private String baseUrl; + private String baseUrl; - private String protocol; + private String protocol; - private Map params = new HashMap<>(); + private Map params = new HashMap<>(); - public String getBaseUrl() { - return baseUrl; - } + public String getBaseUrl() { + return baseUrl; + } - public void setBaseUrl(final String baseUrl) { - this.baseUrl = baseUrl; - } + public void setBaseUrl(final String baseUrl) { + this.baseUrl = baseUrl; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(final String id) { - this.id = id; - } + public void setId(final String id) { + this.id = id; + } - public Map getParams() { - return params; - } + public Map getParams() { + return params; + } - public void setParams(final HashMap params) { - this.params = params; - } + public void setParams(final HashMap params) { + this.params = params; + } - public String getProtocol() { - return protocol; - } + public String getProtocol() { + return protocol; + } - public void setProtocol(final String protocol) { - this.protocol = protocol; - } + public void setProtocol(final String protocol) { + this.protocol = protocol; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStore.java b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStore.java index f076bd188..68fc024af 100644 --- a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStore.java +++ b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStore.java @@ -1,7 +1,9 @@ + package eu.dnetlib.data.mdstore.manager.common.model; import java.io.Serializable; import java.util.UUID; + import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.Id; @@ -11,107 +13,107 @@ import javax.persistence.Table; @Table(name = "mdstores") public class MDStore implements Serializable { - /** */ - private static final long serialVersionUID = 3160530489149700055L; + /** */ + private static final long serialVersionUID = 3160530489149700055L; - @Id - @Column(name = "id") - private String id; + @Id + @Column(name = "id") + private String id; - @Column(name = "format") - private String format; + @Column(name = "format") + private String format; - @Column(name = "layout") - private String layout; + @Column(name = "layout") + private String layout; - @Column(name = "interpretation") - private String interpretation; + @Column(name = "interpretation") + private String interpretation; - @Column(name = "datasource_name") - private String datasourceName; + @Column(name = "datasource_name") + private String datasourceName; - @Column(name = "datasource_id") - private String datasourceId; + @Column(name = "datasource_id") + private String datasourceId; - @Column(name = "api_id") - private String apiId; + @Column(name = "api_id") + private String apiId; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(final String id) { - this.id = id; - } + public void setId(final String id) { + this.id = id; + } - public String getFormat() { - return format; - } + public String getFormat() { + return format; + } - public void setFormat(final String format) { - this.format = format; - } + public void setFormat(final String format) { + this.format = format; + } - public String getLayout() { - return layout; - } + public String getLayout() { + return layout; + } - public void setLayout(final String layout) { - this.layout = layout; - } + public void setLayout(final String layout) { + this.layout = layout; + } - public String getInterpretation() { - return interpretation; - } + public String getInterpretation() { + return interpretation; + } - public void setInterpretation(final String interpretation) { - this.interpretation = interpretation; - } + public void setInterpretation(final String interpretation) { + this.interpretation = interpretation; + } - public String getDatasourceName() { - return datasourceName; - } + public String getDatasourceName() { + return datasourceName; + } - public void setDatasourceName(final String datasourceName) { - this.datasourceName = datasourceName; - } + public void setDatasourceName(final String datasourceName) { + this.datasourceName = datasourceName; + } - public String getDatasourceId() { - return datasourceId; - } + public String getDatasourceId() { + return datasourceId; + } - public void setDatasourceId(final String datasourceId) { - this.datasourceId = datasourceId; - } + public void setDatasourceId(final String datasourceId) { + this.datasourceId = datasourceId; + } - public String getApiId() { - return apiId; - } + public String getApiId() { + return apiId; + } - public void setApiId(final String apiId) { - this.apiId = apiId; - } + public void setApiId(final String apiId) { + this.apiId = apiId; + } - public static MDStore newInstance( - final String format, final String layout, final String interpretation) { - return newInstance(format, layout, interpretation, null, null, null); - } + public static MDStore newInstance( + final String format, final String layout, final String interpretation) { + return newInstance(format, layout, interpretation, null, null, null); + } - public static MDStore newInstance( - final String format, - final String layout, - final String interpretation, - final String dsName, - final String dsId, - final String apiId) { - final MDStore md = new MDStore(); - md.setId("md-" + UUID.randomUUID()); - md.setFormat(format); - md.setLayout(layout); - md.setInterpretation(interpretation); - md.setDatasourceName(dsName); - md.setDatasourceId(dsId); - md.setApiId(apiId); - return md; - } + public static MDStore newInstance( + final String format, + final String layout, + final String interpretation, + final String dsName, + final String dsId, + final String apiId) { + final MDStore md = new MDStore(); + md.setId("md-" + UUID.randomUUID()); + md.setFormat(format); + md.setLayout(layout); + md.setInterpretation(interpretation); + md.setDatasourceName(dsName); + md.setDatasourceId(dsId); + md.setApiId(apiId); + return md; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreCurrentVersion.java b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreCurrentVersion.java index 0f8f04322..f74ab39be 100644 --- a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreCurrentVersion.java +++ b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreCurrentVersion.java @@ -1,6 +1,8 @@ + package eu.dnetlib.data.mdstore.manager.common.model; import java.io.Serializable; + import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.Id; @@ -10,40 +12,40 @@ import javax.persistence.Table; @Table(name = "mdstore_current_versions") public class MDStoreCurrentVersion implements Serializable { - /** */ - private static final long serialVersionUID = -4757725888593745773L; + /** */ + private static final long serialVersionUID = -4757725888593745773L; - @Id - @Column(name = "mdstore") - private String mdstore; + @Id + @Column(name = "mdstore") + private String mdstore; - @Column(name = "current_version") - private String currentVersion; + @Column(name = "current_version") + private String currentVersion; - public String getMdstore() { - return mdstore; - } + public String getMdstore() { + return mdstore; + } - public void setMdstore(final String mdstore) { - this.mdstore = mdstore; - } + public void setMdstore(final String mdstore) { + this.mdstore = mdstore; + } - public String getCurrentVersion() { - return currentVersion; - } + public String getCurrentVersion() { + return currentVersion; + } - public void setCurrentVersion(final String currentVersion) { - this.currentVersion = currentVersion; - } + public void setCurrentVersion(final String currentVersion) { + this.currentVersion = currentVersion; + } - public static MDStoreCurrentVersion newInstance(final String mdId, final String versionId) { - final MDStoreCurrentVersion cv = new MDStoreCurrentVersion(); - cv.setMdstore(mdId); - cv.setCurrentVersion(versionId); - return cv; - } + public static MDStoreCurrentVersion newInstance(final String mdId, final String versionId) { + final MDStoreCurrentVersion cv = new MDStoreCurrentVersion(); + cv.setMdstore(mdId); + cv.setCurrentVersion(versionId); + return cv; + } - public static MDStoreCurrentVersion newInstance(final MDStoreVersion v) { - return newInstance(v.getMdstore(), v.getId()); - } + public static MDStoreCurrentVersion newInstance(final MDStoreVersion v) { + return newInstance(v.getMdstore(), v.getId()); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreVersion.java b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreVersion.java index ca784b2fb..7ef24f191 100644 --- a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreVersion.java +++ b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreVersion.java @@ -1,7 +1,9 @@ + package eu.dnetlib.data.mdstore.manager.common.model; import java.io.Serializable; import java.util.Date; + import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.Id; @@ -13,85 +15,85 @@ import javax.persistence.TemporalType; @Table(name = "mdstore_versions") public class MDStoreVersion implements Serializable { - /** */ - private static final long serialVersionUID = -4763494442274298339L; + /** */ + private static final long serialVersionUID = -4763494442274298339L; - @Id - @Column(name = "id") - private String id; + @Id + @Column(name = "id") + private String id; - @Column(name = "mdstore") - private String mdstore; + @Column(name = "mdstore") + private String mdstore; - @Column(name = "writing") - private boolean writing; + @Column(name = "writing") + private boolean writing; - @Column(name = "readcount") - private int readCount = 0; + @Column(name = "readcount") + private int readCount = 0; - @Column(name = "lastupdate") - @Temporal(TemporalType.TIMESTAMP) - private Date lastUpdate; + @Column(name = "lastupdate") + @Temporal(TemporalType.TIMESTAMP) + private Date lastUpdate; - @Column(name = "size") - private long size = 0; + @Column(name = "size") + private long size = 0; - public static MDStoreVersion newInstance(final String mdId, final boolean writing) { - final MDStoreVersion t = new MDStoreVersion(); - t.setId(mdId + "-" + new Date().getTime()); - t.setMdstore(mdId); - t.setLastUpdate(null); - t.setWriting(writing); - t.setReadCount(0); - t.setSize(0); - return t; - } + public static MDStoreVersion newInstance(final String mdId, final boolean writing) { + final MDStoreVersion t = new MDStoreVersion(); + t.setId(mdId + "-" + new Date().getTime()); + t.setMdstore(mdId); + t.setLastUpdate(null); + t.setWriting(writing); + t.setReadCount(0); + t.setSize(0); + return t; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(final String id) { - this.id = id; - } + public void setId(final String id) { + this.id = id; + } - public String getMdstore() { - return mdstore; - } + public String getMdstore() { + return mdstore; + } - public void setMdstore(final String mdstore) { - this.mdstore = mdstore; - } + public void setMdstore(final String mdstore) { + this.mdstore = mdstore; + } - public boolean isWriting() { - return writing; - } + public boolean isWriting() { + return writing; + } - public void setWriting(final boolean writing) { - this.writing = writing; - } + public void setWriting(final boolean writing) { + this.writing = writing; + } - public int getReadCount() { - return readCount; - } + public int getReadCount() { + return readCount; + } - public void setReadCount(final int readCount) { - this.readCount = readCount; - } + public void setReadCount(final int readCount) { + this.readCount = readCount; + } - public Date getLastUpdate() { - return lastUpdate; - } + public Date getLastUpdate() { + return lastUpdate; + } - public void setLastUpdate(final Date lastUpdate) { - this.lastUpdate = lastUpdate; - } + public void setLastUpdate(final Date lastUpdate) { + this.lastUpdate = lastUpdate; + } - public long getSize() { - return size; - } + public long getSize() { + return size; + } - public void setSize(final long size) { - this.size = size; - } + public void setSize(final long size) { + this.size = size; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreWithInfo.java b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreWithInfo.java index 9225a4876..438359241 100644 --- a/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreWithInfo.java +++ b/dhp-common/src/main/java/eu/dnetlib/data/mdstore/manager/common/model/MDStoreWithInfo.java @@ -1,7 +1,9 @@ + package eu.dnetlib.data.mdstore.manager.common.model; import java.io.Serializable; import java.util.Date; + import javax.persistence.Column; import javax.persistence.Entity; import javax.persistence.Id; @@ -13,129 +15,129 @@ import javax.persistence.TemporalType; @Table(name = "mdstores_with_info") public class MDStoreWithInfo implements Serializable { - /** */ - private static final long serialVersionUID = -8445784770687571492L; + /** */ + private static final long serialVersionUID = -8445784770687571492L; - @Id - @Column(name = "id") - private String id; + @Id + @Column(name = "id") + private String id; - @Column(name = "format") - private String format; + @Column(name = "format") + private String format; - @Column(name = "layout") - private String layout; + @Column(name = "layout") + private String layout; - @Column(name = "interpretation") - private String interpretation; + @Column(name = "interpretation") + private String interpretation; - @Column(name = "datasource_name") - private String datasourceName; + @Column(name = "datasource_name") + private String datasourceName; - @Column(name = "datasource_id") - private String datasourceId; + @Column(name = "datasource_id") + private String datasourceId; - @Column(name = "api_id") - private String apiId; + @Column(name = "api_id") + private String apiId; - @Column(name = "current_version") - private String currentVersion; + @Column(name = "current_version") + private String currentVersion; - @Column(name = "lastupdate") - @Temporal(TemporalType.TIMESTAMP) - private Date lastUpdate; + @Column(name = "lastupdate") + @Temporal(TemporalType.TIMESTAMP) + private Date lastUpdate; - @Column(name = "size") - private long size = 0; + @Column(name = "size") + private long size = 0; - @Column(name = "n_versions") - private long numberOfVersions = 0; + @Column(name = "n_versions") + private long numberOfVersions = 0; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(final String id) { - this.id = id; - } + public void setId(final String id) { + this.id = id; + } - public String getFormat() { - return format; - } + public String getFormat() { + return format; + } - public void setFormat(final String format) { - this.format = format; - } + public void setFormat(final String format) { + this.format = format; + } - public String getLayout() { - return layout; - } + public String getLayout() { + return layout; + } - public void setLayout(final String layout) { - this.layout = layout; - } + public void setLayout(final String layout) { + this.layout = layout; + } - public String getInterpretation() { - return interpretation; - } + public String getInterpretation() { + return interpretation; + } - public void setInterpretation(final String interpretation) { - this.interpretation = interpretation; - } + public void setInterpretation(final String interpretation) { + this.interpretation = interpretation; + } - public String getDatasourceName() { - return datasourceName; - } + public String getDatasourceName() { + return datasourceName; + } - public void setDatasourceName(final String datasourceName) { - this.datasourceName = datasourceName; - } + public void setDatasourceName(final String datasourceName) { + this.datasourceName = datasourceName; + } - public String getDatasourceId() { - return datasourceId; - } + public String getDatasourceId() { + return datasourceId; + } - public void setDatasourceId(final String datasourceId) { - this.datasourceId = datasourceId; - } + public void setDatasourceId(final String datasourceId) { + this.datasourceId = datasourceId; + } - public String getApiId() { - return apiId; - } + public String getApiId() { + return apiId; + } - public void setApiId(final String apiId) { - this.apiId = apiId; - } + public void setApiId(final String apiId) { + this.apiId = apiId; + } - public String getCurrentVersion() { - return currentVersion; - } + public String getCurrentVersion() { + return currentVersion; + } - public void setCurrentVersion(final String currentVersion) { - this.currentVersion = currentVersion; - } + public void setCurrentVersion(final String currentVersion) { + this.currentVersion = currentVersion; + } - public Date getLastUpdate() { - return lastUpdate; - } + public Date getLastUpdate() { + return lastUpdate; + } - public void setLastUpdate(final Date lastUpdate) { - this.lastUpdate = lastUpdate; - } + public void setLastUpdate(final Date lastUpdate) { + this.lastUpdate = lastUpdate; + } - public long getSize() { - return size; - } + public long getSize() { + return size; + } - public void setSize(final long size) { - this.size = size; - } + public void setSize(final long size) { + this.size = size; + } - public long getNumberOfVersions() { - return numberOfVersions; - } + public long getNumberOfVersions() { + return numberOfVersions; + } - public void setNumberOfVersions(final long numberOfVersions) { - this.numberOfVersions = numberOfVersions; - } + public void setNumberOfVersions(final long numberOfVersions) { + this.numberOfVersions = numberOfVersions; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java b/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java index d98874bf3..e65b4bb0b 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/ArgumentApplicationParser.java @@ -1,6 +1,6 @@ + package eu.dnetlib.dhp.application; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Serializable; @@ -8,87 +8,91 @@ import java.io.StringWriter; import java.util.*; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; + import org.apache.commons.cli.*; import org.apache.commons.codec.binary.Base64; import org.apache.commons.io.IOUtils; +import com.fasterxml.jackson.databind.ObjectMapper; + public class ArgumentApplicationParser implements Serializable { - private final Options options = new Options(); - private final Map objectMap = new HashMap<>(); + private final Options options = new Options(); + private final Map objectMap = new HashMap<>(); - private final List compressedValues = new ArrayList<>(); + private final List compressedValues = new ArrayList<>(); - public ArgumentApplicationParser(final String json_configuration) throws Exception { - final ObjectMapper mapper = new ObjectMapper(); - final OptionsParameter[] configuration = - mapper.readValue(json_configuration, OptionsParameter[].class); - createOptionMap(configuration); - } + public ArgumentApplicationParser(final String json_configuration) throws Exception { + final ObjectMapper mapper = new ObjectMapper(); + final OptionsParameter[] configuration = mapper.readValue(json_configuration, OptionsParameter[].class); + createOptionMap(configuration); + } - public ArgumentApplicationParser(final OptionsParameter[] configuration) { - createOptionMap(configuration); - } + public ArgumentApplicationParser(final OptionsParameter[] configuration) { + createOptionMap(configuration); + } - private void createOptionMap(final OptionsParameter[] configuration) { + private void createOptionMap(final OptionsParameter[] configuration) { - Arrays.stream(configuration) - .map( - conf -> { - final Option o = new Option(conf.getParamName(), true, conf.getParamDescription()); - o.setLongOpt(conf.getParamLongName()); - o.setRequired(conf.isParamRequired()); - if (conf.isCompressed()) { - compressedValues.add(conf.getParamLongName()); - } - return o; - }) - .forEach(options::addOption); + Arrays + .stream(configuration) + .map( + conf -> { + final Option o = new Option(conf.getParamName(), true, conf.getParamDescription()); + o.setLongOpt(conf.getParamLongName()); + o.setRequired(conf.isParamRequired()); + if (conf.isCompressed()) { + compressedValues.add(conf.getParamLongName()); + } + return o; + }) + .forEach(options::addOption); - // HelpFormatter formatter = new HelpFormatter(); - // formatter.printHelp("myapp", null, options, null, true); + // HelpFormatter formatter = new HelpFormatter(); + // formatter.printHelp("myapp", null, options, null, true); - } + } - public static String decompressValue(final String abstractCompressed) { - try { - byte[] byteArray = Base64.decodeBase64(abstractCompressed.getBytes()); - GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(byteArray)); - final StringWriter stringWriter = new StringWriter(); - IOUtils.copy(gis, stringWriter); - return stringWriter.toString(); - } catch (Throwable e) { - System.out.println("Wrong value to decompress:" + abstractCompressed); - throw new RuntimeException(e); - } - } + public static String decompressValue(final String abstractCompressed) { + try { + byte[] byteArray = Base64.decodeBase64(abstractCompressed.getBytes()); + GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream(byteArray)); + final StringWriter stringWriter = new StringWriter(); + IOUtils.copy(gis, stringWriter); + return stringWriter.toString(); + } catch (Throwable e) { + System.out.println("Wrong value to decompress:" + abstractCompressed); + throw new RuntimeException(e); + } + } - public static String compressArgument(final String value) throws Exception { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - GZIPOutputStream gzip = new GZIPOutputStream(out); - gzip.write(value.getBytes()); - gzip.close(); - return java.util.Base64.getEncoder().encodeToString(out.toByteArray()); - } + public static String compressArgument(final String value) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GZIPOutputStream gzip = new GZIPOutputStream(out); + gzip.write(value.getBytes()); + gzip.close(); + return java.util.Base64.getEncoder().encodeToString(out.toByteArray()); + } - public void parseArgument(final String[] args) throws Exception { - CommandLineParser parser = new BasicParser(); - CommandLine cmd = parser.parse(options, args); - Arrays.stream(cmd.getOptions()) - .forEach( - it -> - objectMap.put( - it.getLongOpt(), - compressedValues.contains(it.getLongOpt()) - ? decompressValue(it.getValue()) - : it.getValue())); - } + public void parseArgument(final String[] args) throws Exception { + CommandLineParser parser = new BasicParser(); + CommandLine cmd = parser.parse(options, args); + Arrays + .stream(cmd.getOptions()) + .forEach( + it -> objectMap + .put( + it.getLongOpt(), + compressedValues.contains(it.getLongOpt()) + ? decompressValue(it.getValue()) + : it.getValue())); + } - public String get(final String key) { - return objectMap.get(key); - } + public String get(final String key) { + return objectMap.get(key); + } - public Map getObjectMap() { - return objectMap; - } + public Map getObjectMap() { + return objectMap; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java b/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java index 03227d316..7004112e4 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/application/OptionsParameter.java @@ -1,36 +1,38 @@ + package eu.dnetlib.dhp.application; public class OptionsParameter { - private String paramName; - private String paramLongName; - private String paramDescription; - private boolean paramRequired; - private boolean compressed; + private String paramName; + private String paramLongName; + private String paramDescription; + private boolean paramRequired; + private boolean compressed; - public OptionsParameter() {} + public OptionsParameter() { + } - public String getParamName() { - return paramName; - } + public String getParamName() { + return paramName; + } - public String getParamLongName() { - return paramLongName; - } + public String getParamLongName() { + return paramLongName; + } - public String getParamDescription() { - return paramDescription; - } + public String getParamDescription() { + return paramDescription; + } - public boolean isParamRequired() { - return paramRequired; - } + public boolean isParamRequired() { + return paramRequired; + } - public boolean isCompressed() { - return compressed; - } + public boolean isCompressed() { + return compressed; + } - public void setCompressed(boolean compressed) { - this.compressed = compressed; - } + public void setCompressed(boolean compressed) { + this.compressed = compressed; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/FunctionalInterfaceSupport.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/FunctionalInterfaceSupport.java index 4b0e1506e..e793e3f29 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/FunctionalInterfaceSupport.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/FunctionalInterfaceSupport.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.common; import java.io.Serializable; @@ -6,46 +7,48 @@ import java.util.function.Supplier; /** Provides serializable and throwing extensions to standard functional interfaces. */ public class FunctionalInterfaceSupport { - private FunctionalInterfaceSupport() {} + private FunctionalInterfaceSupport() { + } - /** - * Serializable supplier of any kind of objects. To be used withing spark processing pipelines - * when supplying functions externally. - * - * @param - */ - @FunctionalInterface - public interface SerializableSupplier extends Supplier, Serializable {} + /** + * Serializable supplier of any kind of objects. To be used withing spark processing pipelines when supplying + * functions externally. + * + * @param + */ + @FunctionalInterface + public interface SerializableSupplier extends Supplier, Serializable { + } - /** - * Extension of consumer accepting functions throwing an exception. - * - * @param - * @param - */ - @FunctionalInterface - public interface ThrowingConsumer { - void accept(T t) throws E; - } + /** + * Extension of consumer accepting functions throwing an exception. + * + * @param + * @param + */ + @FunctionalInterface + public interface ThrowingConsumer { + void accept(T t) throws E; + } - /** - * Extension of supplier accepting functions throwing an exception. - * - * @param - * @param - */ - @FunctionalInterface - public interface ThrowingSupplier { - T get() throws E; - } + /** + * Extension of supplier accepting functions throwing an exception. + * + * @param + * @param + */ + @FunctionalInterface + public interface ThrowingSupplier { + T get() throws E; + } - /** - * Extension of runnable accepting functions throwing an exception. - * - * @param - */ - @FunctionalInterface - public interface ThrowingRunnable { - void run() throws E; - } + /** + * Extension of runnable accepting functions throwing an exception. + * + * @param + */ + @FunctionalInterface + public interface ThrowingRunnable { + void run() throws E; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/HdfsSupport.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/HdfsSupport.java index 1e5c264d1..0b2cd571f 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/HdfsSupport.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/HdfsSupport.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.common; import static eu.dnetlib.dhp.common.ThrowingSupport.rethrowAsRuntimeException; @@ -5,6 +6,7 @@ import static eu.dnetlib.dhp.common.ThrowingSupport.rethrowAsRuntimeException; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -14,58 +16,59 @@ import org.slf4j.LoggerFactory; /** HDFS utility methods. */ public class HdfsSupport { - private static final Logger logger = LoggerFactory.getLogger(HdfsSupport.class); + private static final Logger logger = LoggerFactory.getLogger(HdfsSupport.class); - private HdfsSupport() {} + private HdfsSupport() { + } - /** - * Checks a path (file or dir) exists on HDFS. - * - * @param path Path to be checked - * @param configuration Configuration of hadoop env - */ - public static boolean exists(String path, Configuration configuration) { - logger.info("Removing path: {}", path); - return rethrowAsRuntimeException( - () -> { - Path f = new Path(path); - FileSystem fileSystem = FileSystem.get(configuration); - return fileSystem.exists(f); - }); - } + /** + * Checks a path (file or dir) exists on HDFS. + * + * @param path Path to be checked + * @param configuration Configuration of hadoop env + */ + public static boolean exists(String path, Configuration configuration) { + logger.info("Removing path: {}", path); + return rethrowAsRuntimeException( + () -> { + Path f = new Path(path); + FileSystem fileSystem = FileSystem.get(configuration); + return fileSystem.exists(f); + }); + } - /** - * Removes a path (file or dir) from HDFS. - * - * @param path Path to be removed - * @param configuration Configuration of hadoop env - */ - public static void remove(String path, Configuration configuration) { - logger.info("Removing path: {}", path); - rethrowAsRuntimeException( - () -> { - Path f = new Path(path); - FileSystem fileSystem = FileSystem.get(configuration); - if (fileSystem.exists(f)) { - fileSystem.delete(f, true); - } - }); - } + /** + * Removes a path (file or dir) from HDFS. + * + * @param path Path to be removed + * @param configuration Configuration of hadoop env + */ + public static void remove(String path, Configuration configuration) { + logger.info("Removing path: {}", path); + rethrowAsRuntimeException( + () -> { + Path f = new Path(path); + FileSystem fileSystem = FileSystem.get(configuration); + if (fileSystem.exists(f)) { + fileSystem.delete(f, true); + } + }); + } - /** - * Lists hadoop files located below path or alternatively lists subdirs under path. - * - * @param path Path to be listed for hadoop files - * @param configuration Configuration of hadoop env - * @return List with string locations of hadoop files - */ - public static List listFiles(String path, Configuration configuration) { - logger.info("Listing files in path: {}", path); - return rethrowAsRuntimeException( - () -> - Arrays.stream(FileSystem.get(configuration).listStatus(new Path(path))) - .filter(FileStatus::isDirectory) - .map(x -> x.getPath().toString()) - .collect(Collectors.toList())); - } + /** + * Lists hadoop files located below path or alternatively lists subdirs under path. + * + * @param path Path to be listed for hadoop files + * @param configuration Configuration of hadoop env + * @return List with string locations of hadoop files + */ + public static List listFiles(String path, Configuration configuration) { + logger.info("Listing files in path: {}", path); + return rethrowAsRuntimeException( + () -> Arrays + .stream(FileSystem.get(configuration).listStatus(new Path(path))) + .filter(FileStatus::isDirectory) + .map(x -> x.getPath().toString()) + .collect(Collectors.toList())); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/SparkSessionSupport.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/SparkSessionSupport.java index 433f64ecd..03cc94961 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/SparkSessionSupport.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/SparkSessionSupport.java @@ -1,74 +1,75 @@ + package eu.dnetlib.dhp.common; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingConsumer; import java.util.Objects; import java.util.function.Function; + import org.apache.spark.SparkConf; import org.apache.spark.sql.SparkSession; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingConsumer; + /** SparkSession utility methods. */ public class SparkSessionSupport { - private SparkSessionSupport() {} + private SparkSessionSupport() { + } - /** - * Runs a given function using SparkSession created using default builder and supplied SparkConf. - * Stops SparkSession when SparkSession is managed. Allows to reuse SparkSession created - * externally. - * - * @param conf SparkConf instance - * @param isSparkSessionManaged When true will stop SparkSession - * @param fn Consumer to be applied to constructed SparkSession - */ - public static void runWithSparkSession( - SparkConf conf, Boolean isSparkSessionManaged, ThrowingConsumer fn) { - runWithSparkSession( - c -> SparkSession.builder().config(c).getOrCreate(), conf, isSparkSessionManaged, fn); - } + /** + * Runs a given function using SparkSession created using default builder and supplied SparkConf. Stops SparkSession + * when SparkSession is managed. Allows to reuse SparkSession created externally. + * + * @param conf SparkConf instance + * @param isSparkSessionManaged When true will stop SparkSession + * @param fn Consumer to be applied to constructed SparkSession + */ + public static void runWithSparkSession( + SparkConf conf, Boolean isSparkSessionManaged, ThrowingConsumer fn) { + runWithSparkSession( + c -> SparkSession.builder().config(c).getOrCreate(), conf, isSparkSessionManaged, fn); + } - /** - * Runs a given function using SparkSession created with hive support and using default builder - * and supplied SparkConf. Stops SparkSession when SparkSession is managed. Allows to reuse - * SparkSession created externally. - * - * @param conf SparkConf instance - * @param isSparkSessionManaged When true will stop SparkSession - * @param fn Consumer to be applied to constructed SparkSession - */ - public static void runWithSparkHiveSession( - SparkConf conf, Boolean isSparkSessionManaged, ThrowingConsumer fn) { - runWithSparkSession( - c -> SparkSession.builder().config(c).enableHiveSupport().getOrCreate(), - conf, - isSparkSessionManaged, - fn); - } + /** + * Runs a given function using SparkSession created with hive support and using default builder and supplied + * SparkConf. Stops SparkSession when SparkSession is managed. Allows to reuse SparkSession created externally. + * + * @param conf SparkConf instance + * @param isSparkSessionManaged When true will stop SparkSession + * @param fn Consumer to be applied to constructed SparkSession + */ + public static void runWithSparkHiveSession( + SparkConf conf, Boolean isSparkSessionManaged, ThrowingConsumer fn) { + runWithSparkSession( + c -> SparkSession.builder().config(c).enableHiveSupport().getOrCreate(), + conf, + isSparkSessionManaged, + fn); + } - /** - * Runs a given function using SparkSession created using supplied builder and supplied SparkConf. - * Stops SparkSession when SparkSession is managed. Allows to reuse SparkSession created - * externally. - * - * @param sparkSessionBuilder Builder of SparkSession - * @param conf SparkConf instance - * @param isSparkSessionManaged When true will stop SparkSession - * @param fn Consumer to be applied to constructed SparkSession - */ - public static void runWithSparkSession( - Function sparkSessionBuilder, - SparkConf conf, - Boolean isSparkSessionManaged, - ThrowingConsumer fn) { - SparkSession spark = null; - try { - spark = sparkSessionBuilder.apply(conf); - fn.accept(spark); - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - if (Objects.nonNull(spark) && isSparkSessionManaged) { - spark.stop(); - } - } - } + /** + * Runs a given function using SparkSession created using supplied builder and supplied SparkConf. Stops + * SparkSession when SparkSession is managed. Allows to reuse SparkSession created externally. + * + * @param sparkSessionBuilder Builder of SparkSession + * @param conf SparkConf instance + * @param isSparkSessionManaged When true will stop SparkSession + * @param fn Consumer to be applied to constructed SparkSession + */ + public static void runWithSparkSession( + Function sparkSessionBuilder, + SparkConf conf, + Boolean isSparkSessionManaged, + ThrowingConsumer fn) { + SparkSession spark = null; + try { + spark = sparkSessionBuilder.apply(conf); + fn.accept(spark); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + if (Objects.nonNull(spark) && isSparkSessionManaged) { + spark.stop(); + } + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/ThrowingSupport.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/ThrowingSupport.java index 54342a46a..f3f59b2a2 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/ThrowingSupport.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/ThrowingSupport.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.common; import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingRunnable; @@ -6,69 +7,70 @@ import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingSupplier; /** Exception handling utility methods. */ public class ThrowingSupport { - private ThrowingSupport() {} + private ThrowingSupport() { + } - /** - * Executes given runnable and rethrows any exceptions as RuntimeException. - * - * @param fn Runnable to be executed - * @param Type of exception thrown - */ - public static void rethrowAsRuntimeException(ThrowingRunnable fn) { - try { - fn.run(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + /** + * Executes given runnable and rethrows any exceptions as RuntimeException. + * + * @param fn Runnable to be executed + * @param Type of exception thrown + */ + public static void rethrowAsRuntimeException(ThrowingRunnable fn) { + try { + fn.run(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } - /** - * Executes given runnable and rethrows any exceptions as RuntimeException with custom message. - * - * @param fn Runnable to be executed - * @param msg Message to be set for rethrown exception - * @param Type of exception thrown - */ - public static void rethrowAsRuntimeException( - ThrowingRunnable fn, String msg) { - try { - fn.run(); - } catch (Exception e) { - throw new RuntimeException(msg, e); - } - } + /** + * Executes given runnable and rethrows any exceptions as RuntimeException with custom message. + * + * @param fn Runnable to be executed + * @param msg Message to be set for rethrown exception + * @param Type of exception thrown + */ + public static void rethrowAsRuntimeException( + ThrowingRunnable fn, String msg) { + try { + fn.run(); + } catch (Exception e) { + throw new RuntimeException(msg, e); + } + } - /** - * Executes given supplier and rethrows any exceptions as RuntimeException. - * - * @param fn Supplier to be executed - * @param Type of returned value - * @param Type of exception thrown - * @return Result of supplier execution - */ - public static T rethrowAsRuntimeException(ThrowingSupplier fn) { - try { - return fn.get(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + /** + * Executes given supplier and rethrows any exceptions as RuntimeException. + * + * @param fn Supplier to be executed + * @param Type of returned value + * @param Type of exception thrown + * @return Result of supplier execution + */ + public static T rethrowAsRuntimeException(ThrowingSupplier fn) { + try { + return fn.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } - /** - * Executes given supplier and rethrows any exceptions as RuntimeException with custom message. - * - * @param fn Supplier to be executed - * @param msg Message to be set for rethrown exception - * @param Type of returned value - * @param Type of exception thrown - * @return Result of supplier execution - */ - public static T rethrowAsRuntimeException( - ThrowingSupplier fn, String msg) { - try { - return fn.get(); - } catch (Exception e) { - throw new RuntimeException(msg, e); - } - } + /** + * Executes given supplier and rethrows any exceptions as RuntimeException with custom message. + * + * @param fn Supplier to be executed + * @param msg Message to be set for rethrown exception + * @param Type of returned value + * @param Type of exception thrown + * @return Result of supplier execution + */ + public static T rethrowAsRuntimeException( + ThrowingSupplier fn, String msg) { + try { + return fn.get(); + } catch (Exception e) { + throw new RuntimeException(msg, e); + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/MetadataRecord.java b/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/MetadataRecord.java index 56d7217ff..ce65e710f 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/MetadataRecord.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/MetadataRecord.java @@ -1,120 +1,121 @@ + package eu.dnetlib.dhp.model.mdstore; -import eu.dnetlib.dhp.utils.DHPUtils; import java.io.Serializable; +import eu.dnetlib.dhp.utils.DHPUtils; + /** This class models a record inside the new Metadata store collection on HDFS * */ public class MetadataRecord implements Serializable { - /** The D-Net Identifier associated to the record */ - private String id; + /** The D-Net Identifier associated to the record */ + private String id; - /** The original Identifier of the record */ - private String originalId; + /** The original Identifier of the record */ + private String originalId; - /** The encoding of the record, should be JSON or XML */ - private String encoding; + /** The encoding of the record, should be JSON or XML */ + private String encoding; - /** - * The information about the provenance of the record see @{@link Provenance} for the model of - * this information - */ - private Provenance provenance; + /** + * The information about the provenance of the record see @{@link Provenance} for the model of this information + */ + private Provenance provenance; - /** The content of the metadata */ - private String body; + /** The content of the metadata */ + private String body; - /** the date when the record has been stored */ - private long dateOfCollection; + /** the date when the record has been stored */ + private long dateOfCollection; - /** the date when the record has been stored */ - private long dateOfTransformation; + /** the date when the record has been stored */ + private long dateOfTransformation; - public MetadataRecord() { - this.dateOfCollection = System.currentTimeMillis(); - } + public MetadataRecord() { + this.dateOfCollection = System.currentTimeMillis(); + } - public MetadataRecord( - String originalId, - String encoding, - Provenance provenance, - String body, - long dateOfCollection) { + public MetadataRecord( + String originalId, + String encoding, + Provenance provenance, + String body, + long dateOfCollection) { - this.originalId = originalId; - this.encoding = encoding; - this.provenance = provenance; - this.body = body; - this.dateOfCollection = dateOfCollection; - this.id = DHPUtils.generateIdentifier(originalId, this.provenance.getNsPrefix()); - } + this.originalId = originalId; + this.encoding = encoding; + this.provenance = provenance; + this.body = body; + this.dateOfCollection = dateOfCollection; + this.id = DHPUtils.generateIdentifier(originalId, this.provenance.getNsPrefix()); + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public String getOriginalId() { - return originalId; - } + public String getOriginalId() { + return originalId; + } - public void setOriginalId(String originalId) { - this.originalId = originalId; - } + public void setOriginalId(String originalId) { + this.originalId = originalId; + } - public String getEncoding() { - return encoding; - } + public String getEncoding() { + return encoding; + } - public void setEncoding(String encoding) { - this.encoding = encoding; - } + public void setEncoding(String encoding) { + this.encoding = encoding; + } - public Provenance getProvenance() { - return provenance; - } + public Provenance getProvenance() { + return provenance; + } - public void setProvenance(Provenance provenance) { - this.provenance = provenance; - } + public void setProvenance(Provenance provenance) { + this.provenance = provenance; + } - public String getBody() { - return body; - } + public String getBody() { + return body; + } - public void setBody(String body) { - this.body = body; - } + public void setBody(String body) { + this.body = body; + } - public long getDateOfCollection() { - return dateOfCollection; - } + public long getDateOfCollection() { + return dateOfCollection; + } - public void setDateOfCollection(long dateOfCollection) { - this.dateOfCollection = dateOfCollection; - } + public void setDateOfCollection(long dateOfCollection) { + this.dateOfCollection = dateOfCollection; + } - public long getDateOfTransformation() { - return dateOfTransformation; - } + public long getDateOfTransformation() { + return dateOfTransformation; + } - public void setDateOfTransformation(long dateOfTransformation) { - this.dateOfTransformation = dateOfTransformation; - } + public void setDateOfTransformation(long dateOfTransformation) { + this.dateOfTransformation = dateOfTransformation; + } - @Override - public boolean equals(Object o) { - if (!(o instanceof MetadataRecord)) { - return false; - } - return ((MetadataRecord) o).getId().equalsIgnoreCase(id); - } + @Override + public boolean equals(Object o) { + if (!(o instanceof MetadataRecord)) { + return false; + } + return ((MetadataRecord) o).getId().equalsIgnoreCase(id); + } - @Override - public int hashCode() { - return id.hashCode(); - } + @Override + public int hashCode() { + return id.hashCode(); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/Provenance.java b/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/Provenance.java index 90897c5c4..556535022 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/Provenance.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/model/mdstore/Provenance.java @@ -1,49 +1,52 @@ + package eu.dnetlib.dhp.model.mdstore; import java.io.Serializable; /** * @author Sandro La Bruzzo - *

Provenace class models the provenance of the record in the metadataStore It contains the - * identifier and the name of the datasource that gives the record + *

+ * Provenace class models the provenance of the record in the metadataStore It contains the identifier and the + * name of the datasource that gives the record */ public class Provenance implements Serializable { - private String datasourceId; + private String datasourceId; - private String datasourceName; + private String datasourceName; - private String nsPrefix; + private String nsPrefix; - public Provenance() {} + public Provenance() { + } - public Provenance(String datasourceId, String datasourceName, String nsPrefix) { - this.datasourceId = datasourceId; - this.datasourceName = datasourceName; - this.nsPrefix = nsPrefix; - } + public Provenance(String datasourceId, String datasourceName, String nsPrefix) { + this.datasourceId = datasourceId; + this.datasourceName = datasourceName; + this.nsPrefix = nsPrefix; + } - public String getDatasourceId() { - return datasourceId; - } + public String getDatasourceId() { + return datasourceId; + } - public void setDatasourceId(String datasourceId) { - this.datasourceId = datasourceId; - } + public void setDatasourceId(String datasourceId) { + this.datasourceId = datasourceId; + } - public String getDatasourceName() { - return datasourceName; - } + public String getDatasourceName() { + return datasourceName; + } - public void setDatasourceName(String datasourceName) { - this.datasourceName = datasourceName; - } + public void setDatasourceName(String datasourceName) { + this.datasourceName = datasourceName; + } - public String getNsPrefix() { - return nsPrefix; - } + public String getNsPrefix() { + return nsPrefix; + } - public void setNsPrefix(String nsPrefix) { - this.nsPrefix = nsPrefix; - } + public void setNsPrefix(String nsPrefix) { + this.nsPrefix = nsPrefix; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdException.java b/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdException.java index 3576dc92b..22945309c 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdException.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdException.java @@ -1,12 +1,13 @@ + package eu.dnetlib.dhp.parser.utility; public class VtdException extends Exception { - public VtdException(final Exception e) { - super(e); - } + public VtdException(final Exception e) { + super(e); + } - public VtdException(final Throwable e) { - super(e); - } + public VtdException(final Throwable e) { + super(e); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdUtilityParser.java b/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdUtilityParser.java index a12662d1f..9ac0a0bf7 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdUtilityParser.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/parser/utility/VtdUtilityParser.java @@ -1,105 +1,110 @@ + package eu.dnetlib.dhp.parser.utility; -import com.ximpleware.AutoPilot; -import com.ximpleware.VTDNav; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import com.ximpleware.AutoPilot; +import com.ximpleware.VTDNav; + /** Created by sandro on 9/29/16. */ public class VtdUtilityParser { - public static List getTextValuesWithAttributes( - final AutoPilot ap, final VTDNav vn, final String xpath, final List attributes) - throws VtdException { - final List results = new ArrayList<>(); - try { - ap.selectXPath(xpath); + public static List getTextValuesWithAttributes( + final AutoPilot ap, final VTDNav vn, final String xpath, final List attributes) + throws VtdException { + final List results = new ArrayList<>(); + try { + ap.selectXPath(xpath); - while (ap.evalXPath() != -1) { - final Node currentNode = new Node(); - int t = vn.getText(); - if (t >= 0) { - currentNode.setTextValue(vn.toNormalizedString(t)); - } - currentNode.setAttributes(getAttributes(vn, attributes)); - results.add(currentNode); - } - return results; - } catch (Exception e) { - throw new VtdException(e); - } - } + while (ap.evalXPath() != -1) { + final Node currentNode = new Node(); + int t = vn.getText(); + if (t >= 0) { + currentNode.setTextValue(vn.toNormalizedString(t)); + } + currentNode.setAttributes(getAttributes(vn, attributes)); + results.add(currentNode); + } + return results; + } catch (Exception e) { + throw new VtdException(e); + } + } - private static Map getAttributes(final VTDNav vn, final List attributes) { - final Map currentAttributes = new HashMap<>(); - if (attributes != null) { + private static Map getAttributes(final VTDNav vn, final List attributes) { + final Map currentAttributes = new HashMap<>(); + if (attributes != null) { - attributes.forEach( - attributeKey -> { - try { - int attr = vn.getAttrVal(attributeKey); - if (attr > -1) { - currentAttributes.put(attributeKey, vn.toNormalizedString(attr)); - } - } catch (Throwable e) { - throw new RuntimeException(e); - } - }); - } - return currentAttributes; - } + attributes + .forEach( + attributeKey -> { + try { + int attr = vn.getAttrVal(attributeKey); + if (attr > -1) { + currentAttributes.put(attributeKey, vn.toNormalizedString(attr)); + } + } catch (Throwable e) { + throw new RuntimeException(e); + } + }); + } + return currentAttributes; + } - public static List getTextValue(final AutoPilot ap, final VTDNav vn, final String xpath) - throws VtdException { - List results = new ArrayList<>(); - try { - ap.selectXPath(xpath); - while (ap.evalXPath() != -1) { - int t = vn.getText(); - if (t > -1) results.add(vn.toNormalizedString(t)); - } - return results; - } catch (Exception e) { - throw new VtdException(e); - } - } + public static List getTextValue(final AutoPilot ap, final VTDNav vn, final String xpath) + throws VtdException { + List results = new ArrayList<>(); + try { + ap.selectXPath(xpath); + while (ap.evalXPath() != -1) { + int t = vn.getText(); + if (t > -1) + results.add(vn.toNormalizedString(t)); + } + return results; + } catch (Exception e) { + throw new VtdException(e); + } + } - public static String getSingleValue(final AutoPilot ap, final VTDNav nav, final String xpath) - throws VtdException { - try { - ap.selectXPath(xpath); - while (ap.evalXPath() != -1) { - int it = nav.getText(); - if (it > -1) return nav.toNormalizedString(it); - } - return null; - } catch (Exception e) { - throw new VtdException(e); - } - } + public static String getSingleValue(final AutoPilot ap, final VTDNav nav, final String xpath) + throws VtdException { + try { + ap.selectXPath(xpath); + while (ap.evalXPath() != -1) { + int it = nav.getText(); + if (it > -1) + return nav.toNormalizedString(it); + } + return null; + } catch (Exception e) { + throw new VtdException(e); + } + } - public static class Node { + public static class Node { - private String textValue; + private String textValue; - private Map attributes; + private Map attributes; - public String getTextValue() { - return textValue; - } + public String getTextValue() { + return textValue; + } - public void setTextValue(final String textValue) { - this.textValue = textValue; - } + public void setTextValue(final String textValue) { + this.textValue = textValue; + } - public Map getAttributes() { - return attributes; - } + public Map getAttributes() { + return attributes; + } - public void setAttributes(final Map attributes) { - this.attributes = attributes; - } - } + public void setAttributes(final Map attributes) { + this.attributes = attributes; + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java index f5800cdaf..18e489a21 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/DHPUtils.java @@ -1,70 +1,75 @@ + package eu.dnetlib.dhp.utils; -import com.jayway.jsonpath.JsonPath; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; -import net.minidev.json.JSONArray; + import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64OutputStream; import org.apache.commons.codec.binary.Hex; +import com.jayway.jsonpath.JsonPath; + +import net.minidev.json.JSONArray; + public class DHPUtils { - 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; - } - } + 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; + } + } - public static String generateIdentifier(final String originalId, final String nsPrefix) { - return String.format("%s::%s", nsPrefix, DHPUtils.md5(originalId)); - } + public static String generateIdentifier(final String originalId, final String nsPrefix) { + return String.format("%s::%s", nsPrefix, DHPUtils.md5(originalId)); + } - public static String compressString(final String input) { - try (ByteArrayOutputStream out = new ByteArrayOutputStream(); - Base64OutputStream b64os = new Base64OutputStream(out)) { - GZIPOutputStream gzip = new GZIPOutputStream(b64os); - gzip.write(input.getBytes(StandardCharsets.UTF_8)); - gzip.close(); - return out.toString(); - } catch (Throwable e) { - return null; - } - } + public static String compressString(final String input) { + try (ByteArrayOutputStream out = new ByteArrayOutputStream(); + Base64OutputStream b64os = new Base64OutputStream(out)) { + GZIPOutputStream gzip = new GZIPOutputStream(b64os); + gzip.write(input.getBytes(StandardCharsets.UTF_8)); + gzip.close(); + return out.toString(); + } catch (Throwable e) { + return null; + } + } - public static String decompressString(final String input) { - byte[] byteArray = Base64.decodeBase64(input.getBytes()); - int len; - try (GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream((byteArray))); - ByteArrayOutputStream bos = new ByteArrayOutputStream(byteArray.length)) { - byte[] buffer = new byte[1024]; - while ((len = gis.read(buffer)) != -1) { - bos.write(buffer, 0, len); - } - return bos.toString(); - } catch (Exception e) { - return null; - } - } + public static String decompressString(final String input) { + byte[] byteArray = Base64.decodeBase64(input.getBytes()); + int len; + try (GZIPInputStream gis = new GZIPInputStream(new ByteArrayInputStream((byteArray))); + ByteArrayOutputStream bos = new ByteArrayOutputStream(byteArray.length)) { + byte[] buffer = new byte[1024]; + while ((len = gis.read(buffer)) != -1) { + bos.write(buffer, 0, len); + } + return bos.toString(); + } catch (Exception e) { + return null; + } + } - public static String getJPathString(final String jsonPath, final String json) { - try { - Object o = JsonPath.read(json, jsonPath); - if (o instanceof String) return (String) o; - if (o instanceof JSONArray && ((JSONArray) o).size() > 0) - return (String) ((JSONArray) o).get(0); - return o.toString(); - } catch (Exception e) { - return ""; - } - } + public static String getJPathString(final String jsonPath, final String json) { + try { + Object o = JsonPath.read(json, jsonPath); + if (o instanceof String) + return (String) o; + if (o instanceof JSONArray && ((JSONArray) o).size() > 0) + return (String) ((JSONArray) o).get(0); + return o.toString(); + } catch (Exception e) { + return ""; + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/ISLookupClientFactory.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/ISLookupClientFactory.java index b6f3f111a..97fe4b9d8 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/ISLookupClientFactory.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/ISLookupClientFactory.java @@ -1,24 +1,26 @@ + package eu.dnetlib.dhp.utils; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.cxf.jaxws.JaxWsProxyFactoryBean; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + public class ISLookupClientFactory { - private static final Log log = LogFactory.getLog(ISLookupClientFactory.class); + private static final Log log = LogFactory.getLog(ISLookupClientFactory.class); - public static ISLookUpService getLookUpService(final String isLookupUrl) { - return getServiceStub(ISLookUpService.class, isLookupUrl); - } + public static ISLookUpService getLookUpService(final String isLookupUrl) { + return getServiceStub(ISLookUpService.class, isLookupUrl); + } - @SuppressWarnings("unchecked") - private static T getServiceStub(final Class clazz, final String endpoint) { - log.info(String.format("creating %s stub from %s", clazz.getName(), endpoint)); - final JaxWsProxyFactoryBean jaxWsProxyFactory = new JaxWsProxyFactoryBean(); - jaxWsProxyFactory.setServiceClass(clazz); - jaxWsProxyFactory.setAddress(endpoint); - return (T) jaxWsProxyFactory.create(); - } + @SuppressWarnings("unchecked") + private static T getServiceStub(final Class clazz, final String endpoint) { + log.info(String.format("creating %s stub from %s", clazz.getName(), endpoint)); + final JaxWsProxyFactoryBean jaxWsProxyFactory = new JaxWsProxyFactoryBean(); + jaxWsProxyFactory.setServiceClass(clazz); + jaxWsProxyFactory.setAddress(endpoint); + return (T) jaxWsProxyFactory.create(); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/AbstractExtensionFunction.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/AbstractExtensionFunction.java index 57bd130cb..9b00b908c 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/AbstractExtensionFunction.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/AbstractExtensionFunction.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.utils.saxon; import net.sf.saxon.expr.XPathContext; @@ -9,25 +10,24 @@ import net.sf.saxon.trans.XPathException; public abstract class AbstractExtensionFunction extends ExtensionFunctionDefinition { - public static String DEFAULT_SAXON_EXT_NS_URI = - "http://www.d-net.research-infrastructures.eu/saxon-extension"; + public static String DEFAULT_SAXON_EXT_NS_URI = "http://www.d-net.research-infrastructures.eu/saxon-extension"; - public abstract String getName(); + public abstract String getName(); - public abstract Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException; + public abstract Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException; - @Override - public StructuredQName getFunctionQName() { - return new StructuredQName("dnet", DEFAULT_SAXON_EXT_NS_URI, getName()); - } + @Override + public StructuredQName getFunctionQName() { + return new StructuredQName("dnet", DEFAULT_SAXON_EXT_NS_URI, getName()); + } - @Override - public ExtensionFunctionCall makeCallExpression() { - return new ExtensionFunctionCall() { - @Override - public Sequence call(XPathContext context, Sequence[] arguments) throws XPathException { - return doCall(context, arguments); - } - }; - } + @Override + public ExtensionFunctionCall makeCallExpression() { + return new ExtensionFunctionCall() { + @Override + public Sequence call(XPathContext context, Sequence[] arguments) throws XPathException { + return doCall(context, arguments); + } + }; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/ExtractYear.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/ExtractYear.java index 38ecb6377..c7e311b02 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/ExtractYear.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/ExtractYear.java @@ -1,9 +1,11 @@ + package eu.dnetlib.dhp.utils.saxon; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Calendar; import java.util.GregorianCalendar; + import net.sf.saxon.expr.XPathContext; import net.sf.saxon.om.Item; import net.sf.saxon.om.Sequence; @@ -13,55 +15,59 @@ import net.sf.saxon.value.StringValue; public class ExtractYear extends AbstractExtensionFunction { - private static final String[] dateFormats = {"yyyy-MM-dd", "yyyy/MM/dd"}; + private static final String[] dateFormats = { + "yyyy-MM-dd", "yyyy/MM/dd" + }; - @Override - public String getName() { - return "extractYear"; - } + @Override + public String getName() { + return "extractYear"; + } - @Override - public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { - if (arguments == null | arguments.length == 0) { - return new StringValue(""); - } - final Item item = arguments[0].head(); - if (item == null) { - return new StringValue(""); - } - return new StringValue(_year(item.getStringValue())); - } + @Override + public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { + if (arguments == null | arguments.length == 0) { + return new StringValue(""); + } + final Item item = arguments[0].head(); + if (item == null) { + return new StringValue(""); + } + return new StringValue(_year(item.getStringValue())); + } - @Override - public int getMinimumNumberOfArguments() { - return 0; - } + @Override + public int getMinimumNumberOfArguments() { + return 0; + } - @Override - public int getMaximumNumberOfArguments() { - return 1; - } + @Override + public int getMaximumNumberOfArguments() { + return 1; + } - @Override - public SequenceType[] getArgumentTypes() { - return new SequenceType[] {SequenceType.OPTIONAL_ITEM}; - } + @Override + public SequenceType[] getArgumentTypes() { + return new SequenceType[] { + SequenceType.OPTIONAL_ITEM + }; + } - @Override - public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { - return SequenceType.SINGLE_STRING; - } + @Override + public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { + return SequenceType.SINGLE_STRING; + } - private String _year(String s) { - Calendar c = new GregorianCalendar(); - for (String format : dateFormats) { - try { - c.setTime(new SimpleDateFormat(format).parse(s)); - String year = String.valueOf(c.get(Calendar.YEAR)); - return year; - } catch (ParseException e) { - } - } - return ""; - } + private String _year(String s) { + Calendar c = new GregorianCalendar(); + for (String format : dateFormats) { + try { + c.setTime(new SimpleDateFormat(format).parse(s)); + String year = String.valueOf(c.get(Calendar.YEAR)); + return year; + } catch (ParseException e) { + } + } + return ""; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/NormalizeDate.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/NormalizeDate.java index def4fdfc7..4a719909a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/NormalizeDate.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/NormalizeDate.java @@ -1,8 +1,10 @@ + package eu.dnetlib.dhp.utils.saxon; import java.text.ParseException; import java.text.SimpleDateFormat; import java.util.Date; + import net.sf.saxon.expr.XPathContext; import net.sf.saxon.om.Sequence; import net.sf.saxon.trans.XPathException; @@ -11,57 +13,59 @@ import net.sf.saxon.value.StringValue; public class NormalizeDate extends AbstractExtensionFunction { - private static final String[] normalizeDateFormats = { - "yyyy-MM-dd'T'hh:mm:ss", "yyyy-MM-dd", "yyyy/MM/dd", "yyyy" - }; + private static final String[] normalizeDateFormats = { + "yyyy-MM-dd'T'hh:mm:ss", "yyyy-MM-dd", "yyyy/MM/dd", "yyyy" + }; - private static final String normalizeOutFormat = new String("yyyy-MM-dd'T'hh:mm:ss'Z'"); + private static final String normalizeOutFormat = new String("yyyy-MM-dd'T'hh:mm:ss'Z'"); - @Override - public String getName() { - return "normalizeDate"; - } + @Override + public String getName() { + return "normalizeDate"; + } - @Override - public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { - if (arguments == null | arguments.length == 0) { - return new StringValue(""); - } - String s = arguments[0].head().getStringValue(); - return new StringValue(_year(s)); - } + @Override + public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { + if (arguments == null | arguments.length == 0) { + return new StringValue(""); + } + String s = arguments[0].head().getStringValue(); + return new StringValue(_year(s)); + } - @Override - public int getMinimumNumberOfArguments() { - return 0; - } + @Override + public int getMinimumNumberOfArguments() { + return 0; + } - @Override - public int getMaximumNumberOfArguments() { - return 1; - } + @Override + public int getMaximumNumberOfArguments() { + return 1; + } - @Override - public SequenceType[] getArgumentTypes() { - return new SequenceType[] {SequenceType.OPTIONAL_ITEM}; - } + @Override + public SequenceType[] getArgumentTypes() { + return new SequenceType[] { + SequenceType.OPTIONAL_ITEM + }; + } - @Override - public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { - return SequenceType.SINGLE_STRING; - } + @Override + public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { + return SequenceType.SINGLE_STRING; + } - private String _year(String s) { - final String date = s != null ? s.trim() : ""; + private String _year(String s) { + final String date = s != null ? s.trim() : ""; - for (String format : normalizeDateFormats) { - try { - Date parse = new SimpleDateFormat(format).parse(date); - String res = new SimpleDateFormat(normalizeOutFormat).format(parse); - return res; - } catch (ParseException e) { - } - } - return ""; - } + for (String format : normalizeDateFormats) { + try { + Date parse = new SimpleDateFormat(format).parse(date); + String res = new SimpleDateFormat(normalizeOutFormat).format(parse); + return res; + } catch (ParseException e) { + } + } + return ""; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/PickFirst.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/PickFirst.java index 73159c617..46ecafd0a 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/PickFirst.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/PickFirst.java @@ -1,59 +1,63 @@ + package eu.dnetlib.dhp.utils.saxon; +import org.apache.commons.lang3.StringUtils; + import net.sf.saxon.expr.XPathContext; import net.sf.saxon.om.Item; import net.sf.saxon.om.Sequence; import net.sf.saxon.trans.XPathException; import net.sf.saxon.value.SequenceType; import net.sf.saxon.value.StringValue; -import org.apache.commons.lang3.StringUtils; public class PickFirst extends AbstractExtensionFunction { - @Override - public String getName() { - return "pickFirst"; - } + @Override + public String getName() { + return "pickFirst"; + } - @Override - public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { - if (arguments == null | arguments.length == 0) { - return new StringValue(""); - } + @Override + public Sequence doCall(XPathContext context, Sequence[] arguments) throws XPathException { + if (arguments == null | arguments.length == 0) { + return new StringValue(""); + } - final String s1 = getValue(arguments[0]); - final String s2 = getValue(arguments[1]); + final String s1 = getValue(arguments[0]); + final String s2 = getValue(arguments[1]); - return new StringValue(StringUtils.isNotBlank(s1) ? s1 : StringUtils.isNotBlank(s2) ? s2 : ""); - } + return new StringValue(StringUtils.isNotBlank(s1) ? s1 : StringUtils.isNotBlank(s2) ? s2 : ""); + } - private String getValue(final Sequence arg) throws XPathException { - if (arg != null) { - final Item item = arg.head(); - if (item != null) { - return item.getStringValue(); - } - } - return ""; - } + private String getValue(final Sequence arg) throws XPathException { + if (arg != null) { + final Item item = arg.head(); + if (item != null) { + return item.getStringValue(); + } + } + return ""; + } - @Override - public int getMinimumNumberOfArguments() { - return 0; - } + @Override + public int getMinimumNumberOfArguments() { + return 0; + } - @Override - public int getMaximumNumberOfArguments() { - return 2; - } + @Override + public int getMaximumNumberOfArguments() { + return 2; + } - @Override - public SequenceType[] getArgumentTypes() { - return new SequenceType[] {SequenceType.OPTIONAL_ITEM}; - } + @Override + public SequenceType[] getArgumentTypes() { + return new SequenceType[] { + SequenceType.OPTIONAL_ITEM + }; + } - @Override - public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { - return SequenceType.SINGLE_STRING; - } + @Override + public SequenceType getResultType(SequenceType[] suppliedArgumentTypes) { + return SequenceType.SINGLE_STRING; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/SaxonTransformerFactory.java b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/SaxonTransformerFactory.java index 18ce51887..b85d866f1 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/SaxonTransformerFactory.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/utils/saxon/SaxonTransformerFactory.java @@ -1,29 +1,32 @@ + package eu.dnetlib.dhp.utils.saxon; import java.io.StringReader; + import javax.xml.transform.Transformer; import javax.xml.transform.TransformerException; import javax.xml.transform.stream.StreamSource; + import net.sf.saxon.Configuration; import net.sf.saxon.TransformerFactoryImpl; public class SaxonTransformerFactory { - /** - * Creates the index record transformer from the given XSLT - * - * @param xslt - * @return - * @throws TransformerException - */ - public static Transformer newInstance(final String xslt) throws TransformerException { + /** + * Creates the index record transformer from the given XSLT + * + * @param xslt + * @return + * @throws TransformerException + */ + public static Transformer newInstance(final String xslt) throws TransformerException { - final TransformerFactoryImpl factory = new TransformerFactoryImpl(); - final Configuration conf = factory.getConfiguration(); - conf.registerExtensionFunction(new ExtractYear()); - conf.registerExtensionFunction(new NormalizeDate()); - conf.registerExtensionFunction(new PickFirst()); + final TransformerFactoryImpl factory = new TransformerFactoryImpl(); + final Configuration conf = factory.getConfiguration(); + conf.registerExtensionFunction(new ExtractYear()); + conf.registerExtensionFunction(new NormalizeDate()); + conf.registerExtensionFunction(new PickFirst()); - return factory.newTransformer(new StreamSource(new StringReader(xslt))); - } + return factory.newTransformer(new StreamSource(new StringReader(xslt))); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/message/Message.java b/dhp-common/src/main/java/eu/dnetlib/message/Message.java index b62afb19a..fc1c38291 100644 --- a/dhp-common/src/main/java/eu/dnetlib/message/Message.java +++ b/dhp-common/src/main/java/eu/dnetlib/message/Message.java @@ -1,73 +1,76 @@ + package eu.dnetlib.message; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; import java.util.Map; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + public class Message { - private String workflowId; + private String workflowId; - private String jobName; + private String jobName; - private MessageType type; + private MessageType type; - private Map body; + private Map body; - public static Message fromJson(final String json) throws IOException { - final ObjectMapper jsonMapper = new ObjectMapper(); - return jsonMapper.readValue(json, Message.class); - } + public static Message fromJson(final String json) throws IOException { + final ObjectMapper jsonMapper = new ObjectMapper(); + return jsonMapper.readValue(json, Message.class); + } - public Message() {} + public Message() { + } - public Message(String workflowId, String jobName, MessageType type, Map body) { - this.workflowId = workflowId; - this.jobName = jobName; - this.type = type; - this.body = body; - } + public Message(String workflowId, String jobName, MessageType type, Map body) { + this.workflowId = workflowId; + this.jobName = jobName; + this.type = type; + this.body = body; + } - public String getWorkflowId() { - return workflowId; - } + public String getWorkflowId() { + return workflowId; + } - public void setWorkflowId(String workflowId) { - this.workflowId = workflowId; - } + public void setWorkflowId(String workflowId) { + this.workflowId = workflowId; + } - public String getJobName() { - return jobName; - } + public String getJobName() { + return jobName; + } - public void setJobName(String jobName) { - this.jobName = jobName; - } + public void setJobName(String jobName) { + this.jobName = jobName; + } - public MessageType getType() { - return type; - } + public MessageType getType() { + return type; + } - public void setType(MessageType type) { - this.type = type; - } + public void setType(MessageType type) { + this.type = type; + } - public Map getBody() { - return body; - } + public Map getBody() { + return body; + } - public void setBody(Map body) { - this.body = body; - } + public void setBody(Map body) { + this.body = body; + } - @Override - public String toString() { - final ObjectMapper jsonMapper = new ObjectMapper(); - try { - return jsonMapper.writeValueAsString(this); - } catch (JsonProcessingException e) { - return null; - } - } + @Override + public String toString() { + final ObjectMapper jsonMapper = new ObjectMapper(); + try { + return jsonMapper.writeValueAsString(this); + } catch (JsonProcessingException e) { + return null; + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/message/MessageConsumer.java b/dhp-common/src/main/java/eu/dnetlib/message/MessageConsumer.java index 3df712a62..fb3f0bd95 100644 --- a/dhp-common/src/main/java/eu/dnetlib/message/MessageConsumer.java +++ b/dhp-common/src/main/java/eu/dnetlib/message/MessageConsumer.java @@ -1,45 +1,47 @@ + package eu.dnetlib.message; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.LinkedBlockingQueue; + import com.rabbitmq.client.AMQP; import com.rabbitmq.client.Channel; import com.rabbitmq.client.DefaultConsumer; import com.rabbitmq.client.Envelope; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.LinkedBlockingQueue; public class MessageConsumer extends DefaultConsumer { - final LinkedBlockingQueue queueMessages; + final LinkedBlockingQueue queueMessages; - /** - * Constructs a new instance and records its association to the passed-in channel. - * - * @param channel the channel to which this consumer is attached - * @param queueMessages - */ - public MessageConsumer(Channel channel, LinkedBlockingQueue queueMessages) { - super(channel); - this.queueMessages = queueMessages; - } + /** + * Constructs a new instance and records its association to the passed-in channel. + * + * @param channel the channel to which this consumer is attached + * @param queueMessages + */ + public MessageConsumer(Channel channel, LinkedBlockingQueue queueMessages) { + super(channel); + this.queueMessages = queueMessages; + } - @Override - public void handleDelivery( - String consumerTag, Envelope envelope, AMQP.BasicProperties properties, byte[] body) - throws IOException { - final String json = new String(body, StandardCharsets.UTF_8); - Message message = Message.fromJson(json); - try { - this.queueMessages.put(message); - System.out.println("Receiving Message " + message); - } catch (InterruptedException e) { - if (message.getType() == MessageType.REPORT) - throw new RuntimeException("Error on sending message"); - else { - // TODO LOGGING EXCEPTION - } - } finally { - getChannel().basicAck(envelope.getDeliveryTag(), false); - } - } + @Override + public void handleDelivery( + String consumerTag, Envelope envelope, AMQP.BasicProperties properties, byte[] body) + throws IOException { + final String json = new String(body, StandardCharsets.UTF_8); + Message message = Message.fromJson(json); + try { + this.queueMessages.put(message); + System.out.println("Receiving Message " + message); + } catch (InterruptedException e) { + if (message.getType() == MessageType.REPORT) + throw new RuntimeException("Error on sending message"); + else { + // TODO LOGGING EXCEPTION + } + } finally { + getChannel().basicAck(envelope.getDeliveryTag(), false); + } + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/message/MessageManager.java b/dhp-common/src/main/java/eu/dnetlib/message/MessageManager.java index 8370a6cc8..4c5c48c55 100644 --- a/dhp-common/src/main/java/eu/dnetlib/message/MessageManager.java +++ b/dhp-common/src/main/java/eu/dnetlib/message/MessageManager.java @@ -1,134 +1,136 @@ + package eu.dnetlib.message; -import com.rabbitmq.client.Channel; -import com.rabbitmq.client.Connection; -import com.rabbitmq.client.ConnectionFactory; import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeoutException; +import com.rabbitmq.client.Channel; +import com.rabbitmq.client.Connection; +import com.rabbitmq.client.ConnectionFactory; + public class MessageManager { - private final String messageHost; + private final String messageHost; - private final String username; + private final String username; - private final String password; + private final String password; - private Connection connection; + private Connection connection; - private Map channels = new HashMap<>(); + private Map channels = new HashMap<>(); - private boolean durable; + private boolean durable; - private boolean autodelete; + private boolean autodelete; - private final LinkedBlockingQueue queueMessages; + private final LinkedBlockingQueue queueMessages; - public MessageManager( - String messageHost, - String username, - String password, - final LinkedBlockingQueue queueMessages) { - this.queueMessages = queueMessages; - this.messageHost = messageHost; - this.username = username; - this.password = password; - } + public MessageManager( + String messageHost, + String username, + String password, + final LinkedBlockingQueue queueMessages) { + this.queueMessages = queueMessages; + this.messageHost = messageHost; + this.username = username; + this.password = password; + } - public MessageManager( - String messageHost, - String username, - String password, - boolean durable, - boolean autodelete, - final LinkedBlockingQueue queueMessages) { - this.queueMessages = queueMessages; - this.messageHost = messageHost; - this.username = username; - this.password = password; + public MessageManager( + String messageHost, + String username, + String password, + boolean durable, + boolean autodelete, + final LinkedBlockingQueue queueMessages) { + this.queueMessages = queueMessages; + this.messageHost = messageHost; + this.username = username; + this.password = password; - this.durable = durable; - this.autodelete = autodelete; - } + this.durable = durable; + this.autodelete = autodelete; + } - private Connection createConnection() throws IOException, TimeoutException { - ConnectionFactory factory = new ConnectionFactory(); - factory.setHost(this.messageHost); - factory.setUsername(this.username); - factory.setPassword(this.password); - return factory.newConnection(); - } + private Connection createConnection() throws IOException, TimeoutException { + ConnectionFactory factory = new ConnectionFactory(); + factory.setHost(this.messageHost); + factory.setUsername(this.username); + factory.setPassword(this.password); + return factory.newConnection(); + } - private Channel createChannel( - final Connection connection, - final String queueName, - final boolean durable, - final boolean autodelete) - throws Exception { - Map args = new HashMap<>(); - args.put("x-message-ttl", 10000); - Channel channel = connection.createChannel(); - channel.queueDeclare(queueName, durable, false, this.autodelete, args); - return channel; - } + private Channel createChannel( + final Connection connection, + final String queueName, + final boolean durable, + final boolean autodelete) + throws Exception { + Map args = new HashMap<>(); + args.put("x-message-ttl", 10000); + Channel channel = connection.createChannel(); + channel.queueDeclare(queueName, durable, false, this.autodelete, args); + return channel; + } - private Channel getOrCreateChannel(final String queueName, boolean durable, boolean autodelete) - throws Exception { - if (channels.containsKey(queueName)) { - return channels.get(queueName); - } + private Channel getOrCreateChannel(final String queueName, boolean durable, boolean autodelete) + throws Exception { + if (channels.containsKey(queueName)) { + return channels.get(queueName); + } - if (this.connection == null) { - this.connection = createConnection(); - } - channels.put(queueName, createChannel(this.connection, queueName, durable, autodelete)); - return channels.get(queueName); - } + if (this.connection == null) { + this.connection = createConnection(); + } + channels.put(queueName, createChannel(this.connection, queueName, durable, autodelete)); + return channels.get(queueName); + } - public void close() throws IOException { - channels - .values() - .forEach( - ch -> { - try { - ch.close(); - } catch (Exception e) { - // TODO LOG - } - }); + public void close() throws IOException { + channels + .values() + .forEach( + ch -> { + try { + ch.close(); + } catch (Exception e) { + // TODO LOG + } + }); - this.connection.close(); - } + this.connection.close(); + } - public boolean sendMessage(final Message message, String queueName) throws Exception { - try { - Channel channel = getOrCreateChannel(queueName, this.durable, this.autodelete); - channel.basicPublish("", queueName, null, message.toString().getBytes()); - return true; - } catch (Throwable e) { - throw new RuntimeException(e); - } - } + public boolean sendMessage(final Message message, String queueName) throws Exception { + try { + Channel channel = getOrCreateChannel(queueName, this.durable, this.autodelete); + channel.basicPublish("", queueName, null, message.toString().getBytes()); + return true; + } catch (Throwable e) { + throw new RuntimeException(e); + } + } - public boolean sendMessage( - final Message message, String queueName, boolean durable_var, boolean autodelete_var) - throws Exception { - try { - Channel channel = getOrCreateChannel(queueName, durable_var, autodelete_var); - channel.basicPublish("", queueName, null, message.toString().getBytes()); - return true; - } catch (Throwable e) { - throw new RuntimeException(e); - } - } + public boolean sendMessage( + final Message message, String queueName, boolean durable_var, boolean autodelete_var) + throws Exception { + try { + Channel channel = getOrCreateChannel(queueName, durable_var, autodelete_var); + channel.basicPublish("", queueName, null, message.toString().getBytes()); + return true; + } catch (Throwable e) { + throw new RuntimeException(e); + } + } - public void startConsumingMessage( - final String queueName, final boolean durable, final boolean autodelete) throws Exception { + public void startConsumingMessage( + final String queueName, final boolean durable, final boolean autodelete) throws Exception { - Channel channel = createChannel(createConnection(), queueName, durable, autodelete); - channel.basicConsume(queueName, false, new MessageConsumer(channel, queueMessages)); - } + Channel channel = createChannel(createConnection(), queueName, durable, autodelete); + channel.basicConsume(queueName, false, new MessageConsumer(channel, queueMessages)); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/message/MessageType.java b/dhp-common/src/main/java/eu/dnetlib/message/MessageType.java index edca90061..72cbda252 100644 --- a/dhp-common/src/main/java/eu/dnetlib/message/MessageType.java +++ b/dhp-common/src/main/java/eu/dnetlib/message/MessageType.java @@ -1,6 +1,6 @@ + package eu.dnetlib.message; public enum MessageType { - ONGOING, - REPORT + ONGOING, REPORT } diff --git a/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelInfo.java b/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelInfo.java index 1ae6e8ead..e07fcef66 100644 --- a/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelInfo.java +++ b/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelInfo.java @@ -1,24 +1,25 @@ + package eu.dnetlib.scholexplorer.relation; import java.io.Serializable; public class RelInfo implements Serializable { - private String original; - private String inverse; + private String original; + private String inverse; - public String getOriginal() { - return original; - } + public String getOriginal() { + return original; + } - public void setOriginal(String original) { - this.original = original; - } + public void setOriginal(String original) { + this.original = original; + } - public String getInverse() { - return inverse; - } + public String getInverse() { + return inverse; + } - public void setInverse(String inverse) { - this.inverse = inverse; - } + public void setInverse(String inverse) { + this.inverse = inverse; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelationMapper.java b/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelationMapper.java index 9cc995821..eb708c390 100644 --- a/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelationMapper.java +++ b/dhp-common/src/main/java/eu/dnetlib/scholexplorer/relation/RelationMapper.java @@ -1,18 +1,20 @@ + package eu.dnetlib.scholexplorer.relation; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.Serializable; import java.util.HashMap; + import org.apache.commons.io.IOUtils; +import com.fasterxml.jackson.databind.ObjectMapper; + public class RelationMapper extends HashMap implements Serializable { - public static RelationMapper load() throws Exception { + public static RelationMapper load() throws Exception { - final String json = - IOUtils.toString(RelationMapper.class.getResourceAsStream("relations.json")); + final String json = IOUtils.toString(RelationMapper.class.getResourceAsStream("relations.json")); - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(json, RelationMapper.class); - } + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue(json, RelationMapper.class); + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java index bb7351745..e14020830 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/application/ArgumentApplicationParserTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.application; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -8,58 +9,59 @@ import org.junit.jupiter.api.Test; public class ArgumentApplicationParserTest { - @Test - public void testParseParameter() throws Exception { - final String jsonConfiguration = - IOUtils.toString( - this.getClass().getResourceAsStream("/eu/dnetlib/application/parameters.json")); - assertNotNull(jsonConfiguration); - ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument( - new String[] { - "-p", - "value0", - "-a", - "value1", - "-n", - "value2", - "-u", - "value3", - "-ru", - "value4", - "-rp", - "value5", - "-rh", - "value6", - "-ro", - "value7", - "-rr", - "value8", - "-w", - "value9", - "-cc", - ArgumentApplicationParser.compressArgument(jsonConfiguration) - }); - assertNotNull(parser.get("hdfsPath")); - assertNotNull(parser.get("apidescriptor")); - assertNotNull(parser.get("namenode")); - assertNotNull(parser.get("userHDFS")); - assertNotNull(parser.get("rabbitUser")); - assertNotNull(parser.get("rabbitPassWord")); - assertNotNull(parser.get("rabbitHost")); - assertNotNull(parser.get("rabbitOngoingQueue")); - assertNotNull(parser.get("rabbitReportQueue")); - assertNotNull(parser.get("workflowId")); - assertEquals("value0", parser.get("hdfsPath")); - assertEquals("value1", parser.get("apidescriptor")); - assertEquals("value2", parser.get("namenode")); - assertEquals("value3", parser.get("userHDFS")); - assertEquals("value4", parser.get("rabbitUser")); - assertEquals("value5", parser.get("rabbitPassWord")); - assertEquals("value6", parser.get("rabbitHost")); - assertEquals("value7", parser.get("rabbitOngoingQueue")); - assertEquals("value8", parser.get("rabbitReportQueue")); - assertEquals("value9", parser.get("workflowId")); - assertEquals(jsonConfiguration, parser.get("ccCoco")); - } + @Test + public void testParseParameter() throws Exception { + final String jsonConfiguration = IOUtils + .toString( + this.getClass().getResourceAsStream("/eu/dnetlib/application/parameters.json")); + assertNotNull(jsonConfiguration); + ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser + .parseArgument( + new String[] { + "-p", + "value0", + "-a", + "value1", + "-n", + "value2", + "-u", + "value3", + "-ru", + "value4", + "-rp", + "value5", + "-rh", + "value6", + "-ro", + "value7", + "-rr", + "value8", + "-w", + "value9", + "-cc", + ArgumentApplicationParser.compressArgument(jsonConfiguration) + }); + assertNotNull(parser.get("hdfsPath")); + assertNotNull(parser.get("apidescriptor")); + assertNotNull(parser.get("namenode")); + assertNotNull(parser.get("userHDFS")); + assertNotNull(parser.get("rabbitUser")); + assertNotNull(parser.get("rabbitPassWord")); + assertNotNull(parser.get("rabbitHost")); + assertNotNull(parser.get("rabbitOngoingQueue")); + assertNotNull(parser.get("rabbitReportQueue")); + assertNotNull(parser.get("workflowId")); + assertEquals("value0", parser.get("hdfsPath")); + assertEquals("value1", parser.get("apidescriptor")); + assertEquals("value2", parser.get("namenode")); + assertEquals("value3", parser.get("userHDFS")); + assertEquals("value4", parser.get("rabbitUser")); + assertEquals("value5", parser.get("rabbitPassWord")); + assertEquals("value6", parser.get("rabbitHost")); + assertEquals("value7", parser.get("rabbitOngoingQueue")); + assertEquals("value8", parser.get("rabbitReportQueue")); + assertEquals("value9", parser.get("workflowId")); + assertEquals(jsonConfiguration, parser.get("ccCoco")); + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/common/HdfsSupportTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/common/HdfsSupportTest.java index a8f0bbb0d..870943816 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/common/HdfsSupportTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/common/HdfsSupportTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.common; import static org.junit.jupiter.api.Assertions.*; @@ -8,6 +9,7 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; @@ -15,63 +17,64 @@ import org.junit.jupiter.api.io.TempDir; public class HdfsSupportTest { - @Nested - class Remove { + @Nested + class Remove { - @Test - public void shouldThrowARuntimeExceptionOnError() { - // when - assertThrows(RuntimeException.class, () -> HdfsSupport.remove(null, new Configuration())); - } + @Test + public void shouldThrowARuntimeExceptionOnError() { + // when + assertThrows(RuntimeException.class, () -> HdfsSupport.remove(null, new Configuration())); + } - @Test - public void shouldRemoveADirFromHDFS(@TempDir Path tempDir) { - // when - HdfsSupport.remove(tempDir.toString(), new Configuration()); + @Test + public void shouldRemoveADirFromHDFS(@TempDir Path tempDir) { + // when + HdfsSupport.remove(tempDir.toString(), new Configuration()); - // then - assertFalse(Files.exists(tempDir)); - } + // then + assertFalse(Files.exists(tempDir)); + } - @Test - public void shouldRemoveAFileFromHDFS(@TempDir Path tempDir) throws IOException { - // given - Path file = Files.createTempFile(tempDir, "p", "s"); + @Test + public void shouldRemoveAFileFromHDFS(@TempDir Path tempDir) throws IOException { + // given + Path file = Files.createTempFile(tempDir, "p", "s"); - // when - HdfsSupport.remove(file.toString(), new Configuration()); + // when + HdfsSupport.remove(file.toString(), new Configuration()); - // then - assertFalse(Files.exists(file)); - } - } + // then + assertFalse(Files.exists(file)); + } + } - @Nested - class ListFiles { + @Nested + class ListFiles { - @Test - public void shouldThrowARuntimeExceptionOnError() { - // when - assertThrows(RuntimeException.class, () -> HdfsSupport.listFiles(null, new Configuration())); - } + @Test + public void shouldThrowARuntimeExceptionOnError() { + // when + assertThrows(RuntimeException.class, () -> HdfsSupport.listFiles(null, new Configuration())); + } - @Test - public void shouldListFilesLocatedInPath(@TempDir Path tempDir) throws IOException { - Path subDir1 = Files.createTempDirectory(tempDir, "list_me"); - Path subDir2 = Files.createTempDirectory(tempDir, "list_me"); + @Test + public void shouldListFilesLocatedInPath(@TempDir Path tempDir) throws IOException { + Path subDir1 = Files.createTempDirectory(tempDir, "list_me"); + Path subDir2 = Files.createTempDirectory(tempDir, "list_me"); - // when - List paths = HdfsSupport.listFiles(tempDir.toString(), new Configuration()); + // when + List paths = HdfsSupport.listFiles(tempDir.toString(), new Configuration()); - // then - assertEquals(2, paths.size()); - List expecteds = - Arrays.stream(new String[] {subDir1.toString(), subDir2.toString()}) - .sorted() - .collect(Collectors.toList()); - List actuals = paths.stream().sorted().collect(Collectors.toList()); - assertTrue(actuals.get(0).contains(expecteds.get(0))); - assertTrue(actuals.get(1).contains(expecteds.get(1))); - } - } + // then + assertEquals(2, paths.size()); + List expecteds = Arrays.stream(new String[] { + subDir1.toString(), subDir2.toString() + }) + .sorted() + .collect(Collectors.toList()); + List actuals = paths.stream().sorted().collect(Collectors.toList()); + assertTrue(actuals.get(0).contains(expecteds.get(0))); + assertTrue(actuals.get(1).contains(expecteds.get(1))); + } + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/common/SparkSessionSupportTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/common/SparkSessionSupportTest.java index 698b9cea5..2f01c0863 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/common/SparkSessionSupportTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/common/SparkSessionSupportTest.java @@ -1,55 +1,58 @@ + package eu.dnetlib.dhp.common; import static org.mockito.Mockito.*; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingConsumer; import java.util.function.Function; + import org.apache.spark.SparkConf; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.ThrowingConsumer; + public class SparkSessionSupportTest { - @Nested - class RunWithSparkSession { + @Nested + class RunWithSparkSession { - @Test - public void shouldExecuteFunctionAndNotStopSparkSessionWhenSparkSessionIsNotManaged() - throws Exception { - // given - SparkSession spark = mock(SparkSession.class); - SparkConf conf = mock(SparkConf.class); - Function sparkSessionBuilder = mock(Function.class); - when(sparkSessionBuilder.apply(conf)).thenReturn(spark); - ThrowingConsumer fn = mock(ThrowingConsumer.class); + @Test + public void shouldExecuteFunctionAndNotStopSparkSessionWhenSparkSessionIsNotManaged() + throws Exception { + // given + SparkSession spark = mock(SparkSession.class); + SparkConf conf = mock(SparkConf.class); + Function sparkSessionBuilder = mock(Function.class); + when(sparkSessionBuilder.apply(conf)).thenReturn(spark); + ThrowingConsumer fn = mock(ThrowingConsumer.class); - // when - SparkSessionSupport.runWithSparkSession(sparkSessionBuilder, conf, false, fn); + // when + SparkSessionSupport.runWithSparkSession(sparkSessionBuilder, conf, false, fn); - // then - verify(sparkSessionBuilder).apply(conf); - verify(fn).accept(spark); - verify(spark, never()).stop(); - } + // then + verify(sparkSessionBuilder).apply(conf); + verify(fn).accept(spark); + verify(spark, never()).stop(); + } - @Test - public void shouldExecuteFunctionAndStopSparkSessionWhenSparkSessionIsManaged() - throws Exception { - // given - SparkSession spark = mock(SparkSession.class); - SparkConf conf = mock(SparkConf.class); - Function sparkSessionBuilder = mock(Function.class); - when(sparkSessionBuilder.apply(conf)).thenReturn(spark); - ThrowingConsumer fn = mock(ThrowingConsumer.class); + @Test + public void shouldExecuteFunctionAndStopSparkSessionWhenSparkSessionIsManaged() + throws Exception { + // given + SparkSession spark = mock(SparkSession.class); + SparkConf conf = mock(SparkConf.class); + Function sparkSessionBuilder = mock(Function.class); + when(sparkSessionBuilder.apply(conf)).thenReturn(spark); + ThrowingConsumer fn = mock(ThrowingConsumer.class); - // when - SparkSessionSupport.runWithSparkSession(sparkSessionBuilder, conf, true, fn); + // when + SparkSessionSupport.runWithSparkSession(sparkSessionBuilder, conf, true, fn); - // then - verify(sparkSessionBuilder).apply(conf); - verify(fn).accept(spark); - verify(spark, times(1)).stop(); - } - } + // then + verify(sparkSessionBuilder).apply(conf); + verify(fn).accept(spark); + verify(spark, times(1)).stop(); + } + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/model/mdstore/MetadataRecordTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/model/mdstore/MetadataRecordTest.java index 84cb08d95..cb4d0ab50 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/model/mdstore/MetadataRecordTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/model/mdstore/MetadataRecordTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.model.mdstore; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -6,10 +7,10 @@ import org.junit.jupiter.api.Test; public class MetadataRecordTest { - @Test - public void getTimestamp() { + @Test + public void getTimestamp() { - MetadataRecord r = new MetadataRecord(); - assertTrue(r.getDateOfCollection() > 0); - } + MetadataRecord r = new MetadataRecord(); + assertTrue(r.getDateOfCollection() > 0); + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/message/MessageTest.java b/dhp-common/src/test/java/eu/dnetlib/message/MessageTest.java index a514f8573..442f7b5c2 100644 --- a/dhp-common/src/test/java/eu/dnetlib/message/MessageTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/message/MessageTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.message; import static org.junit.jupiter.api.Assertions.*; @@ -5,46 +6,46 @@ import static org.junit.jupiter.api.Assertions.*; import java.io.IOException; import java.util.HashMap; import java.util.Map; + import org.junit.jupiter.api.Test; public class MessageTest { - @Test - public void fromJsonTest() throws IOException { - Message m = new Message(); - m.setWorkflowId("wId"); - m.setType(MessageType.ONGOING); - m.setJobName("Collection"); - Map body = new HashMap<>(); - body.put("parsedItem", "300"); - body.put("ExecutionTime", "30s"); + @Test + public void fromJsonTest() throws IOException { + Message m = new Message(); + m.setWorkflowId("wId"); + m.setType(MessageType.ONGOING); + m.setJobName("Collection"); + Map body = new HashMap<>(); + body.put("parsedItem", "300"); + body.put("ExecutionTime", "30s"); - m.setBody(body); - System.out.println("m = " + m); - Message m1 = Message.fromJson(m.toString()); - assertEquals(m1.getWorkflowId(), m.getWorkflowId()); - assertEquals(m1.getType(), m.getType()); - assertEquals(m1.getJobName(), m.getJobName()); + m.setBody(body); + System.out.println("m = " + m); + Message m1 = Message.fromJson(m.toString()); + assertEquals(m1.getWorkflowId(), m.getWorkflowId()); + assertEquals(m1.getType(), m.getType()); + assertEquals(m1.getJobName(), m.getJobName()); - assertNotNull(m1.getBody()); - m1.getBody().keySet().forEach(it -> assertEquals(m1.getBody().get(it), m.getBody().get(it))); - assertEquals(m1.getJobName(), m.getJobName()); - } + assertNotNull(m1.getBody()); + m1.getBody().keySet().forEach(it -> assertEquals(m1.getBody().get(it), m.getBody().get(it))); + assertEquals(m1.getJobName(), m.getJobName()); + } - @Test - public void toStringTest() { - final String expectedJson = - "{\"workflowId\":\"wId\",\"jobName\":\"Collection\",\"type\":\"ONGOING\",\"body\":{\"ExecutionTime\":\"30s\",\"parsedItem\":\"300\"}}"; - Message m = new Message(); - m.setWorkflowId("wId"); - m.setType(MessageType.ONGOING); - m.setJobName("Collection"); - Map body = new HashMap<>(); - body.put("parsedItem", "300"); - body.put("ExecutionTime", "30s"); + @Test + public void toStringTest() { + final String expectedJson = "{\"workflowId\":\"wId\",\"jobName\":\"Collection\",\"type\":\"ONGOING\",\"body\":{\"ExecutionTime\":\"30s\",\"parsedItem\":\"300\"}}"; + Message m = new Message(); + m.setWorkflowId("wId"); + m.setType(MessageType.ONGOING); + m.setJobName("Collection"); + Map body = new HashMap<>(); + body.put("parsedItem", "300"); + body.put("ExecutionTime", "30s"); - m.setBody(body); + m.setBody(body); - assertEquals(expectedJson, m.toString()); - } + assertEquals(expectedJson, m.toString()); + } } diff --git a/dhp-common/src/test/java/eu/dnetlib/scholexplorer/relation/RelationMapperTest.java b/dhp-common/src/test/java/eu/dnetlib/scholexplorer/relation/RelationMapperTest.java index 9381cb01f..d1d1ada71 100644 --- a/dhp-common/src/test/java/eu/dnetlib/scholexplorer/relation/RelationMapperTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/scholexplorer/relation/RelationMapperTest.java @@ -1,13 +1,14 @@ + package eu.dnetlib.scholexplorer.relation; import org.junit.jupiter.api.Test; public class RelationMapperTest { - @Test - public void testLoadRels() throws Exception { + @Test + public void testLoadRels() throws Exception { - RelationMapper relationMapper = RelationMapper.load(); - relationMapper.keySet().forEach(System.out::println); - } + RelationMapper relationMapper = RelationMapper.load(); + relationMapper.keySet().forEach(System.out::println); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java index c803fab52..84b22c81c 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicAction.java @@ -1,36 +1,40 @@ + package eu.dnetlib.dhp.schema.action; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import eu.dnetlib.dhp.schema.oaf.Oaf; import java.io.Serializable; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; + +import eu.dnetlib.dhp.schema.oaf.Oaf; + @JsonDeserialize(using = AtomicActionDeserializer.class) public class AtomicAction implements Serializable { - private Class clazz; + private Class clazz; - private T payload; + private T payload; - public AtomicAction() {} + public AtomicAction() { + } - public AtomicAction(Class clazz, T payload) { - this.clazz = clazz; - this.payload = payload; - } + public AtomicAction(Class clazz, T payload) { + this.clazz = clazz; + this.payload = payload; + } - public Class getClazz() { - return clazz; - } + public Class getClazz() { + return clazz; + } - public void setClazz(Class clazz) { - this.clazz = clazz; - } + public void setClazz(Class clazz) { + this.clazz = clazz; + } - public T getPayload() { - return payload; - } + public T getPayload() { + return payload; + } - public void setPayload(T payload) { - this.payload = payload; - } + public void setPayload(T payload) { + this.payload = payload; + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java index 701833c42..a9543d27a 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/action/AtomicActionDeserializer.java @@ -1,29 +1,32 @@ + package eu.dnetlib.dhp.schema.action; +import java.io.IOException; + import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; + import eu.dnetlib.dhp.schema.oaf.Oaf; -import java.io.IOException; public class AtomicActionDeserializer extends JsonDeserializer { - @Override - public Object deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException, JsonProcessingException { - JsonNode node = jp.getCodec().readTree(jp); - String classTag = node.get("clazz").asText(); - JsonNode payload = node.get("payload"); - ObjectMapper mapper = new ObjectMapper(); + @Override + public Object deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException, JsonProcessingException { + JsonNode node = jp.getCodec().readTree(jp); + String classTag = node.get("clazz").asText(); + JsonNode payload = node.get("payload"); + ObjectMapper mapper = new ObjectMapper(); - try { - final Class clazz = Class.forName(classTag); - return new AtomicAction(clazz, (Oaf) mapper.readValue(payload.toString(), clazz)); - } catch (ClassNotFoundException e) { - throw new IOException(e); - } - } + try { + final Class clazz = Class.forName(classTag); + return new AtomicAction(clazz, (Oaf) mapper.readValue(payload.toString(), clazz)); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/EntityType.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/EntityType.java index d597ecb53..54f30cf33 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/EntityType.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/EntityType.java @@ -1,26 +1,21 @@ + package eu.dnetlib.dhp.schema.common; import eu.dnetlib.dhp.schema.oaf.OafEntity; /** Actual entity types in the Graph */ public enum EntityType { - publication, - dataset, - otherresearchproduct, - software, - datasource, - organization, - project; + publication, dataset, otherresearchproduct, software, datasource, organization, project; - /** - * Resolves the EntityType, given the relative class name - * - * @param clazz the given class name - * @param actual OafEntity subclass - * @return the EntityType associated to the given class - */ - public static EntityType fromClass(Class clazz) { + /** + * Resolves the EntityType, given the relative class name + * + * @param clazz the given class name + * @param actual OafEntity subclass + * @return the EntityType associated to the given class + */ + public static EntityType fromClass(Class clazz) { - return EntityType.valueOf(clazz.getSimpleName().toLowerCase()); - } + return EntityType.valueOf(clazz.getSimpleName().toLowerCase()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/MainEntityType.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/MainEntityType.java index 466cdc9e9..cda8ba484 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/MainEntityType.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/MainEntityType.java @@ -1,9 +1,7 @@ + package eu.dnetlib.dhp.schema.common; /** Main entity types in the Graph */ public enum MainEntityType { - result, - datasource, - organization, - project + result, datasource, organization, project } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java index 0dfdaad52..c6bfff12d 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelConstants.java @@ -1,40 +1,41 @@ + package eu.dnetlib.dhp.schema.common; import eu.dnetlib.dhp.schema.oaf.Qualifier; public class ModelConstants { - public static final String DNET_RESULT_TYPOLOGIES = "dnet:result_typologies"; + public static final String DNET_RESULT_TYPOLOGIES = "dnet:result_typologies"; - public static final String DATASET_RESULTTYPE_CLASSID = "dataset"; - public static final String PUBLICATION_RESULTTYPE_CLASSID = "publication"; - public static final String SOFTWARE_RESULTTYPE_CLASSID = "software"; - public static final String ORP_RESULTTYPE_CLASSID = "other"; + public static final String DATASET_RESULTTYPE_CLASSID = "dataset"; + public static final String PUBLICATION_RESULTTYPE_CLASSID = "publication"; + public static final String SOFTWARE_RESULTTYPE_CLASSID = "software"; + public static final String ORP_RESULTTYPE_CLASSID = "other"; - public static Qualifier PUBLICATION_DEFAULT_RESULTTYPE = new Qualifier(); - public static Qualifier DATASET_DEFAULT_RESULTTYPE = new Qualifier(); - public static Qualifier SOFTWARE_DEFAULT_RESULTTYPE = new Qualifier(); - public static Qualifier ORP_DEFAULT_RESULTTYPE = new Qualifier(); + public static Qualifier PUBLICATION_DEFAULT_RESULTTYPE = new Qualifier(); + public static Qualifier DATASET_DEFAULT_RESULTTYPE = new Qualifier(); + public static Qualifier SOFTWARE_DEFAULT_RESULTTYPE = new Qualifier(); + public static Qualifier ORP_DEFAULT_RESULTTYPE = new Qualifier(); - static { - PUBLICATION_DEFAULT_RESULTTYPE.setClassid(PUBLICATION_RESULTTYPE_CLASSID); - PUBLICATION_DEFAULT_RESULTTYPE.setClassname(PUBLICATION_RESULTTYPE_CLASSID); - PUBLICATION_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); - PUBLICATION_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); + static { + PUBLICATION_DEFAULT_RESULTTYPE.setClassid(PUBLICATION_RESULTTYPE_CLASSID); + PUBLICATION_DEFAULT_RESULTTYPE.setClassname(PUBLICATION_RESULTTYPE_CLASSID); + PUBLICATION_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); + PUBLICATION_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); - DATASET_DEFAULT_RESULTTYPE.setClassid(DATASET_RESULTTYPE_CLASSID); - DATASET_DEFAULT_RESULTTYPE.setClassname(DATASET_RESULTTYPE_CLASSID); - DATASET_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); - DATASET_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); + DATASET_DEFAULT_RESULTTYPE.setClassid(DATASET_RESULTTYPE_CLASSID); + DATASET_DEFAULT_RESULTTYPE.setClassname(DATASET_RESULTTYPE_CLASSID); + DATASET_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); + DATASET_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); - SOFTWARE_DEFAULT_RESULTTYPE.setClassid(SOFTWARE_RESULTTYPE_CLASSID); - SOFTWARE_DEFAULT_RESULTTYPE.setClassname(SOFTWARE_RESULTTYPE_CLASSID); - SOFTWARE_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); - SOFTWARE_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); + SOFTWARE_DEFAULT_RESULTTYPE.setClassid(SOFTWARE_RESULTTYPE_CLASSID); + SOFTWARE_DEFAULT_RESULTTYPE.setClassname(SOFTWARE_RESULTTYPE_CLASSID); + SOFTWARE_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); + SOFTWARE_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); - ORP_DEFAULT_RESULTTYPE.setClassid(ORP_RESULTTYPE_CLASSID); - ORP_DEFAULT_RESULTTYPE.setClassname(ORP_RESULTTYPE_CLASSID); - ORP_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); - ORP_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); - } + ORP_DEFAULT_RESULTTYPE.setClassid(ORP_RESULTTYPE_CLASSID); + ORP_DEFAULT_RESULTTYPE.setClassname(ORP_RESULTTYPE_CLASSID); + ORP_DEFAULT_RESULTTYPE.setSchemeid(DNET_RESULT_TYPOLOGIES); + ORP_DEFAULT_RESULTTYPE.setSchemename(DNET_RESULT_TYPOLOGIES); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java index e004e5800..89398455e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,92 +6,95 @@ import java.util.*; public class Author implements Serializable { - private String fullname; + private String fullname; - private String name; + private String name; - private String surname; + private String surname; - private Integer rank; + private Integer rank; - private List pid; + private List pid; - private List> affiliation; + private List> affiliation; - public String getFullname() { - return fullname; - } + public String getFullname() { + return fullname; + } - public void setFullname(String fullname) { - this.fullname = fullname; - } + public void setFullname(String fullname) { + this.fullname = fullname; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getSurname() { - return surname; - } + public String getSurname() { + return surname; + } - public void setSurname(String surname) { - this.surname = surname; - } + public void setSurname(String surname) { + this.surname = surname; + } - public Integer getRank() { - return rank; - } + public Integer getRank() { + return rank; + } - public void setRank(Integer rank) { - this.rank = rank; - } + public void setRank(Integer rank) { + this.rank = rank; + } - public List getPid() { - return pid; - } + public List getPid() { + return pid; + } - public void setPid(List pid) { - this.pid = pid; - } + public void setPid(List pid) { + this.pid = pid; + } - public List> getAffiliation() { - return affiliation; - } + public List> getAffiliation() { + return affiliation; + } - public void setAffiliation(List> affiliation) { - this.affiliation = affiliation; - } + public void setAffiliation(List> affiliation) { + this.affiliation = affiliation; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Author author = (Author) o; - return Objects.equals(fullname, author.fullname) - && Objects.equals(name, author.name) - && Objects.equals(surname, author.surname) - && Objects.equals(rank, author.rank) - && Objects.equals(pid, author.pid) - && Objects.equals(affiliation, author.affiliation); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + Author author = (Author) o; + return Objects.equals(fullname, author.fullname) + && Objects.equals(name, author.name) + && Objects.equals(surname, author.surname) + && Objects.equals(rank, author.rank) + && Objects.equals(pid, author.pid) + && Objects.equals(affiliation, author.affiliation); + } - @Override - public int hashCode() { - return Objects.hash(fullname, name, surname, rank, pid, affiliation); - } + @Override + public int hashCode() { + return Objects.hash(fullname, name, surname, rank, pid, affiliation); + } - public void addPid(StructuredProperty pid) { + public void addPid(StructuredProperty pid) { - if (pid == null) return; + if (pid == null) + return; - if (this.pid == null) { - this.pid = Arrays.asList(pid); - } else { - this.pid.add(pid); - } - } + if (this.pid == null) { + this.pid = Arrays.asList(pid); + } else { + this.pid.add(pid); + } + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java index 7d930630d..57912c463 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Context.java @@ -1,42 +1,46 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; import java.util.List; public class Context implements Serializable { - private String id; + private String id; - private List dataInfo; + private List dataInfo; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public List getDataInfo() { - return dataInfo; - } + public List getDataInfo() { + return dataInfo; + } - public void setDataInfo(List dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(List dataInfo) { + this.dataInfo = dataInfo; + } - @Override - public int hashCode() { - return id == null ? 0 : id.hashCode(); - } + @Override + public int hashCode() { + return id == null ? 0 : id.hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - Context other = (Context) obj; + Context other = (Context) obj; - return id.equals(other.getId()); - } + return id.equals(other.getId()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Country.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Country.java index 388b9aab6..e25fdcade 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Country.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Country.java @@ -1,30 +1,34 @@ + package eu.dnetlib.dhp.schema.oaf; import java.util.Objects; public class Country extends Qualifier { - private DataInfo dataInfo; + private DataInfo dataInfo; - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - Country country = (Country) o; - return Objects.equals(dataInfo, country.dataInfo); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + Country country = (Country) o; + return Objects.equals(dataInfo, country.dataInfo); + } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), dataInfo); - } + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), dataInfo); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/DataInfo.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/DataInfo.java index f65518a1f..cc77e1ea0 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/DataInfo.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/DataInfo.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,77 +6,80 @@ import java.util.Objects; public class DataInfo implements Serializable { - private Boolean invisible = false; - private Boolean inferred; - private Boolean deletedbyinference; - private String trust; - private String inferenceprovenance; - private Qualifier provenanceaction; + private Boolean invisible = false; + private Boolean inferred; + private Boolean deletedbyinference; + private String trust; + private String inferenceprovenance; + private Qualifier provenanceaction; - public Boolean getInvisible() { - return invisible; - } + public Boolean getInvisible() { + return invisible; + } - public void setInvisible(Boolean invisible) { - this.invisible = invisible; - } + public void setInvisible(Boolean invisible) { + this.invisible = invisible; + } - public Boolean getInferred() { - return inferred; - } + public Boolean getInferred() { + return inferred; + } - public void setInferred(Boolean inferred) { - this.inferred = inferred; - } + public void setInferred(Boolean inferred) { + this.inferred = inferred; + } - public Boolean getDeletedbyinference() { - return deletedbyinference; - } + public Boolean getDeletedbyinference() { + return deletedbyinference; + } - public void setDeletedbyinference(Boolean deletedbyinference) { - this.deletedbyinference = deletedbyinference; - } + public void setDeletedbyinference(Boolean deletedbyinference) { + this.deletedbyinference = deletedbyinference; + } - public String getTrust() { - return trust; - } + public String getTrust() { + return trust; + } - public void setTrust(String trust) { - this.trust = trust; - } + public void setTrust(String trust) { + this.trust = trust; + } - public String getInferenceprovenance() { - return inferenceprovenance; - } + public String getInferenceprovenance() { + return inferenceprovenance; + } - public void setInferenceprovenance(String inferenceprovenance) { - this.inferenceprovenance = inferenceprovenance; - } + public void setInferenceprovenance(String inferenceprovenance) { + this.inferenceprovenance = inferenceprovenance; + } - public Qualifier getProvenanceaction() { - return provenanceaction; - } + public Qualifier getProvenanceaction() { + return provenanceaction; + } - public void setProvenanceaction(Qualifier provenanceaction) { - this.provenanceaction = provenanceaction; - } + public void setProvenanceaction(Qualifier provenanceaction) { + this.provenanceaction = provenanceaction; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - DataInfo dataInfo = (DataInfo) o; - return Objects.equals(invisible, dataInfo.invisible) - && Objects.equals(inferred, dataInfo.inferred) - && Objects.equals(deletedbyinference, dataInfo.deletedbyinference) - && Objects.equals(trust, dataInfo.trust) - && Objects.equals(inferenceprovenance, dataInfo.inferenceprovenance) - && Objects.equals(provenanceaction, dataInfo.provenanceaction); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + DataInfo dataInfo = (DataInfo) o; + return Objects.equals(invisible, dataInfo.invisible) + && Objects.equals(inferred, dataInfo.inferred) + && Objects.equals(deletedbyinference, dataInfo.deletedbyinference) + && Objects.equals(trust, dataInfo.trust) + && Objects.equals(inferenceprovenance, dataInfo.inferenceprovenance) + && Objects.equals(provenanceaction, dataInfo.provenanceaction); + } - @Override - public int hashCode() { - return Objects.hash( - invisible, inferred, deletedbyinference, trust, inferenceprovenance, provenanceaction); - } + @Override + public int hashCode() { + return Objects + .hash( + invisible, inferred, deletedbyinference, trust, inferenceprovenance, provenanceaction); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java index 93b51f352..07ddbb00e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Dataset.java @@ -1,116 +1,115 @@ + package eu.dnetlib.dhp.schema.oaf; -import eu.dnetlib.dhp.schema.common.ModelConstants; import java.io.Serializable; import java.util.List; +import eu.dnetlib.dhp.schema.common.ModelConstants; + public class Dataset extends Result implements Serializable { - private Field storagedate; + private Field storagedate; - private Field device; + private Field device; - private Field size; + private Field size; - private Field version; + private Field version; - private Field lastmetadataupdate; + private Field lastmetadataupdate; - private Field metadataversionnumber; + private Field metadataversionnumber; - private List geolocation; + private List geolocation; - public Dataset() { - setResulttype(ModelConstants.DATASET_DEFAULT_RESULTTYPE); - } + public Dataset() { + setResulttype(ModelConstants.DATASET_DEFAULT_RESULTTYPE); + } - public Field getStoragedate() { - return storagedate; - } + public Field getStoragedate() { + return storagedate; + } - public void setStoragedate(Field storagedate) { - this.storagedate = storagedate; - } + public void setStoragedate(Field storagedate) { + this.storagedate = storagedate; + } - public Field getDevice() { - return device; - } + public Field getDevice() { + return device; + } - public void setDevice(Field device) { - this.device = device; - } + public void setDevice(Field device) { + this.device = device; + } - public Field getSize() { - return size; - } + public Field getSize() { + return size; + } - public void setSize(Field size) { - this.size = size; - } + public void setSize(Field size) { + this.size = size; + } - public Field getVersion() { - return version; - } + public Field getVersion() { + return version; + } - public void setVersion(Field version) { - this.version = version; - } + public void setVersion(Field version) { + this.version = version; + } - public Field getLastmetadataupdate() { - return lastmetadataupdate; - } + public Field getLastmetadataupdate() { + return lastmetadataupdate; + } - public void setLastmetadataupdate(Field lastmetadataupdate) { - this.lastmetadataupdate = lastmetadataupdate; - } + public void setLastmetadataupdate(Field lastmetadataupdate) { + this.lastmetadataupdate = lastmetadataupdate; + } - public Field getMetadataversionnumber() { - return metadataversionnumber; - } + public Field getMetadataversionnumber() { + return metadataversionnumber; + } - public void setMetadataversionnumber(Field metadataversionnumber) { - this.metadataversionnumber = metadataversionnumber; - } + public void setMetadataversionnumber(Field metadataversionnumber) { + this.metadataversionnumber = metadataversionnumber; + } - public List getGeolocation() { - return geolocation; - } + public List getGeolocation() { + return geolocation; + } - public void setGeolocation(List geolocation) { - this.geolocation = geolocation; - } + public void setGeolocation(List geolocation) { + this.geolocation = geolocation; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!Dataset.class.isAssignableFrom(e.getClass())) { - return; - } + if (!Dataset.class.isAssignableFrom(e.getClass())) { + return; + } - final Dataset d = (Dataset) e; + final Dataset d = (Dataset) e; - storagedate = - d.getStoragedate() != null && compareTrust(this, e) < 0 ? d.getStoragedate() : storagedate; + storagedate = d.getStoragedate() != null && compareTrust(this, e) < 0 ? d.getStoragedate() : storagedate; - device = d.getDevice() != null && compareTrust(this, e) < 0 ? d.getDevice() : device; + device = d.getDevice() != null && compareTrust(this, e) < 0 ? d.getDevice() : device; - size = d.getSize() != null && compareTrust(this, e) < 0 ? d.getSize() : size; + size = d.getSize() != null && compareTrust(this, e) < 0 ? d.getSize() : size; - version = d.getVersion() != null && compareTrust(this, e) < 0 ? d.getVersion() : version; + version = d.getVersion() != null && compareTrust(this, e) < 0 ? d.getVersion() : version; - lastmetadataupdate = - d.getLastmetadataupdate() != null && compareTrust(this, e) < 0 - ? d.getLastmetadataupdate() - : lastmetadataupdate; + lastmetadataupdate = d.getLastmetadataupdate() != null && compareTrust(this, e) < 0 + ? d.getLastmetadataupdate() + : lastmetadataupdate; - metadataversionnumber = - d.getMetadataversionnumber() != null && compareTrust(this, e) < 0 - ? d.getMetadataversionnumber() - : metadataversionnumber; + metadataversionnumber = d.getMetadataversionnumber() != null && compareTrust(this, e) < 0 + ? d.getMetadataversionnumber() + : metadataversionnumber; - geolocation = mergeLists(geolocation, d.getGeolocation()); + geolocation = mergeLists(geolocation, d.getGeolocation()); - mergeOAFDataInfo(d); - } + mergeOAFDataInfo(d); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java index f0c797631..721798206 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Datasource.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,494 +6,467 @@ import java.util.List; public class Datasource extends OafEntity implements Serializable { - private Qualifier datasourcetype; + private Qualifier datasourcetype; - private Qualifier openairecompatibility; + private Qualifier openairecompatibility; - private Field officialname; + private Field officialname; - private Field englishname; + private Field englishname; - private Field websiteurl; + private Field websiteurl; - private Field logourl; + private Field logourl; - private Field contactemail; + private Field contactemail; - private Field namespaceprefix; + private Field namespaceprefix; - private Field latitude; + private Field latitude; - private Field longitude; + private Field longitude; - private Field dateofvalidation; + private Field dateofvalidation; - private Field description; + private Field description; - private List subjects; + private List subjects; - // opendoar specific fields (od*) - private Field odnumberofitems; + // opendoar specific fields (od*) + private Field odnumberofitems; - private Field odnumberofitemsdate; + private Field odnumberofitemsdate; - private Field odpolicies; + private Field odpolicies; - private List> odlanguages; + private List> odlanguages; - private List> odcontenttypes; + private List> odcontenttypes; - private List> accessinfopackage; + private List> accessinfopackage; - // re3data fields - private Field releasestartdate; + // re3data fields + private Field releasestartdate; - private Field releaseenddate; + private Field releaseenddate; - private Field missionstatementurl; + private Field missionstatementurl; - private Field dataprovider; + private Field dataprovider; - private Field serviceprovider; + private Field serviceprovider; - // {open, restricted or closed} - private Field databaseaccesstype; + // {open, restricted or closed} + private Field databaseaccesstype; - // {open, restricted or closed} - private Field datauploadtype; + // {open, restricted or closed} + private Field datauploadtype; - // {feeRequired, registration, other} - private Field databaseaccessrestriction; + // {feeRequired, registration, other} + private Field databaseaccessrestriction; - // {feeRequired, registration, other} - private Field datauploadrestriction; + // {feeRequired, registration, other} + private Field datauploadrestriction; - private Field versioning; + private Field versioning; - private Field citationguidelineurl; + private Field citationguidelineurl; - // {yes, no, uknown} - private Field qualitymanagementkind; + // {yes, no, uknown} + private Field qualitymanagementkind; - private Field pidsystems; + private Field pidsystems; - private Field certificates; + private Field certificates; - private List policies; + private List policies; - private Journal journal; + private Journal journal; - public Qualifier getDatasourcetype() { - return datasourcetype; - } + public Qualifier getDatasourcetype() { + return datasourcetype; + } - public void setDatasourcetype(Qualifier datasourcetype) { - this.datasourcetype = datasourcetype; - } + public void setDatasourcetype(Qualifier datasourcetype) { + this.datasourcetype = datasourcetype; + } - public Qualifier getOpenairecompatibility() { - return openairecompatibility; - } + public Qualifier getOpenairecompatibility() { + return openairecompatibility; + } - public void setOpenairecompatibility(Qualifier openairecompatibility) { - this.openairecompatibility = openairecompatibility; - } + public void setOpenairecompatibility(Qualifier openairecompatibility) { + this.openairecompatibility = openairecompatibility; + } - public Field getOfficialname() { - return officialname; - } + public Field getOfficialname() { + return officialname; + } - public void setOfficialname(Field officialname) { - this.officialname = officialname; - } + public void setOfficialname(Field officialname) { + this.officialname = officialname; + } - public Field getEnglishname() { - return englishname; - } + public Field getEnglishname() { + return englishname; + } - public void setEnglishname(Field englishname) { - this.englishname = englishname; - } + public void setEnglishname(Field englishname) { + this.englishname = englishname; + } - public Field getWebsiteurl() { - return websiteurl; - } + public Field getWebsiteurl() { + return websiteurl; + } - public void setWebsiteurl(Field websiteurl) { - this.websiteurl = websiteurl; - } + public void setWebsiteurl(Field websiteurl) { + this.websiteurl = websiteurl; + } - public Field getLogourl() { - return logourl; - } + public Field getLogourl() { + return logourl; + } - public void setLogourl(Field logourl) { - this.logourl = logourl; - } + public void setLogourl(Field logourl) { + this.logourl = logourl; + } - public Field getContactemail() { - return contactemail; - } + public Field getContactemail() { + return contactemail; + } - public void setContactemail(Field contactemail) { - this.contactemail = contactemail; - } + public void setContactemail(Field contactemail) { + this.contactemail = contactemail; + } - public Field getNamespaceprefix() { - return namespaceprefix; - } + public Field getNamespaceprefix() { + return namespaceprefix; + } - public void setNamespaceprefix(Field namespaceprefix) { - this.namespaceprefix = namespaceprefix; - } + public void setNamespaceprefix(Field namespaceprefix) { + this.namespaceprefix = namespaceprefix; + } - public Field getLatitude() { - return latitude; - } + public Field getLatitude() { + return latitude; + } - public void setLatitude(Field latitude) { - this.latitude = latitude; - } + public void setLatitude(Field latitude) { + this.latitude = latitude; + } - public Field getLongitude() { - return longitude; - } + public Field getLongitude() { + return longitude; + } - public void setLongitude(Field longitude) { - this.longitude = longitude; - } + public void setLongitude(Field longitude) { + this.longitude = longitude; + } - public Field getDateofvalidation() { - return dateofvalidation; - } + public Field getDateofvalidation() { + return dateofvalidation; + } - public void setDateofvalidation(Field dateofvalidation) { - this.dateofvalidation = dateofvalidation; - } + public void setDateofvalidation(Field dateofvalidation) { + this.dateofvalidation = dateofvalidation; + } - public Field getDescription() { - return description; - } + public Field getDescription() { + return description; + } - public void setDescription(Field description) { - this.description = description; - } + public void setDescription(Field description) { + this.description = description; + } - public List getSubjects() { - return subjects; - } + public List getSubjects() { + return subjects; + } - public void setSubjects(List subjects) { - this.subjects = subjects; - } + public void setSubjects(List subjects) { + this.subjects = subjects; + } - public Field getOdnumberofitems() { - return odnumberofitems; - } + public Field getOdnumberofitems() { + return odnumberofitems; + } - public void setOdnumberofitems(Field odnumberofitems) { - this.odnumberofitems = odnumberofitems; - } + public void setOdnumberofitems(Field odnumberofitems) { + this.odnumberofitems = odnumberofitems; + } - public Field getOdnumberofitemsdate() { - return odnumberofitemsdate; - } + public Field getOdnumberofitemsdate() { + return odnumberofitemsdate; + } - public void setOdnumberofitemsdate(Field odnumberofitemsdate) { - this.odnumberofitemsdate = odnumberofitemsdate; - } + public void setOdnumberofitemsdate(Field odnumberofitemsdate) { + this.odnumberofitemsdate = odnumberofitemsdate; + } - public Field getOdpolicies() { - return odpolicies; - } + public Field getOdpolicies() { + return odpolicies; + } - public void setOdpolicies(Field odpolicies) { - this.odpolicies = odpolicies; - } + public void setOdpolicies(Field odpolicies) { + this.odpolicies = odpolicies; + } - public List> getOdlanguages() { - return odlanguages; - } + public List> getOdlanguages() { + return odlanguages; + } - public void setOdlanguages(List> odlanguages) { - this.odlanguages = odlanguages; - } + public void setOdlanguages(List> odlanguages) { + this.odlanguages = odlanguages; + } - public List> getOdcontenttypes() { - return odcontenttypes; - } + public List> getOdcontenttypes() { + return odcontenttypes; + } - public void setOdcontenttypes(List> odcontenttypes) { - this.odcontenttypes = odcontenttypes; - } + public void setOdcontenttypes(List> odcontenttypes) { + this.odcontenttypes = odcontenttypes; + } - public List> getAccessinfopackage() { - return accessinfopackage; - } + public List> getAccessinfopackage() { + return accessinfopackage; + } - public void setAccessinfopackage(List> accessinfopackage) { - this.accessinfopackage = accessinfopackage; - } + public void setAccessinfopackage(List> accessinfopackage) { + this.accessinfopackage = accessinfopackage; + } - public Field getReleasestartdate() { - return releasestartdate; - } + public Field getReleasestartdate() { + return releasestartdate; + } - public void setReleasestartdate(Field releasestartdate) { - this.releasestartdate = releasestartdate; - } + public void setReleasestartdate(Field releasestartdate) { + this.releasestartdate = releasestartdate; + } - public Field getReleaseenddate() { - return releaseenddate; - } + public Field getReleaseenddate() { + return releaseenddate; + } - public void setReleaseenddate(Field releaseenddate) { - this.releaseenddate = releaseenddate; - } + public void setReleaseenddate(Field releaseenddate) { + this.releaseenddate = releaseenddate; + } - public Field getMissionstatementurl() { - return missionstatementurl; - } + public Field getMissionstatementurl() { + return missionstatementurl; + } - public void setMissionstatementurl(Field missionstatementurl) { - this.missionstatementurl = missionstatementurl; - } + public void setMissionstatementurl(Field missionstatementurl) { + this.missionstatementurl = missionstatementurl; + } - public Field getDataprovider() { - return dataprovider; - } + public Field getDataprovider() { + return dataprovider; + } - public void setDataprovider(Field dataprovider) { - this.dataprovider = dataprovider; - } + public void setDataprovider(Field dataprovider) { + this.dataprovider = dataprovider; + } - public Field getServiceprovider() { - return serviceprovider; - } + public Field getServiceprovider() { + return serviceprovider; + } - public void setServiceprovider(Field serviceprovider) { - this.serviceprovider = serviceprovider; - } + public void setServiceprovider(Field serviceprovider) { + this.serviceprovider = serviceprovider; + } - public Field getDatabaseaccesstype() { - return databaseaccesstype; - } + public Field getDatabaseaccesstype() { + return databaseaccesstype; + } - public void setDatabaseaccesstype(Field databaseaccesstype) { - this.databaseaccesstype = databaseaccesstype; - } - - public Field getDatauploadtype() { - return datauploadtype; - } - - public void setDatauploadtype(Field datauploadtype) { - this.datauploadtype = datauploadtype; - } - - public Field getDatabaseaccessrestriction() { - return databaseaccessrestriction; - } - - public void setDatabaseaccessrestriction(Field databaseaccessrestriction) { - this.databaseaccessrestriction = databaseaccessrestriction; - } - - public Field getDatauploadrestriction() { - return datauploadrestriction; - } - - public void setDatauploadrestriction(Field datauploadrestriction) { - this.datauploadrestriction = datauploadrestriction; - } - - public Field getVersioning() { - return versioning; - } - - public void setVersioning(Field versioning) { - this.versioning = versioning; - } - - public Field getCitationguidelineurl() { - return citationguidelineurl; - } - - public void setCitationguidelineurl(Field citationguidelineurl) { - this.citationguidelineurl = citationguidelineurl; - } - - public Field getQualitymanagementkind() { - return qualitymanagementkind; - } - - public void setQualitymanagementkind(Field qualitymanagementkind) { - this.qualitymanagementkind = qualitymanagementkind; - } - - public Field getPidsystems() { - return pidsystems; - } - - public void setPidsystems(Field pidsystems) { - this.pidsystems = pidsystems; - } - - public Field getCertificates() { - return certificates; - } - - public void setCertificates(Field certificates) { - this.certificates = certificates; - } - - public List getPolicies() { - return policies; - } - - public void setPolicies(List policies) { - this.policies = policies; - } - - public Journal getJournal() { - return journal; - } - - public void setJournal(Journal journal) { - this.journal = journal; - } - - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); - - if (!Datasource.class.isAssignableFrom(e.getClass())) { - return; - } - - Datasource d = (Datasource) e; - - datasourcetype = - d.getDatasourcetype() != null && compareTrust(this, e) < 0 - ? d.getDatasourcetype() - : datasourcetype; - openairecompatibility = - d.getOpenairecompatibility() != null && compareTrust(this, e) < 0 - ? d.getOpenairecompatibility() - : openairecompatibility; - officialname = - d.getOfficialname() != null && compareTrust(this, e) < 0 - ? d.getOfficialname() - : officialname; - englishname = - d.getEnglishname() != null && compareTrust(this, e) < 0 ? d.getEnglishname() : officialname; - websiteurl = - d.getWebsiteurl() != null && compareTrust(this, e) < 0 ? d.getWebsiteurl() : websiteurl; - logourl = d.getLogourl() != null && compareTrust(this, e) < 0 ? d.getLogourl() : getLogourl(); - contactemail = - d.getContactemail() != null && compareTrust(this, e) < 0 - ? d.getContactemail() - : contactemail; - namespaceprefix = - d.getNamespaceprefix() != null && compareTrust(this, e) < 0 - ? d.getNamespaceprefix() - : namespaceprefix; - latitude = d.getLatitude() != null && compareTrust(this, e) < 0 ? d.getLatitude() : latitude; - longitude = - d.getLongitude() != null && compareTrust(this, e) < 0 ? d.getLongitude() : longitude; - dateofvalidation = - d.getDateofvalidation() != null && compareTrust(this, e) < 0 - ? d.getDateofvalidation() - : dateofvalidation; - description = - d.getDescription() != null && compareTrust(this, e) < 0 ? d.getDescription() : description; - subjects = mergeLists(subjects, d.getSubjects()); - - // opendoar specific fields (od*) - odnumberofitems = - d.getOdnumberofitems() != null && compareTrust(this, e) < 0 - ? d.getOdnumberofitems() - : odnumberofitems; - odnumberofitemsdate = - d.getOdnumberofitemsdate() != null && compareTrust(this, e) < 0 - ? d.getOdnumberofitemsdate() - : odnumberofitemsdate; - odpolicies = - d.getOdpolicies() != null && compareTrust(this, e) < 0 ? d.getOdpolicies() : odpolicies; - odlanguages = mergeLists(odlanguages, d.getOdlanguages()); - odcontenttypes = mergeLists(odcontenttypes, d.getOdcontenttypes()); - accessinfopackage = mergeLists(accessinfopackage, d.getAccessinfopackage()); - - // re3data fields - releasestartdate = - d.getReleasestartdate() != null && compareTrust(this, e) < 0 - ? d.getReleasestartdate() - : releasestartdate; - releaseenddate = - d.getReleaseenddate() != null && compareTrust(this, e) < 0 - ? d.getReleaseenddate() - : releaseenddate; - missionstatementurl = - d.getMissionstatementurl() != null && compareTrust(this, e) < 0 - ? d.getMissionstatementurl() - : missionstatementurl; - dataprovider = - d.getDataprovider() != null && compareTrust(this, e) < 0 - ? d.getDataprovider() - : dataprovider; - serviceprovider = - d.getServiceprovider() != null && compareTrust(this, e) < 0 - ? d.getServiceprovider() - : serviceprovider; - - // {open, restricted or closed} - databaseaccesstype = - d.getDatabaseaccesstype() != null && compareTrust(this, e) < 0 - ? d.getDatabaseaccesstype() - : databaseaccesstype; - - // {open, restricted or closed} - datauploadtype = - d.getDatauploadtype() != null && compareTrust(this, e) < 0 - ? d.getDatauploadtype() - : datauploadtype; - - // {feeRequired, registration, other} - databaseaccessrestriction = - d.getDatabaseaccessrestriction() != null && compareTrust(this, e) < 0 - ? d.getDatabaseaccessrestriction() - : databaseaccessrestriction; - - // {feeRequired, registration, other} - datauploadrestriction = - d.getDatauploadrestriction() != null && compareTrust(this, e) < 0 - ? d.getDatauploadrestriction() - : datauploadrestriction; - - versioning = - d.getVersioning() != null && compareTrust(this, e) < 0 ? d.getVersioning() : versioning; - citationguidelineurl = - d.getCitationguidelineurl() != null && compareTrust(this, e) < 0 - ? d.getCitationguidelineurl() - : citationguidelineurl; - - // {yes, no, unknown} - qualitymanagementkind = - d.getQualitymanagementkind() != null && compareTrust(this, e) < 0 - ? d.getQualitymanagementkind() - : qualitymanagementkind; - pidsystems = - d.getPidsystems() != null && compareTrust(this, e) < 0 ? d.getPidsystems() : pidsystems; - - certificates = - d.getCertificates() != null && compareTrust(this, e) < 0 - ? d.getCertificates() - : certificates; - - policies = mergeLists(policies, d.getPolicies()); - - journal = d.getJournal() != null && compareTrust(this, e) < 0 ? d.getJournal() : journal; - - mergeOAFDataInfo(e); - } + public void setDatabaseaccesstype(Field databaseaccesstype) { + this.databaseaccesstype = databaseaccesstype; + } + + public Field getDatauploadtype() { + return datauploadtype; + } + + public void setDatauploadtype(Field datauploadtype) { + this.datauploadtype = datauploadtype; + } + + public Field getDatabaseaccessrestriction() { + return databaseaccessrestriction; + } + + public void setDatabaseaccessrestriction(Field databaseaccessrestriction) { + this.databaseaccessrestriction = databaseaccessrestriction; + } + + public Field getDatauploadrestriction() { + return datauploadrestriction; + } + + public void setDatauploadrestriction(Field datauploadrestriction) { + this.datauploadrestriction = datauploadrestriction; + } + + public Field getVersioning() { + return versioning; + } + + public void setVersioning(Field versioning) { + this.versioning = versioning; + } + + public Field getCitationguidelineurl() { + return citationguidelineurl; + } + + public void setCitationguidelineurl(Field citationguidelineurl) { + this.citationguidelineurl = citationguidelineurl; + } + + public Field getQualitymanagementkind() { + return qualitymanagementkind; + } + + public void setQualitymanagementkind(Field qualitymanagementkind) { + this.qualitymanagementkind = qualitymanagementkind; + } + + public Field getPidsystems() { + return pidsystems; + } + + public void setPidsystems(Field pidsystems) { + this.pidsystems = pidsystems; + } + + public Field getCertificates() { + return certificates; + } + + public void setCertificates(Field certificates) { + this.certificates = certificates; + } + + public List getPolicies() { + return policies; + } + + public void setPolicies(List policies) { + this.policies = policies; + } + + public Journal getJournal() { + return journal; + } + + public void setJournal(Journal journal) { + this.journal = journal; + } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + if (!Datasource.class.isAssignableFrom(e.getClass())) { + return; + } + + Datasource d = (Datasource) e; + + datasourcetype = d.getDatasourcetype() != null && compareTrust(this, e) < 0 + ? d.getDatasourcetype() + : datasourcetype; + openairecompatibility = d.getOpenairecompatibility() != null && compareTrust(this, e) < 0 + ? d.getOpenairecompatibility() + : openairecompatibility; + officialname = d.getOfficialname() != null && compareTrust(this, e) < 0 + ? d.getOfficialname() + : officialname; + englishname = d.getEnglishname() != null && compareTrust(this, e) < 0 ? d.getEnglishname() : officialname; + websiteurl = d.getWebsiteurl() != null && compareTrust(this, e) < 0 ? d.getWebsiteurl() : websiteurl; + logourl = d.getLogourl() != null && compareTrust(this, e) < 0 ? d.getLogourl() : getLogourl(); + contactemail = d.getContactemail() != null && compareTrust(this, e) < 0 + ? d.getContactemail() + : contactemail; + namespaceprefix = d.getNamespaceprefix() != null && compareTrust(this, e) < 0 + ? d.getNamespaceprefix() + : namespaceprefix; + latitude = d.getLatitude() != null && compareTrust(this, e) < 0 ? d.getLatitude() : latitude; + longitude = d.getLongitude() != null && compareTrust(this, e) < 0 ? d.getLongitude() : longitude; + dateofvalidation = d.getDateofvalidation() != null && compareTrust(this, e) < 0 + ? d.getDateofvalidation() + : dateofvalidation; + description = d.getDescription() != null && compareTrust(this, e) < 0 ? d.getDescription() : description; + subjects = mergeLists(subjects, d.getSubjects()); + + // opendoar specific fields (od*) + odnumberofitems = d.getOdnumberofitems() != null && compareTrust(this, e) < 0 + ? d.getOdnumberofitems() + : odnumberofitems; + odnumberofitemsdate = d.getOdnumberofitemsdate() != null && compareTrust(this, e) < 0 + ? d.getOdnumberofitemsdate() + : odnumberofitemsdate; + odpolicies = d.getOdpolicies() != null && compareTrust(this, e) < 0 ? d.getOdpolicies() : odpolicies; + odlanguages = mergeLists(odlanguages, d.getOdlanguages()); + odcontenttypes = mergeLists(odcontenttypes, d.getOdcontenttypes()); + accessinfopackage = mergeLists(accessinfopackage, d.getAccessinfopackage()); + + // re3data fields + releasestartdate = d.getReleasestartdate() != null && compareTrust(this, e) < 0 + ? d.getReleasestartdate() + : releasestartdate; + releaseenddate = d.getReleaseenddate() != null && compareTrust(this, e) < 0 + ? d.getReleaseenddate() + : releaseenddate; + missionstatementurl = d.getMissionstatementurl() != null && compareTrust(this, e) < 0 + ? d.getMissionstatementurl() + : missionstatementurl; + dataprovider = d.getDataprovider() != null && compareTrust(this, e) < 0 + ? d.getDataprovider() + : dataprovider; + serviceprovider = d.getServiceprovider() != null && compareTrust(this, e) < 0 + ? d.getServiceprovider() + : serviceprovider; + + // {open, restricted or closed} + databaseaccesstype = d.getDatabaseaccesstype() != null && compareTrust(this, e) < 0 + ? d.getDatabaseaccesstype() + : databaseaccesstype; + + // {open, restricted or closed} + datauploadtype = d.getDatauploadtype() != null && compareTrust(this, e) < 0 + ? d.getDatauploadtype() + : datauploadtype; + + // {feeRequired, registration, other} + databaseaccessrestriction = d.getDatabaseaccessrestriction() != null && compareTrust(this, e) < 0 + ? d.getDatabaseaccessrestriction() + : databaseaccessrestriction; + + // {feeRequired, registration, other} + datauploadrestriction = d.getDatauploadrestriction() != null && compareTrust(this, e) < 0 + ? d.getDatauploadrestriction() + : datauploadrestriction; + + versioning = d.getVersioning() != null && compareTrust(this, e) < 0 ? d.getVersioning() : versioning; + citationguidelineurl = d.getCitationguidelineurl() != null && compareTrust(this, e) < 0 + ? d.getCitationguidelineurl() + : citationguidelineurl; + + // {yes, no, unknown} + qualitymanagementkind = d.getQualitymanagementkind() != null && compareTrust(this, e) < 0 + ? d.getQualitymanagementkind() + : qualitymanagementkind; + pidsystems = d.getPidsystems() != null && compareTrust(this, e) < 0 ? d.getPidsystems() : pidsystems; + + certificates = d.getCertificates() != null && compareTrust(this, e) < 0 + ? d.getCertificates() + : certificates; + + policies = mergeLists(policies, d.getPolicies()); + + journal = d.getJournal() != null && compareTrust(this, e) < 0 ? d.getJournal() : journal; + + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExternalReference.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExternalReference.java index 67b48ed16..d509b954e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExternalReference.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExternalReference.java @@ -1,115 +1,119 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; import java.util.Objects; public class ExternalReference implements Serializable { - // source - private String sitename; + // source + private String sitename; - // title - private String label; + // title + private String label; - // text() - private String url; + // text() + private String url; - // ?? not mapped yet ?? - private String description; + // ?? not mapped yet ?? + private String description; - // type - private Qualifier qualifier; + // type + private Qualifier qualifier; - // site internal identifier - private String refidentifier; + // site internal identifier + private String refidentifier; - // maps the oaf:reference/@query attribute - private String query; + // maps the oaf:reference/@query attribute + private String query; - // ExternalReferences might be also inferred - private DataInfo dataInfo; + // ExternalReferences might be also inferred + private DataInfo dataInfo; - public String getSitename() { - return sitename; - } + public String getSitename() { + return sitename; + } - public void setSitename(String sitename) { - this.sitename = sitename; - } + public void setSitename(String sitename) { + this.sitename = sitename; + } - public String getLabel() { - return label; - } + public String getLabel() { + return label; + } - public void setLabel(String label) { - this.label = label; - } + public void setLabel(String label) { + this.label = label; + } - public String getUrl() { - return url; - } + public String getUrl() { + return url; + } - public void setUrl(String url) { - this.url = url; - } + public void setUrl(String url) { + this.url = url; + } - public String getDescription() { - return description; - } + public String getDescription() { + return description; + } - public void setDescription(String description) { - this.description = description; - } + public void setDescription(String description) { + this.description = description; + } - public Qualifier getQualifier() { - return qualifier; - } + public Qualifier getQualifier() { + return qualifier; + } - public void setQualifier(Qualifier qualifier) { - this.qualifier = qualifier; - } + public void setQualifier(Qualifier qualifier) { + this.qualifier = qualifier; + } - public String getRefidentifier() { - return refidentifier; - } + public String getRefidentifier() { + return refidentifier; + } - public void setRefidentifier(String refidentifier) { - this.refidentifier = refidentifier; - } + public void setRefidentifier(String refidentifier) { + this.refidentifier = refidentifier; + } - public String getQuery() { - return query; - } + public String getQuery() { + return query; + } - public void setQuery(String query) { - this.query = query; - } + public void setQuery(String query) { + this.query = query; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ExternalReference that = (ExternalReference) o; - return Objects.equals(sitename, that.sitename) - && Objects.equals(label, that.label) - && Objects.equals(url, that.url) - && Objects.equals(description, that.description) - && Objects.equals(qualifier, that.qualifier) - && Objects.equals(refidentifier, that.refidentifier) - && Objects.equals(query, that.query) - && Objects.equals(dataInfo, that.dataInfo); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + ExternalReference that = (ExternalReference) o; + return Objects.equals(sitename, that.sitename) + && Objects.equals(label, that.label) + && Objects.equals(url, that.url) + && Objects.equals(description, that.description) + && Objects.equals(qualifier, that.qualifier) + && Objects.equals(refidentifier, that.refidentifier) + && Objects.equals(query, that.query) + && Objects.equals(dataInfo, that.dataInfo); + } - @Override - public int hashCode() { - return Objects.hash( - sitename, label, url, description, qualifier, refidentifier, query, dataInfo); - } + @Override + public int hashCode() { + return Objects + .hash( + sitename, label, url, description, qualifier, refidentifier, query, dataInfo); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExtraInfo.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExtraInfo.java index c19c08f5f..3682cc2aa 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExtraInfo.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/ExtraInfo.java @@ -1,74 +1,77 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; import java.util.Objects; public class ExtraInfo implements Serializable { - private String name; + private String name; - private String typology; + private String typology; - private String provenance; + private String provenance; - private String trust; + private String trust; - // json containing a Citation or Statistics - private String value; + // json containing a Citation or Statistics + private String value; - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getTypology() { - return typology; - } + public String getTypology() { + return typology; + } - public void setTypology(String typology) { - this.typology = typology; - } + public void setTypology(String typology) { + this.typology = typology; + } - public String getProvenance() { - return provenance; - } + public String getProvenance() { + return provenance; + } - public void setProvenance(String provenance) { - this.provenance = provenance; - } + public void setProvenance(String provenance) { + this.provenance = provenance; + } - public String getTrust() { - return trust; - } + public String getTrust() { + return trust; + } - public void setTrust(String trust) { - this.trust = trust; - } + public void setTrust(String trust) { + this.trust = trust; + } - public String getValue() { - return value; - } + public String getValue() { + return value; + } - public void setValue(String value) { - this.value = value; - } + public void setValue(String value) { + this.value = value; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ExtraInfo extraInfo = (ExtraInfo) o; - return Objects.equals(name, extraInfo.name) - && Objects.equals(typology, extraInfo.typology) - && Objects.equals(provenance, extraInfo.provenance) - && Objects.equals(trust, extraInfo.trust) - && Objects.equals(value, extraInfo.value); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + ExtraInfo extraInfo = (ExtraInfo) o; + return Objects.equals(name, extraInfo.name) + && Objects.equals(typology, extraInfo.typology) + && Objects.equals(provenance, extraInfo.provenance) + && Objects.equals(trust, extraInfo.trust) + && Objects.equals(value, extraInfo.value); + } - @Override - public int hashCode() { - return Objects.hash(name, typology, provenance, trust, value); - } + @Override + public int hashCode() { + return Objects.hash(name, typology, provenance, trust, value); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java index 1854b85c1..1a85c6842 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Field.java @@ -1,40 +1,44 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; public class Field implements Serializable { - private T value; + private T value; - private DataInfo dataInfo; + private DataInfo dataInfo; - public T getValue() { - return value; - } + public T getValue() { + return value; + } - public void setValue(T value) { - this.value = value; - } + public void setValue(T value) { + this.value = value; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - @Override - public int hashCode() { - return getValue() == null ? 0 : getValue().hashCode(); - } + @Override + public int hashCode() { + return getValue() == null ? 0 : getValue().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; - Field other = (Field) obj; - return getValue().equals(other.getValue()); - } + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Field other = (Field) obj; + return getValue().equals(other.getValue()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java index 741f19002..7ed313a59 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/GeoLocation.java @@ -1,69 +1,76 @@ + package eu.dnetlib.dhp.schema.oaf; -import com.fasterxml.jackson.annotation.JsonIgnore; import java.io.Serializable; + import org.apache.commons.lang3.StringUtils; +import com.fasterxml.jackson.annotation.JsonIgnore; + public class GeoLocation implements Serializable { - private String point; + private String point; - private String box; + private String box; - private String place; + private String place; - public String getPoint() { - return point; - } + public String getPoint() { + return point; + } - public void setPoint(String point) { - this.point = point; - } + public void setPoint(String point) { + this.point = point; + } - public String getBox() { - return box; - } + public String getBox() { + return box; + } - public void setBox(String box) { - this.box = box; - } + public void setBox(String box) { + this.box = box; + } - public String getPlace() { - return place; - } + public String getPlace() { + return place; + } - public void setPlace(String place) { - this.place = place; - } + public void setPlace(String place) { + this.place = place; + } - @JsonIgnore - public boolean isBlank() { - return StringUtils.isBlank(point) && StringUtils.isBlank(box) && StringUtils.isBlank(place); - } + @JsonIgnore + public boolean isBlank() { + return StringUtils.isBlank(point) && StringUtils.isBlank(box) && StringUtils.isBlank(place); + } - public String toComparableString() { - return isBlank() - ? "" - : String.format( - "%s::%s%s", - point != null ? point.toLowerCase() : "", - box != null ? box.toLowerCase() : "", - place != null ? place.toLowerCase() : ""); - } + public String toComparableString() { + return isBlank() + ? "" + : String + .format( + "%s::%s%s", + point != null ? point.toLowerCase() : "", + box != null ? box.toLowerCase() : "", + place != null ? place.toLowerCase() : ""); + } - @Override - public int hashCode() { - return toComparableString().hashCode(); - } + @Override + public int hashCode() { + return toComparableString().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - GeoLocation other = (GeoLocation) obj; + GeoLocation other = (GeoLocation) obj; - return toComparableString().equals(other.toComparableString()); - } + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java index 76b72cfbc..2b7d3846c 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Instance.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,143 +6,147 @@ import java.util.List; public class Instance implements Serializable { - private Field license; + private Field license; - private Qualifier accessright; + private Qualifier accessright; - private Qualifier instancetype; + private Qualifier instancetype; - private KeyValue hostedby; + private KeyValue hostedby; - private List url; + private List url; - // other research products specifc - private String distributionlocation; + // other research products specifc + private String distributionlocation; - private KeyValue collectedfrom; + private KeyValue collectedfrom; - private Field dateofacceptance; + private Field dateofacceptance; - // ( article | book ) processing charges. Defined here to cope with possible wrongly typed - // results - private Field processingchargeamount; + // ( article | book ) processing charges. Defined here to cope with possible wrongly typed + // results + private Field processingchargeamount; - // currency - alphabetic code describe in ISO-4217. Defined here to cope with possible wrongly - // typed results - private Field processingchargecurrency; + // currency - alphabetic code describe in ISO-4217. Defined here to cope with possible wrongly + // typed results + private Field processingchargecurrency; - private Field refereed; // peer-review status + private Field refereed; // peer-review status - public Field getLicense() { - return license; - } + public Field getLicense() { + return license; + } - public void setLicense(Field license) { - this.license = license; - } + public void setLicense(Field license) { + this.license = license; + } - public Qualifier getAccessright() { - return accessright; - } + public Qualifier getAccessright() { + return accessright; + } - public void setAccessright(Qualifier accessright) { - this.accessright = accessright; - } + public void setAccessright(Qualifier accessright) { + this.accessright = accessright; + } - public Qualifier getInstancetype() { - return instancetype; - } + public Qualifier getInstancetype() { + return instancetype; + } - public void setInstancetype(Qualifier instancetype) { - this.instancetype = instancetype; - } + public void setInstancetype(Qualifier instancetype) { + this.instancetype = instancetype; + } - public KeyValue getHostedby() { - return hostedby; - } + public KeyValue getHostedby() { + return hostedby; + } - public void setHostedby(KeyValue hostedby) { - this.hostedby = hostedby; - } + public void setHostedby(KeyValue hostedby) { + this.hostedby = hostedby; + } - public List getUrl() { - return url; - } + public List getUrl() { + return url; + } - public void setUrl(List url) { - this.url = url; - } + public void setUrl(List url) { + this.url = url; + } - public String getDistributionlocation() { - return distributionlocation; - } + public String getDistributionlocation() { + return distributionlocation; + } - public void setDistributionlocation(String distributionlocation) { - this.distributionlocation = distributionlocation; - } + public void setDistributionlocation(String distributionlocation) { + this.distributionlocation = distributionlocation; + } - public KeyValue getCollectedfrom() { - return collectedfrom; - } + public KeyValue getCollectedfrom() { + return collectedfrom; + } - public void setCollectedfrom(KeyValue collectedfrom) { - this.collectedfrom = collectedfrom; - } + public void setCollectedfrom(KeyValue collectedfrom) { + this.collectedfrom = collectedfrom; + } - public Field getDateofacceptance() { - return dateofacceptance; - } + public Field getDateofacceptance() { + return dateofacceptance; + } - public void setDateofacceptance(Field dateofacceptance) { - this.dateofacceptance = dateofacceptance; - } + public void setDateofacceptance(Field dateofacceptance) { + this.dateofacceptance = dateofacceptance; + } - public Field getProcessingchargeamount() { - return processingchargeamount; - } + public Field getProcessingchargeamount() { + return processingchargeamount; + } - public void setProcessingchargeamount(Field processingchargeamount) { - this.processingchargeamount = processingchargeamount; - } + public void setProcessingchargeamount(Field processingchargeamount) { + this.processingchargeamount = processingchargeamount; + } - public Field getProcessingchargecurrency() { - return processingchargecurrency; - } + public Field getProcessingchargecurrency() { + return processingchargecurrency; + } - public void setProcessingchargecurrency(Field processingchargecurrency) { - this.processingchargecurrency = processingchargecurrency; - } + public void setProcessingchargecurrency(Field processingchargecurrency) { + this.processingchargecurrency = processingchargecurrency; + } - public Field getRefereed() { - return refereed; - } + public Field getRefereed() { + return refereed; + } - public void setRefereed(Field refereed) { - this.refereed = refereed; - } + public void setRefereed(Field refereed) { + this.refereed = refereed; + } - public String toComparableString() { - return String.format( - "%s::%s::%s::%s", - hostedby != null && hostedby.getKey() != null ? hostedby.getKey().toLowerCase() : "", - accessright != null && accessright.getClassid() != null ? accessright.getClassid() : "", - instancetype != null && instancetype.getClassid() != null ? instancetype.getClassid() : "", - url != null ? url : ""); - } + public String toComparableString() { + return String + .format( + "%s::%s::%s::%s", + hostedby != null && hostedby.getKey() != null ? hostedby.getKey().toLowerCase() : "", + accessright != null && accessright.getClassid() != null ? accessright.getClassid() : "", + instancetype != null && instancetype.getClassid() != null ? instancetype.getClassid() : "", + url != null ? url : ""); + } - @Override - public int hashCode() { - return toComparableString().hashCode(); - } + @Override + public int hashCode() { + return toComparableString().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - Instance other = (Instance) obj; + Instance other = (Instance) obj; - return toComparableString().equals(other.toComparableString()); - } + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Journal.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Journal.java index bdf64f812..7a375e28b 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Journal.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Journal.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,159 +6,162 @@ import java.util.Objects; public class Journal implements Serializable { - private String name; + private String name; - private String issnPrinted; + private String issnPrinted; - private String issnOnline; + private String issnOnline; - private String issnLinking; + private String issnLinking; - private String ep; + private String ep; - private String iss; + private String iss; - private String sp; + private String sp; - private String vol; + private String vol; - private String edition; + private String edition; - private String conferenceplace; + private String conferenceplace; - private String conferencedate; + private String conferencedate; - private DataInfo dataInfo; + private DataInfo dataInfo; - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getIssnPrinted() { - return issnPrinted; - } + public String getIssnPrinted() { + return issnPrinted; + } - public void setIssnPrinted(String issnPrinted) { - this.issnPrinted = issnPrinted; - } + public void setIssnPrinted(String issnPrinted) { + this.issnPrinted = issnPrinted; + } - public String getIssnOnline() { - return issnOnline; - } + public String getIssnOnline() { + return issnOnline; + } - public void setIssnOnline(String issnOnline) { - this.issnOnline = issnOnline; - } + public void setIssnOnline(String issnOnline) { + this.issnOnline = issnOnline; + } - public String getIssnLinking() { - return issnLinking; - } + public String getIssnLinking() { + return issnLinking; + } - public void setIssnLinking(String issnLinking) { - this.issnLinking = issnLinking; - } + public void setIssnLinking(String issnLinking) { + this.issnLinking = issnLinking; + } - public String getEp() { - return ep; - } + public String getEp() { + return ep; + } - public void setEp(String ep) { - this.ep = ep; - } + public void setEp(String ep) { + this.ep = ep; + } - public String getIss() { - return iss; - } + public String getIss() { + return iss; + } - public void setIss(String iss) { - this.iss = iss; - } + public void setIss(String iss) { + this.iss = iss; + } - public String getSp() { - return sp; - } + public String getSp() { + return sp; + } - public void setSp(String sp) { - this.sp = sp; - } + public void setSp(String sp) { + this.sp = sp; + } - public String getVol() { - return vol; - } + public String getVol() { + return vol; + } - public void setVol(String vol) { - this.vol = vol; - } + public void setVol(String vol) { + this.vol = vol; + } - public String getEdition() { - return edition; - } + public String getEdition() { + return edition; + } - public void setEdition(String edition) { - this.edition = edition; - } + public void setEdition(String edition) { + this.edition = edition; + } - public String getConferenceplace() { - return conferenceplace; - } + public String getConferenceplace() { + return conferenceplace; + } - public void setConferenceplace(String conferenceplace) { - this.conferenceplace = conferenceplace; - } + public void setConferenceplace(String conferenceplace) { + this.conferenceplace = conferenceplace; + } - public String getConferencedate() { - return conferencedate; - } + public String getConferencedate() { + return conferencedate; + } - public void setConferencedate(String conferencedate) { - this.conferencedate = conferencedate; - } + public void setConferencedate(String conferencedate) { + this.conferencedate = conferencedate; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Journal journal = (Journal) o; - return Objects.equals(name, journal.name) - && Objects.equals(issnPrinted, journal.issnPrinted) - && Objects.equals(issnOnline, journal.issnOnline) - && Objects.equals(issnLinking, journal.issnLinking) - && Objects.equals(ep, journal.ep) - && Objects.equals(iss, journal.iss) - && Objects.equals(sp, journal.sp) - && Objects.equals(vol, journal.vol) - && Objects.equals(edition, journal.edition) - && Objects.equals(conferenceplace, journal.conferenceplace) - && Objects.equals(conferencedate, journal.conferencedate) - && Objects.equals(dataInfo, journal.dataInfo); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + Journal journal = (Journal) o; + return Objects.equals(name, journal.name) + && Objects.equals(issnPrinted, journal.issnPrinted) + && Objects.equals(issnOnline, journal.issnOnline) + && Objects.equals(issnLinking, journal.issnLinking) + && Objects.equals(ep, journal.ep) + && Objects.equals(iss, journal.iss) + && Objects.equals(sp, journal.sp) + && Objects.equals(vol, journal.vol) + && Objects.equals(edition, journal.edition) + && Objects.equals(conferenceplace, journal.conferenceplace) + && Objects.equals(conferencedate, journal.conferencedate) + && Objects.equals(dataInfo, journal.dataInfo); + } - @Override - public int hashCode() { - return Objects.hash( - name, - issnPrinted, - issnOnline, - issnLinking, - ep, - iss, - sp, - vol, - edition, - conferenceplace, - conferencedate, - dataInfo); - } + @Override + public int hashCode() { + return Objects + .hash( + name, + issnPrinted, + issnOnline, + issnLinking, + ep, + iss, + sp, + vol, + edition, + conferenceplace, + conferencedate, + dataInfo); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java index 31b898788..4e2d60138 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/KeyValue.java @@ -1,67 +1,74 @@ + package eu.dnetlib.dhp.schema.oaf; -import com.fasterxml.jackson.annotation.JsonIgnore; import java.io.Serializable; + import org.apache.commons.lang3.StringUtils; +import com.fasterxml.jackson.annotation.JsonIgnore; + public class KeyValue implements Serializable { - private String key; + private String key; - private String value; + private String value; - private DataInfo dataInfo; + private DataInfo dataInfo; - public String getKey() { - return key; - } + public String getKey() { + return key; + } - public void setKey(String key) { - this.key = key; - } + public void setKey(String key) { + this.key = key; + } - public String getValue() { - return value; - } + public String getValue() { + return value; + } - public void setValue(String value) { - this.value = value; - } + public void setValue(String value) { + this.value = value; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - public String toComparableString() { - return isBlank() - ? "" - : String.format( - "%s::%s", - key != null ? key.toLowerCase() : "", value != null ? value.toLowerCase() : ""); - } + public String toComparableString() { + return isBlank() + ? "" + : String + .format( + "%s::%s", + key != null ? key.toLowerCase() : "", value != null ? value.toLowerCase() : ""); + } - @JsonIgnore - public boolean isBlank() { - return StringUtils.isBlank(key) && StringUtils.isBlank(value); - } + @JsonIgnore + public boolean isBlank() { + return StringUtils.isBlank(key) && StringUtils.isBlank(value); + } - @Override - public int hashCode() { - return toComparableString().hashCode(); - } + @Override + public int hashCode() { + return toComparableString().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - KeyValue other = (KeyValue) obj; + KeyValue other = (KeyValue) obj; - return toComparableString().equals(other.toComparableString()); - } + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OAIProvenance.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OAIProvenance.java index 5798adae9..88d74afbf 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OAIProvenance.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OAIProvenance.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,26 +6,28 @@ import java.util.Objects; public class OAIProvenance implements Serializable { - private OriginDescription originDescription; + private OriginDescription originDescription; - public OriginDescription getOriginDescription() { - return originDescription; - } + public OriginDescription getOriginDescription() { + return originDescription; + } - public void setOriginDescription(OriginDescription originDescription) { - this.originDescription = originDescription; - } + public void setOriginDescription(OriginDescription originDescription) { + this.originDescription = originDescription; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - OAIProvenance that = (OAIProvenance) o; - return Objects.equals(originDescription, that.originDescription); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + OAIProvenance that = (OAIProvenance) o; + return Objects.equals(originDescription, that.originDescription); + } - @Override - public int hashCode() { - return Objects.hash(originDescription); - } + @Override + public int hashCode() { + return Objects.hash(originDescription); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java index d6561f5cb..4bfc05039 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Oaf.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -6,60 +7,64 @@ import java.util.Objects; public abstract class Oaf implements Serializable { - protected List collectedfrom; + protected List collectedfrom; - private DataInfo dataInfo; + private DataInfo dataInfo; - private Long lastupdatetimestamp; + private Long lastupdatetimestamp; - public List getCollectedfrom() { - return collectedfrom; - } + public List getCollectedfrom() { + return collectedfrom; + } - public void setCollectedfrom(List collectedfrom) { - this.collectedfrom = collectedfrom; - } + public void setCollectedfrom(List collectedfrom) { + this.collectedfrom = collectedfrom; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - public Long getLastupdatetimestamp() { - return lastupdatetimestamp; - } + public Long getLastupdatetimestamp() { + return lastupdatetimestamp; + } - public void setLastupdatetimestamp(Long lastupdatetimestamp) { - this.lastupdatetimestamp = lastupdatetimestamp; - } + public void setLastupdatetimestamp(Long lastupdatetimestamp) { + this.lastupdatetimestamp = lastupdatetimestamp; + } - public void mergeOAFDataInfo(Oaf e) { - if (e.getDataInfo() != null && compareTrust(this, e) < 0) dataInfo = e.getDataInfo(); - } + public void mergeOAFDataInfo(Oaf e) { + if (e.getDataInfo() != null && compareTrust(this, e) < 0) + dataInfo = e.getDataInfo(); + } - protected String extractTrust(Oaf e) { - if (e == null || e.getDataInfo() == null || e.getDataInfo().getTrust() == null) return "0.0"; - return e.getDataInfo().getTrust(); - } + protected String extractTrust(Oaf e) { + if (e == null || e.getDataInfo() == null || e.getDataInfo().getTrust() == null) + return "0.0"; + return e.getDataInfo().getTrust(); + } - protected int compareTrust(Oaf a, Oaf b) { - return extractTrust(a).compareTo(extractTrust(b)); - } + protected int compareTrust(Oaf a, Oaf b) { + return extractTrust(a).compareTo(extractTrust(b)); + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Oaf oaf = (Oaf) o; - return Objects.equals(dataInfo, oaf.dataInfo) - && Objects.equals(lastupdatetimestamp, oaf.lastupdatetimestamp); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + Oaf oaf = (Oaf) o; + return Objects.equals(dataInfo, oaf.dataInfo) + && Objects.equals(lastupdatetimestamp, oaf.lastupdatetimestamp); + } - @Override - public int hashCode() { - return Objects.hash(dataInfo, lastupdatetimestamp); - } + @Override + public int hashCode() { + return Objects.hash(dataInfo, lastupdatetimestamp); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java index fbc73bb0a..09742748d 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OafEntity.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -6,118 +7,123 @@ import java.util.stream.Collectors; public abstract class OafEntity extends Oaf implements Serializable { - private String id; + private String id; - private List originalId; + private List originalId; - private List pid; + private List pid; - private String dateofcollection; + private String dateofcollection; - private String dateoftransformation; + private String dateoftransformation; - private List extraInfo; + private List extraInfo; - private OAIProvenance oaiprovenance; + private OAIProvenance oaiprovenance; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public List getOriginalId() { - return originalId; - } + public List getOriginalId() { + return originalId; + } - public void setOriginalId(List originalId) { - this.originalId = originalId; - } + public void setOriginalId(List originalId) { + this.originalId = originalId; + } - public List getPid() { - return pid; - } + public List getPid() { + return pid; + } - public void setPid(List pid) { - this.pid = pid; - } + public void setPid(List pid) { + this.pid = pid; + } - public String getDateofcollection() { - return dateofcollection; - } + public String getDateofcollection() { + return dateofcollection; + } - public void setDateofcollection(String dateofcollection) { - this.dateofcollection = dateofcollection; - } + public void setDateofcollection(String dateofcollection) { + this.dateofcollection = dateofcollection; + } - public String getDateoftransformation() { - return dateoftransformation; - } + public String getDateoftransformation() { + return dateoftransformation; + } - public void setDateoftransformation(String dateoftransformation) { - this.dateoftransformation = dateoftransformation; - } + public void setDateoftransformation(String dateoftransformation) { + this.dateoftransformation = dateoftransformation; + } - public List getExtraInfo() { - return extraInfo; - } + public List getExtraInfo() { + return extraInfo; + } - public void setExtraInfo(List extraInfo) { - this.extraInfo = extraInfo; - } + public void setExtraInfo(List extraInfo) { + this.extraInfo = extraInfo; + } - public OAIProvenance getOaiprovenance() { - return oaiprovenance; - } + public OAIProvenance getOaiprovenance() { + return oaiprovenance; + } - public void setOaiprovenance(OAIProvenance oaiprovenance) { - this.oaiprovenance = oaiprovenance; - } + public void setOaiprovenance(OAIProvenance oaiprovenance) { + this.oaiprovenance = oaiprovenance; + } - public void mergeFrom(OafEntity e) { + public void mergeFrom(OafEntity e) { - if (e == null) return; + if (e == null) + return; - originalId = mergeLists(originalId, e.getOriginalId()); + originalId = mergeLists(originalId, e.getOriginalId()); - collectedfrom = mergeLists(collectedfrom, e.getCollectedfrom()); + collectedfrom = mergeLists(collectedfrom, e.getCollectedfrom()); - pid = mergeLists(pid, e.getPid()); + pid = mergeLists(pid, e.getPid()); - if (e.getDateofcollection() != null && compareTrust(this, e) < 0) - dateofcollection = e.getDateofcollection(); + if (e.getDateofcollection() != null && compareTrust(this, e) < 0) + dateofcollection = e.getDateofcollection(); - if (e.getDateoftransformation() != null && compareTrust(this, e) < 0) - dateoftransformation = e.getDateoftransformation(); + if (e.getDateoftransformation() != null && compareTrust(this, e) < 0) + dateoftransformation = e.getDateoftransformation(); - extraInfo = mergeLists(extraInfo, e.getExtraInfo()); + extraInfo = mergeLists(extraInfo, e.getExtraInfo()); - if (e.getOaiprovenance() != null && compareTrust(this, e) < 0) - oaiprovenance = e.getOaiprovenance(); - } + if (e.getOaiprovenance() != null && compareTrust(this, e) < 0) + oaiprovenance = e.getOaiprovenance(); + } - protected List mergeLists(final List... lists) { + protected List mergeLists(final List... lists) { - return Arrays.stream(lists) - .filter(Objects::nonNull) - .flatMap(List::stream) - .distinct() - .collect(Collectors.toList()); - } + return Arrays + .stream(lists) + .filter(Objects::nonNull) + .flatMap(List::stream) + .distinct() + .collect(Collectors.toList()); + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; - OafEntity oafEntity = (OafEntity) o; - return Objects.equals(id, oafEntity.id); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + if (!super.equals(o)) + return false; + OafEntity oafEntity = (OafEntity) o; + return Objects.equals(id, oafEntity.id); + } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), id); - } + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), id); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java index 4339ff5b4..a5f9bce30 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Organization.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,221 +6,209 @@ import java.util.List; public class Organization extends OafEntity implements Serializable { - private Field legalshortname; + private Field legalshortname; - private Field legalname; + private Field legalname; - private List> alternativeNames; + private List> alternativeNames; - private Field websiteurl; + private Field websiteurl; - private Field logourl; + private Field logourl; - private Field eclegalbody; + private Field eclegalbody; - private Field eclegalperson; + private Field eclegalperson; - private Field ecnonprofit; + private Field ecnonprofit; - private Field ecresearchorganization; + private Field ecresearchorganization; - private Field echighereducation; + private Field echighereducation; - private Field ecinternationalorganizationeurinterests; + private Field ecinternationalorganizationeurinterests; - private Field ecinternationalorganization; + private Field ecinternationalorganization; - private Field ecenterprise; + private Field ecenterprise; - private Field ecsmevalidated; + private Field ecsmevalidated; - private Field ecnutscode; + private Field ecnutscode; - private Qualifier country; + private Qualifier country; - public Field getLegalshortname() { - return legalshortname; - } + public Field getLegalshortname() { + return legalshortname; + } - public void setLegalshortname(Field legalshortname) { - this.legalshortname = legalshortname; - } + public void setLegalshortname(Field legalshortname) { + this.legalshortname = legalshortname; + } - public Field getLegalname() { - return legalname; - } + public Field getLegalname() { + return legalname; + } - public void setLegalname(Field legalname) { - this.legalname = legalname; - } + public void setLegalname(Field legalname) { + this.legalname = legalname; + } - public List> getAlternativeNames() { - return alternativeNames; - } + public List> getAlternativeNames() { + return alternativeNames; + } - public void setAlternativeNames(List> alternativeNames) { - this.alternativeNames = alternativeNames; - } + public void setAlternativeNames(List> alternativeNames) { + this.alternativeNames = alternativeNames; + } - public Field getWebsiteurl() { - return websiteurl; - } + public Field getWebsiteurl() { + return websiteurl; + } - public void setWebsiteurl(Field websiteurl) { - this.websiteurl = websiteurl; - } + public void setWebsiteurl(Field websiteurl) { + this.websiteurl = websiteurl; + } - public Field getLogourl() { - return logourl; - } + public Field getLogourl() { + return logourl; + } - public void setLogourl(Field logourl) { - this.logourl = logourl; - } + public void setLogourl(Field logourl) { + this.logourl = logourl; + } - public Field getEclegalbody() { - return eclegalbody; - } + public Field getEclegalbody() { + return eclegalbody; + } - public void setEclegalbody(Field eclegalbody) { - this.eclegalbody = eclegalbody; - } + public void setEclegalbody(Field eclegalbody) { + this.eclegalbody = eclegalbody; + } - public Field getEclegalperson() { - return eclegalperson; - } + public Field getEclegalperson() { + return eclegalperson; + } - public void setEclegalperson(Field eclegalperson) { - this.eclegalperson = eclegalperson; - } + public void setEclegalperson(Field eclegalperson) { + this.eclegalperson = eclegalperson; + } - public Field getEcnonprofit() { - return ecnonprofit; - } + public Field getEcnonprofit() { + return ecnonprofit; + } - public void setEcnonprofit(Field ecnonprofit) { - this.ecnonprofit = ecnonprofit; - } + public void setEcnonprofit(Field ecnonprofit) { + this.ecnonprofit = ecnonprofit; + } - public Field getEcresearchorganization() { - return ecresearchorganization; - } + public Field getEcresearchorganization() { + return ecresearchorganization; + } - public void setEcresearchorganization(Field ecresearchorganization) { - this.ecresearchorganization = ecresearchorganization; - } + public void setEcresearchorganization(Field ecresearchorganization) { + this.ecresearchorganization = ecresearchorganization; + } - public Field getEchighereducation() { - return echighereducation; - } + public Field getEchighereducation() { + return echighereducation; + } - public void setEchighereducation(Field echighereducation) { - this.echighereducation = echighereducation; - } + public void setEchighereducation(Field echighereducation) { + this.echighereducation = echighereducation; + } - public Field getEcinternationalorganizationeurinterests() { - return ecinternationalorganizationeurinterests; - } + public Field getEcinternationalorganizationeurinterests() { + return ecinternationalorganizationeurinterests; + } - public void setEcinternationalorganizationeurinterests( - Field ecinternationalorganizationeurinterests) { - this.ecinternationalorganizationeurinterests = ecinternationalorganizationeurinterests; - } + public void setEcinternationalorganizationeurinterests( + Field ecinternationalorganizationeurinterests) { + this.ecinternationalorganizationeurinterests = ecinternationalorganizationeurinterests; + } - public Field getEcinternationalorganization() { - return ecinternationalorganization; - } + public Field getEcinternationalorganization() { + return ecinternationalorganization; + } - public void setEcinternationalorganization(Field ecinternationalorganization) { - this.ecinternationalorganization = ecinternationalorganization; - } + public void setEcinternationalorganization(Field ecinternationalorganization) { + this.ecinternationalorganization = ecinternationalorganization; + } - public Field getEcenterprise() { - return ecenterprise; - } + public Field getEcenterprise() { + return ecenterprise; + } - public void setEcenterprise(Field ecenterprise) { - this.ecenterprise = ecenterprise; - } + public void setEcenterprise(Field ecenterprise) { + this.ecenterprise = ecenterprise; + } - public Field getEcsmevalidated() { - return ecsmevalidated; - } + public Field getEcsmevalidated() { + return ecsmevalidated; + } - public void setEcsmevalidated(Field ecsmevalidated) { - this.ecsmevalidated = ecsmevalidated; - } + public void setEcsmevalidated(Field ecsmevalidated) { + this.ecsmevalidated = ecsmevalidated; + } - public Field getEcnutscode() { - return ecnutscode; - } + public Field getEcnutscode() { + return ecnutscode; + } - public void setEcnutscode(Field ecnutscode) { - this.ecnutscode = ecnutscode; - } + public void setEcnutscode(Field ecnutscode) { + this.ecnutscode = ecnutscode; + } - public Qualifier getCountry() { - return country; - } + public Qualifier getCountry() { + return country; + } - public void setCountry(Qualifier country) { - this.country = country; - } + public void setCountry(Qualifier country) { + this.country = country; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!Organization.class.isAssignableFrom(e.getClass())) { - return; - } + if (!Organization.class.isAssignableFrom(e.getClass())) { + return; + } - final Organization o = (Organization) e; - legalshortname = - o.getLegalshortname() != null && compareTrust(this, e) < 0 - ? o.getLegalshortname() - : legalshortname; - legalname = - o.getLegalname() != null && compareTrust(this, e) < 0 ? o.getLegalname() : legalname; - alternativeNames = mergeLists(o.getAlternativeNames(), alternativeNames); - websiteurl = - o.getWebsiteurl() != null && compareTrust(this, e) < 0 ? o.getWebsiteurl() : websiteurl; - logourl = o.getLogourl() != null && compareTrust(this, e) < 0 ? o.getLogourl() : logourl; - eclegalbody = - o.getEclegalbody() != null && compareTrust(this, e) < 0 ? o.getEclegalbody() : eclegalbody; - eclegalperson = - o.getEclegalperson() != null && compareTrust(this, e) < 0 - ? o.getEclegalperson() - : eclegalperson; - ecnonprofit = - o.getEcnonprofit() != null && compareTrust(this, e) < 0 ? o.getEcnonprofit() : ecnonprofit; - ecresearchorganization = - o.getEcresearchorganization() != null && compareTrust(this, e) < 0 - ? o.getEcresearchorganization() - : ecresearchorganization; - echighereducation = - o.getEchighereducation() != null && compareTrust(this, e) < 0 - ? o.getEchighereducation() - : echighereducation; - ecinternationalorganizationeurinterests = - o.getEcinternationalorganizationeurinterests() != null && compareTrust(this, e) < 0 - ? o.getEcinternationalorganizationeurinterests() - : ecinternationalorganizationeurinterests; - ecinternationalorganization = - o.getEcinternationalorganization() != null && compareTrust(this, e) < 0 - ? o.getEcinternationalorganization() - : ecinternationalorganization; - ecenterprise = - o.getEcenterprise() != null && compareTrust(this, e) < 0 - ? o.getEcenterprise() - : ecenterprise; - ecsmevalidated = - o.getEcsmevalidated() != null && compareTrust(this, e) < 0 - ? o.getEcsmevalidated() - : ecsmevalidated; - ecnutscode = - o.getEcnutscode() != null && compareTrust(this, e) < 0 ? o.getEcnutscode() : ecnutscode; - country = o.getCountry() != null && compareTrust(this, e) < 0 ? o.getCountry() : country; - mergeOAFDataInfo(o); - } + final Organization o = (Organization) e; + legalshortname = o.getLegalshortname() != null && compareTrust(this, e) < 0 + ? o.getLegalshortname() + : legalshortname; + legalname = o.getLegalname() != null && compareTrust(this, e) < 0 ? o.getLegalname() : legalname; + alternativeNames = mergeLists(o.getAlternativeNames(), alternativeNames); + websiteurl = o.getWebsiteurl() != null && compareTrust(this, e) < 0 ? o.getWebsiteurl() : websiteurl; + logourl = o.getLogourl() != null && compareTrust(this, e) < 0 ? o.getLogourl() : logourl; + eclegalbody = o.getEclegalbody() != null && compareTrust(this, e) < 0 ? o.getEclegalbody() : eclegalbody; + eclegalperson = o.getEclegalperson() != null && compareTrust(this, e) < 0 + ? o.getEclegalperson() + : eclegalperson; + ecnonprofit = o.getEcnonprofit() != null && compareTrust(this, e) < 0 ? o.getEcnonprofit() : ecnonprofit; + ecresearchorganization = o.getEcresearchorganization() != null && compareTrust(this, e) < 0 + ? o.getEcresearchorganization() + : ecresearchorganization; + echighereducation = o.getEchighereducation() != null && compareTrust(this, e) < 0 + ? o.getEchighereducation() + : echighereducation; + ecinternationalorganizationeurinterests = o.getEcinternationalorganizationeurinterests() != null + && compareTrust(this, e) < 0 + ? o.getEcinternationalorganizationeurinterests() + : ecinternationalorganizationeurinterests; + ecinternationalorganization = o.getEcinternationalorganization() != null && compareTrust(this, e) < 0 + ? o.getEcinternationalorganization() + : ecinternationalorganization; + ecenterprise = o.getEcenterprise() != null && compareTrust(this, e) < 0 + ? o.getEcenterprise() + : ecenterprise; + ecsmevalidated = o.getEcsmevalidated() != null && compareTrust(this, e) < 0 + ? o.getEcsmevalidated() + : ecsmevalidated; + ecnutscode = o.getEcnutscode() != null && compareTrust(this, e) < 0 ? o.getEcnutscode() : ecnutscode; + country = o.getCountry() != null && compareTrust(this, e) < 0 ? o.getCountry() : country; + mergeOAFDataInfo(o); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OriginDescription.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OriginDescription.java index 5bdabb558..a275fc1a9 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OriginDescription.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OriginDescription.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,81 +6,83 @@ import java.util.Objects; public class OriginDescription implements Serializable { - private String harvestDate; + private String harvestDate; - private Boolean altered = true; + private Boolean altered = true; - private String baseURL; + private String baseURL; - private String identifier; + private String identifier; - private String datestamp; + private String datestamp; - private String metadataNamespace; + private String metadataNamespace; - public String getHarvestDate() { - return harvestDate; - } + public String getHarvestDate() { + return harvestDate; + } - public void setHarvestDate(String harvestDate) { - this.harvestDate = harvestDate; - } + public void setHarvestDate(String harvestDate) { + this.harvestDate = harvestDate; + } - public Boolean getAltered() { - return altered; - } + public Boolean getAltered() { + return altered; + } - public void setAltered(Boolean altered) { - this.altered = altered; - } + public void setAltered(Boolean altered) { + this.altered = altered; + } - public String getBaseURL() { - return baseURL; - } + public String getBaseURL() { + return baseURL; + } - public void setBaseURL(String baseURL) { - this.baseURL = baseURL; - } + public void setBaseURL(String baseURL) { + this.baseURL = baseURL; + } - public String getIdentifier() { - return identifier; - } + public String getIdentifier() { + return identifier; + } - public void setIdentifier(String identifier) { - this.identifier = identifier; - } + public void setIdentifier(String identifier) { + this.identifier = identifier; + } - public String getDatestamp() { - return datestamp; - } + public String getDatestamp() { + return datestamp; + } - public void setDatestamp(String datestamp) { - this.datestamp = datestamp; - } + public void setDatestamp(String datestamp) { + this.datestamp = datestamp; + } - public String getMetadataNamespace() { - return metadataNamespace; - } + public String getMetadataNamespace() { + return metadataNamespace; + } - public void setMetadataNamespace(String metadataNamespace) { - this.metadataNamespace = metadataNamespace; - } + public void setMetadataNamespace(String metadataNamespace) { + this.metadataNamespace = metadataNamespace; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - OriginDescription that = (OriginDescription) o; - return Objects.equals(harvestDate, that.harvestDate) - && Objects.equals(altered, that.altered) - && Objects.equals(baseURL, that.baseURL) - && Objects.equals(identifier, that.identifier) - && Objects.equals(datestamp, that.datestamp) - && Objects.equals(metadataNamespace, that.metadataNamespace); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + OriginDescription that = (OriginDescription) o; + return Objects.equals(harvestDate, that.harvestDate) + && Objects.equals(altered, that.altered) + && Objects.equals(baseURL, that.baseURL) + && Objects.equals(identifier, that.identifier) + && Objects.equals(datestamp, that.datestamp) + && Objects.equals(metadataNamespace, that.metadataNamespace); + } - @Override - public int hashCode() { - return Objects.hash(harvestDate, altered, baseURL, identifier, datestamp, metadataNamespace); - } + @Override + public int hashCode() { + return Objects.hash(harvestDate, altered, baseURL, identifier, datestamp, metadataNamespace); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java index 6cd803ec5..b04934c23 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/OtherResearchProduct.java @@ -1,58 +1,60 @@ + package eu.dnetlib.dhp.schema.oaf; -import eu.dnetlib.dhp.schema.common.ModelConstants; import java.io.Serializable; import java.util.List; +import eu.dnetlib.dhp.schema.common.ModelConstants; + public class OtherResearchProduct extends Result implements Serializable { - private List> contactperson; + private List> contactperson; - private List> contactgroup; + private List> contactgroup; - private List> tool; + private List> tool; - public OtherResearchProduct() { - setResulttype(ModelConstants.ORP_DEFAULT_RESULTTYPE); - } + public OtherResearchProduct() { + setResulttype(ModelConstants.ORP_DEFAULT_RESULTTYPE); + } - public List> getContactperson() { - return contactperson; - } + public List> getContactperson() { + return contactperson; + } - public void setContactperson(List> contactperson) { - this.contactperson = contactperson; - } + public void setContactperson(List> contactperson) { + this.contactperson = contactperson; + } - public List> getContactgroup() { - return contactgroup; - } + public List> getContactgroup() { + return contactgroup; + } - public void setContactgroup(List> contactgroup) { - this.contactgroup = contactgroup; - } + public void setContactgroup(List> contactgroup) { + this.contactgroup = contactgroup; + } - public List> getTool() { - return tool; - } + public List> getTool() { + return tool; + } - public void setTool(List> tool) { - this.tool = tool; - } + public void setTool(List> tool) { + this.tool = tool; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!OtherResearchProduct.class.isAssignableFrom(e.getClass())) { - return; - } + if (!OtherResearchProduct.class.isAssignableFrom(e.getClass())) { + return; + } - OtherResearchProduct o = (OtherResearchProduct) e; + OtherResearchProduct o = (OtherResearchProduct) e; - contactperson = mergeLists(contactperson, o.getContactperson()); - contactgroup = mergeLists(contactgroup, o.getContactgroup()); - tool = mergeLists(tool, o.getTool()); - mergeOAFDataInfo(e); - } + contactperson = mergeLists(contactperson, o.getContactperson()); + contactgroup = mergeLists(contactgroup, o.getContactgroup()); + tool = mergeLists(tool, o.getTool()); + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java index 2187a8828..924c08cc9 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Project.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -5,335 +6,320 @@ import java.util.List; public class Project extends OafEntity implements Serializable { - private Field websiteurl; + private Field websiteurl; - private Field code; + private Field code; - private Field acronym; + private Field acronym; - private Field title; + private Field title; - private Field startdate; + private Field startdate; - private Field enddate; + private Field enddate; - private Field callidentifier; + private Field callidentifier; - private Field keywords; + private Field keywords; - private Field duration; + private Field duration; - private Field ecsc39; + private Field ecsc39; - private Field oamandatepublications; + private Field oamandatepublications; - private Field ecarticle29_3; + private Field ecarticle29_3; - private List subjects; + private List subjects; - private List> fundingtree; + private List> fundingtree; - private Qualifier contracttype; + private Qualifier contracttype; - private Field optional1; + private Field optional1; - private Field optional2; + private Field optional2; - private Field jsonextrainfo; + private Field jsonextrainfo; - private Field contactfullname; + private Field contactfullname; - private Field contactfax; + private Field contactfax; - private Field contactphone; + private Field contactphone; - private Field contactemail; + private Field contactemail; - private Field summary; + private Field summary; - private Field currency; + private Field currency; - private Float totalcost; + private Float totalcost; - private Float fundedamount; + private Float fundedamount; - public Field getWebsiteurl() { - return websiteurl; - } + public Field getWebsiteurl() { + return websiteurl; + } - public void setWebsiteurl(Field websiteurl) { - this.websiteurl = websiteurl; - } + public void setWebsiteurl(Field websiteurl) { + this.websiteurl = websiteurl; + } - public Field getCode() { - return code; - } + public Field getCode() { + return code; + } - public void setCode(Field code) { - this.code = code; - } + public void setCode(Field code) { + this.code = code; + } - public Field getAcronym() { - return acronym; - } + public Field getAcronym() { + return acronym; + } - public void setAcronym(Field acronym) { - this.acronym = acronym; - } + public void setAcronym(Field acronym) { + this.acronym = acronym; + } - public Field getTitle() { - return title; - } + public Field getTitle() { + return title; + } - public void setTitle(Field title) { - this.title = title; - } + public void setTitle(Field title) { + this.title = title; + } - public Field getStartdate() { - return startdate; - } + public Field getStartdate() { + return startdate; + } - public void setStartdate(Field startdate) { - this.startdate = startdate; - } + public void setStartdate(Field startdate) { + this.startdate = startdate; + } - public Field getEnddate() { - return enddate; - } + public Field getEnddate() { + return enddate; + } - public void setEnddate(Field enddate) { - this.enddate = enddate; - } + public void setEnddate(Field enddate) { + this.enddate = enddate; + } - public Field getCallidentifier() { - return callidentifier; - } + public Field getCallidentifier() { + return callidentifier; + } - public void setCallidentifier(Field callidentifier) { - this.callidentifier = callidentifier; - } + public void setCallidentifier(Field callidentifier) { + this.callidentifier = callidentifier; + } - public Field getKeywords() { - return keywords; - } + public Field getKeywords() { + return keywords; + } - public void setKeywords(Field keywords) { - this.keywords = keywords; - } + public void setKeywords(Field keywords) { + this.keywords = keywords; + } - public Field getDuration() { - return duration; - } + public Field getDuration() { + return duration; + } - public void setDuration(Field duration) { - this.duration = duration; - } + public void setDuration(Field duration) { + this.duration = duration; + } - public Field getEcsc39() { - return ecsc39; - } + public Field getEcsc39() { + return ecsc39; + } - public void setEcsc39(Field ecsc39) { - this.ecsc39 = ecsc39; - } + public void setEcsc39(Field ecsc39) { + this.ecsc39 = ecsc39; + } - public Field getOamandatepublications() { - return oamandatepublications; - } + public Field getOamandatepublications() { + return oamandatepublications; + } - public void setOamandatepublications(Field oamandatepublications) { - this.oamandatepublications = oamandatepublications; - } + public void setOamandatepublications(Field oamandatepublications) { + this.oamandatepublications = oamandatepublications; + } - public Field getEcarticle29_3() { - return ecarticle29_3; - } + public Field getEcarticle29_3() { + return ecarticle29_3; + } - public void setEcarticle29_3(Field ecarticle29_3) { - this.ecarticle29_3 = ecarticle29_3; - } + public void setEcarticle29_3(Field ecarticle29_3) { + this.ecarticle29_3 = ecarticle29_3; + } - public List getSubjects() { - return subjects; - } + public List getSubjects() { + return subjects; + } - public void setSubjects(List subjects) { - this.subjects = subjects; - } + public void setSubjects(List subjects) { + this.subjects = subjects; + } - public List> getFundingtree() { - return fundingtree; - } + public List> getFundingtree() { + return fundingtree; + } - public void setFundingtree(List> fundingtree) { - this.fundingtree = fundingtree; - } + public void setFundingtree(List> fundingtree) { + this.fundingtree = fundingtree; + } - public Qualifier getContracttype() { - return contracttype; - } + public Qualifier getContracttype() { + return contracttype; + } - public void setContracttype(Qualifier contracttype) { - this.contracttype = contracttype; - } + public void setContracttype(Qualifier contracttype) { + this.contracttype = contracttype; + } - public Field getOptional1() { - return optional1; - } + public Field getOptional1() { + return optional1; + } - public void setOptional1(Field optional1) { - this.optional1 = optional1; - } - - public Field getOptional2() { - return optional2; - } - - public void setOptional2(Field optional2) { - this.optional2 = optional2; - } - - public Field getJsonextrainfo() { - return jsonextrainfo; - } - - public void setJsonextrainfo(Field jsonextrainfo) { - this.jsonextrainfo = jsonextrainfo; - } - - public Field getContactfullname() { - return contactfullname; - } - - public void setContactfullname(Field contactfullname) { - this.contactfullname = contactfullname; - } - - public Field getContactfax() { - return contactfax; - } - - public void setContactfax(Field contactfax) { - this.contactfax = contactfax; - } - - public Field getContactphone() { - return contactphone; - } - - public void setContactphone(Field contactphone) { - this.contactphone = contactphone; - } - - public Field getContactemail() { - return contactemail; - } - - public void setContactemail(Field contactemail) { - this.contactemail = contactemail; - } - - public Field getSummary() { - return summary; - } - - public void setSummary(Field summary) { - this.summary = summary; - } - - public Field getCurrency() { - return currency; - } - - public void setCurrency(Field currency) { - this.currency = currency; - } - - public Float getTotalcost() { - return totalcost; - } - - public void setTotalcost(Float totalcost) { - this.totalcost = totalcost; - } - - public Float getFundedamount() { - return fundedamount; - } - - public void setFundedamount(Float fundedamount) { - this.fundedamount = fundedamount; - } - - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); - - if (!Project.class.isAssignableFrom(e.getClass())) { - return; - } - - Project p = (Project) e; - - websiteurl = - p.getWebsiteurl() != null && compareTrust(this, e) < 0 ? p.getWebsiteurl() : websiteurl; - code = p.getCode() != null && compareTrust(this, e) < 0 ? p.getCode() : code; - acronym = p.getAcronym() != null && compareTrust(this, e) < 0 ? p.getAcronym() : acronym; - title = p.getTitle() != null && compareTrust(this, e) < 0 ? p.getTitle() : title; - startdate = - p.getStartdate() != null && compareTrust(this, e) < 0 ? p.getStartdate() : startdate; - enddate = p.getEnddate() != null && compareTrust(this, e) < 0 ? p.getEnddate() : enddate; - callidentifier = - p.getCallidentifier() != null && compareTrust(this, e) < 0 - ? p.getCallidentifier() - : callidentifier; - keywords = p.getKeywords() != null && compareTrust(this, e) < 0 ? p.getKeywords() : keywords; - duration = p.getDuration() != null && compareTrust(this, e) < 0 ? p.getDuration() : duration; - ecsc39 = p.getEcsc39() != null && compareTrust(this, e) < 0 ? p.getEcsc39() : ecsc39; - oamandatepublications = - p.getOamandatepublications() != null && compareTrust(this, e) < 0 - ? p.getOamandatepublications() - : oamandatepublications; - ecarticle29_3 = - p.getEcarticle29_3() != null && compareTrust(this, e) < 0 - ? p.getEcarticle29_3() - : ecarticle29_3; - subjects = mergeLists(subjects, p.getSubjects()); - fundingtree = mergeLists(fundingtree, p.getFundingtree()); - contracttype = - p.getContracttype() != null && compareTrust(this, e) < 0 - ? p.getContracttype() - : contracttype; - optional1 = - p.getOptional1() != null && compareTrust(this, e) < 0 ? p.getOptional1() : optional1; - optional2 = - p.getOptional2() != null && compareTrust(this, e) < 0 ? p.getOptional2() : optional2; - jsonextrainfo = - p.getJsonextrainfo() != null && compareTrust(this, e) < 0 - ? p.getJsonextrainfo() - : jsonextrainfo; - contactfullname = - p.getContactfullname() != null && compareTrust(this, e) < 0 - ? p.getContactfullname() - : contactfullname; - contactfax = - p.getContactfax() != null && compareTrust(this, e) < 0 ? p.getContactfax() : contactfax; - contactphone = - p.getContactphone() != null && compareTrust(this, e) < 0 - ? p.getContactphone() - : contactphone; - contactemail = - p.getContactemail() != null && compareTrust(this, e) < 0 - ? p.getContactemail() - : contactemail; - summary = p.getSummary() != null && compareTrust(this, e) < 0 ? p.getSummary() : summary; - currency = p.getCurrency() != null && compareTrust(this, e) < 0 ? p.getCurrency() : currency; - totalcost = - p.getTotalcost() != null && compareTrust(this, e) < 0 ? p.getTotalcost() : totalcost; - fundedamount = - p.getFundedamount() != null && compareTrust(this, e) < 0 - ? p.getFundedamount() - : fundedamount; - mergeOAFDataInfo(e); - } + public void setOptional1(Field optional1) { + this.optional1 = optional1; + } + + public Field getOptional2() { + return optional2; + } + + public void setOptional2(Field optional2) { + this.optional2 = optional2; + } + + public Field getJsonextrainfo() { + return jsonextrainfo; + } + + public void setJsonextrainfo(Field jsonextrainfo) { + this.jsonextrainfo = jsonextrainfo; + } + + public Field getContactfullname() { + return contactfullname; + } + + public void setContactfullname(Field contactfullname) { + this.contactfullname = contactfullname; + } + + public Field getContactfax() { + return contactfax; + } + + public void setContactfax(Field contactfax) { + this.contactfax = contactfax; + } + + public Field getContactphone() { + return contactphone; + } + + public void setContactphone(Field contactphone) { + this.contactphone = contactphone; + } + + public Field getContactemail() { + return contactemail; + } + + public void setContactemail(Field contactemail) { + this.contactemail = contactemail; + } + + public Field getSummary() { + return summary; + } + + public void setSummary(Field summary) { + this.summary = summary; + } + + public Field getCurrency() { + return currency; + } + + public void setCurrency(Field currency) { + this.currency = currency; + } + + public Float getTotalcost() { + return totalcost; + } + + public void setTotalcost(Float totalcost) { + this.totalcost = totalcost; + } + + public Float getFundedamount() { + return fundedamount; + } + + public void setFundedamount(Float fundedamount) { + this.fundedamount = fundedamount; + } + + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + + if (!Project.class.isAssignableFrom(e.getClass())) { + return; + } + + Project p = (Project) e; + + websiteurl = p.getWebsiteurl() != null && compareTrust(this, e) < 0 ? p.getWebsiteurl() : websiteurl; + code = p.getCode() != null && compareTrust(this, e) < 0 ? p.getCode() : code; + acronym = p.getAcronym() != null && compareTrust(this, e) < 0 ? p.getAcronym() : acronym; + title = p.getTitle() != null && compareTrust(this, e) < 0 ? p.getTitle() : title; + startdate = p.getStartdate() != null && compareTrust(this, e) < 0 ? p.getStartdate() : startdate; + enddate = p.getEnddate() != null && compareTrust(this, e) < 0 ? p.getEnddate() : enddate; + callidentifier = p.getCallidentifier() != null && compareTrust(this, e) < 0 + ? p.getCallidentifier() + : callidentifier; + keywords = p.getKeywords() != null && compareTrust(this, e) < 0 ? p.getKeywords() : keywords; + duration = p.getDuration() != null && compareTrust(this, e) < 0 ? p.getDuration() : duration; + ecsc39 = p.getEcsc39() != null && compareTrust(this, e) < 0 ? p.getEcsc39() : ecsc39; + oamandatepublications = p.getOamandatepublications() != null && compareTrust(this, e) < 0 + ? p.getOamandatepublications() + : oamandatepublications; + ecarticle29_3 = p.getEcarticle29_3() != null && compareTrust(this, e) < 0 + ? p.getEcarticle29_3() + : ecarticle29_3; + subjects = mergeLists(subjects, p.getSubjects()); + fundingtree = mergeLists(fundingtree, p.getFundingtree()); + contracttype = p.getContracttype() != null && compareTrust(this, e) < 0 + ? p.getContracttype() + : contracttype; + optional1 = p.getOptional1() != null && compareTrust(this, e) < 0 ? p.getOptional1() : optional1; + optional2 = p.getOptional2() != null && compareTrust(this, e) < 0 ? p.getOptional2() : optional2; + jsonextrainfo = p.getJsonextrainfo() != null && compareTrust(this, e) < 0 + ? p.getJsonextrainfo() + : jsonextrainfo; + contactfullname = p.getContactfullname() != null && compareTrust(this, e) < 0 + ? p.getContactfullname() + : contactfullname; + contactfax = p.getContactfax() != null && compareTrust(this, e) < 0 ? p.getContactfax() : contactfax; + contactphone = p.getContactphone() != null && compareTrust(this, e) < 0 + ? p.getContactphone() + : contactphone; + contactemail = p.getContactemail() != null && compareTrust(this, e) < 0 + ? p.getContactemail() + : contactemail; + summary = p.getSummary() != null && compareTrust(this, e) < 0 ? p.getSummary() : summary; + currency = p.getCurrency() != null && compareTrust(this, e) < 0 ? p.getCurrency() : currency; + totalcost = p.getTotalcost() != null && compareTrust(this, e) < 0 ? p.getTotalcost() : totalcost; + fundedamount = p.getFundedamount() != null && compareTrust(this, e) < 0 + ? p.getFundedamount() + : fundedamount; + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java index 9227df6ee..3058c262b 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Publication.java @@ -1,36 +1,39 @@ + package eu.dnetlib.dhp.schema.oaf; -import eu.dnetlib.dhp.schema.common.ModelConstants; import java.io.Serializable; +import eu.dnetlib.dhp.schema.common.ModelConstants; + public class Publication extends Result implements Serializable { - // publication specific - private Journal journal; + // publication specific + private Journal journal; - public Publication() { - setResulttype(ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE); - } + public Publication() { + setResulttype(ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE); + } - public Journal getJournal() { - return journal; - } + public Journal getJournal() { + return journal; + } - public void setJournal(Journal journal) { - this.journal = journal; - } + public void setJournal(Journal journal) { + this.journal = journal; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!Publication.class.isAssignableFrom(e.getClass())) { - return; - } + if (!Publication.class.isAssignableFrom(e.getClass())) { + return; + } - Publication p = (Publication) e; + Publication p = (Publication) e; - if (p.getJournal() != null && compareTrust(this, e) < 0) journal = p.getJournal(); - mergeOAFDataInfo(e); - } + if (p.getJournal() != null && compareTrust(this, e) < 0) + journal = p.getJournal(); + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java index 003d4a7a4..87ecb55f1 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Qualifier.java @@ -1,80 +1,87 @@ + package eu.dnetlib.dhp.schema.oaf; -import com.fasterxml.jackson.annotation.JsonIgnore; import java.io.Serializable; + import org.apache.commons.lang3.StringUtils; +import com.fasterxml.jackson.annotation.JsonIgnore; + public class Qualifier implements Serializable { - private String classid; - private String classname; - private String schemeid; - private String schemename; + private String classid; + private String classname; + private String schemeid; + private String schemename; - public String getClassid() { - return classid; - } + public String getClassid() { + return classid; + } - public void setClassid(String classid) { - this.classid = classid; - } + public void setClassid(String classid) { + this.classid = classid; + } - public String getClassname() { - return classname; - } + public String getClassname() { + return classname; + } - public void setClassname(String classname) { - this.classname = classname; - } + public void setClassname(String classname) { + this.classname = classname; + } - public String getSchemeid() { - return schemeid; - } + public String getSchemeid() { + return schemeid; + } - public void setSchemeid(String schemeid) { - this.schemeid = schemeid; - } + public void setSchemeid(String schemeid) { + this.schemeid = schemeid; + } - public String getSchemename() { - return schemename; - } + public String getSchemename() { + return schemename; + } - public void setSchemename(String schemename) { - this.schemename = schemename; - } + public void setSchemename(String schemename) { + this.schemename = schemename; + } - public String toComparableString() { - return isBlank() - ? "" - : String.format( - "%s::%s::%s::%s", - classid != null ? classid : "", - classname != null ? classname : "", - schemeid != null ? schemeid : "", - schemename != null ? schemename : ""); - } + public String toComparableString() { + return isBlank() + ? "" + : String + .format( + "%s::%s::%s::%s", + classid != null ? classid : "", + classname != null ? classname : "", + schemeid != null ? schemeid : "", + schemename != null ? schemename : ""); + } - @JsonIgnore - public boolean isBlank() { - return StringUtils.isBlank(classid) - && StringUtils.isBlank(classname) - && StringUtils.isBlank(schemeid) - && StringUtils.isBlank(schemename); - } + @JsonIgnore + public boolean isBlank() { + return StringUtils.isBlank(classid) + && StringUtils.isBlank(classname) + && StringUtils.isBlank(schemeid) + && StringUtils.isBlank(schemename); + } - @Override - public int hashCode() { - return toComparableString().hashCode(); - } + @Override + public int hashCode() { + return toComparableString().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - Qualifier other = (Qualifier) obj; + Qualifier other = (Qualifier) obj; - return toComparableString().equals(other.toComparableString()); - } + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java index 197adfb81..2c282c29e 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Relation.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import static com.google.common.base.Preconditions.checkArgument; @@ -8,91 +9,96 @@ import java.util.stream.Stream; public class Relation extends Oaf { - private String relType; + private String relType; - private String subRelType; + private String subRelType; - private String relClass; + private String relClass; - private String source; + private String source; - private String target; + private String target; - public String getRelType() { - return relType; - } + public String getRelType() { + return relType; + } - public void setRelType(final String relType) { - this.relType = relType; - } + public void setRelType(final String relType) { + this.relType = relType; + } - public String getSubRelType() { - return subRelType; - } + public String getSubRelType() { + return subRelType; + } - public void setSubRelType(final String subRelType) { - this.subRelType = subRelType; - } + public void setSubRelType(final String subRelType) { + this.subRelType = subRelType; + } - public String getRelClass() { - return relClass; - } + public String getRelClass() { + return relClass; + } - public void setRelClass(final String relClass) { - this.relClass = relClass; - } + public void setRelClass(final String relClass) { + this.relClass = relClass; + } - public String getSource() { - return source; - } + public String getSource() { + return source; + } - public void setSource(final String source) { - this.source = source; - } + public void setSource(final String source) { + this.source = source; + } - public String getTarget() { - return target; - } + public String getTarget() { + return target; + } - public void setTarget(final String target) { - this.target = target; - } + public void setTarget(final String target) { + this.target = target; + } - public void mergeFrom(final Relation r) { + public void mergeFrom(final Relation r) { - checkArgument(Objects.equals(getSource(), r.getSource()), "source ids must be equal"); - checkArgument(Objects.equals(getTarget(), r.getTarget()), "target ids must be equal"); - checkArgument(Objects.equals(getRelType(), r.getRelType()), "relType(s) must be equal"); - checkArgument( - Objects.equals(getSubRelType(), r.getSubRelType()), "subRelType(s) must be equal"); - checkArgument(Objects.equals(getRelClass(), r.getRelClass()), "relClass(es) must be equal"); + checkArgument(Objects.equals(getSource(), r.getSource()), "source ids must be equal"); + checkArgument(Objects.equals(getTarget(), r.getTarget()), "target ids must be equal"); + checkArgument(Objects.equals(getRelType(), r.getRelType()), "relType(s) must be equal"); + checkArgument( + Objects.equals(getSubRelType(), r.getSubRelType()), "subRelType(s) must be equal"); + checkArgument(Objects.equals(getRelClass(), r.getRelClass()), "relClass(es) must be equal"); - setCollectedfrom( - Stream.concat( - Optional.ofNullable(getCollectedfrom()) - .map(Collection::stream) - .orElse(Stream.empty()), - Optional.ofNullable(r.getCollectedfrom()) - .map(Collection::stream) - .orElse(Stream.empty())) - .distinct() // relies on KeyValue.equals - .collect(Collectors.toList())); - } + setCollectedfrom( + Stream + .concat( + Optional + .ofNullable(getCollectedfrom()) + .map(Collection::stream) + .orElse(Stream.empty()), + Optional + .ofNullable(r.getCollectedfrom()) + .map(Collection::stream) + .orElse(Stream.empty())) + .distinct() // relies on KeyValue.equals + .collect(Collectors.toList())); + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Relation relation = (Relation) o; - return relType.equals(relation.relType) - && subRelType.equals(relation.subRelType) - && relClass.equals(relation.relClass) - && source.equals(relation.source) - && target.equals(relation.target); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + Relation relation = (Relation) o; + return relType.equals(relation.relType) + && subRelType.equals(relation.subRelType) + && relClass.equals(relation.relClass) + && source.equals(relation.source) + && target.equals(relation.target); + } - @Override - public int hashCode() { - return Objects.hash(relType, subRelType, relClass, source, target, collectedfrom); - } + @Override + public int hashCode() { + return Objects.hash(relType, subRelType, relClass, source, target, collectedfrom); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java index 5da50b921..711b1ca68 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Result.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; @@ -6,286 +7,291 @@ import java.util.List; public class Result extends OafEntity implements Serializable { - private List author; + private List author; - // resulttype allows subclassing results into publications | datasets | software - private Qualifier resulttype; + // resulttype allows subclassing results into publications | datasets | software + private Qualifier resulttype; - // common fields - private Qualifier language; + // common fields + private Qualifier language; - private List country; + private List country; - private List subject; + private List subject; - private List title; + private List title; - private List relevantdate; + private List relevantdate; - private List> description; + private List> description; - private Field dateofacceptance; + private Field dateofacceptance; - private Field publisher; + private Field publisher; - private Field embargoenddate; + private Field embargoenddate; - private List> source; + private List> source; - private List> fulltext; // remove candidate + private List> fulltext; // remove candidate - private List> format; + private List> format; - private List> contributor; + private List> contributor; - private Qualifier resourcetype; + private Qualifier resourcetype; - private List> coverage; + private List> coverage; - private Qualifier bestaccessright; + private Qualifier bestaccessright; - private List context; + private List context; - private List externalReference; + private List externalReference; - private List instance; + private List instance; - public List getAuthor() { - return author; - } + public List getAuthor() { + return author; + } - public void setAuthor(List author) { - this.author = author; - } + public void setAuthor(List author) { + this.author = author; + } - public Qualifier getResulttype() { - return resulttype; - } + public Qualifier getResulttype() { + return resulttype; + } - public void setResulttype(Qualifier resulttype) { - this.resulttype = resulttype; - } + public void setResulttype(Qualifier resulttype) { + this.resulttype = resulttype; + } - public Qualifier getLanguage() { - return language; - } + public Qualifier getLanguage() { + return language; + } - public void setLanguage(Qualifier language) { - this.language = language; - } + public void setLanguage(Qualifier language) { + this.language = language; + } - public List getCountry() { - return country; - } + public List getCountry() { + return country; + } - public void setCountry(List country) { - this.country = country; - } + public void setCountry(List country) { + this.country = country; + } - public List getSubject() { - return subject; - } + public List getSubject() { + return subject; + } - public void setSubject(List subject) { - this.subject = subject; - } + public void setSubject(List subject) { + this.subject = subject; + } - public List getTitle() { - return title; - } + public List getTitle() { + return title; + } - public void setTitle(List title) { - this.title = title; - } + public void setTitle(List title) { + this.title = title; + } - public List getRelevantdate() { - return relevantdate; - } + public List getRelevantdate() { + return relevantdate; + } - public void setRelevantdate(List relevantdate) { - this.relevantdate = relevantdate; - } + public void setRelevantdate(List relevantdate) { + this.relevantdate = relevantdate; + } - public List> getDescription() { - return description; - } + public List> getDescription() { + return description; + } - public void setDescription(List> description) { - this.description = description; - } + public void setDescription(List> description) { + this.description = description; + } - public Field getDateofacceptance() { - return dateofacceptance; - } + public Field getDateofacceptance() { + return dateofacceptance; + } - public void setDateofacceptance(Field dateofacceptance) { - this.dateofacceptance = dateofacceptance; - } + public void setDateofacceptance(Field dateofacceptance) { + this.dateofacceptance = dateofacceptance; + } - public Field getPublisher() { - return publisher; - } + public Field getPublisher() { + return publisher; + } - public void setPublisher(Field publisher) { - this.publisher = publisher; - } + public void setPublisher(Field publisher) { + this.publisher = publisher; + } - public Field getEmbargoenddate() { - return embargoenddate; - } + public Field getEmbargoenddate() { + return embargoenddate; + } - public void setEmbargoenddate(Field embargoenddate) { - this.embargoenddate = embargoenddate; - } + public void setEmbargoenddate(Field embargoenddate) { + this.embargoenddate = embargoenddate; + } - public List> getSource() { - return source; - } + public List> getSource() { + return source; + } - public void setSource(List> source) { - this.source = source; - } + public void setSource(List> source) { + this.source = source; + } - public List> getFulltext() { - return fulltext; - } + public List> getFulltext() { + return fulltext; + } - public void setFulltext(List> fulltext) { - this.fulltext = fulltext; - } + public void setFulltext(List> fulltext) { + this.fulltext = fulltext; + } - public List> getFormat() { - return format; - } + public List> getFormat() { + return format; + } - public void setFormat(List> format) { - this.format = format; - } + public void setFormat(List> format) { + this.format = format; + } - public List> getContributor() { - return contributor; - } + public List> getContributor() { + return contributor; + } - public void setContributor(List> contributor) { - this.contributor = contributor; - } + public void setContributor(List> contributor) { + this.contributor = contributor; + } - public Qualifier getResourcetype() { - return resourcetype; - } + public Qualifier getResourcetype() { + return resourcetype; + } - public void setResourcetype(Qualifier resourcetype) { - this.resourcetype = resourcetype; - } + public void setResourcetype(Qualifier resourcetype) { + this.resourcetype = resourcetype; + } - public List> getCoverage() { - return coverage; - } + public List> getCoverage() { + return coverage; + } - public void setCoverage(List> coverage) { - this.coverage = coverage; - } + public void setCoverage(List> coverage) { + this.coverage = coverage; + } - public Qualifier getBestaccessright() { - return bestaccessright; - } + public Qualifier getBestaccessright() { + return bestaccessright; + } - public void setBestaccessright(Qualifier bestaccessright) { - this.bestaccessright = bestaccessright; - } + public void setBestaccessright(Qualifier bestaccessright) { + this.bestaccessright = bestaccessright; + } - public List getContext() { - return context; - } + public List getContext() { + return context; + } - public void setContext(List context) { - this.context = context; - } + public void setContext(List context) { + this.context = context; + } - public List getExternalReference() { - return externalReference; - } + public List getExternalReference() { + return externalReference; + } - public void setExternalReference(List externalReference) { - this.externalReference = externalReference; - } + public void setExternalReference(List externalReference) { + this.externalReference = externalReference; + } - public List getInstance() { - return instance; - } + public List getInstance() { + return instance; + } - public void setInstance(List instance) { - this.instance = instance; - } + public void setInstance(List instance) { + this.instance = instance; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!Result.class.isAssignableFrom(e.getClass())) { - return; - } + if (!Result.class.isAssignableFrom(e.getClass())) { + return; + } - Result r = (Result) e; + Result r = (Result) e; - instance = mergeLists(instance, r.getInstance()); + instance = mergeLists(instance, r.getInstance()); - if (r.getBestaccessright() != null && compareTrust(this, r) < 0) - bestaccessright = r.getBestaccessright(); + if (r.getBestaccessright() != null && compareTrust(this, r) < 0) + bestaccessright = r.getBestaccessright(); - if (r.getResulttype() != null && compareTrust(this, r) < 0) resulttype = r.getResulttype(); + if (r.getResulttype() != null && compareTrust(this, r) < 0) + resulttype = r.getResulttype(); - if (r.getLanguage() != null && compareTrust(this, r) < 0) language = r.getLanguage(); + if (r.getLanguage() != null && compareTrust(this, r) < 0) + language = r.getLanguage(); - country = mergeLists(country, r.getCountry()); + country = mergeLists(country, r.getCountry()); - subject = mergeLists(subject, r.getSubject()); + subject = mergeLists(subject, r.getSubject()); - title = mergeLists(title, r.getTitle()); + title = mergeLists(title, r.getTitle()); - relevantdate = mergeLists(relevantdate, r.getRelevantdate()); + relevantdate = mergeLists(relevantdate, r.getRelevantdate()); - description = longestLists(description, r.getDescription()); + description = longestLists(description, r.getDescription()); - if (r.getPublisher() != null && compareTrust(this, r) < 0) publisher = r.getPublisher(); + if (r.getPublisher() != null && compareTrust(this, r) < 0) + publisher = r.getPublisher(); - if (r.getEmbargoenddate() != null && compareTrust(this, r) < 0) - embargoenddate = r.getEmbargoenddate(); + if (r.getEmbargoenddate() != null && compareTrust(this, r) < 0) + embargoenddate = r.getEmbargoenddate(); - source = mergeLists(source, r.getSource()); + source = mergeLists(source, r.getSource()); - fulltext = mergeLists(fulltext, r.getFulltext()); + fulltext = mergeLists(fulltext, r.getFulltext()); - format = mergeLists(format, r.getFormat()); + format = mergeLists(format, r.getFormat()); - contributor = mergeLists(contributor, r.getContributor()); + contributor = mergeLists(contributor, r.getContributor()); - if (r.getResourcetype() != null) resourcetype = r.getResourcetype(); + if (r.getResourcetype() != null) + resourcetype = r.getResourcetype(); - coverage = mergeLists(coverage, r.getCoverage()); + coverage = mergeLists(coverage, r.getCoverage()); - context = mergeLists(context, r.getContext()); + context = mergeLists(context, r.getContext()); - externalReference = mergeLists(externalReference, r.getExternalReference()); - } + externalReference = mergeLists(externalReference, r.getExternalReference()); + } - private List> longestLists(List> a, List> b) { - if (a == null || b == null) return a == null ? b : a; - if (a.size() == b.size()) { - int msa = - a.stream() - .filter(i -> i.getValue() != null) - .map(i -> i.getValue().length()) - .max(Comparator.naturalOrder()) - .orElse(0); - int msb = - b.stream() - .filter(i -> i.getValue() != null) - .map(i -> i.getValue().length()) - .max(Comparator.naturalOrder()) - .orElse(0); - return msa > msb ? a : b; - } - return a.size() > b.size() ? a : b; - } + private List> longestLists(List> a, List> b) { + if (a == null || b == null) + return a == null ? b : a; + if (a.size() == b.size()) { + int msa = a + .stream() + .filter(i -> i.getValue() != null) + .map(i -> i.getValue().length()) + .max(Comparator.naturalOrder()) + .orElse(0); + int msb = b + .stream() + .filter(i -> i.getValue() != null) + .map(i -> i.getValue().length()) + .max(Comparator.naturalOrder()) + .orElse(0); + return msa > msb ? a : b; + } + return a.size() > b.size() ? a : b; + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java index ffb7e03f7..40332bf53 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Software.java @@ -1,78 +1,78 @@ + package eu.dnetlib.dhp.schema.oaf; -import eu.dnetlib.dhp.schema.common.ModelConstants; import java.io.Serializable; import java.util.List; +import eu.dnetlib.dhp.schema.common.ModelConstants; + public class Software extends Result implements Serializable { - private List> documentationUrl; + private List> documentationUrl; - private List license; + private List license; - private Field codeRepositoryUrl; + private Field codeRepositoryUrl; - private Qualifier programmingLanguage; + private Qualifier programmingLanguage; - public Software() { - setResulttype(ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE); - } + public Software() { + setResulttype(ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE); + } - public List> getDocumentationUrl() { - return documentationUrl; - } + public List> getDocumentationUrl() { + return documentationUrl; + } - public void setDocumentationUrl(List> documentationUrl) { - this.documentationUrl = documentationUrl; - } + public void setDocumentationUrl(List> documentationUrl) { + this.documentationUrl = documentationUrl; + } - public List getLicense() { - return license; - } + public List getLicense() { + return license; + } - public void setLicense(List license) { - this.license = license; - } + public void setLicense(List license) { + this.license = license; + } - public Field getCodeRepositoryUrl() { - return codeRepositoryUrl; - } + public Field getCodeRepositoryUrl() { + return codeRepositoryUrl; + } - public void setCodeRepositoryUrl(Field codeRepositoryUrl) { - this.codeRepositoryUrl = codeRepositoryUrl; - } + public void setCodeRepositoryUrl(Field codeRepositoryUrl) { + this.codeRepositoryUrl = codeRepositoryUrl; + } - public Qualifier getProgrammingLanguage() { - return programmingLanguage; - } + public Qualifier getProgrammingLanguage() { + return programmingLanguage; + } - public void setProgrammingLanguage(Qualifier programmingLanguage) { - this.programmingLanguage = programmingLanguage; - } + public void setProgrammingLanguage(Qualifier programmingLanguage) { + this.programmingLanguage = programmingLanguage; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); - if (!Software.class.isAssignableFrom(e.getClass())) { - return; - } + if (!Software.class.isAssignableFrom(e.getClass())) { + return; + } - final Software s = (Software) e; - documentationUrl = mergeLists(documentationUrl, s.getDocumentationUrl()); + final Software s = (Software) e; + documentationUrl = mergeLists(documentationUrl, s.getDocumentationUrl()); - license = mergeLists(license, s.getLicense()); + license = mergeLists(license, s.getLicense()); - codeRepositoryUrl = - s.getCodeRepositoryUrl() != null && compareTrust(this, s) < 0 - ? s.getCodeRepositoryUrl() - : codeRepositoryUrl; + codeRepositoryUrl = s.getCodeRepositoryUrl() != null && compareTrust(this, s) < 0 + ? s.getCodeRepositoryUrl() + : codeRepositoryUrl; - programmingLanguage = - s.getProgrammingLanguage() != null && compareTrust(this, s) < 0 - ? s.getProgrammingLanguage() - : programmingLanguage; + programmingLanguage = s.getProgrammingLanguage() != null && compareTrust(this, s) < 0 + ? s.getProgrammingLanguage() + : programmingLanguage; - mergeOAFDataInfo(e); - } + mergeOAFDataInfo(e); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java index 2e77389a3..1fa0de0be 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/StructuredProperty.java @@ -1,56 +1,60 @@ + package eu.dnetlib.dhp.schema.oaf; import java.io.Serializable; public class StructuredProperty implements Serializable { - private String value; + private String value; - private Qualifier qualifier; + private Qualifier qualifier; - private DataInfo dataInfo; + private DataInfo dataInfo; - public String getValue() { - return value; - } + public String getValue() { + return value; + } - public void setValue(String value) { - this.value = value; - } + public void setValue(String value) { + this.value = value; + } - public Qualifier getQualifier() { - return qualifier; - } + public Qualifier getQualifier() { + return qualifier; + } - public void setQualifier(Qualifier qualifier) { - this.qualifier = qualifier; - } + public void setQualifier(Qualifier qualifier) { + this.qualifier = qualifier; + } - public DataInfo getDataInfo() { - return dataInfo; - } + public DataInfo getDataInfo() { + return dataInfo; + } - public void setDataInfo(DataInfo dataInfo) { - this.dataInfo = dataInfo; - } + public void setDataInfo(DataInfo dataInfo) { + this.dataInfo = dataInfo; + } - public String toComparableString() { - return value != null ? value.toLowerCase() : ""; - } + public String toComparableString() { + return value != null ? value.toLowerCase() : ""; + } - @Override - public int hashCode() { - return toComparableString().hashCode(); - } + @Override + public int hashCode() { + return toComparableString().hashCode(); + } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null) return false; - if (getClass() != obj.getClass()) return false; + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; - StructuredProperty other = (StructuredProperty) obj; + StructuredProperty other = (StructuredProperty) obj; - return toComparableString().equals(other.toComparableString()); - } + return toComparableString().equals(other.toComparableString()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIDataset.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIDataset.java index e1569787b..421b4ecaa 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIDataset.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIDataset.java @@ -1,83 +1,89 @@ + package eu.dnetlib.dhp.schema.scholexplorer; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.OafEntity; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; + import org.apache.commons.lang3.StringUtils; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.OafEntity; + public class DLIDataset extends Dataset { - private String originalObjIdentifier; + private String originalObjIdentifier; - private List dlicollectedfrom; + private List dlicollectedfrom; - private String completionStatus; + private String completionStatus; - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } - public List getDlicollectedfrom() { - return dlicollectedfrom; - } + public List getDlicollectedfrom() { + return dlicollectedfrom; + } - public void setDlicollectedfrom(List dlicollectedfrom) { - this.dlicollectedfrom = dlicollectedfrom; - } + public void setDlicollectedfrom(List dlicollectedfrom) { + this.dlicollectedfrom = dlicollectedfrom; + } - public String getOriginalObjIdentifier() { - return originalObjIdentifier; - } + public String getOriginalObjIdentifier() { + return originalObjIdentifier; + } - public void setOriginalObjIdentifier(String originalObjIdentifier) { - this.originalObjIdentifier = originalObjIdentifier; - } + public void setOriginalObjIdentifier(String originalObjIdentifier) { + this.originalObjIdentifier = originalObjIdentifier; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); - DLIDataset p = (DLIDataset) e; - if (StringUtils.isBlank(completionStatus) && StringUtils.isNotBlank(p.completionStatus)) - completionStatus = p.completionStatus; - if ("complete".equalsIgnoreCase(p.completionStatus)) completionStatus = "complete"; - dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); - } + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + DLIDataset p = (DLIDataset) e; + if (StringUtils.isBlank(completionStatus) && StringUtils.isNotBlank(p.completionStatus)) + completionStatus = p.completionStatus; + if ("complete".equalsIgnoreCase(p.completionStatus)) + completionStatus = "complete"; + dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); + } - private List mergeProvenance( - final List a, final List b) { - Map result = new HashMap<>(); - if (a != null) - a.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + private List mergeProvenance( + final List a, final List b) { + Map result = new HashMap<>(); + if (a != null) + a + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); - if (b != null) - b.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); + if (b != null) + b + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); - return new ArrayList<>(result.values()); - } + return new ArrayList<>(result.values()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIPublication.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIPublication.java index 2cfb6515c..c899a899c 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIPublication.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIPublication.java @@ -1,81 +1,87 @@ + package eu.dnetlib.dhp.schema.scholexplorer; +import java.io.Serializable; +import java.util.*; + +import org.apache.commons.lang3.StringUtils; + import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.Publication; -import java.io.Serializable; -import java.util.*; -import org.apache.commons.lang3.StringUtils; public class DLIPublication extends Publication implements Serializable { - private String originalObjIdentifier; + private String originalObjIdentifier; - private List dlicollectedfrom; + private List dlicollectedfrom; - private String completionStatus; + private String completionStatus; - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } - public List getDlicollectedfrom() { - return dlicollectedfrom; - } + public List getDlicollectedfrom() { + return dlicollectedfrom; + } - public void setDlicollectedfrom(List dlicollectedfrom) { - this.dlicollectedfrom = dlicollectedfrom; - } + public void setDlicollectedfrom(List dlicollectedfrom) { + this.dlicollectedfrom = dlicollectedfrom; + } - public String getOriginalObjIdentifier() { - return originalObjIdentifier; - } + public String getOriginalObjIdentifier() { + return originalObjIdentifier; + } - public void setOriginalObjIdentifier(String originalObjIdentifier) { - this.originalObjIdentifier = originalObjIdentifier; - } + public void setOriginalObjIdentifier(String originalObjIdentifier) { + this.originalObjIdentifier = originalObjIdentifier; + } - @Override - public void mergeFrom(OafEntity e) { - super.mergeFrom(e); - DLIPublication p = (DLIPublication) e; - if (StringUtils.isBlank(completionStatus) && StringUtils.isNotBlank(p.completionStatus)) - completionStatus = p.completionStatus; - if ("complete".equalsIgnoreCase(p.completionStatus)) completionStatus = "complete"; - dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); - } + @Override + public void mergeFrom(OafEntity e) { + super.mergeFrom(e); + DLIPublication p = (DLIPublication) e; + if (StringUtils.isBlank(completionStatus) && StringUtils.isNotBlank(p.completionStatus)) + completionStatus = p.completionStatus; + if ("complete".equalsIgnoreCase(p.completionStatus)) + completionStatus = "complete"; + dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); + } - private List mergeProvenance( - final List a, final List b) { - Map result = new HashMap<>(); - if (a != null) - a.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + private List mergeProvenance( + final List a, final List b) { + Map result = new HashMap<>(); + if (a != null) + a + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); - if (b != null) - b.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); + if (b != null) + b + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); - return new ArrayList<>(result.values()); - } + return new ArrayList<>(result.values()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java index b58483cbb..d2d2089c0 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIRelation.java @@ -1,15 +1,16 @@ + package eu.dnetlib.dhp.schema.scholexplorer; import eu.dnetlib.dhp.schema.oaf.Relation; public class DLIRelation extends Relation { - private String dateOfCollection; + private String dateOfCollection; - public String getDateOfCollection() { - return dateOfCollection; - } + public String getDateOfCollection() { + return dateOfCollection; + } - public void setDateOfCollection(String dateOfCollection) { - this.dateOfCollection = dateOfCollection; - } + public void setDateOfCollection(String dateOfCollection) { + this.dateOfCollection = dateOfCollection; + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIUnknown.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIUnknown.java index 6a58ab54f..e9b670d03 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIUnknown.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/DLIUnknown.java @@ -1,109 +1,115 @@ + package eu.dnetlib.dhp.schema.scholexplorer; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; + import org.apache.commons.lang3.StringUtils; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; + public class DLIUnknown extends Oaf implements Serializable { - private String id; + private String id; - private List pid; + private List pid; - private String dateofcollection; + private String dateofcollection; - private String dateoftransformation; + private String dateoftransformation; - private List dlicollectedfrom; + private List dlicollectedfrom; - private String completionStatus = "incomplete"; + private String completionStatus = "incomplete"; - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } - public List getDlicollectedfrom() { - return dlicollectedfrom; - } + public List getDlicollectedfrom() { + return dlicollectedfrom; + } - public void setDlicollectedfrom(List dlicollectedfrom) { - this.dlicollectedfrom = dlicollectedfrom; - } + public void setDlicollectedfrom(List dlicollectedfrom) { + this.dlicollectedfrom = dlicollectedfrom; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public List getPid() { - return pid; - } + public List getPid() { + return pid; + } - public void setPid(List pid) { - this.pid = pid; - } + public void setPid(List pid) { + this.pid = pid; + } - public String getDateofcollection() { - return dateofcollection; - } + public String getDateofcollection() { + return dateofcollection; + } - public void setDateofcollection(String dateofcollection) { - this.dateofcollection = dateofcollection; - } + public void setDateofcollection(String dateofcollection) { + this.dateofcollection = dateofcollection; + } - public String getDateoftransformation() { - return dateoftransformation; - } + public String getDateoftransformation() { + return dateoftransformation; + } - public void setDateoftransformation(String dateoftransformation) { - this.dateoftransformation = dateoftransformation; - } + public void setDateoftransformation(String dateoftransformation) { + this.dateoftransformation = dateoftransformation; + } - public void mergeFrom(DLIUnknown p) { - if ("complete".equalsIgnoreCase(p.completionStatus)) completionStatus = "complete"; - dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); - } + public void mergeFrom(DLIUnknown p) { + if ("complete".equalsIgnoreCase(p.completionStatus)) + completionStatus = "complete"; + dlicollectedfrom = mergeProvenance(dlicollectedfrom, p.getDlicollectedfrom()); + } - private List mergeProvenance( - final List a, final List b) { - Map result = new HashMap<>(); - if (a != null) - a.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + private List mergeProvenance( + final List a, final List b) { + Map result = new HashMap<>(); + if (a != null) + a + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); - if (b != null) - b.forEach( - p -> { - if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { - if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) - && StringUtils.isNotBlank(p.getCompletionStatus())) { - result.put(p.getId(), p); - } + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); + if (b != null) + b + .forEach( + p -> { + if (p != null && StringUtils.isNotBlank(p.getId()) && result.containsKey(p.getId())) { + if ("incomplete".equalsIgnoreCase(result.get(p.getId()).getCompletionStatus()) + && StringUtils.isNotBlank(p.getCompletionStatus())) { + result.put(p.getId(), p); + } - } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) - result.put(p.getId(), p); - }); + } else if (p != null && p.getId() != null && !result.containsKey(p.getId())) + result.put(p.getId(), p); + }); - return new ArrayList<>(result.values()); - } + return new ArrayList<>(result.values()); + } } diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/ProvenaceInfo.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/ProvenaceInfo.java index 52f7161b9..b1188f064 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/ProvenaceInfo.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/ProvenaceInfo.java @@ -1,46 +1,47 @@ + package eu.dnetlib.dhp.schema.scholexplorer; import java.io.Serializable; public class ProvenaceInfo implements Serializable { - private String id; + private String id; - private String name; + private String name; - private String completionStatus; + private String completionStatus; - private String collectionMode = "collected"; + private String collectionMode = "collected"; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } - public String getCollectionMode() { - return collectionMode; - } + public String getCollectionMode() { + return collectionMode; + } - public void setCollectionMode(String collectionMode) { - this.collectionMode = collectionMode; - } + public void setCollectionMode(String collectionMode) { + this.collectionMode = collectionMode; + } } diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java index 482c1c223..4d31591a0 100644 --- a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java +++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/action/AtomicActionTest.java @@ -1,36 +1,40 @@ + package eu.dnetlib.dhp.schema.action; import static org.junit.jupiter.api.Assertions.*; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.io.IOException; + import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.Test; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Relation; + /** @author claudio.atzori */ public class AtomicActionTest { - @Test - public void serializationTest() throws IOException { + @Test + public void serializationTest() throws IOException { - Relation rel = new Relation(); - rel.setSource("1"); - rel.setTarget("2"); - rel.setRelType("resultResult"); - rel.setSubRelType("dedup"); - rel.setRelClass("merges"); + Relation rel = new Relation(); + rel.setSource("1"); + rel.setTarget("2"); + rel.setRelType("resultResult"); + rel.setSubRelType("dedup"); + rel.setRelClass("merges"); - AtomicAction aa1 = new AtomicAction(Relation.class, rel); + AtomicAction aa1 = new AtomicAction(Relation.class, rel); - final ObjectMapper mapper = new ObjectMapper(); - String json = mapper.writeValueAsString(aa1); + final ObjectMapper mapper = new ObjectMapper(); + String json = mapper.writeValueAsString(aa1); - assertTrue(StringUtils.isNotBlank(json)); + assertTrue(StringUtils.isNotBlank(json)); - AtomicAction aa2 = mapper.readValue(json, AtomicAction.class); + AtomicAction aa2 = mapper.readValue(json, AtomicAction.class); - assertEquals(aa1.getClazz(), aa2.getClazz()); - assertEquals(aa1.getPayload(), aa2.getPayload()); - } + assertEquals(aa1.getClazz(), aa2.getClazz()); + assertEquals(aa1.getPayload(), aa2.getPayload()); + } } diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/common/ModelSupportTest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/common/ModelSupportTest.java index 3e07ea87c..73e8c47ff 100644 --- a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/common/ModelSupportTest.java +++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/common/ModelSupportTest.java @@ -1,35 +1,37 @@ + package eu.dnetlib.dhp.schema.common; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; + public class ModelSupportTest { - @Nested - class IsSubClass { + @Nested + class IsSubClass { - @Test - public void shouldReturnFalseWhenSubClassDoesNotExtendSuperClass() { - // when - Boolean result = ModelSupport.isSubClass(Relation.class, OafEntity.class); + @Test + public void shouldReturnFalseWhenSubClassDoesNotExtendSuperClass() { + // when + Boolean result = ModelSupport.isSubClass(Relation.class, OafEntity.class); - // then - assertFalse(result); - } + // then + assertFalse(result); + } - @Test - public void shouldReturnTrueWhenSubClassExtendsSuperClass() { - // when - Boolean result = ModelSupport.isSubClass(Result.class, OafEntity.class); + @Test + public void shouldReturnTrueWhenSubClassExtendsSuperClass() { + // when + Boolean result = ModelSupport.isSubClass(Result.class, OafEntity.class); - // then - assertTrue(result); - } - } + // then + assertTrue(result); + } + } } diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java index bb5b824f6..f91646f2c 100644 --- a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java +++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/oaf/MergeTest.java @@ -1,86 +1,88 @@ + package eu.dnetlib.dhp.schema.oaf; import static org.junit.jupiter.api.Assertions.*; import java.util.Arrays; import java.util.List; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; public class MergeTest { - OafEntity oaf; + OafEntity oaf; - @BeforeEach - public void setUp() { - oaf = new Publication(); - } + @BeforeEach + public void setUp() { + oaf = new Publication(); + } - @Test - public void mergeListsTest() { + @Test + public void mergeListsTest() { - // string list merge test - List a = Arrays.asList("a", "b", "c", "e"); - List b = Arrays.asList("a", "b", "c", "d"); - List c = null; + // string list merge test + List a = Arrays.asList("a", "b", "c", "e"); + List b = Arrays.asList("a", "b", "c", "d"); + List c = null; - System.out.println("merge result 1 = " + oaf.mergeLists(a, b)); + System.out.println("merge result 1 = " + oaf.mergeLists(a, b)); - System.out.println("merge result 2 = " + oaf.mergeLists(a, c)); + System.out.println("merge result 2 = " + oaf.mergeLists(a, c)); - System.out.println("merge result 3 = " + oaf.mergeLists(c, c)); - } + System.out.println("merge result 3 = " + oaf.mergeLists(c, c)); + } - @Test - public void mergePublicationCollectedFromTest() { + @Test + public void mergePublicationCollectedFromTest() { - Publication a = new Publication(); - Publication b = new Publication(); + Publication a = new Publication(); + Publication b = new Publication(); - a.setCollectedfrom(Arrays.asList(setKV("a", "open"), setKV("b", "closed"))); - b.setCollectedfrom(Arrays.asList(setKV("A", "open"), setKV("b", "Open"))); + a.setCollectedfrom(Arrays.asList(setKV("a", "open"), setKV("b", "closed"))); + b.setCollectedfrom(Arrays.asList(setKV("A", "open"), setKV("b", "Open"))); - a.mergeFrom(b); + a.mergeFrom(b); - assertNotNull(a.getCollectedfrom()); - assertEquals(3, a.getCollectedfrom().size()); - } + assertNotNull(a.getCollectedfrom()); + assertEquals(3, a.getCollectedfrom().size()); + } - @Test - public void mergePublicationSubjectTest() { + @Test + public void mergePublicationSubjectTest() { - Publication a = new Publication(); - Publication b = new Publication(); + Publication a = new Publication(); + Publication b = new Publication(); - a.setSubject(Arrays.asList(setSP("a", "open", "classe"), setSP("b", "open", "classe"))); - b.setSubject(Arrays.asList(setSP("A", "open", "classe"), setSP("c", "open", "classe"))); + a.setSubject(Arrays.asList(setSP("a", "open", "classe"), setSP("b", "open", "classe"))); + b.setSubject(Arrays.asList(setSP("A", "open", "classe"), setSP("c", "open", "classe"))); - a.mergeFrom(b); + a.mergeFrom(b); - assertNotNull(a.getSubject()); - assertEquals(3, a.getSubject().size()); - } + assertNotNull(a.getSubject()); + assertEquals(3, a.getSubject().size()); + } - private KeyValue setKV(final String key, final String value) { + private KeyValue setKV(final String key, final String value) { - KeyValue k = new KeyValue(); + KeyValue k = new KeyValue(); - k.setKey(key); - k.setValue(value); + k.setKey(key); + k.setValue(value); - return k; - } + return k; + } - private StructuredProperty setSP( - final String value, final String schema, final String classname) { - StructuredProperty s = new StructuredProperty(); - s.setValue(value); - Qualifier q = new Qualifier(); - q.setClassname(classname); - q.setClassid(classname); - q.setSchemename(schema); - q.setSchemeid(schema); - s.setQualifier(q); - return s; - } + private StructuredProperty setSP( + final String value, final String schema, final String classname) { + StructuredProperty s = new StructuredProperty(); + s.setValue(value); + Qualifier q = new Qualifier(); + q.setClassname(classname); + q.setClassid(classname); + q.setSchemename(schema); + q.setSchemeid(schema); + s.setQualifier(q); + return s; + } } diff --git a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/scholexplorer/DLItest.java b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/scholexplorer/DLItest.java index 4f82cfe10..e4596fcdd 100644 --- a/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/scholexplorer/DLItest.java +++ b/dhp-schemas/src/test/java/eu/dnetlib/dhp/schema/scholexplorer/DLItest.java @@ -1,76 +1,83 @@ + package eu.dnetlib.dhp.schema.scholexplorer; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; + +import org.junit.jupiter.api.Test; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; + import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import org.junit.jupiter.api.Test; public class DLItest { - @Test - public void testMergePublication() throws JsonProcessingException { - DLIPublication a1 = new DLIPublication(); - a1.setPid(Arrays.asList(createSP("123456", "pdb", "dnet:pid_types"))); - a1.setTitle(Collections.singletonList(createSP("Un Titolo", "title", "dnetTitle"))); - a1.setDlicollectedfrom(Arrays.asList(createCollectedFrom("znd", "Zenodo", "complete"))); - a1.setCompletionStatus("complete"); + @Test + public void testMergePublication() throws JsonProcessingException { + DLIPublication a1 = new DLIPublication(); + a1.setPid(Arrays.asList(createSP("123456", "pdb", "dnet:pid_types"))); + a1.setTitle(Collections.singletonList(createSP("Un Titolo", "title", "dnetTitle"))); + a1.setDlicollectedfrom(Arrays.asList(createCollectedFrom("znd", "Zenodo", "complete"))); + a1.setCompletionStatus("complete"); - DLIPublication a = new DLIPublication(); - a.setPid( - Arrays.asList( - createSP("10.11", "doi", "dnet:pid_types"), - createSP("123456", "pdb", "dnet:pid_types"))); - a.setTitle(Collections.singletonList(createSP("A Title", "title", "dnetTitle"))); - a.setDlicollectedfrom( - Arrays.asList( - createCollectedFrom("dct", "datacite", "complete"), - createCollectedFrom("dct", "datacite", "incomplete"))); - a.setCompletionStatus("incomplete"); + DLIPublication a = new DLIPublication(); + a + .setPid( + Arrays + .asList( + createSP("10.11", "doi", "dnet:pid_types"), + createSP("123456", "pdb", "dnet:pid_types"))); + a.setTitle(Collections.singletonList(createSP("A Title", "title", "dnetTitle"))); + a + .setDlicollectedfrom( + Arrays + .asList( + createCollectedFrom("dct", "datacite", "complete"), + createCollectedFrom("dct", "datacite", "incomplete"))); + a.setCompletionStatus("incomplete"); - a.mergeFrom(a1); + a.mergeFrom(a1); - ObjectMapper mapper = new ObjectMapper(); - System.out.println(mapper.writeValueAsString(a)); - } + ObjectMapper mapper = new ObjectMapper(); + System.out.println(mapper.writeValueAsString(a)); + } - @Test - public void testDeserialization() throws IOException { + @Test + public void testDeserialization() throws IOException { - final String json = - "{\"dataInfo\":{\"invisible\":false,\"inferred\":null,\"deletedbyinference\":false,\"trust\":\"0.9\",\"inferenceprovenance\":null,\"provenanceaction\":null},\"lastupdatetimestamp\":null,\"id\":\"60|bd9352547098929a394655ad1a44a479\",\"originalId\":[\"bd9352547098929a394655ad1a44a479\"],\"collectedfrom\":[{\"key\":\"dli_________::datacite\",\"value\":\"Datasets in Datacite\",\"dataInfo\":null,\"blank\":false}],\"pid\":[{\"value\":\"10.7925/DRS1.DUCHAS_5078760\",\"qualifier\":{\"classid\":\"doi\",\"classname\":\"doi\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\",\"blank\":false},\"dataInfo\":null}],\"dateofcollection\":\"2020-01-09T08:29:31.885Z\",\"dateoftransformation\":null,\"extraInfo\":null,\"oaiprovenance\":null,\"author\":[{\"fullname\":\"Cathail, S. Ó\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Donnell, Breda Mc\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Ireland. Department of Arts, Culture, and the Gaeltacht\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"University College Dublin\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"National Folklore Foundation\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Cathail, S. Ó\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Donnell, Breda Mc\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null}],\"resulttype\":null,\"language\":null,\"country\":null,\"subject\":[{\"value\":\"Recreation\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null},{\"value\":\"Entertainments and recreational activities\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null},{\"value\":\"Siamsaíocht agus caitheamh aimsire\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null}],\"title\":[{\"value\":\"Games We Play\",\"qualifier\":null,\"dataInfo\":null}],\"relevantdate\":[{\"value\":\"1938-09-28\",\"qualifier\":{\"classid\":\"date\",\"classname\":\"date\",\"schemeid\":\"dnet::date\",\"schemename\":\"dnet::date\",\"blank\":false},\"dataInfo\":null}],\"description\":[{\"value\":\"Story collected by Breda Mc Donnell, a student at Tenure school (Tinure, Co. Louth) (no informant identified).\",\"dataInfo\":null}],\"dateofacceptance\":null,\"publisher\":{\"value\":\"University College Dublin\",\"dataInfo\":null},\"embargoenddate\":null,\"source\":null,\"fulltext\":null,\"format\":null,\"contributor\":null,\"resourcetype\":null,\"coverage\":null,\"refereed\":null,\"context\":null,\"processingchargeamount\":null,\"processingchargecurrency\":null,\"externalReference\":null,\"instance\":[],\"storagedate\":null,\"device\":null,\"size\":null,\"version\":null,\"lastmetadataupdate\":null,\"metadataversionnumber\":null,\"geolocation\":null,\"dlicollectedfrom\":[{\"id\":\"dli_________::datacite\",\"name\":\"Datasets in Datacite\",\"completionStatus\":\"complete\",\"collectionMode\":\"resolved\"}],\"completionStatus\":\"complete\"}"; + final String json = "{\"dataInfo\":{\"invisible\":false,\"inferred\":null,\"deletedbyinference\":false,\"trust\":\"0.9\",\"inferenceprovenance\":null,\"provenanceaction\":null},\"lastupdatetimestamp\":null,\"id\":\"60|bd9352547098929a394655ad1a44a479\",\"originalId\":[\"bd9352547098929a394655ad1a44a479\"],\"collectedfrom\":[{\"key\":\"dli_________::datacite\",\"value\":\"Datasets in Datacite\",\"dataInfo\":null,\"blank\":false}],\"pid\":[{\"value\":\"10.7925/DRS1.DUCHAS_5078760\",\"qualifier\":{\"classid\":\"doi\",\"classname\":\"doi\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\",\"blank\":false},\"dataInfo\":null}],\"dateofcollection\":\"2020-01-09T08:29:31.885Z\",\"dateoftransformation\":null,\"extraInfo\":null,\"oaiprovenance\":null,\"author\":[{\"fullname\":\"Cathail, S. Ó\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Donnell, Breda Mc\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Ireland. Department of Arts, Culture, and the Gaeltacht\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"University College Dublin\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"National Folklore Foundation\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Cathail, S. Ó\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null},{\"fullname\":\"Donnell, Breda Mc\",\"name\":null,\"surname\":null,\"rank\":null,\"pid\":null,\"affiliation\":null}],\"resulttype\":null,\"language\":null,\"country\":null,\"subject\":[{\"value\":\"Recreation\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null},{\"value\":\"Entertainments and recreational activities\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null},{\"value\":\"Siamsaíocht agus caitheamh aimsire\",\"qualifier\":{\"classid\":\"dnet:subject\",\"classname\":\"dnet:subject\",\"schemeid\":\"unknown\",\"schemename\":\"unknown\",\"blank\":false},\"dataInfo\":null}],\"title\":[{\"value\":\"Games We Play\",\"qualifier\":null,\"dataInfo\":null}],\"relevantdate\":[{\"value\":\"1938-09-28\",\"qualifier\":{\"classid\":\"date\",\"classname\":\"date\",\"schemeid\":\"dnet::date\",\"schemename\":\"dnet::date\",\"blank\":false},\"dataInfo\":null}],\"description\":[{\"value\":\"Story collected by Breda Mc Donnell, a student at Tenure school (Tinure, Co. Louth) (no informant identified).\",\"dataInfo\":null}],\"dateofacceptance\":null,\"publisher\":{\"value\":\"University College Dublin\",\"dataInfo\":null},\"embargoenddate\":null,\"source\":null,\"fulltext\":null,\"format\":null,\"contributor\":null,\"resourcetype\":null,\"coverage\":null,\"refereed\":null,\"context\":null,\"processingchargeamount\":null,\"processingchargecurrency\":null,\"externalReference\":null,\"instance\":[],\"storagedate\":null,\"device\":null,\"size\":null,\"version\":null,\"lastmetadataupdate\":null,\"metadataversionnumber\":null,\"geolocation\":null,\"dlicollectedfrom\":[{\"id\":\"dli_________::datacite\",\"name\":\"Datasets in Datacite\",\"completionStatus\":\"complete\",\"collectionMode\":\"resolved\"}],\"completionStatus\":\"complete\"}"; - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - DLIDataset dliDataset = mapper.readValue(json, DLIDataset.class); - mapper.enable(SerializationFeature.INDENT_OUTPUT); - System.out.println(mapper.writeValueAsString(dliDataset)); - } + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + DLIDataset dliDataset = mapper.readValue(json, DLIDataset.class); + mapper.enable(SerializationFeature.INDENT_OUTPUT); + System.out.println(mapper.writeValueAsString(dliDataset)); + } - private ProvenaceInfo createCollectedFrom( - final String id, final String name, final String completionStatus) { - ProvenaceInfo p = new ProvenaceInfo(); - p.setId(id); - p.setName(name); - p.setCompletionStatus(completionStatus); - return p; - } + private ProvenaceInfo createCollectedFrom( + final String id, final String name, final String completionStatus) { + ProvenaceInfo p = new ProvenaceInfo(); + p.setId(id); + p.setName(name); + p.setCompletionStatus(completionStatus); + return p; + } - private StructuredProperty createSP( - final String value, final String className, final String schemeName) { - StructuredProperty p = new StructuredProperty(); - p.setValue(value); - Qualifier schema = new Qualifier(); - schema.setClassname(className); - schema.setClassid(className); - schema.setSchemename(schemeName); - schema.setSchemeid(schemeName); - p.setQualifier(schema); - return p; - } + private StructuredProperty createSP( + final String value, final String className, final String schemeName) { + StructuredProperty p = new StructuredProperty(); + p.setValue(value); + Qualifier schema = new Qualifier(); + schema.setClassname(className); + schema.setClassid(className); + schema.setSchemename(schemeName); + schema.setSchemeid(schemeName); + p.setQualifier(schema); + return p; + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java index 42ca86f5f..091438195 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/ISClient.java @@ -1,8 +1,23 @@ + package eu.dnetlib.dhp.actionmanager; +import java.io.Serializable; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; + +import org.dom4j.Document; +import org.dom4j.Element; +import org.dom4j.io.SAXReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; + import eu.dnetlib.actionmanager.rmi.ActionManagerException; import eu.dnetlib.actionmanager.set.ActionManagerSet; import eu.dnetlib.actionmanager.set.ActionManagerSet.ImpactTypes; @@ -10,130 +25,120 @@ import eu.dnetlib.dhp.actionmanager.partition.PartitionActionSetsByPayloadTypeJo import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import java.io.Serializable; -import java.io.StringReader; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.stream.Collectors; -import org.dom4j.Document; -import org.dom4j.Element; -import org.dom4j.io.SAXReader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ISClient implements Serializable { - private static final Logger log = - LoggerFactory.getLogger(PartitionActionSetsByPayloadTypeJob.class); + private static final Logger log = LoggerFactory.getLogger(PartitionActionSetsByPayloadTypeJob.class); - private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; + private static final String INPUT_ACTION_SET_ID_SEPARATOR = ","; - private ISLookUpService isLookup; + private ISLookUpService isLookup; - public ISClient(String isLookupUrl) { - isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); - } + public ISClient(String isLookupUrl) { + isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); + } - public List getLatestRawsetPaths(String setIds) { + public List getLatestRawsetPaths(String setIds) { - List ids = - Lists.newArrayList( - Splitter.on(INPUT_ACTION_SET_ID_SEPARATOR) - .omitEmptyStrings() - .trimResults() - .split(setIds)); + List ids = Lists + .newArrayList( + Splitter + .on(INPUT_ACTION_SET_ID_SEPARATOR) + .omitEmptyStrings() + .trimResults() + .split(setIds)); - return ids.stream() - .map(id -> getSet(isLookup, id)) - .map(as -> as.getPathToLatest()) - .collect(Collectors.toCollection(ArrayList::new)); - } + return ids + .stream() + .map(id -> getSet(isLookup, id)) + .map(as -> as.getPathToLatest()) + .collect(Collectors.toCollection(ArrayList::new)); + } - private ActionManagerSet getSet(ISLookUpService isLookup, final String setId) { + private ActionManagerSet getSet(ISLookUpService isLookup, final String setId) { - final String q = - "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') " - + "where $x//SET/@id = '" - + setId - + "' return $x"; + final String q = "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') " + + "where $x//SET/@id = '" + + setId + + "' return $x"; - try { - final String basePath = getBasePathHDFS(isLookup); - final String setProfile = isLookup.getResourceProfileByQuery(q); - return getActionManagerSet(basePath, setProfile); - } catch (ISLookUpException | ActionManagerException e) { - throw new RuntimeException("Error accessing Sets, using query: " + q); - } - } + try { + final String basePath = getBasePathHDFS(isLookup); + final String setProfile = isLookup.getResourceProfileByQuery(q); + return getActionManagerSet(basePath, setProfile); + } catch (ISLookUpException | ActionManagerException e) { + throw new RuntimeException("Error accessing Sets, using query: " + q); + } + } - private ActionManagerSet getActionManagerSet(final String basePath, final String profile) - throws ActionManagerException { - final SAXReader reader = new SAXReader(); - final ActionManagerSet set = new ActionManagerSet(); + private ActionManagerSet getActionManagerSet(final String basePath, final String profile) + throws ActionManagerException { + final SAXReader reader = new SAXReader(); + final ActionManagerSet set = new ActionManagerSet(); - try { - final Document doc = reader.read(new StringReader(profile)); + try { + final Document doc = reader.read(new StringReader(profile)); - set.setId(doc.valueOf("//SET/@id").trim()); - set.setName(doc.valueOf("//SET").trim()); - set.setImpact(ImpactTypes.valueOf(doc.valueOf("//IMPACT").trim())); - set.setLatest( - doc.valueOf("//RAW_SETS/LATEST/@id"), - doc.valueOf("//RAW_SETS/LATEST/@creationDate"), - doc.valueOf("//RAW_SETS/LATEST/@lastUpdate")); - set.setDirectory(doc.valueOf("//SET/@directory")); - final List expiredNodes = doc.selectNodes("//RAW_SETS/EXPIRED"); - if (expiredNodes != null) { - for (int i = 0; i < expiredNodes.size(); i++) { - Element ex = (Element) expiredNodes.get(i); - set.addExpired( - ex.attributeValue("id"), - ex.attributeValue("creationDate"), - ex.attributeValue("lastUpdate")); - } - } + set.setId(doc.valueOf("//SET/@id").trim()); + set.setName(doc.valueOf("//SET").trim()); + set.setImpact(ImpactTypes.valueOf(doc.valueOf("//IMPACT").trim())); + set + .setLatest( + doc.valueOf("//RAW_SETS/LATEST/@id"), + doc.valueOf("//RAW_SETS/LATEST/@creationDate"), + doc.valueOf("//RAW_SETS/LATEST/@lastUpdate")); + set.setDirectory(doc.valueOf("//SET/@directory")); + final List expiredNodes = doc.selectNodes("//RAW_SETS/EXPIRED"); + if (expiredNodes != null) { + for (int i = 0; i < expiredNodes.size(); i++) { + Element ex = (Element) expiredNodes.get(i); + set + .addExpired( + ex.attributeValue("id"), + ex.attributeValue("creationDate"), + ex.attributeValue("lastUpdate")); + } + } - final StringBuilder sb = new StringBuilder(); - sb.append(basePath); - sb.append("/"); - sb.append(doc.valueOf("//SET/@directory")); - sb.append("/"); - sb.append(doc.valueOf("//RAW_SETS/LATEST/@id")); - set.setPathToLatest(sb.toString()); + final StringBuilder sb = new StringBuilder(); + sb.append(basePath); + sb.append("/"); + sb.append(doc.valueOf("//SET/@directory")); + sb.append("/"); + sb.append(doc.valueOf("//RAW_SETS/LATEST/@id")); + set.setPathToLatest(sb.toString()); - return set; - } catch (Exception e) { - throw new ActionManagerException("Error creating set from profile: " + profile, e); - } - } + return set; + } catch (Exception e) { + throw new ActionManagerException("Error creating set from profile: " + profile, e); + } + } - private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { - return queryServiceProperty(isLookup, "basePath"); - } + private String getBasePathHDFS(ISLookUpService isLookup) throws ActionManagerException { + return queryServiceProperty(isLookup, "basePath"); + } - private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) - throws ActionManagerException { - final String q = - "for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='ActionManagerServiceResourceType'] return $x//SERVICE_PROPERTIES/PROPERTY[./@ key='" - + propertyName - + "']/@value/string()"; - log.debug("quering for service property: " + q); - try { - final List value = isLookup.quickSearchProfile(q); - return Iterables.getOnlyElement(value); - } catch (ISLookUpException e) { - String msg = "Error accessing service profile, using query: " + q; - log.error(msg, e); - throw new ActionManagerException(msg, e); - } catch (NoSuchElementException e) { - String msg = "missing service property: " + propertyName; - log.error(msg, e); - throw new ActionManagerException(msg, e); - } catch (IllegalArgumentException e) { - String msg = "found more than one service property: " + propertyName; - log.error(msg, e); - throw new ActionManagerException(msg, e); - } - } + private String queryServiceProperty(ISLookUpService isLookup, final String propertyName) + throws ActionManagerException { + final String q = "for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='ActionManagerServiceResourceType'] return $x//SERVICE_PROPERTIES/PROPERTY[./@ key='" + + propertyName + + "']/@value/string()"; + log.debug("quering for service property: " + q); + try { + final List value = isLookup.quickSearchProfile(q); + return Iterables.getOnlyElement(value); + } catch (ISLookUpException e) { + String msg = "Error accessing service profile, using query: " + q; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } catch (NoSuchElementException e) { + String msg = "missing service property: " + propertyName; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } catch (IllegalArgumentException e) { + String msg = "found more than one service property: " + propertyName; + log.error(msg, e); + throw new ActionManagerException(msg, e); + } + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/LicenseComparator.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/LicenseComparator.java index ae498c411..7b6046f8b 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/LicenseComparator.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/LicenseComparator.java @@ -1,47 +1,69 @@ + package eu.dnetlib.dhp.actionmanager.migration; -import eu.dnetlib.data.proto.FieldTypeProtos.Qualifier; import java.util.Comparator; +import eu.dnetlib.data.proto.FieldTypeProtos.Qualifier; + public class LicenseComparator implements Comparator { - @Override - public int compare(Qualifier left, Qualifier right) { + @Override + public int compare(Qualifier left, Qualifier right) { - if (left == null && right == null) return 0; - if (left == null) return 1; - if (right == null) return -1; + if (left == null && right == null) + return 0; + if (left == null) + return 1; + if (right == null) + return -1; - String lClass = left.getClassid(); - String rClass = right.getClassid(); + String lClass = left.getClassid(); + String rClass = right.getClassid(); - if (lClass.equals(rClass)) return 0; + if (lClass.equals(rClass)) + return 0; - if (lClass.equals("OPEN SOURCE")) return -1; - if (rClass.equals("OPEN SOURCE")) return 1; + if (lClass.equals("OPEN SOURCE")) + return -1; + if (rClass.equals("OPEN SOURCE")) + return 1; - if (lClass.equals("OPEN")) return -1; - if (rClass.equals("OPEN")) return 1; + if (lClass.equals("OPEN")) + return -1; + if (rClass.equals("OPEN")) + return 1; - if (lClass.equals("6MONTHS")) return -1; - if (rClass.equals("6MONTHS")) return 1; + if (lClass.equals("6MONTHS")) + return -1; + if (rClass.equals("6MONTHS")) + return 1; - if (lClass.equals("12MONTHS")) return -1; - if (rClass.equals("12MONTHS")) return 1; + if (lClass.equals("12MONTHS")) + return -1; + if (rClass.equals("12MONTHS")) + return 1; - if (lClass.equals("EMBARGO")) return -1; - if (rClass.equals("EMBARGO")) return 1; + if (lClass.equals("EMBARGO")) + return -1; + if (rClass.equals("EMBARGO")) + return 1; - if (lClass.equals("RESTRICTED")) return -1; - if (rClass.equals("RESTRICTED")) return 1; + if (lClass.equals("RESTRICTED")) + return -1; + if (rClass.equals("RESTRICTED")) + return 1; - if (lClass.equals("CLOSED")) return -1; - if (rClass.equals("CLOSED")) return 1; + if (lClass.equals("CLOSED")) + return -1; + if (rClass.equals("CLOSED")) + return 1; - if (lClass.equals("UNKNOWN")) return -1; - if (rClass.equals("UNKNOWN")) return 1; + if (lClass.equals("UNKNOWN")) + return -1; + if (rClass.equals("UNKNOWN")) + return 1; - // Else (but unlikely), lexicographical ordering will do. - return lClass.compareTo(rClass); - } + // Else (but unlikely), lexicographical ordering will do. + return lClass.compareTo(rClass); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/MigrateActionSet.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/MigrateActionSet.java index 43ad7c5e3..89cb63fab 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/MigrateActionSet.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/MigrateActionSet.java @@ -1,11 +1,6 @@ + package eu.dnetlib.dhp.actionmanager.migration; -import com.google.common.base.Splitter; -import com.google.common.collect.Lists; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import java.io.File; import java.io.FileOutputStream; import java.io.OutputStream; @@ -14,6 +9,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Properties; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -25,164 +21,174 @@ import org.apache.hadoop.util.ToolRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Splitter; +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + public class MigrateActionSet { - private static final Logger log = LoggerFactory.getLogger(MigrateActionSet.class); + private static final Logger log = LoggerFactory.getLogger(MigrateActionSet.class); - private static final String SEPARATOR = "/"; - private static final String TARGET_PATHS = "target_paths"; - private static final String RAWSET_PREFIX = "rawset_"; + private static final String SEPARATOR = "/"; + private static final String TARGET_PATHS = "target_paths"; + private static final String RAWSET_PREFIX = "rawset_"; - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateActionSet.class.getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/migration/migrate_actionsets_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateActionSet.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/migration/migrate_actionsets_parameters.json"))); + parser.parseArgument(args); - new MigrateActionSet().run(parser); - } + new MigrateActionSet().run(parser); + } - private void run(ArgumentApplicationParser parser) throws Exception { + private void run(ArgumentApplicationParser parser) throws Exception { - final String isLookupUrl = parser.get("isLookupUrl"); - final String sourceNN = parser.get("sourceNameNode"); - final String targetNN = parser.get("targetNameNode"); - final String workDir = parser.get("workingDirectory"); - final Integer distcp_num_maps = Integer.parseInt(parser.get("distcp_num_maps")); + final String isLookupUrl = parser.get("isLookupUrl"); + final String sourceNN = parser.get("sourceNameNode"); + final String targetNN = parser.get("targetNameNode"); + final String workDir = parser.get("workingDirectory"); + final Integer distcp_num_maps = Integer.parseInt(parser.get("distcp_num_maps")); - final String distcp_memory_mb = parser.get("distcp_memory_mb"); - final String distcp_task_timeout = parser.get("distcp_task_timeout"); + final String distcp_memory_mb = parser.get("distcp_memory_mb"); + final String distcp_task_timeout = parser.get("distcp_task_timeout"); - final String transform_only_s = parser.get("transform_only"); + final String transform_only_s = parser.get("transform_only"); - log.info("transform only param: {}", transform_only_s); + log.info("transform only param: {}", transform_only_s); - final Boolean transformOnly = Boolean.valueOf(parser.get("transform_only")); + final Boolean transformOnly = Boolean.valueOf(parser.get("transform_only")); - log.info("transform only: {}", transformOnly); + log.info("transform only: {}", transformOnly); - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - Configuration conf = getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps); - FileSystem targetFS = FileSystem.get(conf); + Configuration conf = getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps); + FileSystem targetFS = FileSystem.get(conf); - Configuration sourceConf = - getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps); - sourceConf.set(FileSystem.FS_DEFAULT_NAME_KEY, sourceNN); - FileSystem sourceFS = FileSystem.get(sourceConf); + Configuration sourceConf = getConfiguration(distcp_task_timeout, distcp_memory_mb, distcp_num_maps); + sourceConf.set(FileSystem.FS_DEFAULT_NAME_KEY, sourceNN); + FileSystem sourceFS = FileSystem.get(sourceConf); - Properties props = new Properties(); + Properties props = new Properties(); - List targetPaths = new ArrayList<>(); + List targetPaths = new ArrayList<>(); - final List sourcePaths = getSourcePaths(sourceNN, isLookUp); - log.info( - "paths to process:\n{}", - sourcePaths.stream().map(p -> p.toString()).collect(Collectors.joining("\n"))); - for (Path source : sourcePaths) { + final List sourcePaths = getSourcePaths(sourceNN, isLookUp); + log + .info( + "paths to process:\n{}", + sourcePaths.stream().map(p -> p.toString()).collect(Collectors.joining("\n"))); + for (Path source : sourcePaths) { - if (!sourceFS.exists(source)) { - log.warn("skipping unexisting path: {}", source); - } else { + if (!sourceFS.exists(source)) { + log.warn("skipping unexisting path: {}", source); + } else { - LinkedList pathQ = - Lists.newLinkedList(Splitter.on(SEPARATOR).split(source.toUri().getPath())); + LinkedList pathQ = Lists.newLinkedList(Splitter.on(SEPARATOR).split(source.toUri().getPath())); - final String rawSet = pathQ.pollLast(); - log.info("got RAWSET: {}", rawSet); + final String rawSet = pathQ.pollLast(); + log.info("got RAWSET: {}", rawSet); - if (StringUtils.isNotBlank(rawSet) && rawSet.startsWith(RAWSET_PREFIX)) { + if (StringUtils.isNotBlank(rawSet) && rawSet.startsWith(RAWSET_PREFIX)) { - final String actionSetDirectory = pathQ.pollLast(); + final String actionSetDirectory = pathQ.pollLast(); - final Path targetPath = - new Path(targetNN + workDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawSet); + final Path targetPath = new Path( + targetNN + workDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawSet); - log.info("using TARGET PATH: {}", targetPath); + log.info("using TARGET PATH: {}", targetPath); - if (!transformOnly) { - if (targetFS.exists(targetPath)) { - targetFS.delete(targetPath, true); - } - runDistcp( - distcp_num_maps, distcp_memory_mb, distcp_task_timeout, conf, source, targetPath); - } + if (!transformOnly) { + if (targetFS.exists(targetPath)) { + targetFS.delete(targetPath, true); + } + runDistcp( + distcp_num_maps, distcp_memory_mb, distcp_task_timeout, conf, source, targetPath); + } - targetPaths.add(targetPath); - } - } - } + targetPaths.add(targetPath); + } + } + } - props.setProperty( - TARGET_PATHS, targetPaths.stream().map(p -> p.toString()).collect(Collectors.joining(","))); - File file = new File(System.getProperty("oozie.action.output.properties")); + props + .setProperty( + TARGET_PATHS, targetPaths.stream().map(p -> p.toString()).collect(Collectors.joining(","))); + File file = new File(System.getProperty("oozie.action.output.properties")); - try (OutputStream os = new FileOutputStream(file)) { - props.store(os, ""); - } - System.out.println(file.getAbsolutePath()); - } + try (OutputStream os = new FileOutputStream(file)) { + props.store(os, ""); + } + System.out.println(file.getAbsolutePath()); + } - private void runDistcp( - Integer distcp_num_maps, - String distcp_memory_mb, - String distcp_task_timeout, - Configuration conf, - Path source, - Path targetPath) - throws Exception { + private void runDistcp( + Integer distcp_num_maps, + String distcp_memory_mb, + String distcp_task_timeout, + Configuration conf, + Path source, + Path targetPath) + throws Exception { - final DistCpOptions op = new DistCpOptions(source, targetPath); - op.setMaxMaps(distcp_num_maps); - op.preserve(DistCpOptions.FileAttribute.BLOCKSIZE); - op.preserve(DistCpOptions.FileAttribute.REPLICATION); - op.preserve(DistCpOptions.FileAttribute.CHECKSUMTYPE); + final DistCpOptions op = new DistCpOptions(source, targetPath); + op.setMaxMaps(distcp_num_maps); + op.preserve(DistCpOptions.FileAttribute.BLOCKSIZE); + op.preserve(DistCpOptions.FileAttribute.REPLICATION); + op.preserve(DistCpOptions.FileAttribute.CHECKSUMTYPE); - int res = - ToolRunner.run( - new DistCp(conf, op), - new String[] { - "-Dmapred.task.timeout=" + distcp_task_timeout, - "-Dmapreduce.map.memory.mb=" + distcp_memory_mb, - "-pb", - "-m " + distcp_num_maps, - source.toString(), - targetPath.toString() - }); + int res = ToolRunner + .run( + new DistCp(conf, op), + new String[] { + "-Dmapred.task.timeout=" + distcp_task_timeout, + "-Dmapreduce.map.memory.mb=" + distcp_memory_mb, + "-pb", + "-m " + distcp_num_maps, + source.toString(), + targetPath.toString() + }); - if (res != 0) { - throw new RuntimeException(String.format("distcp exited with code %s", res)); - } - } + if (res != 0) { + throw new RuntimeException(String.format("distcp exited with code %s", res)); + } + } - private Configuration getConfiguration( - String distcp_task_timeout, String distcp_memory_mb, Integer distcp_num_maps) { - final Configuration conf = new Configuration(); - conf.set("dfs.webhdfs.socket.connect-timeout", distcp_task_timeout); - conf.set("dfs.webhdfs.socket.read-timeout", distcp_task_timeout); - conf.set("dfs.http.client.retry.policy.enabled", "true"); - conf.set("mapred.task.timeout", distcp_task_timeout); - conf.set("mapreduce.map.memory.mb", distcp_memory_mb); - conf.set("mapred.map.tasks", String.valueOf(distcp_num_maps)); - return conf; - } + private Configuration getConfiguration( + String distcp_task_timeout, String distcp_memory_mb, Integer distcp_num_maps) { + final Configuration conf = new Configuration(); + conf.set("dfs.webhdfs.socket.connect-timeout", distcp_task_timeout); + conf.set("dfs.webhdfs.socket.read-timeout", distcp_task_timeout); + conf.set("dfs.http.client.retry.policy.enabled", "true"); + conf.set("mapred.task.timeout", distcp_task_timeout); + conf.set("mapreduce.map.memory.mb", distcp_memory_mb); + conf.set("mapred.map.tasks", String.valueOf(distcp_num_maps)); + return conf; + } - private List getSourcePaths(String sourceNN, ISLookUpService isLookUp) - throws ISLookUpException { - String XQUERY = - "distinct-values(\n" - + "let $basePath := collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()\n" - + "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') \n" - + "let $setDir := $x//SET/@directory/string()\n" - + "let $rawSet := $x//RAW_SETS/LATEST/@id/string()\n" - + "return concat($basePath, '/', $setDir, '/', $rawSet))"; + private List getSourcePaths(String sourceNN, ISLookUpService isLookUp) + throws ISLookUpException { + String XQUERY = "distinct-values(\n" + + "let $basePath := collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()\n" + + "for $x in collection('/db/DRIVER/ActionManagerSetDSResources/ActionManagerSetDSResourceType') \n" + + "let $setDir := $x//SET/@directory/string()\n" + + "let $rawSet := $x//RAW_SETS/LATEST/@id/string()\n" + + "return concat($basePath, '/', $setDir, '/', $rawSet))"; - log.info(String.format("running xquery:\n%s", XQUERY)); - return isLookUp.quickSearchProfile(XQUERY).stream() - .map(p -> sourceNN + p) - .map(Path::new) - .collect(Collectors.toList()); - } + log.info(String.format("running xquery:\n%s", XQUERY)); + return isLookUp + .quickSearchProfile(XQUERY) + .stream() + .map(p -> sourceNN + p) + .map(Path::new) + .collect(Collectors.toList()); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/ProtoConverter.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/ProtoConverter.java index 894804e25..456113c43 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/ProtoConverter.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/ProtoConverter.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.actionmanager.migration; import static eu.dnetlib.data.proto.KindProtos.Kind.entity; @@ -5,569 +6,659 @@ import static eu.dnetlib.data.proto.KindProtos.Kind.relation; import static eu.dnetlib.data.proto.TypeProtos.*; import static eu.dnetlib.data.proto.TypeProtos.Type.*; -import com.google.common.collect.Lists; -import com.googlecode.protobuf.format.JsonFormat; -import eu.dnetlib.data.proto.*; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.Serializable; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; +import com.google.common.collect.Lists; +import com.googlecode.protobuf.format.JsonFormat; + +import eu.dnetlib.data.proto.*; +import eu.dnetlib.dhp.schema.oaf.*; + public class ProtoConverter implements Serializable { - public static final String UNKNOWN = "UNKNOWN"; - public static final String NOT_AVAILABLE = "not available"; - public static final String DNET_ACCESS_MODES = "dnet:access_modes"; + public static final String UNKNOWN = "UNKNOWN"; + public static final String NOT_AVAILABLE = "not available"; + public static final String DNET_ACCESS_MODES = "dnet:access_modes"; - public static Oaf convert(OafProtos.Oaf oaf) { - try { - switch (oaf.getKind()) { - case entity: - return convertEntity(oaf); - case relation: - return convertRelation(oaf); - default: - throw new IllegalArgumentException("invalid kind " + oaf.getKind()); - } - } catch (Throwable e) { - throw new RuntimeException("error on getting " + JsonFormat.printToString(oaf), e); - } - } + public static Oaf convert(OafProtos.Oaf oaf) { + try { + switch (oaf.getKind()) { + case entity: + return convertEntity(oaf); + case relation: + return convertRelation(oaf); + default: + throw new IllegalArgumentException("invalid kind " + oaf.getKind()); + } + } catch (Throwable e) { + throw new RuntimeException("error on getting " + JsonFormat.printToString(oaf), e); + } + } - private static Relation convertRelation(OafProtos.Oaf oaf) { - final OafProtos.OafRel r = oaf.getRel(); - final Relation rel = new Relation(); - rel.setDataInfo(mapDataInfo(oaf.getDataInfo())); - rel.setLastupdatetimestamp(oaf.getLastupdatetimestamp()); - rel.setSource(r.getSource()); - rel.setTarget(r.getTarget()); - rel.setRelType(r.getRelType().toString()); - rel.setSubRelType(r.getSubRelType().toString()); - rel.setRelClass(r.getRelClass()); - rel.setCollectedfrom( - r.getCollectedfromCount() > 0 - ? r.getCollectedfromList().stream().map(kv -> mapKV(kv)).collect(Collectors.toList()) - : null); - return rel; - } + private static Relation convertRelation(OafProtos.Oaf oaf) { + final OafProtos.OafRel r = oaf.getRel(); + final Relation rel = new Relation(); + rel.setDataInfo(mapDataInfo(oaf.getDataInfo())); + rel.setLastupdatetimestamp(oaf.getLastupdatetimestamp()); + rel.setSource(r.getSource()); + rel.setTarget(r.getTarget()); + rel.setRelType(r.getRelType().toString()); + rel.setSubRelType(r.getSubRelType().toString()); + rel.setRelClass(r.getRelClass()); + rel + .setCollectedfrom( + r.getCollectedfromCount() > 0 + ? r.getCollectedfromList().stream().map(kv -> mapKV(kv)).collect(Collectors.toList()) + : null); + return rel; + } - private static OafEntity convertEntity(OafProtos.Oaf oaf) { + private static OafEntity convertEntity(OafProtos.Oaf oaf) { - switch (oaf.getEntity().getType()) { - case result: - final Result r = convertResult(oaf); - r.setInstance(convertInstances(oaf)); - return r; - case project: - return convertProject(oaf); - case datasource: - return convertDataSource(oaf); - case organization: - return convertOrganization(oaf); - default: - throw new RuntimeException("received unknown type"); - } - } + switch (oaf.getEntity().getType()) { + case result: + final Result r = convertResult(oaf); + r.setInstance(convertInstances(oaf)); + return r; + case project: + return convertProject(oaf); + case datasource: + return convertDataSource(oaf); + case organization: + return convertOrganization(oaf); + default: + throw new RuntimeException("received unknown type"); + } + } - private static List convertInstances(OafProtos.Oaf oaf) { + private static List convertInstances(OafProtos.Oaf oaf) { - final ResultProtos.Result r = oaf.getEntity().getResult(); - if (r.getInstanceCount() > 0) { - return r.getInstanceList().stream().map(i -> convertInstance(i)).collect(Collectors.toList()); - } - return Lists.newArrayList(); - } + final ResultProtos.Result r = oaf.getEntity().getResult(); + if (r.getInstanceCount() > 0) { + return r.getInstanceList().stream().map(i -> convertInstance(i)).collect(Collectors.toList()); + } + return Lists.newArrayList(); + } - private static Instance convertInstance(ResultProtos.Result.Instance ri) { - final Instance i = new Instance(); - i.setAccessright(mapQualifier(ri.getAccessright())); - i.setCollectedfrom(mapKV(ri.getCollectedfrom())); - i.setDateofacceptance(mapStringField(ri.getDateofacceptance())); - i.setDistributionlocation(ri.getDistributionlocation()); - i.setHostedby(mapKV(ri.getHostedby())); - i.setInstancetype(mapQualifier(ri.getInstancetype())); - i.setLicense(mapStringField(ri.getLicense())); - i.setUrl(ri.getUrlList()); - i.setRefereed(mapStringField(ri.getRefereed())); - i.setProcessingchargeamount(mapStringField(ri.getProcessingchargeamount())); - i.setProcessingchargecurrency(mapStringField(ri.getProcessingchargecurrency())); - return i; - } + private static Instance convertInstance(ResultProtos.Result.Instance ri) { + final Instance i = new Instance(); + i.setAccessright(mapQualifier(ri.getAccessright())); + i.setCollectedfrom(mapKV(ri.getCollectedfrom())); + i.setDateofacceptance(mapStringField(ri.getDateofacceptance())); + i.setDistributionlocation(ri.getDistributionlocation()); + i.setHostedby(mapKV(ri.getHostedby())); + i.setInstancetype(mapQualifier(ri.getInstancetype())); + i.setLicense(mapStringField(ri.getLicense())); + i.setUrl(ri.getUrlList()); + i.setRefereed(mapStringField(ri.getRefereed())); + i.setProcessingchargeamount(mapStringField(ri.getProcessingchargeamount())); + i.setProcessingchargecurrency(mapStringField(ri.getProcessingchargecurrency())); + return i; + } - private static Organization convertOrganization(OafProtos.Oaf oaf) { - final OrganizationProtos.Organization.Metadata m = - oaf.getEntity().getOrganization().getMetadata(); - final Organization org = setOaf(new Organization(), oaf); - setEntity(org, oaf); - org.setLegalshortname(mapStringField(m.getLegalshortname())); - org.setLegalname(mapStringField(m.getLegalname())); - org.setAlternativeNames( - m.getAlternativeNamesList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - org.setWebsiteurl(mapStringField(m.getWebsiteurl())); - org.setLogourl(mapStringField(m.getLogourl())); - org.setEclegalbody(mapStringField(m.getEclegalbody())); - org.setEclegalperson(mapStringField(m.getEclegalperson())); - org.setEcnonprofit(mapStringField(m.getEcnonprofit())); - org.setEcresearchorganization(mapStringField(m.getEcresearchorganization())); - org.setEchighereducation(mapStringField(m.getEchighereducation())); - org.setEcinternationalorganizationeurinterests( - mapStringField(m.getEcinternationalorganizationeurinterests())); - org.setEcinternationalorganization(mapStringField(m.getEcinternationalorganization())); - org.setEcenterprise(mapStringField(m.getEcenterprise())); - org.setEcsmevalidated(mapStringField(m.getEcsmevalidated())); - org.setEcnutscode(mapStringField(m.getEcnutscode())); - org.setCountry(mapQualifier(m.getCountry())); + private static Organization convertOrganization(OafProtos.Oaf oaf) { + final OrganizationProtos.Organization.Metadata m = oaf.getEntity().getOrganization().getMetadata(); + final Organization org = setOaf(new Organization(), oaf); + setEntity(org, oaf); + org.setLegalshortname(mapStringField(m.getLegalshortname())); + org.setLegalname(mapStringField(m.getLegalname())); + org + .setAlternativeNames( + m + .getAlternativeNamesList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + org.setWebsiteurl(mapStringField(m.getWebsiteurl())); + org.setLogourl(mapStringField(m.getLogourl())); + org.setEclegalbody(mapStringField(m.getEclegalbody())); + org.setEclegalperson(mapStringField(m.getEclegalperson())); + org.setEcnonprofit(mapStringField(m.getEcnonprofit())); + org.setEcresearchorganization(mapStringField(m.getEcresearchorganization())); + org.setEchighereducation(mapStringField(m.getEchighereducation())); + org + .setEcinternationalorganizationeurinterests( + mapStringField(m.getEcinternationalorganizationeurinterests())); + org.setEcinternationalorganization(mapStringField(m.getEcinternationalorganization())); + org.setEcenterprise(mapStringField(m.getEcenterprise())); + org.setEcsmevalidated(mapStringField(m.getEcsmevalidated())); + org.setEcnutscode(mapStringField(m.getEcnutscode())); + org.setCountry(mapQualifier(m.getCountry())); - return org; - } + return org; + } - private static Datasource convertDataSource(OafProtos.Oaf oaf) { - final DatasourceProtos.Datasource.Metadata m = oaf.getEntity().getDatasource().getMetadata(); - final Datasource datasource = setOaf(new Datasource(), oaf); - setEntity(datasource, oaf); - datasource.setAccessinfopackage( - m.getAccessinfopackageList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - datasource.setCertificates(mapStringField(m.getCertificates())); - datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl())); - datasource.setContactemail(mapStringField(m.getContactemail())); - datasource.setDatabaseaccessrestriction(mapStringField(m.getDatabaseaccessrestriction())); - datasource.setDatabaseaccesstype(mapStringField(m.getDatabaseaccesstype())); - datasource.setDataprovider(mapBoolField(m.getDataprovider())); - datasource.setDatasourcetype(mapQualifier(m.getDatasourcetype())); - datasource.setDatauploadrestriction(mapStringField(m.getDatauploadrestriction())); - datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl())); - datasource.setDatauploadtype(mapStringField(m.getDatauploadtype())); - datasource.setDateofvalidation(mapStringField(m.getDateofvalidation())); - datasource.setDescription(mapStringField(m.getDescription())); - datasource.setEnglishname(mapStringField(m.getEnglishname())); - datasource.setLatitude(mapStringField(m.getLatitude())); - datasource.setLongitude(mapStringField(m.getLongitude())); - datasource.setLogourl(mapStringField(m.getLogourl())); - datasource.setMissionstatementurl(mapStringField(m.getMissionstatementurl())); - datasource.setNamespaceprefix(mapStringField(m.getNamespaceprefix())); - datasource.setOdcontenttypes( - m.getOdcontenttypesList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - datasource.setOdlanguages( - m.getOdlanguagesList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - datasource.setOdnumberofitems(mapStringField(m.getOdnumberofitems())); - datasource.setOdnumberofitemsdate(mapStringField(m.getOdnumberofitemsdate())); - datasource.setOdpolicies(mapStringField(m.getOdpolicies())); - datasource.setOfficialname(mapStringField(m.getOfficialname())); - datasource.setOpenairecompatibility(mapQualifier(m.getOpenairecompatibility())); - datasource.setPidsystems(mapStringField(m.getPidsystems())); - datasource.setPolicies( - m.getPoliciesList().stream().map(ProtoConverter::mapKV).collect(Collectors.toList())); - datasource.setQualitymanagementkind(mapStringField(m.getQualitymanagementkind())); - datasource.setReleaseenddate(mapStringField(m.getReleaseenddate())); - datasource.setServiceprovider(mapBoolField(m.getServiceprovider())); - datasource.setReleasestartdate(mapStringField(m.getReleasestartdate())); - datasource.setSubjects( - m.getSubjectsList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - datasource.setVersioning(mapBoolField(m.getVersioning())); - datasource.setWebsiteurl(mapStringField(m.getWebsiteurl())); - datasource.setJournal(mapJournal(m.getJournal())); + private static Datasource convertDataSource(OafProtos.Oaf oaf) { + final DatasourceProtos.Datasource.Metadata m = oaf.getEntity().getDatasource().getMetadata(); + final Datasource datasource = setOaf(new Datasource(), oaf); + setEntity(datasource, oaf); + datasource + .setAccessinfopackage( + m + .getAccessinfopackageList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + datasource.setCertificates(mapStringField(m.getCertificates())); + datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl())); + datasource.setContactemail(mapStringField(m.getContactemail())); + datasource.setDatabaseaccessrestriction(mapStringField(m.getDatabaseaccessrestriction())); + datasource.setDatabaseaccesstype(mapStringField(m.getDatabaseaccesstype())); + datasource.setDataprovider(mapBoolField(m.getDataprovider())); + datasource.setDatasourcetype(mapQualifier(m.getDatasourcetype())); + datasource.setDatauploadrestriction(mapStringField(m.getDatauploadrestriction())); + datasource.setCitationguidelineurl(mapStringField(m.getCitationguidelineurl())); + datasource.setDatauploadtype(mapStringField(m.getDatauploadtype())); + datasource.setDateofvalidation(mapStringField(m.getDateofvalidation())); + datasource.setDescription(mapStringField(m.getDescription())); + datasource.setEnglishname(mapStringField(m.getEnglishname())); + datasource.setLatitude(mapStringField(m.getLatitude())); + datasource.setLongitude(mapStringField(m.getLongitude())); + datasource.setLogourl(mapStringField(m.getLogourl())); + datasource.setMissionstatementurl(mapStringField(m.getMissionstatementurl())); + datasource.setNamespaceprefix(mapStringField(m.getNamespaceprefix())); + datasource + .setOdcontenttypes( + m + .getOdcontenttypesList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + datasource + .setOdlanguages( + m + .getOdlanguagesList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + datasource.setOdnumberofitems(mapStringField(m.getOdnumberofitems())); + datasource.setOdnumberofitemsdate(mapStringField(m.getOdnumberofitemsdate())); + datasource.setOdpolicies(mapStringField(m.getOdpolicies())); + datasource.setOfficialname(mapStringField(m.getOfficialname())); + datasource.setOpenairecompatibility(mapQualifier(m.getOpenairecompatibility())); + datasource.setPidsystems(mapStringField(m.getPidsystems())); + datasource + .setPolicies( + m.getPoliciesList().stream().map(ProtoConverter::mapKV).collect(Collectors.toList())); + datasource.setQualitymanagementkind(mapStringField(m.getQualitymanagementkind())); + datasource.setReleaseenddate(mapStringField(m.getReleaseenddate())); + datasource.setServiceprovider(mapBoolField(m.getServiceprovider())); + datasource.setReleasestartdate(mapStringField(m.getReleasestartdate())); + datasource + .setSubjects( + m + .getSubjectsList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + datasource.setVersioning(mapBoolField(m.getVersioning())); + datasource.setWebsiteurl(mapStringField(m.getWebsiteurl())); + datasource.setJournal(mapJournal(m.getJournal())); - return datasource; - } + return datasource; + } - private static Project convertProject(OafProtos.Oaf oaf) { - final ProjectProtos.Project.Metadata m = oaf.getEntity().getProject().getMetadata(); - final Project project = setOaf(new Project(), oaf); - setEntity(project, oaf); - project.setAcronym(mapStringField(m.getAcronym())); - project.setCallidentifier(mapStringField(m.getCallidentifier())); - project.setCode(mapStringField(m.getCode())); - project.setContactemail(mapStringField(m.getContactemail())); - project.setContactfax(mapStringField(m.getContactfax())); - project.setContactfullname(mapStringField(m.getContactfullname())); - project.setContactphone(mapStringField(m.getContactphone())); - project.setContracttype(mapQualifier(m.getContracttype())); - project.setCurrency(mapStringField(m.getCurrency())); - project.setDuration(mapStringField(m.getDuration())); - project.setEcarticle29_3(mapStringField(m.getEcarticle293())); - project.setEcsc39(mapStringField(m.getEcsc39())); - project.setOamandatepublications(mapStringField(m.getOamandatepublications())); - project.setStartdate(mapStringField(m.getStartdate())); - project.setEnddate(mapStringField(m.getEnddate())); - project.setFundedamount(m.getFundedamount()); - project.setTotalcost(m.getTotalcost()); - project.setKeywords(mapStringField(m.getKeywords())); - project.setSubjects( - m.getSubjectsList().stream() - .map(sp -> mapStructuredProperty(sp)) - .collect(Collectors.toList())); - project.setTitle(mapStringField(m.getTitle())); - project.setWebsiteurl(mapStringField(m.getWebsiteurl())); - project.setFundingtree( - m.getFundingtreeList().stream().map(f -> mapStringField(f)).collect(Collectors.toList())); - project.setJsonextrainfo(mapStringField(m.getJsonextrainfo())); - project.setSummary(mapStringField(m.getSummary())); - project.setOptional1(mapStringField(m.getOptional1())); - project.setOptional2(mapStringField(m.getOptional2())); - return project; - } + private static Project convertProject(OafProtos.Oaf oaf) { + final ProjectProtos.Project.Metadata m = oaf.getEntity().getProject().getMetadata(); + final Project project = setOaf(new Project(), oaf); + setEntity(project, oaf); + project.setAcronym(mapStringField(m.getAcronym())); + project.setCallidentifier(mapStringField(m.getCallidentifier())); + project.setCode(mapStringField(m.getCode())); + project.setContactemail(mapStringField(m.getContactemail())); + project.setContactfax(mapStringField(m.getContactfax())); + project.setContactfullname(mapStringField(m.getContactfullname())); + project.setContactphone(mapStringField(m.getContactphone())); + project.setContracttype(mapQualifier(m.getContracttype())); + project.setCurrency(mapStringField(m.getCurrency())); + project.setDuration(mapStringField(m.getDuration())); + project.setEcarticle29_3(mapStringField(m.getEcarticle293())); + project.setEcsc39(mapStringField(m.getEcsc39())); + project.setOamandatepublications(mapStringField(m.getOamandatepublications())); + project.setStartdate(mapStringField(m.getStartdate())); + project.setEnddate(mapStringField(m.getEnddate())); + project.setFundedamount(m.getFundedamount()); + project.setTotalcost(m.getTotalcost()); + project.setKeywords(mapStringField(m.getKeywords())); + project + .setSubjects( + m + .getSubjectsList() + .stream() + .map(sp -> mapStructuredProperty(sp)) + .collect(Collectors.toList())); + project.setTitle(mapStringField(m.getTitle())); + project.setWebsiteurl(mapStringField(m.getWebsiteurl())); + project + .setFundingtree( + m.getFundingtreeList().stream().map(f -> mapStringField(f)).collect(Collectors.toList())); + project.setJsonextrainfo(mapStringField(m.getJsonextrainfo())); + project.setSummary(mapStringField(m.getSummary())); + project.setOptional1(mapStringField(m.getOptional1())); + project.setOptional2(mapStringField(m.getOptional2())); + return project; + } - private static Result convertResult(OafProtos.Oaf oaf) { - switch (oaf.getEntity().getResult().getMetadata().getResulttype().getClassid()) { - case "dataset": - return createDataset(oaf); - case "publication": - return createPublication(oaf); - case "software": - return createSoftware(oaf); - case "other": - return createORP(oaf); - default: - Result result = setOaf(new Result(), oaf); - setEntity(result, oaf); - return setResult(result, oaf); - } - } + private static Result convertResult(OafProtos.Oaf oaf) { + switch (oaf.getEntity().getResult().getMetadata().getResulttype().getClassid()) { + case "dataset": + return createDataset(oaf); + case "publication": + return createPublication(oaf); + case "software": + return createSoftware(oaf); + case "other": + return createORP(oaf); + default: + Result result = setOaf(new Result(), oaf); + setEntity(result, oaf); + return setResult(result, oaf); + } + } - private static Software createSoftware(OafProtos.Oaf oaf) { - ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); - Software software = setOaf(new Software(), oaf); - setEntity(software, oaf); - setResult(software, oaf); + private static Software createSoftware(OafProtos.Oaf oaf) { + ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); + Software software = setOaf(new Software(), oaf); + setEntity(software, oaf); + setResult(software, oaf); - software.setDocumentationUrl( - m.getDocumentationUrlList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - software.setLicense( - m.getLicenseList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - software.setCodeRepositoryUrl(mapStringField(m.getCodeRepositoryUrl())); - software.setProgrammingLanguage(mapQualifier(m.getProgrammingLanguage())); - return software; - } + software + .setDocumentationUrl( + m + .getDocumentationUrlList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + software + .setLicense( + m + .getLicenseList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + software.setCodeRepositoryUrl(mapStringField(m.getCodeRepositoryUrl())); + software.setProgrammingLanguage(mapQualifier(m.getProgrammingLanguage())); + return software; + } - private static OtherResearchProduct createORP(OafProtos.Oaf oaf) { - ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); - OtherResearchProduct otherResearchProducts = setOaf(new OtherResearchProduct(), oaf); - setEntity(otherResearchProducts, oaf); - setResult(otherResearchProducts, oaf); - otherResearchProducts.setContactperson( - m.getContactpersonList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - otherResearchProducts.setContactgroup( - m.getContactgroupList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - otherResearchProducts.setTool( - m.getToolList().stream().map(ProtoConverter::mapStringField).collect(Collectors.toList())); + private static OtherResearchProduct createORP(OafProtos.Oaf oaf) { + ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); + OtherResearchProduct otherResearchProducts = setOaf(new OtherResearchProduct(), oaf); + setEntity(otherResearchProducts, oaf); + setResult(otherResearchProducts, oaf); + otherResearchProducts + .setContactperson( + m + .getContactpersonList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + otherResearchProducts + .setContactgroup( + m + .getContactgroupList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + otherResearchProducts + .setTool( + m.getToolList().stream().map(ProtoConverter::mapStringField).collect(Collectors.toList())); - return otherResearchProducts; - } + return otherResearchProducts; + } - private static Publication createPublication(OafProtos.Oaf oaf) { + private static Publication createPublication(OafProtos.Oaf oaf) { - ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); - Publication publication = setOaf(new Publication(), oaf); - setEntity(publication, oaf); - setResult(publication, oaf); - publication.setJournal(mapJournal(m.getJournal())); - return publication; - } + ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); + Publication publication = setOaf(new Publication(), oaf); + setEntity(publication, oaf); + setResult(publication, oaf); + publication.setJournal(mapJournal(m.getJournal())); + return publication; + } - private static Dataset createDataset(OafProtos.Oaf oaf) { + private static Dataset createDataset(OafProtos.Oaf oaf) { - ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); - Dataset dataset = setOaf(new Dataset(), oaf); - setEntity(dataset, oaf); - setResult(dataset, oaf); - dataset.setStoragedate(mapStringField(m.getStoragedate())); - dataset.setDevice(mapStringField(m.getDevice())); - dataset.setSize(mapStringField(m.getSize())); - dataset.setVersion(mapStringField(m.getVersion())); - dataset.setLastmetadataupdate(mapStringField(m.getLastmetadataupdate())); - dataset.setMetadataversionnumber(mapStringField(m.getMetadataversionnumber())); - dataset.setGeolocation( - m.getGeolocationList().stream() - .map(ProtoConverter::mapGeolocation) - .collect(Collectors.toList())); - return dataset; - } + ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); + Dataset dataset = setOaf(new Dataset(), oaf); + setEntity(dataset, oaf); + setResult(dataset, oaf); + dataset.setStoragedate(mapStringField(m.getStoragedate())); + dataset.setDevice(mapStringField(m.getDevice())); + dataset.setSize(mapStringField(m.getSize())); + dataset.setVersion(mapStringField(m.getVersion())); + dataset.setLastmetadataupdate(mapStringField(m.getLastmetadataupdate())); + dataset.setMetadataversionnumber(mapStringField(m.getMetadataversionnumber())); + dataset + .setGeolocation( + m + .getGeolocationList() + .stream() + .map(ProtoConverter::mapGeolocation) + .collect(Collectors.toList())); + return dataset; + } - public static T setOaf(T oaf, OafProtos.Oaf o) { - oaf.setDataInfo(mapDataInfo(o.getDataInfo())); - oaf.setLastupdatetimestamp(o.getLastupdatetimestamp()); - return oaf; - } + public static T setOaf(T oaf, OafProtos.Oaf o) { + oaf.setDataInfo(mapDataInfo(o.getDataInfo())); + oaf.setLastupdatetimestamp(o.getLastupdatetimestamp()); + return oaf; + } - public static T setEntity(T entity, OafProtos.Oaf oaf) { - // setting Entity fields - final OafProtos.OafEntity e = oaf.getEntity(); - entity.setId(e.getId()); - entity.setOriginalId(e.getOriginalIdList()); - entity.setCollectedfrom( - e.getCollectedfromList().stream().map(ProtoConverter::mapKV).collect(Collectors.toList())); - entity.setPid( - e.getPidList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - entity.setDateofcollection(e.getDateofcollection()); - entity.setDateoftransformation(e.getDateoftransformation()); - entity.setExtraInfo( - e.getExtraInfoList().stream() - .map(ProtoConverter::mapExtraInfo) - .collect(Collectors.toList())); - return entity; - } + public static T setEntity(T entity, OafProtos.Oaf oaf) { + // setting Entity fields + final OafProtos.OafEntity e = oaf.getEntity(); + entity.setId(e.getId()); + entity.setOriginalId(e.getOriginalIdList()); + entity + .setCollectedfrom( + e.getCollectedfromList().stream().map(ProtoConverter::mapKV).collect(Collectors.toList())); + entity + .setPid( + e + .getPidList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + entity.setDateofcollection(e.getDateofcollection()); + entity.setDateoftransformation(e.getDateoftransformation()); + entity + .setExtraInfo( + e + .getExtraInfoList() + .stream() + .map(ProtoConverter::mapExtraInfo) + .collect(Collectors.toList())); + return entity; + } - public static T setResult(T entity, OafProtos.Oaf oaf) { - // setting Entity fields - final ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); - entity.setAuthor( - m.getAuthorList().stream().map(ProtoConverter::mapAuthor).collect(Collectors.toList())); - entity.setResulttype(mapQualifier(m.getResulttype())); - entity.setLanguage(mapQualifier(m.getLanguage())); - entity.setCountry( - m.getCountryList().stream() - .map(ProtoConverter::mapQualifierAsCountry) - .collect(Collectors.toList())); - entity.setSubject( - m.getSubjectList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - entity.setTitle( - m.getTitleList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - entity.setRelevantdate( - m.getRelevantdateList().stream() - .map(ProtoConverter::mapStructuredProperty) - .collect(Collectors.toList())); - entity.setDescription( - m.getDescriptionList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setDateofacceptance(mapStringField(m.getDateofacceptance())); - entity.setPublisher(mapStringField(m.getPublisher())); - entity.setEmbargoenddate(mapStringField(m.getEmbargoenddate())); - entity.setSource( - m.getSourceList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setFulltext( - m.getFulltextList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setFormat( - m.getFormatList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setContributor( - m.getContributorList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setResourcetype(mapQualifier(m.getResourcetype())); - entity.setCoverage( - m.getCoverageList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - entity.setContext( - m.getContextList().stream().map(ProtoConverter::mapContext).collect(Collectors.toList())); + public static T setResult(T entity, OafProtos.Oaf oaf) { + // setting Entity fields + final ResultProtos.Result.Metadata m = oaf.getEntity().getResult().getMetadata(); + entity + .setAuthor( + m.getAuthorList().stream().map(ProtoConverter::mapAuthor).collect(Collectors.toList())); + entity.setResulttype(mapQualifier(m.getResulttype())); + entity.setLanguage(mapQualifier(m.getLanguage())); + entity + .setCountry( + m + .getCountryList() + .stream() + .map(ProtoConverter::mapQualifierAsCountry) + .collect(Collectors.toList())); + entity + .setSubject( + m + .getSubjectList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + entity + .setTitle( + m + .getTitleList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + entity + .setRelevantdate( + m + .getRelevantdateList() + .stream() + .map(ProtoConverter::mapStructuredProperty) + .collect(Collectors.toList())); + entity + .setDescription( + m + .getDescriptionList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity.setDateofacceptance(mapStringField(m.getDateofacceptance())); + entity.setPublisher(mapStringField(m.getPublisher())); + entity.setEmbargoenddate(mapStringField(m.getEmbargoenddate())); + entity + .setSource( + m + .getSourceList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity + .setFulltext( + m + .getFulltextList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity + .setFormat( + m + .getFormatList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity + .setContributor( + m + .getContributorList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity.setResourcetype(mapQualifier(m.getResourcetype())); + entity + .setCoverage( + m + .getCoverageList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + entity + .setContext( + m.getContextList().stream().map(ProtoConverter::mapContext).collect(Collectors.toList())); - entity.setBestaccessright(getBestAccessRights(oaf.getEntity().getResult().getInstanceList())); + entity.setBestaccessright(getBestAccessRights(oaf.getEntity().getResult().getInstanceList())); - return entity; - } + return entity; + } - private static Qualifier getBestAccessRights(List instanceList) { - if (instanceList != null) { - final Optional min = - instanceList.stream().map(i -> i.getAccessright()).min(new LicenseComparator()); + private static Qualifier getBestAccessRights(List instanceList) { + if (instanceList != null) { + final Optional min = instanceList + .stream() + .map(i -> i.getAccessright()) + .min(new LicenseComparator()); - final Qualifier rights = min.isPresent() ? mapQualifier(min.get()) : new Qualifier(); + final Qualifier rights = min.isPresent() ? mapQualifier(min.get()) : new Qualifier(); - if (StringUtils.isBlank(rights.getClassid())) { - rights.setClassid(UNKNOWN); - } - if (StringUtils.isBlank(rights.getClassname()) - || UNKNOWN.equalsIgnoreCase(rights.getClassname())) { - rights.setClassname(NOT_AVAILABLE); - } - if (StringUtils.isBlank(rights.getSchemeid())) { - rights.setSchemeid(DNET_ACCESS_MODES); - } - if (StringUtils.isBlank(rights.getSchemename())) { - rights.setSchemename(DNET_ACCESS_MODES); - } + if (StringUtils.isBlank(rights.getClassid())) { + rights.setClassid(UNKNOWN); + } + if (StringUtils.isBlank(rights.getClassname()) + || UNKNOWN.equalsIgnoreCase(rights.getClassname())) { + rights.setClassname(NOT_AVAILABLE); + } + if (StringUtils.isBlank(rights.getSchemeid())) { + rights.setSchemeid(DNET_ACCESS_MODES); + } + if (StringUtils.isBlank(rights.getSchemename())) { + rights.setSchemename(DNET_ACCESS_MODES); + } - return rights; - } - return null; - } + return rights; + } + return null; + } - private static Context mapContext(ResultProtos.Result.Context context) { + private static Context mapContext(ResultProtos.Result.Context context) { - final Context entity = new Context(); - entity.setId(context.getId()); - entity.setDataInfo( - context.getDataInfoList().stream() - .map(ProtoConverter::mapDataInfo) - .collect(Collectors.toList())); - return entity; - } + final Context entity = new Context(); + entity.setId(context.getId()); + entity + .setDataInfo( + context + .getDataInfoList() + .stream() + .map(ProtoConverter::mapDataInfo) + .collect(Collectors.toList())); + return entity; + } - public static KeyValue mapKV(FieldTypeProtos.KeyValue kv) { - final KeyValue keyValue = new KeyValue(); - keyValue.setKey(kv.getKey()); - keyValue.setValue(kv.getValue()); - keyValue.setDataInfo(mapDataInfo(kv.getDataInfo())); - return keyValue; - } + public static KeyValue mapKV(FieldTypeProtos.KeyValue kv) { + final KeyValue keyValue = new KeyValue(); + keyValue.setKey(kv.getKey()); + keyValue.setValue(kv.getValue()); + keyValue.setDataInfo(mapDataInfo(kv.getDataInfo())); + return keyValue; + } - public static DataInfo mapDataInfo(FieldTypeProtos.DataInfo d) { - final DataInfo dataInfo = new DataInfo(); - dataInfo.setDeletedbyinference(d.getDeletedbyinference()); - dataInfo.setInferenceprovenance(d.getInferenceprovenance()); - dataInfo.setInferred(d.getInferred()); - dataInfo.setInvisible(d.getInvisible()); - dataInfo.setProvenanceaction(mapQualifier(d.getProvenanceaction())); - dataInfo.setTrust(d.getTrust()); - return dataInfo; - } + public static DataInfo mapDataInfo(FieldTypeProtos.DataInfo d) { + final DataInfo dataInfo = new DataInfo(); + dataInfo.setDeletedbyinference(d.getDeletedbyinference()); + dataInfo.setInferenceprovenance(d.getInferenceprovenance()); + dataInfo.setInferred(d.getInferred()); + dataInfo.setInvisible(d.getInvisible()); + dataInfo.setProvenanceaction(mapQualifier(d.getProvenanceaction())); + dataInfo.setTrust(d.getTrust()); + return dataInfo; + } - public static Qualifier mapQualifier(FieldTypeProtos.Qualifier q) { - final Qualifier qualifier = new Qualifier(); - qualifier.setClassid(q.getClassid()); - qualifier.setClassname(q.getClassname()); - qualifier.setSchemeid(q.getSchemeid()); - qualifier.setSchemename(q.getSchemename()); - return qualifier; - } + public static Qualifier mapQualifier(FieldTypeProtos.Qualifier q) { + final Qualifier qualifier = new Qualifier(); + qualifier.setClassid(q.getClassid()); + qualifier.setClassname(q.getClassname()); + qualifier.setSchemeid(q.getSchemeid()); + qualifier.setSchemename(q.getSchemename()); + return qualifier; + } - public static Country mapQualifierAsCountry(FieldTypeProtos.Qualifier q) { - final Country c = new Country(); - c.setClassid(q.getClassid()); - c.setClassname(q.getClassname()); - c.setSchemeid(q.getSchemeid()); - c.setSchemename(q.getSchemename()); - c.setDataInfo(mapDataInfo(q.getDataInfo())); - return c; - } + public static Country mapQualifierAsCountry(FieldTypeProtos.Qualifier q) { + final Country c = new Country(); + c.setClassid(q.getClassid()); + c.setClassname(q.getClassname()); + c.setSchemeid(q.getSchemeid()); + c.setSchemename(q.getSchemename()); + c.setDataInfo(mapDataInfo(q.getDataInfo())); + return c; + } - public static StructuredProperty mapStructuredProperty(FieldTypeProtos.StructuredProperty sp) { - final StructuredProperty structuredProperty = new StructuredProperty(); - structuredProperty.setValue(sp.getValue()); - structuredProperty.setQualifier(mapQualifier(sp.getQualifier())); - structuredProperty.setDataInfo(mapDataInfo(sp.getDataInfo())); - return structuredProperty; - } + public static StructuredProperty mapStructuredProperty(FieldTypeProtos.StructuredProperty sp) { + final StructuredProperty structuredProperty = new StructuredProperty(); + structuredProperty.setValue(sp.getValue()); + structuredProperty.setQualifier(mapQualifier(sp.getQualifier())); + structuredProperty.setDataInfo(mapDataInfo(sp.getDataInfo())); + return structuredProperty; + } - public static ExtraInfo mapExtraInfo(FieldTypeProtos.ExtraInfo extraInfo) { - final ExtraInfo entity = new ExtraInfo(); - entity.setName(extraInfo.getName()); - entity.setTypology(extraInfo.getTypology()); - entity.setProvenance(extraInfo.getProvenance()); - entity.setTrust(extraInfo.getTrust()); - entity.setValue(extraInfo.getValue()); - return entity; - } + public static ExtraInfo mapExtraInfo(FieldTypeProtos.ExtraInfo extraInfo) { + final ExtraInfo entity = new ExtraInfo(); + entity.setName(extraInfo.getName()); + entity.setTypology(extraInfo.getTypology()); + entity.setProvenance(extraInfo.getProvenance()); + entity.setTrust(extraInfo.getTrust()); + entity.setValue(extraInfo.getValue()); + return entity; + } - public static OAIProvenance mapOAIProvenance(FieldTypeProtos.OAIProvenance oaiProvenance) { - final OAIProvenance entity = new OAIProvenance(); - entity.setOriginDescription(mapOriginalDescription(oaiProvenance.getOriginDescription())); - return entity; - } + public static OAIProvenance mapOAIProvenance(FieldTypeProtos.OAIProvenance oaiProvenance) { + final OAIProvenance entity = new OAIProvenance(); + entity.setOriginDescription(mapOriginalDescription(oaiProvenance.getOriginDescription())); + return entity; + } - public static OriginDescription mapOriginalDescription( - FieldTypeProtos.OAIProvenance.OriginDescription originDescription) { - final OriginDescription originDescriptionResult = new OriginDescription(); - originDescriptionResult.setHarvestDate(originDescription.getHarvestDate()); - originDescriptionResult.setAltered(originDescription.getAltered()); - originDescriptionResult.setBaseURL(originDescription.getBaseURL()); - originDescriptionResult.setIdentifier(originDescription.getIdentifier()); - originDescriptionResult.setDatestamp(originDescription.getDatestamp()); - originDescriptionResult.setMetadataNamespace(originDescription.getMetadataNamespace()); - return originDescriptionResult; - } + public static OriginDescription mapOriginalDescription( + FieldTypeProtos.OAIProvenance.OriginDescription originDescription) { + final OriginDescription originDescriptionResult = new OriginDescription(); + originDescriptionResult.setHarvestDate(originDescription.getHarvestDate()); + originDescriptionResult.setAltered(originDescription.getAltered()); + originDescriptionResult.setBaseURL(originDescription.getBaseURL()); + originDescriptionResult.setIdentifier(originDescription.getIdentifier()); + originDescriptionResult.setDatestamp(originDescription.getDatestamp()); + originDescriptionResult.setMetadataNamespace(originDescription.getMetadataNamespace()); + return originDescriptionResult; + } - public static Field mapStringField(FieldTypeProtos.StringField s) { - final Field stringField = new Field<>(); - stringField.setValue(s.getValue()); - stringField.setDataInfo(mapDataInfo(s.getDataInfo())); - return stringField; - } + public static Field mapStringField(FieldTypeProtos.StringField s) { + final Field stringField = new Field<>(); + stringField.setValue(s.getValue()); + stringField.setDataInfo(mapDataInfo(s.getDataInfo())); + return stringField; + } - public static Field mapBoolField(FieldTypeProtos.BoolField b) { - final Field booleanField = new Field<>(); - booleanField.setValue(b.getValue()); - booleanField.setDataInfo(mapDataInfo(b.getDataInfo())); - return booleanField; - } + public static Field mapBoolField(FieldTypeProtos.BoolField b) { + final Field booleanField = new Field<>(); + booleanField.setValue(b.getValue()); + booleanField.setDataInfo(mapDataInfo(b.getDataInfo())); + return booleanField; + } - public static Field mapIntField(FieldTypeProtos.IntField b) { - final Field entity = new Field<>(); - entity.setValue(b.getValue()); - entity.setDataInfo(mapDataInfo(b.getDataInfo())); - return entity; - } + public static Field mapIntField(FieldTypeProtos.IntField b) { + final Field entity = new Field<>(); + entity.setValue(b.getValue()); + entity.setDataInfo(mapDataInfo(b.getDataInfo())); + return entity; + } - public static Journal mapJournal(FieldTypeProtos.Journal j) { - final Journal journal = new Journal(); - journal.setConferencedate(j.getConferencedate()); - journal.setConferenceplace(j.getConferenceplace()); - journal.setEdition(j.getEdition()); - journal.setEp(j.getEp()); - journal.setIss(j.getIss()); - journal.setIssnLinking(j.getIssnLinking()); - journal.setIssnOnline(j.getIssnOnline()); - journal.setIssnPrinted(j.getIssnPrinted()); - journal.setName(j.getName()); - journal.setSp(j.getSp()); - journal.setVol(j.getVol()); - journal.setDataInfo(mapDataInfo(j.getDataInfo())); - return journal; - } + public static Journal mapJournal(FieldTypeProtos.Journal j) { + final Journal journal = new Journal(); + journal.setConferencedate(j.getConferencedate()); + journal.setConferenceplace(j.getConferenceplace()); + journal.setEdition(j.getEdition()); + journal.setEp(j.getEp()); + journal.setIss(j.getIss()); + journal.setIssnLinking(j.getIssnLinking()); + journal.setIssnOnline(j.getIssnOnline()); + journal.setIssnPrinted(j.getIssnPrinted()); + journal.setName(j.getName()); + journal.setSp(j.getSp()); + journal.setVol(j.getVol()); + journal.setDataInfo(mapDataInfo(j.getDataInfo())); + return journal; + } - public static Author mapAuthor(FieldTypeProtos.Author author) { - final Author entity = new Author(); - entity.setFullname(author.getFullname()); - entity.setName(author.getName()); - entity.setSurname(author.getSurname()); - entity.setRank(author.getRank()); - entity.setPid( - author.getPidList().stream() - .map( - kv -> { - final StructuredProperty sp = new StructuredProperty(); - sp.setValue(kv.getValue()); - final Qualifier q = new Qualifier(); - q.setClassid(kv.getKey()); - q.setClassname(kv.getKey()); - sp.setQualifier(q); - return sp; - }) - .collect(Collectors.toList())); - entity.setAffiliation( - author.getAffiliationList().stream() - .map(ProtoConverter::mapStringField) - .collect(Collectors.toList())); - return entity; - } + public static Author mapAuthor(FieldTypeProtos.Author author) { + final Author entity = new Author(); + entity.setFullname(author.getFullname()); + entity.setName(author.getName()); + entity.setSurname(author.getSurname()); + entity.setRank(author.getRank()); + entity + .setPid( + author + .getPidList() + .stream() + .map( + kv -> { + final StructuredProperty sp = new StructuredProperty(); + sp.setValue(kv.getValue()); + final Qualifier q = new Qualifier(); + q.setClassid(kv.getKey()); + q.setClassname(kv.getKey()); + sp.setQualifier(q); + return sp; + }) + .collect(Collectors.toList())); + entity + .setAffiliation( + author + .getAffiliationList() + .stream() + .map(ProtoConverter::mapStringField) + .collect(Collectors.toList())); + return entity; + } - public static GeoLocation mapGeolocation(ResultProtos.Result.GeoLocation geoLocation) { - final GeoLocation entity = new GeoLocation(); - entity.setPoint(geoLocation.getPoint()); - entity.setBox(geoLocation.getBox()); - entity.setPlace(geoLocation.getPlace()); - return entity; - } + public static GeoLocation mapGeolocation(ResultProtos.Result.GeoLocation geoLocation) { + final GeoLocation entity = new GeoLocation(); + entity.setPoint(geoLocation.getPoint()); + entity.setBox(geoLocation.getBox()); + entity.setPlace(geoLocation.getPlace()); + return entity; + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/TransformActions.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/TransformActions.java index d200ac18f..490668606 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/TransformActions.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/migration/TransformActions.java @@ -1,23 +1,14 @@ + package eu.dnetlib.dhp.actionmanager.migration; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Splitter; -import com.google.common.collect.Lists; -import com.google.protobuf.InvalidProtocolBufferException; -import eu.dnetlib.data.proto.OafProtos; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import java.io.IOException; import java.io.Serializable; import java.util.LinkedList; import java.util.Objects; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; @@ -29,136 +20,153 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Splitter; +import com.google.common.collect.Lists; +import com.google.protobuf.InvalidProtocolBufferException; + +import eu.dnetlib.data.proto.OafProtos; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import scala.Tuple2; public class TransformActions implements Serializable { - private static final Logger log = LoggerFactory.getLogger(TransformActions.class); + private static final Logger log = LoggerFactory.getLogger(TransformActions.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final String SEPARATOR = "/"; + private static final String SEPARATOR = "/"; - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateActionSet.class.getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/migration/transform_actionsets_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateActionSet.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/migration/transform_actionsets_parameters.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); - final String inputPaths = parser.get("inputPaths"); + final String inputPaths = parser.get("inputPaths"); - if (StringUtils.isBlank(inputPaths)) { - throw new RuntimeException("empty inputPaths"); - } - log.info("inputPaths: {}", inputPaths); + if (StringUtils.isBlank(inputPaths)) { + throw new RuntimeException("empty inputPaths"); + } + log.info("inputPaths: {}", inputPaths); - final String targetBaseDir = getTargetBaseDir(isLookupUrl); + final String targetBaseDir = getTargetBaseDir(isLookupUrl); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, isSparkSessionManaged, spark -> transformActions(inputPaths, targetBaseDir, spark)); - } + runWithSparkSession( + conf, isSparkSessionManaged, spark -> transformActions(inputPaths, targetBaseDir, spark)); + } - private static void transformActions(String inputPaths, String targetBaseDir, SparkSession spark) - throws IOException { - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - final FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration()); + private static void transformActions(String inputPaths, String targetBaseDir, SparkSession spark) + throws IOException { + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + final FileSystem fs = FileSystem.get(spark.sparkContext().hadoopConfiguration()); - for (String sourcePath : Lists.newArrayList(Splitter.on(",").split(inputPaths))) { + for (String sourcePath : Lists.newArrayList(Splitter.on(",").split(inputPaths))) { - LinkedList pathQ = Lists.newLinkedList(Splitter.on(SEPARATOR).split(sourcePath)); + LinkedList pathQ = Lists.newLinkedList(Splitter.on(SEPARATOR).split(sourcePath)); - final String rawset = pathQ.pollLast(); - final String actionSetDirectory = pathQ.pollLast(); + final String rawset = pathQ.pollLast(); + final String actionSetDirectory = pathQ.pollLast(); - final Path targetDirectory = - new Path(targetBaseDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawset); + final Path targetDirectory = new Path(targetBaseDir + SEPARATOR + actionSetDirectory + SEPARATOR + rawset); - if (fs.exists(targetDirectory)) { - log.info("found target directory '{}", targetDirectory); - fs.delete(targetDirectory, true); - log.info("deleted target directory '{}", targetDirectory); - } + if (fs.exists(targetDirectory)) { + log.info("found target directory '{}", targetDirectory); + fs.delete(targetDirectory, true); + log.info("deleted target directory '{}", targetDirectory); + } - log.info("transforming actions from '{}' to '{}'", sourcePath, targetDirectory); + log.info("transforming actions from '{}' to '{}'", sourcePath, targetDirectory); - sc.sequenceFile(sourcePath, Text.class, Text.class) - .map(a -> eu.dnetlib.actionmanager.actions.AtomicAction.fromJSON(a._2().toString())) - .map(TransformActions::doTransform) - .filter(Objects::nonNull) - .mapToPair( - a -> new Tuple2<>(a.getClazz().toString(), OBJECT_MAPPER.writeValueAsString(a))) - .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2()))) - .saveAsNewAPIHadoopFile( - targetDirectory.toString(), - Text.class, - Text.class, - SequenceFileOutputFormat.class, - sc.hadoopConfiguration()); - } - } + sc + .sequenceFile(sourcePath, Text.class, Text.class) + .map(a -> eu.dnetlib.actionmanager.actions.AtomicAction.fromJSON(a._2().toString())) + .map(TransformActions::doTransform) + .filter(Objects::nonNull) + .mapToPair( + a -> new Tuple2<>(a.getClazz().toString(), OBJECT_MAPPER.writeValueAsString(a))) + .mapToPair(t -> new Tuple2(new Text(t._1()), new Text(t._2()))) + .saveAsNewAPIHadoopFile( + targetDirectory.toString(), + Text.class, + Text.class, + SequenceFileOutputFormat.class, + sc.hadoopConfiguration()); + } + } - private static AtomicAction doTransform(eu.dnetlib.actionmanager.actions.AtomicAction aa) - throws InvalidProtocolBufferException { + private static AtomicAction doTransform(eu.dnetlib.actionmanager.actions.AtomicAction aa) + throws InvalidProtocolBufferException { - // dedup similarity relations had empty target value, don't migrate them - if (aa.getTargetValue().length == 0) { - return null; - } - final OafProtos.Oaf proto_oaf = OafProtos.Oaf.parseFrom(aa.getTargetValue()); - final Oaf oaf = ProtoConverter.convert(proto_oaf); - switch (proto_oaf.getKind()) { - case entity: - switch (proto_oaf.getEntity().getType()) { - case datasource: - return new AtomicAction<>(Datasource.class, (Datasource) oaf); - case organization: - return new AtomicAction<>(Organization.class, (Organization) oaf); - case project: - return new AtomicAction<>(Project.class, (Project) oaf); - case result: - final String resulttypeid = - proto_oaf.getEntity().getResult().getMetadata().getResulttype().getClassid(); - switch (resulttypeid) { - case "publication": - return new AtomicAction<>(Publication.class, (Publication) oaf); - case "software": - return new AtomicAction<>(Software.class, (Software) oaf); - case "other": - return new AtomicAction<>(OtherResearchProduct.class, (OtherResearchProduct) oaf); - case "dataset": - return new AtomicAction<>(Dataset.class, (Dataset) oaf); - default: - // can be an update, where the resulttype is not specified - return new AtomicAction<>(Result.class, (Result) oaf); - } - default: - throw new IllegalArgumentException( - "invalid entity type: " + proto_oaf.getEntity().getType()); - } - case relation: - return new AtomicAction<>(Relation.class, (Relation) oaf); - default: - throw new IllegalArgumentException("invalid kind: " + proto_oaf.getKind()); - } - } + // dedup similarity relations had empty target value, don't migrate them + if (aa.getTargetValue().length == 0) { + return null; + } + final OafProtos.Oaf proto_oaf = OafProtos.Oaf.parseFrom(aa.getTargetValue()); + final Oaf oaf = ProtoConverter.convert(proto_oaf); + switch (proto_oaf.getKind()) { + case entity: + switch (proto_oaf.getEntity().getType()) { + case datasource: + return new AtomicAction<>(Datasource.class, (Datasource) oaf); + case organization: + return new AtomicAction<>(Organization.class, (Organization) oaf); + case project: + return new AtomicAction<>(Project.class, (Project) oaf); + case result: + final String resulttypeid = proto_oaf + .getEntity() + .getResult() + .getMetadata() + .getResulttype() + .getClassid(); + switch (resulttypeid) { + case "publication": + return new AtomicAction<>(Publication.class, (Publication) oaf); + case "software": + return new AtomicAction<>(Software.class, (Software) oaf); + case "other": + return new AtomicAction<>(OtherResearchProduct.class, (OtherResearchProduct) oaf); + case "dataset": + return new AtomicAction<>(Dataset.class, (Dataset) oaf); + default: + // can be an update, where the resulttype is not specified + return new AtomicAction<>(Result.class, (Result) oaf); + } + default: + throw new IllegalArgumentException( + "invalid entity type: " + proto_oaf.getEntity().getType()); + } + case relation: + return new AtomicAction<>(Relation.class, (Relation) oaf); + default: + throw new IllegalArgumentException("invalid kind: " + proto_oaf.getKind()); + } + } - private static String getTargetBaseDir(String isLookupUrl) throws ISLookUpException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - String XQUERY = - "collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()"; - return isLookUp.getResourceProfileByQuery(XQUERY); - } + private static String getTargetBaseDir(String isLookupUrl) throws ISLookUpException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + String XQUERY = "collection('/db/DRIVER/ServiceResources/ActionManagerServiceResourceType')//SERVICE_PROPERTIES/PROPERTY[@key = 'basePath']/@value/string()"; + return isLookUp.getResourceProfileByQuery(XQUERY); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java index 6eb0bac3b..af3ef0c12 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJob.java @@ -1,15 +1,13 @@ + package eu.dnetlib.dhp.actionmanager.partition; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static org.apache.spark.sql.functions.*; -import eu.dnetlib.dhp.actionmanager.ISClient; -import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJob; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; import java.util.Arrays; import java.util.List; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.spark.SparkConf; @@ -20,117 +18,127 @@ import org.apache.spark.sql.types.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.actionmanager.ISClient; +import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJob; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; + /** Partitions given set of action sets by payload type. */ public class PartitionActionSetsByPayloadTypeJob { - private static final Logger logger = - LoggerFactory.getLogger(PartitionActionSetsByPayloadTypeJob.class); + private static final Logger logger = LoggerFactory.getLogger(PartitionActionSetsByPayloadTypeJob.class); - private static final StructType KV_SCHEMA = - StructType$.MODULE$.apply( - Arrays.asList( - StructField$.MODULE$.apply("key", DataTypes.StringType, false, Metadata.empty()), - StructField$.MODULE$.apply("value", DataTypes.StringType, false, Metadata.empty()))); + private static final StructType KV_SCHEMA = StructType$.MODULE$ + .apply( + Arrays + .asList( + StructField$.MODULE$.apply("key", DataTypes.StringType, false, Metadata.empty()), + StructField$.MODULE$.apply("value", DataTypes.StringType, false, Metadata.empty()))); - private static final StructType ATOMIC_ACTION_SCHEMA = - StructType$.MODULE$.apply( - Arrays.asList( - StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()), - StructField$.MODULE$.apply( - "payload", DataTypes.StringType, false, Metadata.empty()))); + private static final StructType ATOMIC_ACTION_SCHEMA = StructType$.MODULE$ + .apply( + Arrays + .asList( + StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()), + StructField$.MODULE$ + .apply( + "payload", DataTypes.StringType, false, Metadata.empty()))); - private ISClient isClient; + private ISClient isClient; - public PartitionActionSetsByPayloadTypeJob(String isLookupUrl) { - this.isClient = new ISClient(isLookupUrl); - } + public PartitionActionSetsByPayloadTypeJob(String isLookupUrl) { + this.isClient = new ISClient(isLookupUrl); + } - public PartitionActionSetsByPayloadTypeJob() {} + public PartitionActionSetsByPayloadTypeJob() { + } - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PromoteActionPayloadForGraphTableJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PromoteActionPayloadForGraphTableJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/partition/partition_action_sets_by_payload_type_input_parameters.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputActionSetIds = parser.get("inputActionSetIds"); - logger.info("inputActionSetIds: {}", inputActionSetIds); + String inputActionSetIds = parser.get("inputActionSetIds"); + logger.info("inputActionSetIds: {}", inputActionSetIds); - String outputPath = parser.get("outputPath"); - logger.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + logger.info("outputPath: {}", outputPath); - String isLookupUrl = parser.get("isLookupUrl"); - logger.info("isLookupUrl: {}", isLookupUrl); + String isLookupUrl = parser.get("isLookupUrl"); + logger.info("isLookupUrl: {}", isLookupUrl); - new PartitionActionSetsByPayloadTypeJob(isLookupUrl) - .run(isSparkSessionManaged, inputActionSetIds, outputPath); - } + new PartitionActionSetsByPayloadTypeJob(isLookupUrl) + .run(isSparkSessionManaged, inputActionSetIds, outputPath); + } - protected void run(Boolean isSparkSessionManaged, String inputActionSetIds, String outputPath) { + protected void run(Boolean isSparkSessionManaged, String inputActionSetIds, String outputPath) { - List inputActionSetPaths = getIsClient().getLatestRawsetPaths(inputActionSetIds); - logger.info("inputActionSetPaths: {}", String.join(",", inputActionSetPaths)); + List inputActionSetPaths = getIsClient().getLatestRawsetPaths(inputActionSetIds); + logger.info("inputActionSetPaths: {}", String.join(",", inputActionSetPaths)); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - readAndWriteActionSetsFromPaths(spark, inputActionSetPaths, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + readAndWriteActionSetsFromPaths(spark, inputActionSetPaths, outputPath); + }); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - private static void readAndWriteActionSetsFromPaths( - SparkSession spark, List inputActionSetPaths, String outputPath) { - inputActionSetPaths.stream() - .filter(path -> HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration())) - .forEach( - inputActionSetPath -> { - Dataset actionDS = readActionSetFromPath(spark, inputActionSetPath); - saveActions(actionDS, outputPath); - }); - } + private static void readAndWriteActionSetsFromPaths( + SparkSession spark, List inputActionSetPaths, String outputPath) { + inputActionSetPaths + .stream() + .filter(path -> HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration())) + .forEach( + inputActionSetPath -> { + Dataset actionDS = readActionSetFromPath(spark, inputActionSetPath); + saveActions(actionDS, outputPath); + }); + } - private static Dataset readActionSetFromPath(SparkSession spark, String path) { - logger.info("Reading actions from path: {}", path); + private static Dataset readActionSetFromPath(SparkSession spark, String path) { + logger.info("Reading actions from path: {}", path); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD rdd = - sc.sequenceFile(path, Text.class, Text.class) - .map(x -> RowFactory.create(x._1().toString(), x._2().toString())); + JavaRDD rdd = sc + .sequenceFile(path, Text.class, Text.class) + .map(x -> RowFactory.create(x._1().toString(), x._2().toString())); - return spark - .createDataFrame(rdd, KV_SCHEMA) - .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) - .select(expr("atomic_action.*")); - } + return spark + .createDataFrame(rdd, KV_SCHEMA) + .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) + .select(expr("atomic_action.*")); + } - private static void saveActions(Dataset actionDS, String path) { - logger.info("Saving actions to path: {}", path); - actionDS.write().partitionBy("clazz").mode(SaveMode.Append).parquet(path); - } + private static void saveActions(Dataset actionDS, String path) { + logger.info("Saving actions to path: {}", path); + actionDS.write().partitionBy("clazz").mode(SaveMode.Append).parquet(path); + } - public ISClient getIsClient() { - return isClient; - } + public ISClient getIsClient() { + return isClient; + } - public void setIsClient(ISClient isClient) { - this.isClient = isClient; - } + public void setIsClient(ISClient isClient) { + this.isClient = isClient; + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java index ac8291842..fbb072957 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java @@ -1,82 +1,87 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; +import java.util.function.BiFunction; + import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.Relation; -import java.util.function.BiFunction; /** OAF model merging support. */ public class MergeAndGet { - private MergeAndGet() {} + private MergeAndGet() { + } - /** - * Strategy for merging OAF model objects. - * - *

MERGE_FROM_AND_GET: use OAF 'mergeFrom' method SELECT_NEWER_AND_GET: use last update - * timestamp to return newer instance - */ - public enum Strategy { - MERGE_FROM_AND_GET, - SELECT_NEWER_AND_GET - } + /** + * Strategy for merging OAF model objects. + *

+ * MERGE_FROM_AND_GET: use OAF 'mergeFrom' method SELECT_NEWER_AND_GET: use last update timestamp to return newer + * instance + */ + public enum Strategy { + MERGE_FROM_AND_GET, SELECT_NEWER_AND_GET + } - /** - * Returns a function for merging OAF model objects. - * - * @param strategy Strategy to be used to merge objects - * @param Graph table type - * @param Action payload type - * @return BiFunction to be used to merge OAF objects - */ - public static - SerializableSupplier> functionFor(Strategy strategy) { - switch (strategy) { - case MERGE_FROM_AND_GET: - return () -> MergeAndGet::mergeFromAndGet; - case SELECT_NEWER_AND_GET: - return () -> MergeAndGet::selectNewerAndGet; - } - throw new RuntimeException(); - } + /** + * Returns a function for merging OAF model objects. + * + * @param strategy Strategy to be used to merge objects + * @param Graph table type + * @param Action payload type + * @return BiFunction to be used to merge OAF objects + */ + public static SerializableSupplier> functionFor( + Strategy strategy) { + switch (strategy) { + case MERGE_FROM_AND_GET: + return () -> MergeAndGet::mergeFromAndGet; + case SELECT_NEWER_AND_GET: + return () -> MergeAndGet::selectNewerAndGet; + } + throw new RuntimeException(); + } - private static G mergeFromAndGet(G x, A y) { - if (isSubClass(x, Relation.class) && isSubClass(y, Relation.class)) { - ((Relation) x).mergeFrom((Relation) y); - return x; - } else if (isSubClass(x, OafEntity.class) - && isSubClass(y, OafEntity.class) - && isSubClass(x, y)) { - ((OafEntity) x).mergeFrom((OafEntity) y); - return x; - } - throw new RuntimeException( - String.format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); - } + private static G mergeFromAndGet(G x, A y) { + if (isSubClass(x, Relation.class) && isSubClass(y, Relation.class)) { + ((Relation) x).mergeFrom((Relation) y); + return x; + } else if (isSubClass(x, OafEntity.class) + && isSubClass(y, OafEntity.class) + && isSubClass(x, y)) { + ((OafEntity) x).mergeFrom((OafEntity) y); + return x; + } + throw new RuntimeException( + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); + } - private static G selectNewerAndGet(G x, A y) { - if (x.getClass().equals(y.getClass()) - && x.getLastupdatetimestamp() > y.getLastupdatetimestamp()) { - return x; - } else if (x.getClass().equals(y.getClass()) - && x.getLastupdatetimestamp() < y.getLastupdatetimestamp()) { - return (G) y; - } else if (isSubClass(x, y) && x.getLastupdatetimestamp() > y.getLastupdatetimestamp()) { - return x; - } else if (isSubClass(x, y) && x.getLastupdatetimestamp() < y.getLastupdatetimestamp()) { - throw new RuntimeException( - String.format( - "SELECT_NEWER_AND_GET cannot return right type when it is not the same as left type: %s, %s", - x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); - } - throw new RuntimeException( - String.format( - "SELECT_NEWER_AND_GET cannot be used when left is not subtype of right: %s, %s", - x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); - } + private static G selectNewerAndGet(G x, A y) { + if (x.getClass().equals(y.getClass()) + && x.getLastupdatetimestamp() > y.getLastupdatetimestamp()) { + return x; + } else if (x.getClass().equals(y.getClass()) + && x.getLastupdatetimestamp() < y.getLastupdatetimestamp()) { + return (G) y; + } else if (isSubClass(x, y) && x.getLastupdatetimestamp() > y.getLastupdatetimestamp()) { + return x; + } else if (isSubClass(x, y) && x.getLastupdatetimestamp() < y.getLastupdatetimestamp()) { + throw new RuntimeException( + String + .format( + "SELECT_NEWER_AND_GET cannot return right type when it is not the same as left type: %s, %s", + x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); + } + throw new RuntimeException( + String + .format( + "SELECT_NEWER_AND_GET cannot be used when left is not subtype of right: %s, %s", + x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java index 24af1973f..17bfc4af3 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJob.java @@ -1,18 +1,14 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Objects; import java.util.Optional; import java.util.function.BiFunction; import java.util.function.Function; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -23,204 +19,207 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; + /** Applies a given action payload file to graph table of compatible type. */ public class PromoteActionPayloadForGraphTableJob { - private static final Logger logger = - LoggerFactory.getLogger(PromoteActionPayloadForGraphTableJob.class); + private static final Logger logger = LoggerFactory.getLogger(PromoteActionPayloadForGraphTableJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PromoteActionPayloadForGraphTableJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/promote/promote_action_payload_for_graph_table_input_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PromoteActionPayloadForGraphTableJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/promote/promote_action_payload_for_graph_table_input_parameters.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + logger.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputGraphTablePath = parser.get("inputGraphTablePath"); - logger.info("inputGraphTablePath: {}", inputGraphTablePath); + String inputGraphTablePath = parser.get("inputGraphTablePath"); + logger.info("inputGraphTablePath: {}", inputGraphTablePath); - String graphTableClassName = parser.get("graphTableClassName"); - logger.info("graphTableClassName: {}", graphTableClassName); + String graphTableClassName = parser.get("graphTableClassName"); + logger.info("graphTableClassName: {}", graphTableClassName); - String inputActionPayloadPath = parser.get("inputActionPayloadPath"); - logger.info("inputActionPayloadPath: {}", inputActionPayloadPath); + String inputActionPayloadPath = parser.get("inputActionPayloadPath"); + logger.info("inputActionPayloadPath: {}", inputActionPayloadPath); - String actionPayloadClassName = parser.get("actionPayloadClassName"); - logger.info("actionPayloadClassName: {}", actionPayloadClassName); + String actionPayloadClassName = parser.get("actionPayloadClassName"); + logger.info("actionPayloadClassName: {}", actionPayloadClassName); - String outputGraphTablePath = parser.get("outputGraphTablePath"); - logger.info("outputGraphTablePath: {}", outputGraphTablePath); + String outputGraphTablePath = parser.get("outputGraphTablePath"); + logger.info("outputGraphTablePath: {}", outputGraphTablePath); - MergeAndGet.Strategy strategy = - MergeAndGet.Strategy.valueOf(parser.get("mergeAndGetStrategy").toUpperCase()); - logger.info("strategy: {}", strategy); + MergeAndGet.Strategy strategy = MergeAndGet.Strategy.valueOf(parser.get("mergeAndGetStrategy").toUpperCase()); + logger.info("strategy: {}", strategy); - Class rowClazz = (Class) Class.forName(graphTableClassName); - Class actionPayloadClazz = - (Class) Class.forName(actionPayloadClassName); + Class rowClazz = (Class) Class.forName(graphTableClassName); + Class actionPayloadClazz = (Class) Class.forName(actionPayloadClassName); - throwIfGraphTableClassIsNotSubClassOfActionPayloadClass(rowClazz, actionPayloadClazz); + throwIfGraphTableClassIsNotSubClassOfActionPayloadClass(rowClazz, actionPayloadClazz); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputGraphTablePath); - promoteActionPayloadForGraphTable( - spark, - inputGraphTablePath, - inputActionPayloadPath, - outputGraphTablePath, - strategy, - rowClazz, - actionPayloadClazz); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputGraphTablePath); + promoteActionPayloadForGraphTable( + spark, + inputGraphTablePath, + inputActionPayloadPath, + outputGraphTablePath, + strategy, + rowClazz, + actionPayloadClazz); + }); + } - private static void throwIfGraphTableClassIsNotSubClassOfActionPayloadClass( - Class rowClazz, Class actionPayloadClazz) { - if (!isSubClass(rowClazz, actionPayloadClazz)) { - String msg = - String.format( - "graph table class is not a subclass of action payload class: graph=%s, action=%s", - rowClazz.getCanonicalName(), actionPayloadClazz.getCanonicalName()); - throw new RuntimeException(msg); - } - } + private static void throwIfGraphTableClassIsNotSubClassOfActionPayloadClass( + Class rowClazz, Class actionPayloadClazz) { + if (!isSubClass(rowClazz, actionPayloadClazz)) { + String msg = String + .format( + "graph table class is not a subclass of action payload class: graph=%s, action=%s", + rowClazz.getCanonicalName(), actionPayloadClazz.getCanonicalName()); + throw new RuntimeException(msg); + } + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - private static void promoteActionPayloadForGraphTable( - SparkSession spark, - String inputGraphTablePath, - String inputActionPayloadPath, - String outputGraphTablePath, - MergeAndGet.Strategy strategy, - Class rowClazz, - Class actionPayloadClazz) { - Dataset rowDS = readGraphTable(spark, inputGraphTablePath, rowClazz); - Dataset actionPayloadDS = - readActionPayload(spark, inputActionPayloadPath, actionPayloadClazz); + private static void promoteActionPayloadForGraphTable( + SparkSession spark, + String inputGraphTablePath, + String inputActionPayloadPath, + String outputGraphTablePath, + MergeAndGet.Strategy strategy, + Class rowClazz, + Class actionPayloadClazz) { + Dataset rowDS = readGraphTable(spark, inputGraphTablePath, rowClazz); + Dataset actionPayloadDS = readActionPayload(spark, inputActionPayloadPath, actionPayloadClazz); - Dataset result = - promoteActionPayloadForGraphTable( - rowDS, actionPayloadDS, strategy, rowClazz, actionPayloadClazz) - .map((MapFunction) value -> value, Encoders.bean(rowClazz)); + Dataset result = promoteActionPayloadForGraphTable( + rowDS, actionPayloadDS, strategy, rowClazz, actionPayloadClazz) + .map((MapFunction) value -> value, Encoders.bean(rowClazz)); - saveGraphTable(result, outputGraphTablePath); - } + saveGraphTable(result, outputGraphTablePath); + } - private static Dataset readGraphTable( - SparkSession spark, String path, Class rowClazz) { - logger.info("Reading graph table from path: {}", path); + private static Dataset readGraphTable( + SparkSession spark, String path, Class rowClazz) { + logger.info("Reading graph table from path: {}", path); - return spark - .read() - .textFile(path) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, rowClazz), - Encoders.bean(rowClazz)); + return spark + .read() + .textFile(path) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, rowClazz), + Encoders.bean(rowClazz)); - /* - * return spark .read() .parquet(path) .as(Encoders.bean(rowClazz)); - */ - } + /* + * return spark .read() .parquet(path) .as(Encoders.bean(rowClazz)); + */ + } - private static Dataset readActionPayload( - SparkSession spark, String path, Class actionPayloadClazz) { - logger.info("Reading action payload from path: {}", path); - return spark - .read() - .parquet(path) - .map( - (MapFunction) - value -> - OBJECT_MAPPER.readValue(value.getAs("payload"), actionPayloadClazz), - Encoders.bean(actionPayloadClazz)); - } + private static Dataset readActionPayload( + SparkSession spark, String path, Class actionPayloadClazz) { + logger.info("Reading action payload from path: {}", path); + return spark + .read() + .parquet(path) + .map( + (MapFunction) value -> OBJECT_MAPPER + .readValue(value. getAs("payload"), actionPayloadClazz), + Encoders.bean(actionPayloadClazz)); + } - private static Dataset promoteActionPayloadForGraphTable( - Dataset rowDS, - Dataset actionPayloadDS, - MergeAndGet.Strategy strategy, - Class rowClazz, - Class actionPayloadClazz) { - logger.info( - "Promoting action payload for graph table: payload={}, table={}", - actionPayloadClazz.getSimpleName(), - rowClazz.getSimpleName()); + private static Dataset promoteActionPayloadForGraphTable( + Dataset rowDS, + Dataset actionPayloadDS, + MergeAndGet.Strategy strategy, + Class rowClazz, + Class actionPayloadClazz) { + logger + .info( + "Promoting action payload for graph table: payload={}, table={}", + actionPayloadClazz.getSimpleName(), + rowClazz.getSimpleName()); - SerializableSupplier> rowIdFn = ModelSupport::idFn; - SerializableSupplier> actionPayloadIdFn = ModelSupport::idFn; - SerializableSupplier> mergeRowWithActionPayloadAndGetFn = - MergeAndGet.functionFor(strategy); - SerializableSupplier> mergeRowsAndGetFn = MergeAndGet.functionFor(strategy); - SerializableSupplier zeroFn = zeroFn(rowClazz); - SerializableSupplier> isNotZeroFn = - PromoteActionPayloadForGraphTableJob::isNotZeroFnUsingIdOrSource; + SerializableSupplier> rowIdFn = ModelSupport::idFn; + SerializableSupplier> actionPayloadIdFn = ModelSupport::idFn; + SerializableSupplier> mergeRowWithActionPayloadAndGetFn = MergeAndGet.functionFor(strategy); + SerializableSupplier> mergeRowsAndGetFn = MergeAndGet.functionFor(strategy); + SerializableSupplier zeroFn = zeroFn(rowClazz); + SerializableSupplier> isNotZeroFn = PromoteActionPayloadForGraphTableJob::isNotZeroFnUsingIdOrSource; - Dataset joinedAndMerged = - PromoteActionPayloadFunctions.joinGraphTableWithActionPayloadAndMerge( - rowDS, - actionPayloadDS, - rowIdFn, - actionPayloadIdFn, - mergeRowWithActionPayloadAndGetFn, - rowClazz, - actionPayloadClazz); + Dataset joinedAndMerged = PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + rowDS, + actionPayloadDS, + rowIdFn, + actionPayloadIdFn, + mergeRowWithActionPayloadAndGetFn, + rowClazz, + actionPayloadClazz); - return PromoteActionPayloadFunctions.groupGraphTableByIdAndMerge( - joinedAndMerged, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, rowClazz); - } + return PromoteActionPayloadFunctions + .groupGraphTableByIdAndMerge( + joinedAndMerged, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, rowClazz); + } - private static SerializableSupplier zeroFn(Class clazz) { - switch (clazz.getCanonicalName()) { - case "eu.dnetlib.dhp.schema.oaf.Dataset": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Dataset()); - case "eu.dnetlib.dhp.schema.oaf.Datasource": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Datasource()); - case "eu.dnetlib.dhp.schema.oaf.Organization": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Organization()); - case "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.OtherResearchProduct()); - case "eu.dnetlib.dhp.schema.oaf.Project": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Project()); - case "eu.dnetlib.dhp.schema.oaf.Publication": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Publication()); - case "eu.dnetlib.dhp.schema.oaf.Relation": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Relation()); - case "eu.dnetlib.dhp.schema.oaf.Software": - return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Software()); - default: - throw new RuntimeException("unknown class: " + clazz.getCanonicalName()); - } - } + private static SerializableSupplier zeroFn(Class clazz) { + switch (clazz.getCanonicalName()) { + case "eu.dnetlib.dhp.schema.oaf.Dataset": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Dataset()); + case "eu.dnetlib.dhp.schema.oaf.Datasource": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Datasource()); + case "eu.dnetlib.dhp.schema.oaf.Organization": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Organization()); + case "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.OtherResearchProduct()); + case "eu.dnetlib.dhp.schema.oaf.Project": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Project()); + case "eu.dnetlib.dhp.schema.oaf.Publication": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Publication()); + case "eu.dnetlib.dhp.schema.oaf.Relation": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Relation()); + case "eu.dnetlib.dhp.schema.oaf.Software": + return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Software()); + default: + throw new RuntimeException("unknown class: " + clazz.getCanonicalName()); + } + } - private static Function isNotZeroFnUsingIdOrSource() { - return t -> { - if (isSubClass(t, Relation.class)) { - return Objects.nonNull(((Relation) t).getSource()); - } - return Objects.nonNull(((OafEntity) t).getId()); - }; - } + private static Function isNotZeroFnUsingIdOrSource() { + return t -> { + if (isSubClass(t, Relation.class)) { + return Objects.nonNull(((Relation) t).getSource()); + } + return Objects.nonNull(((OafEntity) t).getId()); + }; + } - private static void saveGraphTable(Dataset result, String path) { - logger.info("Saving graph table to path: {}", path); - result.toJSON().write().option("compression", "gzip").text(path); - } + private static void saveGraphTable(Dataset result, String path) { + logger.info("Saving graph table to path: {}", path); + result.toJSON().write().option("compression", "gzip").text(path); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java index cff964003..ffde658bd 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctions.java @@ -1,13 +1,13 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; -import eu.dnetlib.dhp.schema.oaf.Oaf; import java.util.Objects; import java.util.Optional; import java.util.function.BiFunction; import java.util.function.Function; + import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; @@ -15,171 +15,170 @@ import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.TypedColumn; import org.apache.spark.sql.expressions.Aggregator; + +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; +import eu.dnetlib.dhp.schema.oaf.Oaf; import scala.Tuple2; /** Promote action payload functions. */ public class PromoteActionPayloadFunctions { - private PromoteActionPayloadFunctions() {} + private PromoteActionPayloadFunctions() { + } - /** - * Joins dataset representing graph table with dataset representing action payload using supplied - * functions. - * - * @param rowDS Dataset representing graph table - * @param actionPayloadDS Dataset representing action payload - * @param rowIdFn Function used to get the id of graph table row - * @param actionPayloadIdFn Function used to get id of action payload instance - * @param mergeAndGetFn Function used to merge graph table row and action payload instance - * @param rowClazz Class of graph table - * @param actionPayloadClazz Class of action payload - * @param Type of graph table row - * @param Type of action payload instance - * @return Dataset of merged graph table rows and action payload instances - */ - public static Dataset joinGraphTableWithActionPayloadAndMerge( - Dataset rowDS, - Dataset actionPayloadDS, - SerializableSupplier> rowIdFn, - SerializableSupplier> actionPayloadIdFn, - SerializableSupplier> mergeAndGetFn, - Class rowClazz, - Class actionPayloadClazz) { - if (!isSubClass(rowClazz, actionPayloadClazz)) { - throw new RuntimeException( - "action payload type must be the same or be a super type of table row type"); - } + /** + * Joins dataset representing graph table with dataset representing action payload using supplied functions. + * + * @param rowDS Dataset representing graph table + * @param actionPayloadDS Dataset representing action payload + * @param rowIdFn Function used to get the id of graph table row + * @param actionPayloadIdFn Function used to get id of action payload instance + * @param mergeAndGetFn Function used to merge graph table row and action payload instance + * @param rowClazz Class of graph table + * @param actionPayloadClazz Class of action payload + * @param Type of graph table row + * @param Type of action payload instance + * @return Dataset of merged graph table rows and action payload instances + */ + public static Dataset joinGraphTableWithActionPayloadAndMerge( + Dataset rowDS, + Dataset actionPayloadDS, + SerializableSupplier> rowIdFn, + SerializableSupplier> actionPayloadIdFn, + SerializableSupplier> mergeAndGetFn, + Class rowClazz, + Class actionPayloadClazz) { + if (!isSubClass(rowClazz, actionPayloadClazz)) { + throw new RuntimeException( + "action payload type must be the same or be a super type of table row type"); + } - Dataset> rowWithIdDS = mapToTupleWithId(rowDS, rowIdFn, rowClazz); - Dataset> actionPayloadWithIdDS = - mapToTupleWithId(actionPayloadDS, actionPayloadIdFn, actionPayloadClazz); + Dataset> rowWithIdDS = mapToTupleWithId(rowDS, rowIdFn, rowClazz); + Dataset> actionPayloadWithIdDS = mapToTupleWithId( + actionPayloadDS, actionPayloadIdFn, actionPayloadClazz); - return rowWithIdDS - .joinWith( - actionPayloadWithIdDS, - rowWithIdDS.col("_1").equalTo(actionPayloadWithIdDS.col("_1")), - "full_outer") - .map( - (MapFunction, Tuple2>, G>) - value -> { - Optional rowOpt = Optional.ofNullable(value._1()).map(Tuple2::_2); - Optional actionPayloadOpt = Optional.ofNullable(value._2()).map(Tuple2::_2); - return rowOpt - .map( - row -> - actionPayloadOpt - .map( - actionPayload -> - mergeAndGetFn.get().apply(row, actionPayload)) - .orElse(row)) - .orElseGet( - () -> - actionPayloadOpt - .filter( - actionPayload -> actionPayload.getClass().equals(rowClazz)) - .map(rowClazz::cast) - .orElse(null)); - }, - Encoders.kryo(rowClazz)) - .filter((FilterFunction) Objects::nonNull); - } + return rowWithIdDS + .joinWith( + actionPayloadWithIdDS, + rowWithIdDS.col("_1").equalTo(actionPayloadWithIdDS.col("_1")), + "full_outer") + .map( + (MapFunction, Tuple2>, G>) value -> { + Optional rowOpt = Optional.ofNullable(value._1()).map(Tuple2::_2); + Optional actionPayloadOpt = Optional.ofNullable(value._2()).map(Tuple2::_2); + return rowOpt + .map( + row -> actionPayloadOpt + .map( + actionPayload -> mergeAndGetFn.get().apply(row, actionPayload)) + .orElse(row)) + .orElseGet( + () -> actionPayloadOpt + .filter( + actionPayload -> actionPayload.getClass().equals(rowClazz)) + .map(rowClazz::cast) + .orElse(null)); + }, + Encoders.kryo(rowClazz)) + .filter((FilterFunction) Objects::nonNull); + } - private static Dataset> mapToTupleWithId( - Dataset ds, SerializableSupplier> idFn, Class clazz) { - return ds.map( - (MapFunction>) value -> new Tuple2<>(idFn.get().apply(value), value), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); - } + private static Dataset> mapToTupleWithId( + Dataset ds, SerializableSupplier> idFn, Class clazz) { + return ds + .map( + (MapFunction>) value -> new Tuple2<>(idFn.get().apply(value), value), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + } - /** - * Groups graph table by id and aggregates using supplied functions. - * - * @param rowDS Dataset representing graph table - * @param rowIdFn Function used to get the id of graph table row - * @param mergeAndGetFn Function used to merge graph table rows - * @param zeroFn Function to create a zero/empty instance of graph table row - * @param isNotZeroFn Function to check if graph table row is not zero/empty - * @param rowClazz Class of graph table - * @param Type of graph table row - * @return Dataset of aggregated graph table rows - */ - public static Dataset groupGraphTableByIdAndMerge( - Dataset rowDS, - SerializableSupplier> rowIdFn, - SerializableSupplier> mergeAndGetFn, - SerializableSupplier zeroFn, - SerializableSupplier> isNotZeroFn, - Class rowClazz) { - TypedColumn aggregator = - new TableAggregator<>(zeroFn, mergeAndGetFn, isNotZeroFn, rowClazz).toColumn(); - return rowDS - .groupByKey((MapFunction) x -> rowIdFn.get().apply(x), Encoders.STRING()) - .agg(aggregator) - .map((MapFunction, G>) Tuple2::_2, Encoders.kryo(rowClazz)); - } + /** + * Groups graph table by id and aggregates using supplied functions. + * + * @param rowDS Dataset representing graph table + * @param rowIdFn Function used to get the id of graph table row + * @param mergeAndGetFn Function used to merge graph table rows + * @param zeroFn Function to create a zero/empty instance of graph table row + * @param isNotZeroFn Function to check if graph table row is not zero/empty + * @param rowClazz Class of graph table + * @param Type of graph table row + * @return Dataset of aggregated graph table rows + */ + public static Dataset groupGraphTableByIdAndMerge( + Dataset rowDS, + SerializableSupplier> rowIdFn, + SerializableSupplier> mergeAndGetFn, + SerializableSupplier zeroFn, + SerializableSupplier> isNotZeroFn, + Class rowClazz) { + TypedColumn aggregator = new TableAggregator<>(zeroFn, mergeAndGetFn, isNotZeroFn, rowClazz).toColumn(); + return rowDS + .groupByKey((MapFunction) x -> rowIdFn.get().apply(x), Encoders.STRING()) + .agg(aggregator) + .map((MapFunction, G>) Tuple2::_2, Encoders.kryo(rowClazz)); + } - /** - * Aggregator to be used for aggregating graph table rows during grouping. - * - * @param Type of graph table row - */ - public static class TableAggregator extends Aggregator { - private SerializableSupplier zeroFn; - private SerializableSupplier> mergeAndGetFn; - private SerializableSupplier> isNotZeroFn; - private Class rowClazz; + /** + * Aggregator to be used for aggregating graph table rows during grouping. + * + * @param Type of graph table row + */ + public static class TableAggregator extends Aggregator { + private SerializableSupplier zeroFn; + private SerializableSupplier> mergeAndGetFn; + private SerializableSupplier> isNotZeroFn; + private Class rowClazz; - public TableAggregator( - SerializableSupplier zeroFn, - SerializableSupplier> mergeAndGetFn, - SerializableSupplier> isNotZeroFn, - Class rowClazz) { - this.zeroFn = zeroFn; - this.mergeAndGetFn = mergeAndGetFn; - this.isNotZeroFn = isNotZeroFn; - this.rowClazz = rowClazz; - } + public TableAggregator( + SerializableSupplier zeroFn, + SerializableSupplier> mergeAndGetFn, + SerializableSupplier> isNotZeroFn, + Class rowClazz) { + this.zeroFn = zeroFn; + this.mergeAndGetFn = mergeAndGetFn; + this.isNotZeroFn = isNotZeroFn; + this.rowClazz = rowClazz; + } - @Override - public G zero() { - return zeroFn.get(); - } + @Override + public G zero() { + return zeroFn.get(); + } - @Override - public G reduce(G b, G a) { - return zeroSafeMergeAndGet(b, a); - } + @Override + public G reduce(G b, G a) { + return zeroSafeMergeAndGet(b, a); + } - @Override - public G merge(G b1, G b2) { - return zeroSafeMergeAndGet(b1, b2); - } + @Override + public G merge(G b1, G b2) { + return zeroSafeMergeAndGet(b1, b2); + } - private G zeroSafeMergeAndGet(G left, G right) { - Function isNotZero = isNotZeroFn.get(); - if (isNotZero.apply(left) && isNotZero.apply(right)) { - return mergeAndGetFn.get().apply(left, right); - } else if (isNotZero.apply(left) && !isNotZero.apply(right)) { - return left; - } else if (!isNotZero.apply(left) && isNotZero.apply(right)) { - return right; - } - throw new RuntimeException("internal aggregation error: left and right objects are zero"); - } + private G zeroSafeMergeAndGet(G left, G right) { + Function isNotZero = isNotZeroFn.get(); + if (isNotZero.apply(left) && isNotZero.apply(right)) { + return mergeAndGetFn.get().apply(left, right); + } else if (isNotZero.apply(left) && !isNotZero.apply(right)) { + return left; + } else if (!isNotZero.apply(left) && isNotZero.apply(right)) { + return right; + } + throw new RuntimeException("internal aggregation error: left and right objects are zero"); + } - @Override - public G finish(G reduction) { - return reduction; - } + @Override + public G finish(G reduction) { + return reduction; + } - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(rowClazz); - } + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(rowClazz); + } - @Override - public Encoder outputEncoder() { - return Encoders.kryo(rowClazz); - } - } + @Override + public Encoder outputEncoder() { + return Encoders.kryo(rowClazz); + } + } } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java index 3d36cef69..f51c697f4 100644 --- a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/partition/PartitionActionSetsByPayloadTypeJobTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.actionmanager.partition; import static eu.dnetlib.dhp.common.ThrowingSupport.rethrowAsRuntimeException; @@ -5,16 +6,13 @@ import static org.apache.spark.sql.functions.*; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static scala.collection.JavaConversions.mutableSeqAsJavaList; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.actionmanager.ISClient; -import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJobTest; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.*; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; @@ -32,197 +30,212 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.ISClient; +import eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJobTest; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; import scala.collection.mutable.Seq; @ExtendWith(MockitoExtension.class) public class PartitionActionSetsByPayloadTypeJobTest { - private static final ClassLoader cl = - PartitionActionSetsByPayloadTypeJobTest.class.getClassLoader(); + private static final ClassLoader cl = PartitionActionSetsByPayloadTypeJobTest.class.getClassLoader(); - private static Configuration configuration; - private static SparkSession spark; + private static Configuration configuration; + private static SparkSession spark; - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final StructType ATOMIC_ACTION_SCHEMA = - StructType$.MODULE$.apply( - Arrays.asList( - StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()), - StructField$.MODULE$.apply( - "payload", DataTypes.StringType, false, Metadata.empty()))); + private static final StructType ATOMIC_ACTION_SCHEMA = StructType$.MODULE$ + .apply( + Arrays + .asList( + StructField$.MODULE$.apply("clazz", DataTypes.StringType, false, Metadata.empty()), + StructField$.MODULE$ + .apply( + "payload", DataTypes.StringType, false, Metadata.empty()))); - @BeforeAll - public static void beforeAll() throws IOException { - configuration = Job.getInstance().getConfiguration(); - SparkConf conf = new SparkConf(); - conf.setAppName(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); - conf.setMaster("local"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - spark = SparkSession.builder().config(conf).getOrCreate(); - } + @BeforeAll + public static void beforeAll() throws IOException { + configuration = Job.getInstance().getConfiguration(); + SparkConf conf = new SparkConf(); + conf.setAppName(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); + conf.setMaster("local"); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + spark = SparkSession.builder().config(conf).getOrCreate(); + } - @AfterAll - public static void afterAll() { - spark.stop(); - } + @AfterAll + public static void afterAll() { + spark.stop(); + } - @DisplayName("Job") - @Nested - class Main { + @DisplayName("Job") + @Nested + class Main { - @Mock private ISClient isClient; + @Mock + private ISClient isClient; - @Test - public void shouldPartitionActionSetsByPayloadType(@TempDir Path workingDir) throws Exception { - // given - Path inputActionSetsBaseDir = workingDir.resolve("input").resolve("action_sets"); - Path outputDir = workingDir.resolve("output"); + @Test + public void shouldPartitionActionSetsByPayloadType(@TempDir Path workingDir) throws Exception { + // given + Path inputActionSetsBaseDir = workingDir.resolve("input").resolve("action_sets"); + Path outputDir = workingDir.resolve("output"); - Map> oafsByClassName = createActionSets(inputActionSetsBaseDir); + Map> oafsByClassName = createActionSets(inputActionSetsBaseDir); - List inputActionSetsPaths = resolveInputActionSetPaths(inputActionSetsBaseDir); + List inputActionSetsPaths = resolveInputActionSetPaths(inputActionSetsBaseDir); - // when - Mockito.when(isClient.getLatestRawsetPaths(Mockito.anyString())) - .thenReturn(inputActionSetsPaths); + // when + Mockito + .when(isClient.getLatestRawsetPaths(Mockito.anyString())) + .thenReturn(inputActionSetsPaths); - PartitionActionSetsByPayloadTypeJob job = new PartitionActionSetsByPayloadTypeJob(); - job.setIsClient(isClient); - job.run( - Boolean.FALSE, - "", // it can be empty we're mocking the response from isClient - // to - // resolve the - // paths - outputDir.toString()); + PartitionActionSetsByPayloadTypeJob job = new PartitionActionSetsByPayloadTypeJob(); + job.setIsClient(isClient); + job + .run( + Boolean.FALSE, + "", // it can be empty we're mocking the response from isClient + // to + // resolve the + // paths + outputDir.toString()); - // then - Files.exists(outputDir); + // then + Files.exists(outputDir); - assertForOafType(outputDir, oafsByClassName, eu.dnetlib.dhp.schema.oaf.Dataset.class); - assertForOafType(outputDir, oafsByClassName, Datasource.class); - assertForOafType(outputDir, oafsByClassName, Organization.class); - assertForOafType(outputDir, oafsByClassName, OtherResearchProduct.class); - assertForOafType(outputDir, oafsByClassName, Project.class); - assertForOafType(outputDir, oafsByClassName, Publication.class); - assertForOafType(outputDir, oafsByClassName, Result.class); - assertForOafType(outputDir, oafsByClassName, Relation.class); - assertForOafType(outputDir, oafsByClassName, Software.class); - } - } + assertForOafType(outputDir, oafsByClassName, eu.dnetlib.dhp.schema.oaf.Dataset.class); + assertForOafType(outputDir, oafsByClassName, Datasource.class); + assertForOafType(outputDir, oafsByClassName, Organization.class); + assertForOafType(outputDir, oafsByClassName, OtherResearchProduct.class); + assertForOafType(outputDir, oafsByClassName, Project.class); + assertForOafType(outputDir, oafsByClassName, Publication.class); + assertForOafType(outputDir, oafsByClassName, Result.class); + assertForOafType(outputDir, oafsByClassName, Relation.class); + assertForOafType(outputDir, oafsByClassName, Software.class); + } + } - private List resolveInputActionSetPaths(Path inputActionSetsBaseDir) throws IOException { - Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); - return Files.list(inputActionSetJsonDumpsDir) - .map( - path -> { - String inputActionSetId = path.getFileName().toString(); - return inputActionSetsBaseDir.resolve(inputActionSetId).toString(); - }) - .collect(Collectors.toCollection(ArrayList::new)); - } + private List resolveInputActionSetPaths(Path inputActionSetsBaseDir) throws IOException { + Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); + return Files + .list(inputActionSetJsonDumpsDir) + .map( + path -> { + String inputActionSetId = path.getFileName().toString(); + return inputActionSetsBaseDir.resolve(inputActionSetId).toString(); + }) + .collect(Collectors.toCollection(ArrayList::new)); + } - private static Map> createActionSets(Path inputActionSetsDir) - throws IOException { - Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); + private static Map> createActionSets(Path inputActionSetsDir) + throws IOException { + Path inputActionSetJsonDumpsDir = getInputActionSetJsonDumpsDir(); - Map> oafsByType = new HashMap<>(); - Files.list(inputActionSetJsonDumpsDir) - .forEach( - inputActionSetJsonDumpFile -> { - String inputActionSetId = inputActionSetJsonDumpFile.getFileName().toString(); - Path inputActionSetDir = inputActionSetsDir.resolve(inputActionSetId); + Map> oafsByType = new HashMap<>(); + Files + .list(inputActionSetJsonDumpsDir) + .forEach( + inputActionSetJsonDumpFile -> { + String inputActionSetId = inputActionSetJsonDumpFile.getFileName().toString(); + Path inputActionSetDir = inputActionSetsDir.resolve(inputActionSetId); - Dataset actionDS = - readActionsFromJsonDump(inputActionSetJsonDumpFile.toString()).cache(); + Dataset actionDS = readActionsFromJsonDump(inputActionSetJsonDumpFile.toString()).cache(); - writeActionsAsJobInput(actionDS, inputActionSetId, inputActionSetDir.toString()); + writeActionsAsJobInput(actionDS, inputActionSetId, inputActionSetDir.toString()); - Map> actionSetOafsByType = - actionDS - .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) - .select(expr("atomic_action.*")).groupBy(col("clazz")) - .agg(collect_list(col("payload")).as("payload_list")).collectAsList().stream() - .map( - row -> - new AbstractMap.SimpleEntry<>( - row.getAs("clazz"), - mutableSeqAsJavaList(row.>getAs("payload_list")))) - .collect( - Collectors.toMap( - AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); + Map> actionSetOafsByType = actionDS + .withColumn("atomic_action", from_json(col("value"), ATOMIC_ACTION_SCHEMA)) + .select(expr("atomic_action.*")) + .groupBy(col("clazz")) + .agg(collect_list(col("payload")).as("payload_list")) + .collectAsList() + .stream() + .map( + row -> new AbstractMap.SimpleEntry<>( + row. getAs("clazz"), + mutableSeqAsJavaList(row.> getAs("payload_list")))) + .collect( + Collectors + .toMap( + AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue)); - actionSetOafsByType - .keySet() - .forEach( - x -> { - if (oafsByType.containsKey(x)) { - List collected = new ArrayList<>(); - collected.addAll(oafsByType.get(x)); - collected.addAll(actionSetOafsByType.get(x)); - oafsByType.put(x, collected); - } else { - oafsByType.put(x, actionSetOafsByType.get(x)); - } - }); - }); + actionSetOafsByType + .keySet() + .forEach( + x -> { + if (oafsByType.containsKey(x)) { + List collected = new ArrayList<>(); + collected.addAll(oafsByType.get(x)); + collected.addAll(actionSetOafsByType.get(x)); + oafsByType.put(x, collected); + } else { + oafsByType.put(x, actionSetOafsByType.get(x)); + } + }); + }); - return oafsByType; - } + return oafsByType; + } - private static Path getInputActionSetJsonDumpsDir() { - return Paths.get( - Objects.requireNonNull(cl.getResource("eu/dnetlib/dhp/actionmanager/partition/input/")) - .getFile()); - } + private static Path getInputActionSetJsonDumpsDir() { + return Paths + .get( + Objects + .requireNonNull(cl.getResource("eu/dnetlib/dhp/actionmanager/partition/input/")) + .getFile()); + } - private static Dataset readActionsFromJsonDump(String path) { - return spark.read().textFile(path); - } + private static Dataset readActionsFromJsonDump(String path) { + return spark.read().textFile(path); + } - private static void writeActionsAsJobInput( - Dataset actionDS, String inputActionSetId, String path) { - actionDS - .javaRDD() - .mapToPair(json -> new Tuple2<>(new Text(inputActionSetId), new Text(json))) - .saveAsNewAPIHadoopFile( - path, Text.class, Text.class, SequenceFileOutputFormat.class, configuration); - } + private static void writeActionsAsJobInput( + Dataset actionDS, String inputActionSetId, String path) { + actionDS + .javaRDD() + .mapToPair(json -> new Tuple2<>(new Text(inputActionSetId), new Text(json))) + .saveAsNewAPIHadoopFile( + path, Text.class, Text.class, SequenceFileOutputFormat.class, configuration); + } - private static void assertForOafType( - Path outputDir, Map> oafsByClassName, Class clazz) { - Path outputDatasetDir = outputDir.resolve(String.format("clazz=%s", clazz.getCanonicalName())); - Files.exists(outputDatasetDir); + private static void assertForOafType( + Path outputDir, Map> oafsByClassName, Class clazz) { + Path outputDatasetDir = outputDir.resolve(String.format("clazz=%s", clazz.getCanonicalName())); + Files.exists(outputDatasetDir); - List actuals = - readActionPayloadFromJobOutput(outputDatasetDir.toString(), clazz).collectAsList(); - actuals.sort(Comparator.comparingInt(Object::hashCode)); + List actuals = readActionPayloadFromJobOutput(outputDatasetDir.toString(), clazz).collectAsList(); + actuals.sort(Comparator.comparingInt(Object::hashCode)); - List expecteds = - oafsByClassName.get(clazz.getCanonicalName()).stream() - .map(json -> mapToOaf(json, clazz)) - .sorted(Comparator.comparingInt(Object::hashCode)) - .collect(Collectors.toList()); + List expecteds = oafsByClassName + .get(clazz.getCanonicalName()) + .stream() + .map(json -> mapToOaf(json, clazz)) + .sorted(Comparator.comparingInt(Object::hashCode)) + .collect(Collectors.toList()); - assertIterableEquals(expecteds, actuals); - } + assertIterableEquals(expecteds, actuals); + } - private static Dataset readActionPayloadFromJobOutput( - String path, Class clazz) { - return spark - .read() - .parquet(path) - .map( - (MapFunction) - value -> OBJECT_MAPPER.readValue(value.getAs("payload"), clazz), - Encoders.bean(clazz)); - } + private static Dataset readActionPayloadFromJobOutput( + String path, Class clazz) { + return spark + .read() + .parquet(path) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value. getAs("payload"), clazz), + Encoders.bean(clazz)); + } - private static T mapToOaf(String json, Class clazz) { - return rethrowAsRuntimeException( - () -> OBJECT_MAPPER.readValue(json, clazz), - String.format( - "failed to map json to class: json=%s, class=%s", json, clazz.getCanonicalName())); - } + private static T mapToOaf(String json, Class clazz) { + return rethrowAsRuntimeException( + () -> OBJECT_MAPPER.readValue(json, clazz), + String + .format( + "failed to map json to class: json=%s, class=%s", json, clazz.getCanonicalName())); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGetTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGetTest.java index 0de6f6b4f..b2248d77a 100644 --- a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGetTest.java +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGetTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static eu.dnetlib.dhp.actionmanager.promote.MergeAndGet.Strategy; @@ -5,254 +6,252 @@ import static eu.dnetlib.dhp.actionmanager.promote.MergeAndGet.functionFor; import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.*; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.function.BiFunction; + import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; +import eu.dnetlib.dhp.schema.oaf.*; + public class MergeAndGetTest { - @Nested - class MergeFromAndGetStrategy { + @Nested + class MergeFromAndGetStrategy { - @Test - public void shouldThrowForOafAndOaf() { - // given - Oaf a = mock(Oaf.class); - Oaf b = mock(Oaf.class); + @Test + public void shouldThrowForOafAndOaf() { + // given + Oaf a = mock(Oaf.class); + Oaf b = mock(Oaf.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForOafAndRelation() { - // given - Oaf a = mock(Oaf.class); - Relation b = mock(Relation.class); + @Test + public void shouldThrowForOafAndRelation() { + // given + Oaf a = mock(Oaf.class); + Relation b = mock(Relation.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForOafAndOafEntity() { - // given - Oaf a = mock(Oaf.class); - OafEntity b = mock(OafEntity.class); + @Test + public void shouldThrowForOafAndOafEntity() { + // given + Oaf a = mock(Oaf.class); + OafEntity b = mock(OafEntity.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForRelationAndOaf() { - // given - Relation a = mock(Relation.class); - Oaf b = mock(Oaf.class); + @Test + public void shouldThrowForRelationAndOaf() { + // given + Relation a = mock(Relation.class); + Oaf b = mock(Oaf.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForRelationAndOafEntity() { - // given - Relation a = mock(Relation.class); - OafEntity b = mock(OafEntity.class); + @Test + public void shouldThrowForRelationAndOafEntity() { + // given + Relation a = mock(Relation.class); + OafEntity b = mock(OafEntity.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldBehaveProperlyForRelationAndRelation() { - // given - Relation a = mock(Relation.class); - Relation b = mock(Relation.class); + @Test + public void shouldBehaveProperlyForRelationAndRelation() { + // given + Relation a = mock(Relation.class); + Relation b = mock(Relation.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - Oaf x = fn.get().apply(a, b); - assertTrue(Relation.class.isAssignableFrom(x.getClass())); - verify(a).mergeFrom(b); - assertEquals(a, x); - } + // then + Oaf x = fn.get().apply(a, b); + assertTrue(Relation.class.isAssignableFrom(x.getClass())); + verify(a).mergeFrom(b); + assertEquals(a, x); + } - @Test - public void shouldThrowForOafEntityAndOaf() { - // given - OafEntity a = mock(OafEntity.class); - Oaf b = mock(Oaf.class); + @Test + public void shouldThrowForOafEntityAndOaf() { + // given + OafEntity a = mock(OafEntity.class); + Oaf b = mock(Oaf.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForOafEntityAndRelation() { - // given - OafEntity a = mock(OafEntity.class); - Relation b = mock(Relation.class); + @Test + public void shouldThrowForOafEntityAndRelation() { + // given + OafEntity a = mock(OafEntity.class); + Relation b = mock(Relation.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForOafEntityAndOafEntityButNotSubclasses() { - // given - class OafEntitySub1 extends OafEntity {} - class OafEntitySub2 extends OafEntity {} + @Test + public void shouldThrowForOafEntityAndOafEntityButNotSubclasses() { + // given + class OafEntitySub1 extends OafEntity { + } + class OafEntitySub2 extends OafEntity { + } - OafEntitySub1 a = mock(OafEntitySub1.class); - OafEntitySub2 b = mock(OafEntitySub2.class); + OafEntitySub1 a = mock(OafEntitySub1.class); + OafEntitySub2 b = mock(OafEntitySub2.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldBehaveProperlyForOafEntityAndOafEntity() { - // given - OafEntity a = mock(OafEntity.class); - OafEntity b = mock(OafEntity.class); + @Test + public void shouldBehaveProperlyForOafEntityAndOafEntity() { + // given + OafEntity a = mock(OafEntity.class); + OafEntity b = mock(OafEntity.class); - // when - SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.MERGE_FROM_AND_GET); - // then - Oaf x = fn.get().apply(a, b); - assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); - verify(a).mergeFrom(b); - assertEquals(a, x); - } - } + // then + Oaf x = fn.get().apply(a, b); + assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); + verify(a).mergeFrom(b); + assertEquals(a, x); + } + } - @Nested - class SelectNewerAndGetStrategy { + @Nested + class SelectNewerAndGetStrategy { - @Test - public void shouldThrowForOafEntityAndRelation() { - // given - OafEntity a = mock(OafEntity.class); - Relation b = mock(Relation.class); + @Test + public void shouldThrowForOafEntityAndRelation() { + // given + OafEntity a = mock(OafEntity.class); + Relation b = mock(Relation.class); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForRelationAndOafEntity() { - // given - Relation a = mock(Relation.class); - OafEntity b = mock(OafEntity.class); + @Test + public void shouldThrowForRelationAndOafEntity() { + // given + Relation a = mock(Relation.class); + OafEntity b = mock(OafEntity.class); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowForOafEntityAndResult() { - // given - OafEntity a = mock(OafEntity.class); - Result b = mock(Result.class); + @Test + public void shouldThrowForOafEntityAndResult() { + // given + OafEntity a = mock(OafEntity.class); + Result b = mock(Result.class); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldThrowWhenSuperTypeIsNewerForResultAndOafEntity() { - // given - // real types must be used because subclass-superclass resolution does not work for - // mocks - Dataset a = new Dataset(); - a.setLastupdatetimestamp(1L); - Result b = new Result(); - b.setLastupdatetimestamp(2L); + @Test + public void shouldThrowWhenSuperTypeIsNewerForResultAndOafEntity() { + // given + // real types must be used because subclass-superclass resolution does not work for + // mocks + Dataset a = new Dataset(); + a.setLastupdatetimestamp(1L); + Result b = new Result(); + b.setLastupdatetimestamp(2L); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); - } + // then + assertThrows(RuntimeException.class, () -> fn.get().apply(a, b)); + } - @Test - public void shouldShouldReturnLeftForOafEntityAndOafEntity() { - // given - OafEntity a = mock(OafEntity.class); - when(a.getLastupdatetimestamp()).thenReturn(1L); - OafEntity b = mock(OafEntity.class); - when(b.getLastupdatetimestamp()).thenReturn(2L); + @Test + public void shouldShouldReturnLeftForOafEntityAndOafEntity() { + // given + OafEntity a = mock(OafEntity.class); + when(a.getLastupdatetimestamp()).thenReturn(1L); + OafEntity b = mock(OafEntity.class); + when(b.getLastupdatetimestamp()).thenReturn(2L); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - Oaf x = fn.get().apply(a, b); - assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); - assertEquals(b, x); - } + // then + Oaf x = fn.get().apply(a, b); + assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); + assertEquals(b, x); + } - @Test - public void shouldShouldReturnRightForOafEntityAndOafEntity() { - // given - OafEntity a = mock(OafEntity.class); - when(a.getLastupdatetimestamp()).thenReturn(2L); - OafEntity b = mock(OafEntity.class); - when(b.getLastupdatetimestamp()).thenReturn(1L); + @Test + public void shouldShouldReturnRightForOafEntityAndOafEntity() { + // given + OafEntity a = mock(OafEntity.class); + when(a.getLastupdatetimestamp()).thenReturn(2L); + OafEntity b = mock(OafEntity.class); + when(b.getLastupdatetimestamp()).thenReturn(1L); - // when - SerializableSupplier> fn = - functionFor(Strategy.SELECT_NEWER_AND_GET); + // when + SerializableSupplier> fn = functionFor(Strategy.SELECT_NEWER_AND_GET); - // then - Oaf x = fn.get().apply(a, b); - assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); - assertEquals(a, x); - } - } + // then + Oaf x = fn.get().apply(a, b); + assertTrue(OafEntity.class.isAssignableFrom(x.getClass())); + assertEquals(a, x); + } + } } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJobTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJobTest.java index e8f802585..129daadcc 100644 --- a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJobTest.java +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadForGraphTableJobTest.java @@ -1,11 +1,9 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static org.junit.jupiter.api.Assertions.*; import static org.junit.jupiter.params.provider.Arguments.arguments; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; @@ -15,6 +13,7 @@ import java.util.List; import java.util.Objects; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -26,253 +25,256 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; + public class PromoteActionPayloadForGraphTableJobTest { - private static final ClassLoader cl = - PromoteActionPayloadForGraphTableJobTest.class.getClassLoader(); + private static final ClassLoader cl = PromoteActionPayloadForGraphTableJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private Path workingDir; - private Path inputDir; - private Path inputGraphRootDir; - private Path inputActionPayloadRootDir; - private Path outputDir; + private Path workingDir; + private Path inputDir; + private Path inputGraphRootDir; + private Path inputActionPayloadRootDir; + private Path outputDir; - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - @BeforeAll - public static void beforeAll() { - SparkConf conf = new SparkConf(); - conf.setAppName(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); - conf.setMaster("local"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - spark = SparkSession.builder().config(conf).getOrCreate(); - } + @BeforeAll + public static void beforeAll() { + SparkConf conf = new SparkConf(); + conf.setAppName(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); + conf.setMaster("local"); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + spark = SparkSession.builder().config(conf).getOrCreate(); + } - @BeforeEach - public void beforeEach() throws IOException { - workingDir = - Files.createTempDirectory(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); - inputDir = workingDir.resolve("input"); - inputGraphRootDir = inputDir.resolve("graph"); - inputActionPayloadRootDir = inputDir.resolve("action_payload"); - outputDir = workingDir.resolve("output"); - } + @BeforeEach + public void beforeEach() throws IOException { + workingDir = Files.createTempDirectory(PromoteActionPayloadForGraphTableJobTest.class.getSimpleName()); + inputDir = workingDir.resolve("input"); + inputGraphRootDir = inputDir.resolve("graph"); + inputActionPayloadRootDir = inputDir.resolve("action_payload"); + outputDir = workingDir.resolve("output"); + } - @AfterEach - public void afterEach() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - } + @AfterEach + public void afterEach() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + } - @AfterAll - public static void afterAll() { - spark.stop(); - } + @AfterAll + public static void afterAll() { + spark.stop(); + } - @DisplayName("Job") - @Nested - class Main { + @DisplayName("Job") + @Nested + class Main { - @Test - public void shouldThrowWhenGraphTableClassIsNotASubClassOfActionPayloadClass() { - // given - Class rowClazz = Relation.class; - Class actionPayloadClazz = OafEntity.class; + @Test + public void shouldThrowWhenGraphTableClassIsNotASubClassOfActionPayloadClass() { + // given + Class rowClazz = Relation.class; + Class actionPayloadClazz = OafEntity.class; - // when - RuntimeException exception = - assertThrows( - RuntimeException.class, - () -> - PromoteActionPayloadForGraphTableJob.main( - new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-inputGraphTablePath", - "", - "-graphTableClassName", - rowClazz.getCanonicalName(), - "-inputActionPayloadPath", - "", - "-actionPayloadClassName", - actionPayloadClazz.getCanonicalName(), - "-outputGraphTablePath", - "", - "-mergeAndGetStrategy", - MergeAndGet.Strategy.SELECT_NEWER_AND_GET.name() - })); + // when + RuntimeException exception = assertThrows( + RuntimeException.class, + () -> PromoteActionPayloadForGraphTableJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-inputGraphTablePath", + "", + "-graphTableClassName", + rowClazz.getCanonicalName(), + "-inputActionPayloadPath", + "", + "-actionPayloadClassName", + actionPayloadClazz.getCanonicalName(), + "-outputGraphTablePath", + "", + "-mergeAndGetStrategy", + MergeAndGet.Strategy.SELECT_NEWER_AND_GET.name() + })); - // then - String msg = - String.format( - "graph table class is not a subclass of action payload class: graph=%s, action=%s", - rowClazz.getCanonicalName(), actionPayloadClazz.getCanonicalName()); - assertTrue(exception.getMessage().contains(msg)); - } + // then + String msg = String + .format( + "graph table class is not a subclass of action payload class: graph=%s, action=%s", + rowClazz.getCanonicalName(), actionPayloadClazz.getCanonicalName()); + assertTrue(exception.getMessage().contains(msg)); + } - @ParameterizedTest(name = "strategy: {0}, graph table: {1}, action payload: {2}") - @MethodSource( - "eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJobTest#promoteJobTestParams") - public void shouldPromoteActionPayloadForGraphTable( - MergeAndGet.Strategy strategy, - Class rowClazz, - Class actionPayloadClazz) - throws Exception { - // given - Path inputGraphTableDir = createGraphTable(inputGraphRootDir, rowClazz); - Path inputActionPayloadDir = - createActionPayload(inputActionPayloadRootDir, rowClazz, actionPayloadClazz); - Path outputGraphTableDir = - outputDir.resolve("graph").resolve(rowClazz.getSimpleName().toLowerCase()); + @ParameterizedTest(name = "strategy: {0}, graph table: {1}, action payload: {2}") + @MethodSource("eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJobTest#promoteJobTestParams") + public void shouldPromoteActionPayloadForGraphTable( + MergeAndGet.Strategy strategy, + Class rowClazz, + Class actionPayloadClazz) + throws Exception { + // given + Path inputGraphTableDir = createGraphTable(inputGraphRootDir, rowClazz); + Path inputActionPayloadDir = createActionPayload(inputActionPayloadRootDir, rowClazz, actionPayloadClazz); + Path outputGraphTableDir = outputDir.resolve("graph").resolve(rowClazz.getSimpleName().toLowerCase()); - // when - PromoteActionPayloadForGraphTableJob.main( - new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-inputGraphTablePath", - inputGraphTableDir.toString(), - "-graphTableClassName", - rowClazz.getCanonicalName(), - "-inputActionPayloadPath", - inputActionPayloadDir.toString(), - "-actionPayloadClassName", - actionPayloadClazz.getCanonicalName(), - "-outputGraphTablePath", - outputGraphTableDir.toString(), - "-mergeAndGetStrategy", - strategy.name() - }); + // when + PromoteActionPayloadForGraphTableJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-inputGraphTablePath", + inputGraphTableDir.toString(), + "-graphTableClassName", + rowClazz.getCanonicalName(), + "-inputActionPayloadPath", + inputActionPayloadDir.toString(), + "-actionPayloadClassName", + actionPayloadClazz.getCanonicalName(), + "-outputGraphTablePath", + outputGraphTableDir.toString(), + "-mergeAndGetStrategy", + strategy.name() + }); - // then - assertTrue(Files.exists(outputGraphTableDir)); + // then + assertTrue(Files.exists(outputGraphTableDir)); - List actualOutputRows = - readGraphTableFromJobOutput(outputGraphTableDir.toString(), rowClazz).collectAsList() - .stream() - .sorted(Comparator.comparingInt(Object::hashCode)) - .collect(Collectors.toList()); - String expectedOutputGraphTableJsonDumpPath = - resultFileLocation(strategy, rowClazz, actionPayloadClazz); - Path expectedOutputGraphTableJsonDumpFile = - Paths.get( - Objects.requireNonNull(cl.getResource(expectedOutputGraphTableJsonDumpPath)) - .getFile()); - List expectedOutputRows = - readGraphTableFromJsonDump(expectedOutputGraphTableJsonDumpFile.toString(), rowClazz) - .collectAsList().stream() - .sorted(Comparator.comparingInt(Object::hashCode)) - .collect(Collectors.toList()); - assertIterableEquals(expectedOutputRows, actualOutputRows); - } - } + List actualOutputRows = readGraphTableFromJobOutput(outputGraphTableDir.toString(), rowClazz) + .collectAsList() + .stream() + .sorted(Comparator.comparingInt(Object::hashCode)) + .collect(Collectors.toList()); + String expectedOutputGraphTableJsonDumpPath = resultFileLocation(strategy, rowClazz, actionPayloadClazz); + Path expectedOutputGraphTableJsonDumpFile = Paths + .get( + Objects + .requireNonNull(cl.getResource(expectedOutputGraphTableJsonDumpPath)) + .getFile()); + List expectedOutputRows = readGraphTableFromJsonDump( + expectedOutputGraphTableJsonDumpFile.toString(), rowClazz) + .collectAsList() + .stream() + .sorted(Comparator.comparingInt(Object::hashCode)) + .collect(Collectors.toList()); + assertIterableEquals(expectedOutputRows, actualOutputRows); + } + } - public static Stream promoteJobTestParams() { - return Stream.of( - arguments( - MergeAndGet.Strategy.MERGE_FROM_AND_GET, - eu.dnetlib.dhp.schema.oaf.Dataset.class, - eu.dnetlib.dhp.schema.oaf.Dataset.class), - arguments( - MergeAndGet.Strategy.MERGE_FROM_AND_GET, - eu.dnetlib.dhp.schema.oaf.Dataset.class, - eu.dnetlib.dhp.schema.oaf.Result.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Datasource.class, Datasource.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Organization.class, Organization.class), - arguments( - MergeAndGet.Strategy.MERGE_FROM_AND_GET, - OtherResearchProduct.class, - OtherResearchProduct.class), - arguments( - MergeAndGet.Strategy.MERGE_FROM_AND_GET, OtherResearchProduct.class, Result.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Project.class, Project.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Publication.class, Publication.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Publication.class, Result.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Relation.class, Relation.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Software.class, Software.class), - arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Software.class, Result.class)); - } + public static Stream promoteJobTestParams() { + return Stream + .of( + arguments( + MergeAndGet.Strategy.MERGE_FROM_AND_GET, + eu.dnetlib.dhp.schema.oaf.Dataset.class, + eu.dnetlib.dhp.schema.oaf.Dataset.class), + arguments( + MergeAndGet.Strategy.MERGE_FROM_AND_GET, + eu.dnetlib.dhp.schema.oaf.Dataset.class, + eu.dnetlib.dhp.schema.oaf.Result.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Datasource.class, Datasource.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Organization.class, Organization.class), + arguments( + MergeAndGet.Strategy.MERGE_FROM_AND_GET, + OtherResearchProduct.class, + OtherResearchProduct.class), + arguments( + MergeAndGet.Strategy.MERGE_FROM_AND_GET, OtherResearchProduct.class, Result.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Project.class, Project.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Publication.class, Publication.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Publication.class, Result.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Relation.class, Relation.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Software.class, Software.class), + arguments(MergeAndGet.Strategy.MERGE_FROM_AND_GET, Software.class, Result.class)); + } - private static Path createGraphTable(Path inputGraphRootDir, Class rowClazz) { - String inputGraphTableJsonDumpPath = inputGraphTableJsonDumpLocation(rowClazz); - Path inputGraphTableJsonDumpFile = - Paths.get(Objects.requireNonNull(cl.getResource(inputGraphTableJsonDumpPath)).getFile()); - Dataset rowDS = readGraphTableFromJsonDump(inputGraphTableJsonDumpFile.toString(), rowClazz); - String inputGraphTableName = rowClazz.getSimpleName().toLowerCase(); - Path inputGraphTableDir = inputGraphRootDir.resolve(inputGraphTableName); - writeGraphTableAaJobInput(rowDS, inputGraphTableDir.toString()); - return inputGraphTableDir; - } + private static Path createGraphTable(Path inputGraphRootDir, Class rowClazz) { + String inputGraphTableJsonDumpPath = inputGraphTableJsonDumpLocation(rowClazz); + Path inputGraphTableJsonDumpFile = Paths + .get(Objects.requireNonNull(cl.getResource(inputGraphTableJsonDumpPath)).getFile()); + Dataset rowDS = readGraphTableFromJsonDump(inputGraphTableJsonDumpFile.toString(), rowClazz); + String inputGraphTableName = rowClazz.getSimpleName().toLowerCase(); + Path inputGraphTableDir = inputGraphRootDir.resolve(inputGraphTableName); + writeGraphTableAaJobInput(rowDS, inputGraphTableDir.toString()); + return inputGraphTableDir; + } - private static String inputGraphTableJsonDumpLocation(Class rowClazz) { - return String.format( - "%s/%s.json", - "eu/dnetlib/dhp/actionmanager/promote/input/graph", rowClazz.getSimpleName().toLowerCase()); - } + private static String inputGraphTableJsonDumpLocation(Class rowClazz) { + return String + .format( + "%s/%s.json", + "eu/dnetlib/dhp/actionmanager/promote/input/graph", rowClazz.getSimpleName().toLowerCase()); + } - private static Dataset readGraphTableFromJsonDump( - String path, Class rowClazz) { - return spark - .read() - .textFile(path) - .map( - (MapFunction) json -> OBJECT_MAPPER.readValue(json, rowClazz), - Encoders.bean(rowClazz)); - } + private static Dataset readGraphTableFromJsonDump( + String path, Class rowClazz) { + return spark + .read() + .textFile(path) + .map( + (MapFunction) json -> OBJECT_MAPPER.readValue(json, rowClazz), + Encoders.bean(rowClazz)); + } - private static void writeGraphTableAaJobInput(Dataset rowDS, String path) { - rowDS.write().option("compression", "gzip").json(path); - } + private static void writeGraphTableAaJobInput(Dataset rowDS, String path) { + rowDS.write().option("compression", "gzip").json(path); + } - private static Path createActionPayload( - Path inputActionPayloadRootDir, Class rowClazz, Class actionPayloadClazz) { - String inputActionPayloadJsonDumpPath = - inputActionPayloadJsonDumpLocation(rowClazz, actionPayloadClazz); - Path inputActionPayloadJsonDumpFile = - Paths.get(Objects.requireNonNull(cl.getResource(inputActionPayloadJsonDumpPath)).getFile()); - Dataset actionPayloadDS = - readActionPayloadFromJsonDump(inputActionPayloadJsonDumpFile.toString()); - Path inputActionPayloadDir = - inputActionPayloadRootDir.resolve(actionPayloadClazz.getSimpleName().toLowerCase()); - writeActionPayloadAsJobInput(actionPayloadDS, inputActionPayloadDir.toString()); - return inputActionPayloadDir; - } + private static Path createActionPayload( + Path inputActionPayloadRootDir, Class rowClazz, Class actionPayloadClazz) { + String inputActionPayloadJsonDumpPath = inputActionPayloadJsonDumpLocation(rowClazz, actionPayloadClazz); + Path inputActionPayloadJsonDumpFile = Paths + .get(Objects.requireNonNull(cl.getResource(inputActionPayloadJsonDumpPath)).getFile()); + Dataset actionPayloadDS = readActionPayloadFromJsonDump(inputActionPayloadJsonDumpFile.toString()); + Path inputActionPayloadDir = inputActionPayloadRootDir + .resolve(actionPayloadClazz.getSimpleName().toLowerCase()); + writeActionPayloadAsJobInput(actionPayloadDS, inputActionPayloadDir.toString()); + return inputActionPayloadDir; + } - private static String inputActionPayloadJsonDumpLocation( - Class rowClazz, Class actionPayloadClazz) { + private static String inputActionPayloadJsonDumpLocation( + Class rowClazz, Class actionPayloadClazz) { - return String.format( - "eu/dnetlib/dhp/actionmanager/promote/input/action_payload/%s_table/%s.json", - rowClazz.getSimpleName().toLowerCase(), actionPayloadClazz.getSimpleName().toLowerCase()); - } + return String + .format( + "eu/dnetlib/dhp/actionmanager/promote/input/action_payload/%s_table/%s.json", + rowClazz.getSimpleName().toLowerCase(), actionPayloadClazz.getSimpleName().toLowerCase()); + } - private static Dataset readActionPayloadFromJsonDump(String path) { - return spark.read().textFile(path); - } + private static Dataset readActionPayloadFromJsonDump(String path) { + return spark.read().textFile(path); + } - private static void writeActionPayloadAsJobInput(Dataset actionPayloadDS, String path) { - actionPayloadDS.withColumnRenamed("value", "payload").write().parquet(path); - } + private static void writeActionPayloadAsJobInput(Dataset actionPayloadDS, String path) { + actionPayloadDS.withColumnRenamed("value", "payload").write().parquet(path); + } - private static Dataset readGraphTableFromJobOutput( - String path, Class rowClazz) { - return spark - .read() - .textFile(path) - .map( - (MapFunction) json -> OBJECT_MAPPER.readValue(json, rowClazz), - Encoders.bean(rowClazz)); - } + private static Dataset readGraphTableFromJobOutput( + String path, Class rowClazz) { + return spark + .read() + .textFile(path) + .map( + (MapFunction) json -> OBJECT_MAPPER.readValue(json, rowClazz), + Encoders.bean(rowClazz)); + } - private static String resultFileLocation( - MergeAndGet.Strategy strategy, - Class rowClazz, - Class actionPayloadClazz) { - return String.format( - "eu/dnetlib/dhp/actionmanager/promote/output/graph/%s/%s/%s_action_payload/result.json", - strategy.name().toLowerCase(), - rowClazz.getSimpleName().toLowerCase(), - actionPayloadClazz.getSimpleName().toLowerCase()); - } + private static String resultFileLocation( + MergeAndGet.Strategy strategy, + Class rowClazz, + Class actionPayloadClazz) { + return String + .format( + "eu/dnetlib/dhp/actionmanager/promote/output/graph/%s/%s/%s_action_payload/result.json", + strategy.name().toLowerCase(), + rowClazz.getSimpleName().toLowerCase(), + actionPayloadClazz.getSimpleName().toLowerCase()); + } } diff --git a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctionsTest.java b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctionsTest.java index 9abb0858f..477e4b204 100644 --- a/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctionsTest.java +++ b/dhp-workflows/dhp-actionmanager/src/test/java/eu/dnetlib/dhp/actionmanager/promote/PromoteActionPayloadFunctionsTest.java @@ -1,15 +1,15 @@ + package eu.dnetlib.dhp.actionmanager.promote; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; -import eu.dnetlib.dhp.schema.oaf.Oaf; import java.util.Arrays; import java.util.List; import java.util.Objects; import java.util.function.BiFunction; import java.util.function.Function; + import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -19,314 +19,311 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; +import eu.dnetlib.dhp.schema.oaf.Oaf; + public class PromoteActionPayloadFunctionsTest { - private static SparkSession spark; + private static SparkSession spark; - @BeforeAll - public static void beforeAll() { - SparkConf conf = new SparkConf(); - conf.setMaster("local"); - conf.setAppName(PromoteActionPayloadFunctionsTest.class.getSimpleName()); - conf.set("spark.driver.host", "localhost"); - spark = SparkSession.builder().config(conf).getOrCreate(); - } + @BeforeAll + public static void beforeAll() { + SparkConf conf = new SparkConf(); + conf.setMaster("local"); + conf.setAppName(PromoteActionPayloadFunctionsTest.class.getSimpleName()); + conf.set("spark.driver.host", "localhost"); + spark = SparkSession.builder().config(conf).getOrCreate(); + } - @AfterAll - public static void afterAll() { - spark.stop(); - } + @AfterAll + public static void afterAll() { + spark.stop(); + } - @Nested - class JoinTableWithActionPayloadAndMerge { + @Nested + class JoinTableWithActionPayloadAndMerge { - @Test - public void shouldThrowWhenTableTypeIsNotSubtypeOfActionPayloadType() { - // given - class OafImpl extends Oaf {} + @Test + public void shouldThrowWhenTableTypeIsNotSubtypeOfActionPayloadType() { + // given + class OafImpl extends Oaf { + } - // when - assertThrows( - RuntimeException.class, - () -> - PromoteActionPayloadFunctions.joinGraphTableWithActionPayloadAndMerge( - null, null, null, null, null, OafImplSubSub.class, OafImpl.class)); - } + // when + assertThrows( + RuntimeException.class, + () -> PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + null, null, null, null, null, OafImplSubSub.class, OafImpl.class)); + } - @Test - public void shouldRunProperlyWhenActionPayloadTypeAndTableTypeAreTheSame() { - // given - String id0 = "id0"; - String id1 = "id1"; - String id2 = "id2"; - String id3 = "id3"; - String id4 = "id4"; - List rowData = - Arrays.asList( - createOafImplSubSub(id0), - createOafImplSubSub(id1), - createOafImplSubSub(id2), - createOafImplSubSub(id3)); - Dataset rowDS = - spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); + @Test + public void shouldRunProperlyWhenActionPayloadTypeAndTableTypeAreTheSame() { + // given + String id0 = "id0"; + String id1 = "id1"; + String id2 = "id2"; + String id3 = "id3"; + String id4 = "id4"; + List rowData = Arrays + .asList( + createOafImplSubSub(id0), + createOafImplSubSub(id1), + createOafImplSubSub(id2), + createOafImplSubSub(id3)); + Dataset rowDS = spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); - List actionPayloadData = - Arrays.asList( - createOafImplSubSub(id1), - createOafImplSubSub(id2), - createOafImplSubSub(id2), - createOafImplSubSub(id3), - createOafImplSubSub(id3), - createOafImplSubSub(id3), - createOafImplSubSub(id4), - createOafImplSubSub(id4), - createOafImplSubSub(id4), - createOafImplSubSub(id4)); - Dataset actionPayloadDS = - spark.createDataset(actionPayloadData, Encoders.bean(OafImplSubSub.class)); + List actionPayloadData = Arrays + .asList( + createOafImplSubSub(id1), + createOafImplSubSub(id2), + createOafImplSubSub(id2), + createOafImplSubSub(id3), + createOafImplSubSub(id3), + createOafImplSubSub(id3), + createOafImplSubSub(id4), + createOafImplSubSub(id4), + createOafImplSubSub(id4), + createOafImplSubSub(id4)); + Dataset actionPayloadDS = spark + .createDataset(actionPayloadData, Encoders.bean(OafImplSubSub.class)); - SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; - SerializableSupplier> actionPayloadIdFn = - () -> OafImplRoot::getId; - SerializableSupplier> mergeAndGetFn = - () -> - (x, y) -> { - x.merge(y); - return x; - }; + SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; + SerializableSupplier> actionPayloadIdFn = () -> OafImplRoot::getId; + SerializableSupplier> mergeAndGetFn = () -> (x, + y) -> { + x.merge(y); + return x; + }; - // when - List results = - PromoteActionPayloadFunctions.joinGraphTableWithActionPayloadAndMerge( - rowDS, - actionPayloadDS, - rowIdFn, - actionPayloadIdFn, - mergeAndGetFn, - OafImplSubSub.class, - OafImplSubSub.class) - .collectAsList(); + // when + List results = PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + rowDS, + actionPayloadDS, + rowIdFn, + actionPayloadIdFn, + mergeAndGetFn, + OafImplSubSub.class, + OafImplSubSub.class) + .collectAsList(); - // then - assertEquals(11, results.size()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id0)).count()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); - assertEquals(2, results.stream().filter(x -> x.getId().equals(id2)).count()); - assertEquals(3, results.stream().filter(x -> x.getId().equals(id3)).count()); - assertEquals(4, results.stream().filter(x -> x.getId().equals(id4)).count()); + // then + assertEquals(11, results.size()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id0)).count()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); + assertEquals(2, results.stream().filter(x -> x.getId().equals(id2)).count()); + assertEquals(3, results.stream().filter(x -> x.getId().equals(id3)).count()); + assertEquals(4, results.stream().filter(x -> x.getId().equals(id4)).count()); - results.forEach( - result -> { - switch (result.getId()) { - case "id0": - assertEquals(1, result.getMerged()); - break; - case "id1": - case "id2": - case "id3": - assertEquals(2, result.getMerged()); - break; - case "id4": - assertEquals(1, result.getMerged()); - break; - default: - throw new RuntimeException(); - } - }); - } + results + .forEach( + result -> { + switch (result.getId()) { + case "id0": + assertEquals(1, result.getMerged()); + break; + case "id1": + case "id2": + case "id3": + assertEquals(2, result.getMerged()); + break; + case "id4": + assertEquals(1, result.getMerged()); + break; + default: + throw new RuntimeException(); + } + }); + } - @Test - public void shouldRunProperlyWhenActionPayloadTypeIsSuperTypeOfTableType() { - // given - String id0 = "id0"; - String id1 = "id1"; - String id2 = "id2"; - String id3 = "id3"; - String id4 = "id4"; - List rowData = - Arrays.asList( - createOafImplSubSub(id0), - createOafImplSubSub(id1), - createOafImplSubSub(id2), - createOafImplSubSub(id3)); - Dataset rowDS = - spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); + @Test + public void shouldRunProperlyWhenActionPayloadTypeIsSuperTypeOfTableType() { + // given + String id0 = "id0"; + String id1 = "id1"; + String id2 = "id2"; + String id3 = "id3"; + String id4 = "id4"; + List rowData = Arrays + .asList( + createOafImplSubSub(id0), + createOafImplSubSub(id1), + createOafImplSubSub(id2), + createOafImplSubSub(id3)); + Dataset rowDS = spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); - List actionPayloadData = - Arrays.asList( - createOafImplSub(id1), - createOafImplSub(id2), - createOafImplSub(id2), - createOafImplSub(id3), - createOafImplSub(id3), - createOafImplSub(id3), - createOafImplSub(id4), - createOafImplSub(id4), - createOafImplSub(id4), - createOafImplSub(id4)); - Dataset actionPayloadDS = - spark.createDataset(actionPayloadData, Encoders.bean(OafImplSub.class)); + List actionPayloadData = Arrays + .asList( + createOafImplSub(id1), + createOafImplSub(id2), + createOafImplSub(id2), + createOafImplSub(id3), + createOafImplSub(id3), + createOafImplSub(id3), + createOafImplSub(id4), + createOafImplSub(id4), + createOafImplSub(id4), + createOafImplSub(id4)); + Dataset actionPayloadDS = spark + .createDataset(actionPayloadData, Encoders.bean(OafImplSub.class)); - SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; - SerializableSupplier> actionPayloadIdFn = - () -> OafImplRoot::getId; - SerializableSupplier> mergeAndGetFn = - () -> - (x, y) -> { - x.merge(y); - return x; - }; + SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; + SerializableSupplier> actionPayloadIdFn = () -> OafImplRoot::getId; + SerializableSupplier> mergeAndGetFn = () -> (x, y) -> { + x.merge(y); + return x; + }; - // when - List results = - PromoteActionPayloadFunctions.joinGraphTableWithActionPayloadAndMerge( - rowDS, - actionPayloadDS, - rowIdFn, - actionPayloadIdFn, - mergeAndGetFn, - OafImplSubSub.class, - OafImplSub.class) - .collectAsList(); + // when + List results = PromoteActionPayloadFunctions + .joinGraphTableWithActionPayloadAndMerge( + rowDS, + actionPayloadDS, + rowIdFn, + actionPayloadIdFn, + mergeAndGetFn, + OafImplSubSub.class, + OafImplSub.class) + .collectAsList(); - // then - assertEquals(7, results.size()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id0)).count()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); - assertEquals(2, results.stream().filter(x -> x.getId().equals(id2)).count()); - assertEquals(3, results.stream().filter(x -> x.getId().equals(id3)).count()); - assertEquals(0, results.stream().filter(x -> x.getId().equals(id4)).count()); + // then + assertEquals(7, results.size()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id0)).count()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); + assertEquals(2, results.stream().filter(x -> x.getId().equals(id2)).count()); + assertEquals(3, results.stream().filter(x -> x.getId().equals(id3)).count()); + assertEquals(0, results.stream().filter(x -> x.getId().equals(id4)).count()); - results.forEach( - result -> { - switch (result.getId()) { - case "id0": - assertEquals(1, result.getMerged()); - break; - case "id1": - case "id2": - case "id3": - assertEquals(2, result.getMerged()); - break; - default: - throw new RuntimeException(); - } - }); - } - } + results + .forEach( + result -> { + switch (result.getId()) { + case "id0": + assertEquals(1, result.getMerged()); + break; + case "id1": + case "id2": + case "id3": + assertEquals(2, result.getMerged()); + break; + default: + throw new RuntimeException(); + } + }); + } + } - @Nested - class GroupTableByIdAndMerge { + @Nested + class GroupTableByIdAndMerge { - @Test - public void shouldRunProperly() { - // given - String id1 = "id1"; - String id2 = "id2"; - String id3 = "id3"; - List rowData = - Arrays.asList( - createOafImplSubSub(id1), - createOafImplSubSub(id2), - createOafImplSubSub(id2), - createOafImplSubSub(id3), - createOafImplSubSub(id3), - createOafImplSubSub(id3)); - Dataset rowDS = - spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); + @Test + public void shouldRunProperly() { + // given + String id1 = "id1"; + String id2 = "id2"; + String id3 = "id3"; + List rowData = Arrays + .asList( + createOafImplSubSub(id1), + createOafImplSubSub(id2), + createOafImplSubSub(id2), + createOafImplSubSub(id3), + createOafImplSubSub(id3), + createOafImplSubSub(id3)); + Dataset rowDS = spark.createDataset(rowData, Encoders.bean(OafImplSubSub.class)); - SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; - SerializableSupplier> mergeAndGetFn = - () -> - (x, y) -> { - x.merge(y); - return x; - }; - SerializableSupplier zeroFn = OafImplSubSub::new; - SerializableSupplier> isNotZeroFn = - () -> x -> Objects.nonNull(x.getId()); + SerializableSupplier> rowIdFn = () -> OafImplRoot::getId; + SerializableSupplier> mergeAndGetFn = () -> (x, + y) -> { + x.merge(y); + return x; + }; + SerializableSupplier zeroFn = OafImplSubSub::new; + SerializableSupplier> isNotZeroFn = () -> x -> Objects.nonNull(x.getId()); - // when - List results = - PromoteActionPayloadFunctions.groupGraphTableByIdAndMerge( - rowDS, rowIdFn, mergeAndGetFn, zeroFn, isNotZeroFn, OafImplSubSub.class) - .collectAsList(); + // when + List results = PromoteActionPayloadFunctions + .groupGraphTableByIdAndMerge( + rowDS, rowIdFn, mergeAndGetFn, zeroFn, isNotZeroFn, OafImplSubSub.class) + .collectAsList(); - // then - assertEquals(3, results.size()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id2)).count()); - assertEquals(1, results.stream().filter(x -> x.getId().equals(id3)).count()); + // then + assertEquals(3, results.size()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id1)).count()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id2)).count()); + assertEquals(1, results.stream().filter(x -> x.getId().equals(id3)).count()); - results.forEach( - result -> { - switch (result.getId()) { - case "id1": - assertEquals(1, result.getMerged()); - break; - case "id2": - assertEquals(2, result.getMerged()); - break; - case "id3": - assertEquals(3, result.getMerged()); - break; - default: - throw new RuntimeException(); - } - }); - } - } + results + .forEach( + result -> { + switch (result.getId()) { + case "id1": + assertEquals(1, result.getMerged()); + break; + case "id2": + assertEquals(2, result.getMerged()); + break; + case "id3": + assertEquals(3, result.getMerged()); + break; + default: + throw new RuntimeException(); + } + }); + } + } - public static class OafImplRoot extends Oaf { - private String id; - private int merged = 1; + public static class OafImplRoot extends Oaf { + private String id; + private int merged = 1; - public void merge(OafImplRoot e) { - merged += e.merged; - } + public void merge(OafImplRoot e) { + merged += e.merged; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public int getMerged() { - return merged; - } + public int getMerged() { + return merged; + } - public void setMerged(int merged) { - this.merged = merged; - } - } + public void setMerged(int merged) { + this.merged = merged; + } + } - public static class OafImplSub extends OafImplRoot { + public static class OafImplSub extends OafImplRoot { - @Override - public void merge(OafImplRoot e) { - super.merge(e); - } - } + @Override + public void merge(OafImplRoot e) { + super.merge(e); + } + } - private static OafImplSub createOafImplSub(String id) { - OafImplSub x = new OafImplSub(); - x.setId(id); - return x; - } + private static OafImplSub createOafImplSub(String id) { + OafImplSub x = new OafImplSub(); + x.setId(id); + return x; + } - public static class OafImplSubSub extends OafImplSub { + public static class OafImplSubSub extends OafImplSub { - @Override - public void merge(OafImplRoot e) { - super.merge(e); - } - } + @Override + public void merge(OafImplRoot e) { + super.merge(e); + } + } - private static OafImplSubSub createOafImplSubSub(String id) { - OafImplSubSub x = new OafImplSubSub(); - x.setId(id); - return x; - } + private static OafImplSubSub createOafImplSubSub(String id) { + OafImplSubSub x = new OafImplSubSub(); + x.setId(id); + return x; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java index 526bff2e1..9811fb707 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/GenerateNativeStoreSparkJob.java @@ -1,17 +1,12 @@ + package eu.dnetlib.dhp.collection; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.model.mdstore.Provenance; -import eu.dnetlib.message.Message; -import eu.dnetlib.message.MessageManager; -import eu.dnetlib.message.MessageType; import java.io.ByteArrayInputStream; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; import java.util.Objects; + import org.apache.commons.cli.*; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; @@ -29,127 +24,138 @@ import org.dom4j.Document; import org.dom4j.Node; import org.dom4j.io.SAXReader; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.model.mdstore.MetadataRecord; +import eu.dnetlib.dhp.model.mdstore.Provenance; +import eu.dnetlib.message.Message; +import eu.dnetlib.message.MessageManager; +import eu.dnetlib.message.MessageType; + public class GenerateNativeStoreSparkJob { - public static MetadataRecord parseRecord( - final String input, - final String xpath, - final String encoding, - final Provenance provenance, - final Long dateOfCollection, - final LongAccumulator totalItems, - final LongAccumulator invalidRecords) { + public static MetadataRecord parseRecord( + final String input, + final String xpath, + final String encoding, + final Provenance provenance, + final Long dateOfCollection, + final LongAccumulator totalItems, + final LongAccumulator invalidRecords) { - if (totalItems != null) totalItems.add(1); - try { - SAXReader reader = new SAXReader(); - Document document = - reader.read(new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8))); - Node node = document.selectSingleNode(xpath); - final String originalIdentifier = node.getText(); - if (StringUtils.isBlank(originalIdentifier)) { - if (invalidRecords != null) invalidRecords.add(1); - return null; - } - return new MetadataRecord(originalIdentifier, encoding, provenance, input, dateOfCollection); - } catch (Throwable e) { - if (invalidRecords != null) invalidRecords.add(1); - e.printStackTrace(); - return null; - } - } + if (totalItems != null) + totalItems.add(1); + try { + SAXReader reader = new SAXReader(); + Document document = reader.read(new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8))); + Node node = document.selectSingleNode(xpath); + final String originalIdentifier = node.getText(); + if (StringUtils.isBlank(originalIdentifier)) { + if (invalidRecords != null) + invalidRecords.add(1); + return null; + } + return new MetadataRecord(originalIdentifier, encoding, provenance, input, dateOfCollection); + } catch (Throwable e) { + if (invalidRecords != null) + invalidRecords.add(1); + e.printStackTrace(); + return null; + } + } - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - GenerateNativeStoreSparkJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/collection/collection_input_parameters.json"))); - parser.parseArgument(args); - final ObjectMapper jsonMapper = new ObjectMapper(); - final Provenance provenance = jsonMapper.readValue(parser.get("provenance"), Provenance.class); - final long dateOfCollection = new Long(parser.get("dateOfCollection")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + GenerateNativeStoreSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/collection_input_parameters.json"))); + parser.parseArgument(args); + final ObjectMapper jsonMapper = new ObjectMapper(); + final Provenance provenance = jsonMapper.readValue(parser.get("provenance"), Provenance.class); + final long dateOfCollection = new Long(parser.get("dateOfCollection")); - final SparkSession spark = - SparkSession.builder() - .appName("GenerateNativeStoreSparkJob") - .master(parser.get("master")) - .getOrCreate(); + final SparkSession spark = SparkSession + .builder() + .appName("GenerateNativeStoreSparkJob") + .master(parser.get("master")) + .getOrCreate(); - final Map ongoingMap = new HashMap<>(); - final Map reportMap = new HashMap<>(); + final Map ongoingMap = new HashMap<>(); + final Map reportMap = new HashMap<>(); - final boolean test = - parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); + final boolean test = parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final JavaPairRDD inputRDD = - sc.sequenceFile(parser.get("input"), IntWritable.class, Text.class); + final JavaPairRDD inputRDD = sc + .sequenceFile(parser.get("input"), IntWritable.class, Text.class); - final LongAccumulator totalItems = sc.sc().longAccumulator("TotalItems"); + final LongAccumulator totalItems = sc.sc().longAccumulator("TotalItems"); - final LongAccumulator invalidRecords = sc.sc().longAccumulator("InvalidRecords"); + final LongAccumulator invalidRecords = sc.sc().longAccumulator("InvalidRecords"); - final MessageManager manager = - new MessageManager( - parser.get("rabbitHost"), - parser.get("rabbitUser"), - parser.get("rabbitPassword"), - false, - false, - null); + final MessageManager manager = new MessageManager( + parser.get("rabbitHost"), + parser.get("rabbitUser"), + parser.get("rabbitPassword"), + false, + false, + null); - final JavaRDD mappeRDD = - inputRDD - .map( - item -> - parseRecord( - item._2().toString(), - parser.get("xpath"), - parser.get("encoding"), - provenance, - dateOfCollection, - totalItems, - invalidRecords)) - .filter(Objects::nonNull) - .distinct(); + final JavaRDD mappeRDD = inputRDD + .map( + item -> parseRecord( + item._2().toString(), + parser.get("xpath"), + parser.get("encoding"), + provenance, + dateOfCollection, + totalItems, + invalidRecords)) + .filter(Objects::nonNull) + .distinct(); - ongoingMap.put("ongoing", "0"); - if (!test) { - manager.sendMessage( - new Message( - parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), - parser.get("rabbitOngoingQueue"), - true, - false); - } + ongoingMap.put("ongoing", "0"); + if (!test) { + manager + .sendMessage( + new Message( + parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), + parser.get("rabbitOngoingQueue"), + true, + false); + } - final Encoder encoder = Encoders.bean(MetadataRecord.class); - final Dataset mdstore = spark.createDataset(mappeRDD.rdd(), encoder); - final LongAccumulator mdStoreRecords = sc.sc().longAccumulator("MDStoreRecords"); - mdStoreRecords.add(mdstore.count()); - ongoingMap.put("ongoing", "" + totalItems.value()); - if (!test) { - manager.sendMessage( - new Message( - parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), - parser.get("rabbitOngoingQueue"), - true, - false); - } - mdstore.write().format("parquet").save(parser.get("output")); - reportMap.put("inputItem", "" + totalItems.value()); - reportMap.put("invalidRecords", "" + invalidRecords.value()); - reportMap.put("mdStoreSize", "" + mdStoreRecords.value()); - if (!test) { - manager.sendMessage( - new Message(parser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), - parser.get("rabbitReportQueue"), - true, - false); - manager.close(); - } - } + final Encoder encoder = Encoders.bean(MetadataRecord.class); + final Dataset mdstore = spark.createDataset(mappeRDD.rdd(), encoder); + final LongAccumulator mdStoreRecords = sc.sc().longAccumulator("MDStoreRecords"); + mdStoreRecords.add(mdstore.count()); + ongoingMap.put("ongoing", "" + totalItems.value()); + if (!test) { + manager + .sendMessage( + new Message( + parser.get("workflowId"), "DataFrameCreation", MessageType.ONGOING, ongoingMap), + parser.get("rabbitOngoingQueue"), + true, + false); + } + mdstore.write().format("parquet").save(parser.get("output")); + reportMap.put("inputItem", "" + totalItems.value()); + reportMap.put("invalidRecords", "" + invalidRecords.value()); + reportMap.put("mdStoreSize", "" + mdStoreRecords.value()); + if (!test) { + manager + .sendMessage( + new Message(parser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), + parser.get("rabbitReportQueue"), + true, + false); + manager.close(); + } + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index 82f28afe6..4a0c70c45 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -1,10 +1,12 @@ + package eu.dnetlib.dhp.collection.plugin; +import java.util.stream.Stream; + import eu.dnetlib.collector.worker.model.ApiDescriptor; import eu.dnetlib.dhp.collection.worker.DnetCollectorException; -import java.util.stream.Stream; public interface CollectorPlugin { - Stream collect(ApiDescriptor api) throws DnetCollectorException; + Stream collect(ApiDescriptor api) throws DnetCollectorException; } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java index 415102a1a..7f71f401d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiCollectorPlugin.java @@ -1,11 +1,6 @@ + package eu.dnetlib.dhp.collection.plugin.oai; -import com.google.common.base.Splitter; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import eu.dnetlib.collector.worker.model.ApiDescriptor; -import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -14,65 +9,74 @@ import java.util.Spliterators; import java.util.stream.Stream; import java.util.stream.StreamSupport; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; + +import eu.dnetlib.collector.worker.model.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.collection.worker.DnetCollectorException; + public class OaiCollectorPlugin implements CollectorPlugin { - private static final String FORMAT_PARAM = "format"; - private static final String OAI_SET_PARAM = "set"; - private static final Object OAI_FROM_DATE_PARAM = "fromDate"; - private static final Object OAI_UNTIL_DATE_PARAM = "untilDate"; + private static final String FORMAT_PARAM = "format"; + private static final String OAI_SET_PARAM = "set"; + private static final Object OAI_FROM_DATE_PARAM = "fromDate"; + private static final Object OAI_UNTIL_DATE_PARAM = "untilDate"; - private OaiIteratorFactory oaiIteratorFactory; + private OaiIteratorFactory oaiIteratorFactory; - @Override - public Stream collect(final ApiDescriptor api) throws DnetCollectorException { - final String baseUrl = api.getBaseUrl(); - final String mdFormat = api.getParams().get(FORMAT_PARAM); - final String setParam = api.getParams().get(OAI_SET_PARAM); - final String fromDate = api.getParams().get(OAI_FROM_DATE_PARAM); - final String untilDate = api.getParams().get(OAI_UNTIL_DATE_PARAM); + @Override + public Stream collect(final ApiDescriptor api) throws DnetCollectorException { + final String baseUrl = api.getBaseUrl(); + final String mdFormat = api.getParams().get(FORMAT_PARAM); + final String setParam = api.getParams().get(OAI_SET_PARAM); + final String fromDate = api.getParams().get(OAI_FROM_DATE_PARAM); + final String untilDate = api.getParams().get(OAI_UNTIL_DATE_PARAM); - final List sets = new ArrayList<>(); - if (setParam != null) { - sets.addAll( - Lists.newArrayList(Splitter.on(",").omitEmptyStrings().trimResults().split(setParam))); - } - if (sets.isEmpty()) { - // If no set is defined, ALL the sets must be harvested - sets.add(""); - } + final List sets = new ArrayList<>(); + if (setParam != null) { + sets + .addAll( + Lists.newArrayList(Splitter.on(",").omitEmptyStrings().trimResults().split(setParam))); + } + if (sets.isEmpty()) { + // If no set is defined, ALL the sets must be harvested + sets.add(""); + } - if (baseUrl == null || baseUrl.isEmpty()) { - throw new DnetCollectorException("Param 'baseurl' is null or empty"); - } + if (baseUrl == null || baseUrl.isEmpty()) { + throw new DnetCollectorException("Param 'baseurl' is null or empty"); + } - if (mdFormat == null || mdFormat.isEmpty()) { - throw new DnetCollectorException("Param 'mdFormat' is null or empty"); - } + if (mdFormat == null || mdFormat.isEmpty()) { + throw new DnetCollectorException("Param 'mdFormat' is null or empty"); + } - if (fromDate != null && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + fromDate); - } + if (fromDate != null && !fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { + throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + fromDate); + } - if (untilDate != null && !untilDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + untilDate); - } + if (untilDate != null && !untilDate.matches("\\d{4}-\\d{2}-\\d{2}")) { + throw new DnetCollectorException("Invalid date (YYYY-MM-DD): " + untilDate); + } - final Iterator> iters = - sets.stream() - .map( - set -> - getOaiIteratorFactory() - .newIterator(baseUrl, mdFormat, set, fromDate, untilDate)) - .iterator(); + final Iterator> iters = sets + .stream() + .map( + set -> getOaiIteratorFactory() + .newIterator(baseUrl, mdFormat, set, fromDate, untilDate)) + .iterator(); - return StreamSupport.stream( - Spliterators.spliteratorUnknownSize(Iterators.concat(iters), Spliterator.ORDERED), false); - } + return StreamSupport + .stream( + Spliterators.spliteratorUnknownSize(Iterators.concat(iters), Spliterator.ORDERED), false); + } - public OaiIteratorFactory getOaiIteratorFactory() { - if (oaiIteratorFactory == null) { - oaiIteratorFactory = new OaiIteratorFactory(); - } - return oaiIteratorFactory; - } + public OaiIteratorFactory getOaiIteratorFactory() { + if (oaiIteratorFactory == null) { + oaiIteratorFactory = new OaiIteratorFactory(); + } + return oaiIteratorFactory; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java index 13c40de06..d61f13fb5 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/oai/OaiIterator.java @@ -1,14 +1,13 @@ + package eu.dnetlib.dhp.collection.plugin.oai; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; -import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; -import eu.dnetlib.dhp.collection.worker.utils.XmlCleaner; import java.io.StringReader; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; import java.util.Iterator; import java.util.Queue; import java.util.concurrent.PriorityBlockingQueue; + import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -17,160 +16,162 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; +import eu.dnetlib.dhp.collection.worker.DnetCollectorException; +import eu.dnetlib.dhp.collection.worker.utils.HttpConnector; +import eu.dnetlib.dhp.collection.worker.utils.XmlCleaner; + public class OaiIterator implements Iterator { - private static final Log log = LogFactory.getLog(OaiIterator.class); // NOPMD by marko on - // 11/24/08 5:02 PM + private static final Log log = LogFactory.getLog(OaiIterator.class); // NOPMD by marko on + // 11/24/08 5:02 PM - private final Queue queue = new PriorityBlockingQueue<>(); - private final SAXReader reader = new SAXReader(); + private final Queue queue = new PriorityBlockingQueue<>(); + private final SAXReader reader = new SAXReader(); - private final String baseUrl; - private final String set; - private final String mdFormat; - private final String fromDate; - private final String untilDate; - private String token; - private boolean started; - private final HttpConnector httpConnector; + private final String baseUrl; + private final String set; + private final String mdFormat; + private final String fromDate; + private final String untilDate; + private String token; + private boolean started; + private final HttpConnector httpConnector; - public OaiIterator( - final String baseUrl, - final String mdFormat, - final String set, - final String fromDate, - final String untilDate, - final HttpConnector httpConnector) { - this.baseUrl = baseUrl; - this.mdFormat = mdFormat; - this.set = set; - this.fromDate = fromDate; - this.untilDate = untilDate; - this.started = false; - this.httpConnector = httpConnector; - } + public OaiIterator( + final String baseUrl, + final String mdFormat, + final String set, + final String fromDate, + final String untilDate, + final HttpConnector httpConnector) { + this.baseUrl = baseUrl; + this.mdFormat = mdFormat; + this.set = set; + this.fromDate = fromDate; + this.untilDate = untilDate; + this.started = false; + this.httpConnector = httpConnector; + } - private void verifyStarted() { - if (!this.started) { - this.started = true; - try { - this.token = firstPage(); - } catch (final DnetCollectorException e) { - throw new RuntimeException(e); - } - } - } + private void verifyStarted() { + if (!this.started) { + this.started = true; + try { + this.token = firstPage(); + } catch (final DnetCollectorException e) { + throw new RuntimeException(e); + } + } + } - @Override - public boolean hasNext() { - synchronized (queue) { - verifyStarted(); - return !queue.isEmpty(); - } - } + @Override + public boolean hasNext() { + synchronized (queue) { + verifyStarted(); + return !queue.isEmpty(); + } + } - @Override - public String next() { - synchronized (queue) { - verifyStarted(); - final String res = queue.poll(); - while (queue.isEmpty() && token != null && !token.isEmpty()) { - try { - token = otherPages(token); - } catch (final DnetCollectorException e) { - throw new RuntimeException(e); - } - } - return res; - } - } + @Override + public String next() { + synchronized (queue) { + verifyStarted(); + final String res = queue.poll(); + while (queue.isEmpty() && token != null && !token.isEmpty()) { + try { + token = otherPages(token); + } catch (final DnetCollectorException e) { + throw new RuntimeException(e); + } + } + return res; + } + } - @Override - public void remove() {} + @Override + public void remove() { + } - private String firstPage() throws DnetCollectorException { - try { - String url = - baseUrl + "?verb=ListRecords&metadataPrefix=" + URLEncoder.encode(mdFormat, "UTF-8"); - if (set != null && !set.isEmpty()) { - url += "&set=" + URLEncoder.encode(set, "UTF-8"); - } - if (fromDate != null && fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - url += "&from=" + URLEncoder.encode(fromDate, "UTF-8"); - } - if (untilDate != null && untilDate.matches("\\d{4}-\\d{2}-\\d{2}")) { - url += "&until=" + URLEncoder.encode(untilDate, "UTF-8"); - } - log.info("Start harvesting using url: " + url); + private String firstPage() throws DnetCollectorException { + try { + String url = baseUrl + "?verb=ListRecords&metadataPrefix=" + URLEncoder.encode(mdFormat, "UTF-8"); + if (set != null && !set.isEmpty()) { + url += "&set=" + URLEncoder.encode(set, "UTF-8"); + } + if (fromDate != null && fromDate.matches("\\d{4}-\\d{2}-\\d{2}")) { + url += "&from=" + URLEncoder.encode(fromDate, "UTF-8"); + } + if (untilDate != null && untilDate.matches("\\d{4}-\\d{2}-\\d{2}")) { + url += "&until=" + URLEncoder.encode(untilDate, "UTF-8"); + } + log.info("Start harvesting using url: " + url); - return downloadPage(url); - } catch (final UnsupportedEncodingException e) { - throw new DnetCollectorException(e); - } - } + return downloadPage(url); + } catch (final UnsupportedEncodingException e) { + throw new DnetCollectorException(e); + } + } - private String extractResumptionToken(final String xml) { + private String extractResumptionToken(final String xml) { - final String s = StringUtils.substringAfter(xml, "", "", " newIterator( - final String baseUrl, - final String mdFormat, - final String set, - final String fromDate, - final String untilDate) { - return new OaiIterator(baseUrl, mdFormat, set, fromDate, untilDate, getHttpConnector()); - } + public Iterator newIterator( + final String baseUrl, + final String mdFormat, + final String set, + final String fromDate, + final String untilDate) { + return new OaiIterator(baseUrl, mdFormat, set, fromDate, untilDate, getHttpConnector()); + } - private HttpConnector getHttpConnector() { - if (httpConnector == null) httpConnector = new HttpConnector(); - return httpConnector; - } + private HttpConnector getHttpConnector() { + if (httpConnector == null) + httpConnector = new HttpConnector(); + return httpConnector; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java index 320f735b3..f40962c21 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorException.java @@ -1,31 +1,32 @@ + package eu.dnetlib.dhp.collection.worker; public class DnetCollectorException extends Exception { - /** */ - private static final long serialVersionUID = -290723075076039757L; + /** */ + private static final long serialVersionUID = -290723075076039757L; - public DnetCollectorException() { - super(); - } + public DnetCollectorException() { + super(); + } - public DnetCollectorException( - final String message, - final Throwable cause, - final boolean enableSuppression, - final boolean writableStackTrace) { - super(message, cause, enableSuppression, writableStackTrace); - } + public DnetCollectorException( + final String message, + final Throwable cause, + final boolean enableSuppression, + final boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } - public DnetCollectorException(final String message, final Throwable cause) { - super(message, cause); - } + public DnetCollectorException(final String message, final Throwable cause) { + super(message, cause); + } - public DnetCollectorException(final String message) { - super(message); - } + public DnetCollectorException(final String message) { + super(message); + } - public DnetCollectorException(final Throwable cause) { - super(cause); - } + public DnetCollectorException(final Throwable cause) { + super(cause); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java index d76ec8e37..e686ad518 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorker.java @@ -1,18 +1,12 @@ + package eu.dnetlib.dhp.collection.worker; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.collector.worker.model.ApiDescriptor; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; -import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; -import eu.dnetlib.message.Message; -import eu.dnetlib.message.MessageManager; -import eu.dnetlib.message.MessageType; import java.io.IOException; import java.net.URI; import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -22,111 +16,124 @@ import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.collector.worker.model.ApiDescriptor; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; +import eu.dnetlib.message.Message; +import eu.dnetlib.message.MessageManager; +import eu.dnetlib.message.MessageType; + public class DnetCollectorWorker { - private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorker.class); + private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorker.class); - private final CollectorPluginFactory collectorPluginFactory; + private final CollectorPluginFactory collectorPluginFactory; - private final ArgumentApplicationParser argumentParser; + private final ArgumentApplicationParser argumentParser; - private final MessageManager manager; + private final MessageManager manager; - public DnetCollectorWorker( - final CollectorPluginFactory collectorPluginFactory, - final ArgumentApplicationParser argumentParser, - final MessageManager manager) - throws DnetCollectorException { - this.collectorPluginFactory = collectorPluginFactory; - this.argumentParser = argumentParser; - this.manager = manager; - } + public DnetCollectorWorker( + final CollectorPluginFactory collectorPluginFactory, + final ArgumentApplicationParser argumentParser, + final MessageManager manager) + throws DnetCollectorException { + this.collectorPluginFactory = collectorPluginFactory; + this.argumentParser = argumentParser; + this.manager = manager; + } - public void collect() throws DnetCollectorException { - try { - final ObjectMapper jsonMapper = new ObjectMapper(); - final ApiDescriptor api = - jsonMapper.readValue(argumentParser.get("apidescriptor"), ApiDescriptor.class); + public void collect() throws DnetCollectorException { + try { + final ObjectMapper jsonMapper = new ObjectMapper(); + final ApiDescriptor api = jsonMapper.readValue(argumentParser.get("apidescriptor"), ApiDescriptor.class); - final CollectorPlugin plugin = collectorPluginFactory.getPluginByProtocol(api.getProtocol()); + final CollectorPlugin plugin = collectorPluginFactory.getPluginByProtocol(api.getProtocol()); - final String hdfsuri = argumentParser.get("namenode"); + final String hdfsuri = argumentParser.get("namenode"); - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - System.setProperty("HADOOP_USER_NAME", argumentParser.get("userHDFS")); - System.setProperty("hadoop.home.dir", "/"); - // Get the filesystem - HDFS - FileSystem.get(URI.create(hdfsuri), conf); - Path hdfswritepath = new Path(argumentParser.get("hdfsPath")); + System.setProperty("HADOOP_USER_NAME", argumentParser.get("userHDFS")); + System.setProperty("hadoop.home.dir", "/"); + // Get the filesystem - HDFS + FileSystem.get(URI.create(hdfsuri), conf); + Path hdfswritepath = new Path(argumentParser.get("hdfsPath")); - log.info("Created path " + hdfswritepath.toString()); + log.info("Created path " + hdfswritepath.toString()); - final Map ongoingMap = new HashMap<>(); - final Map reportMap = new HashMap<>(); - final AtomicInteger counter = new AtomicInteger(0); - try (SequenceFile.Writer writer = - SequenceFile.createWriter( - conf, - SequenceFile.Writer.file(hdfswritepath), - SequenceFile.Writer.keyClass(IntWritable.class), - SequenceFile.Writer.valueClass(Text.class))) { - final IntWritable key = new IntWritable(counter.get()); - final Text value = new Text(); - plugin - .collect(api) - .forEach( - content -> { - key.set(counter.getAndIncrement()); - value.set(content); - if (counter.get() % 10 == 0) { - try { - ongoingMap.put("ongoing", "" + counter.get()); - log.debug( - "Sending message: " - + manager.sendMessage( - new Message( - argumentParser.get("workflowId"), - "Collection", - MessageType.ONGOING, - ongoingMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false)); - } catch (Exception e) { - log.error("Error on sending message ", e); - } - } - try { - writer.append(key, value); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - } - ongoingMap.put("ongoing", "" + counter.get()); - manager.sendMessage( - new Message( - argumentParser.get("workflowId"), "Collection", MessageType.ONGOING, ongoingMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false); - reportMap.put("collected", "" + counter.get()); - manager.sendMessage( - new Message( - argumentParser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), - argumentParser.get("rabbitOngoingQueue"), - true, - false); - manager.close(); - } catch (Throwable e) { - throw new DnetCollectorException("Error on collecting ", e); - } - } + final Map ongoingMap = new HashMap<>(); + final Map reportMap = new HashMap<>(); + final AtomicInteger counter = new AtomicInteger(0); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(hdfswritepath), + SequenceFile.Writer.keyClass(IntWritable.class), + SequenceFile.Writer.valueClass(Text.class))) { + final IntWritable key = new IntWritable(counter.get()); + final Text value = new Text(); + plugin + .collect(api) + .forEach( + content -> { + key.set(counter.getAndIncrement()); + value.set(content); + if (counter.get() % 10 == 0) { + try { + ongoingMap.put("ongoing", "" + counter.get()); + log + .debug( + "Sending message: " + + manager + .sendMessage( + new Message( + argumentParser.get("workflowId"), + "Collection", + MessageType.ONGOING, + ongoingMap), + argumentParser.get("rabbitOngoingQueue"), + true, + false)); + } catch (Exception e) { + log.error("Error on sending message ", e); + } + } + try { + writer.append(key, value); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + ongoingMap.put("ongoing", "" + counter.get()); + manager + .sendMessage( + new Message( + argumentParser.get("workflowId"), "Collection", MessageType.ONGOING, ongoingMap), + argumentParser.get("rabbitOngoingQueue"), + true, + false); + reportMap.put("collected", "" + counter.get()); + manager + .sendMessage( + new Message( + argumentParser.get("workflowId"), "Collection", MessageType.REPORT, reportMap), + argumentParser.get("rabbitOngoingQueue"), + true, + false); + manager.close(); + } catch (Throwable e) { + throw new DnetCollectorException("Error on collecting ", e); + } + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java index 7ff61d677..cda07d151 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/DnetCollectorWorkerApplication.java @@ -1,48 +1,49 @@ + package eu.dnetlib.dhp.collection.worker; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; -import eu.dnetlib.message.MessageManager; import org.apache.commons.io.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; +import eu.dnetlib.message.MessageManager; + /** - * DnetCollectortWorkerApplication is the main class responsible to start the Dnet Collection into - * HDFS. This module will be executed on the hadoop cluster and taking in input some parameters that - * tells it which is the right collector plugin to use and where store the data into HDFS path + * DnetCollectortWorkerApplication is the main class responsible to start the Dnet Collection into HDFS. This module + * will be executed on the hadoop cluster and taking in input some parameters that tells it which is the right collector + * plugin to use and where store the data into HDFS path * * @author Sandro La Bruzzo */ public class DnetCollectorWorkerApplication { - private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorkerApplication.class); + private static final Logger log = LoggerFactory.getLogger(DnetCollectorWorkerApplication.class); - private static CollectorPluginFactory collectorPluginFactory = new CollectorPluginFactory(); + private static CollectorPluginFactory collectorPluginFactory = new CollectorPluginFactory(); - private static ArgumentApplicationParser argumentParser; + private static ArgumentApplicationParser argumentParser; - /** @param args */ - public static void main(final String[] args) throws Exception { + /** @param args */ + public static void main(final String[] args) throws Exception { - argumentParser = - new ArgumentApplicationParser( - IOUtils.toString( - DnetCollectorWorker.class.getResourceAsStream( - "/eu/dnetlib/collector/worker/collector_parameter.json"))); - argumentParser.parseArgument(args); - log.info("hdfsPath =" + argumentParser.get("hdfsPath")); - log.info("json = " + argumentParser.get("apidescriptor")); - final MessageManager manager = - new MessageManager( - argumentParser.get("rabbitHost"), - argumentParser.get("rabbitUser"), - argumentParser.get("rabbitPassword"), - false, - false, - null); - final DnetCollectorWorker worker = - new DnetCollectorWorker(collectorPluginFactory, argumentParser, manager); - worker.collect(); - } + argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + DnetCollectorWorker.class + .getResourceAsStream( + "/eu/dnetlib/collector/worker/collector_parameter.json"))); + argumentParser.parseArgument(args); + log.info("hdfsPath =" + argumentParser.get("hdfsPath")); + log.info("json = " + argumentParser.get("apidescriptor")); + final MessageManager manager = new MessageManager( + argumentParser.get("rabbitHost"), + argumentParser.get("rabbitUser"), + argumentParser.get("rabbitPassword"), + false, + false, + null); + final DnetCollectorWorker worker = new DnetCollectorWorker(collectorPluginFactory, argumentParser, manager); + worker.collect(); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginErrorLogList.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginErrorLogList.java index 27d982796..6ee8a8b49 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginErrorLogList.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginErrorLogList.java @@ -1,18 +1,19 @@ + package eu.dnetlib.dhp.collection.worker.utils; import java.util.LinkedList; public class CollectorPluginErrorLogList extends LinkedList { - private static final long serialVersionUID = -6925786561303289704L; + private static final long serialVersionUID = -6925786561303289704L; - @Override - public String toString() { - String log = new String(); - int index = 0; - for (final String errorMessage : this) { - log += String.format("Retry #%s: %s / ", index++, errorMessage); - } - return log; - } + @Override + public String toString() { + String log = new String(); + int index = 0; + for (final String errorMessage : this) { + log += String.format("Retry #%s: %s / ", index++, errorMessage); + } + return log; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java index 857200119..7a0028e79 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/CollectorPluginFactory.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.worker.utils; import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; @@ -6,13 +7,14 @@ import eu.dnetlib.dhp.collection.worker.DnetCollectorException; public class CollectorPluginFactory { - public CollectorPlugin getPluginByProtocol(final String protocol) throws DnetCollectorException { - if (protocol == null) throw new DnetCollectorException("protocol cannot be null"); - switch (protocol.toLowerCase().trim()) { - case "oai": - return new OaiCollectorPlugin(); - default: - throw new DnetCollectorException("UNknown protocol"); - } - } + public CollectorPlugin getPluginByProtocol(final String protocol) throws DnetCollectorException { + if (protocol == null) + throw new DnetCollectorException("protocol cannot be null"); + switch (protocol.toLowerCase().trim()) { + case "oai": + return new OaiCollectorPlugin(); + default: + throw new DnetCollectorException("UNknown protocol"); + } + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java index 36b08008a..5d6108fad 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/HttpConnector.java @@ -1,6 +1,6 @@ + package eu.dnetlib.dhp.collection.worker.utils; -import eu.dnetlib.dhp.collection.worker.DnetCollectorException; import java.io.IOException; import java.io.InputStream; import java.net.*; @@ -8,226 +8,237 @@ import java.security.GeneralSecurityException; import java.security.cert.X509Certificate; import java.util.List; import java.util.Map; + import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLContext; import javax.net.ssl.TrustManager; import javax.net.ssl.X509TrustManager; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang.math.NumberUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.dnetlib.dhp.collection.worker.DnetCollectorException; + public class HttpConnector { - private static final Log log = LogFactory.getLog(HttpConnector.class); + private static final Log log = LogFactory.getLog(HttpConnector.class); - private int maxNumberOfRetry = 6; - private int defaultDelay = 120; // seconds - private int readTimeOut = 120; // seconds + private int maxNumberOfRetry = 6; + private int defaultDelay = 120; // seconds + private int readTimeOut = 120; // seconds - private String responseType = null; + private String responseType = null; - private final String userAgent = "Mozilla/5.0 (compatible; OAI; +http://www.openaire.eu)"; + private final String userAgent = "Mozilla/5.0 (compatible; OAI; +http://www.openaire.eu)"; - public HttpConnector() { - CookieHandler.setDefault(new CookieManager(null, CookiePolicy.ACCEPT_ALL)); - } + public HttpConnector() { + CookieHandler.setDefault(new CookieManager(null, CookiePolicy.ACCEPT_ALL)); + } - /** - * Given the URL returns the content via HTTP GET - * - * @param requestUrl the URL - * @return the content of the downloaded resource - * @throws DnetCollectorException when retrying more than maxNumberOfRetry times - */ - public String getInputSource(final String requestUrl) throws DnetCollectorException { - return attemptDownlaodAsString(requestUrl, 1, new CollectorPluginErrorLogList()); - } + /** + * Given the URL returns the content via HTTP GET + * + * @param requestUrl the URL + * @return the content of the downloaded resource + * @throws DnetCollectorException when retrying more than maxNumberOfRetry times + */ + public String getInputSource(final String requestUrl) throws DnetCollectorException { + return attemptDownlaodAsString(requestUrl, 1, new CollectorPluginErrorLogList()); + } - /** - * Given the URL returns the content as a stream via HTTP GET - * - * @param requestUrl the URL - * @return the content of the downloaded resource as InputStream - * @throws DnetCollectorException when retrying more than maxNumberOfRetry times - */ - public InputStream getInputSourceAsStream(final String requestUrl) throws DnetCollectorException { - return attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); - } + /** + * Given the URL returns the content as a stream via HTTP GET + * + * @param requestUrl the URL + * @return the content of the downloaded resource as InputStream + * @throws DnetCollectorException when retrying more than maxNumberOfRetry times + */ + public InputStream getInputSourceAsStream(final String requestUrl) throws DnetCollectorException { + return attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); + } - private String attemptDownlaodAsString( - final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) - throws DnetCollectorException { - try { - final InputStream s = attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); - try { - return IOUtils.toString(s); - } catch (final IOException e) { - log.error("error while retrieving from http-connection occured: " + requestUrl, e); - Thread.sleep(defaultDelay * 1000); - errorList.add(e.getMessage()); - return attemptDownlaodAsString(requestUrl, retryNumber + 1, errorList); - } finally { - IOUtils.closeQuietly(s); - } - } catch (final InterruptedException e) { - throw new DnetCollectorException(e); - } - } + private String attemptDownlaodAsString( + final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) + throws DnetCollectorException { + try { + final InputStream s = attemptDownload(requestUrl, 1, new CollectorPluginErrorLogList()); + try { + return IOUtils.toString(s); + } catch (final IOException e) { + log.error("error while retrieving from http-connection occured: " + requestUrl, e); + Thread.sleep(defaultDelay * 1000); + errorList.add(e.getMessage()); + return attemptDownlaodAsString(requestUrl, retryNumber + 1, errorList); + } finally { + IOUtils.closeQuietly(s); + } + } catch (final InterruptedException e) { + throw new DnetCollectorException(e); + } + } - private InputStream attemptDownload( - final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) - throws DnetCollectorException { + private InputStream attemptDownload( + final String requestUrl, final int retryNumber, final CollectorPluginErrorLogList errorList) + throws DnetCollectorException { - if (retryNumber > maxNumberOfRetry) { - throw new DnetCollectorException("Max number of retries exceeded. Cause: \n " + errorList); - } + if (retryNumber > maxNumberOfRetry) { + throw new DnetCollectorException("Max number of retries exceeded. Cause: \n " + errorList); + } - log.debug("Downloading " + requestUrl + " - try: " + retryNumber); - try { - InputStream input = null; + log.debug("Downloading " + requestUrl + " - try: " + retryNumber); + try { + InputStream input = null; - try { - final HttpURLConnection urlConn = (HttpURLConnection) new URL(requestUrl).openConnection(); - urlConn.setInstanceFollowRedirects(false); - urlConn.setReadTimeout(readTimeOut * 1000); - urlConn.addRequestProperty("User-Agent", userAgent); + try { + final HttpURLConnection urlConn = (HttpURLConnection) new URL(requestUrl).openConnection(); + urlConn.setInstanceFollowRedirects(false); + urlConn.setReadTimeout(readTimeOut * 1000); + urlConn.addRequestProperty("User-Agent", userAgent); - if (log.isDebugEnabled()) { - logHeaderFields(urlConn); - } + if (log.isDebugEnabled()) { + logHeaderFields(urlConn); + } - final int retryAfter = obtainRetryAfter(urlConn.getHeaderFields()); - if (retryAfter > 0 && urlConn.getResponseCode() == HttpURLConnection.HTTP_UNAVAILABLE) { - log.warn("waiting and repeating request after " + retryAfter + " sec."); - Thread.sleep(retryAfter * 1000); - errorList.add("503 Service Unavailable"); - urlConn.disconnect(); - return attemptDownload(requestUrl, retryNumber + 1, errorList); - } else if (urlConn.getResponseCode() == HttpURLConnection.HTTP_MOVED_PERM - || urlConn.getResponseCode() == HttpURLConnection.HTTP_MOVED_TEMP) { - final String newUrl = obtainNewLocation(urlConn.getHeaderFields()); - log.debug("The requested url has been moved to " + newUrl); - errorList.add( - String.format( - "%s %s. Moved to: %s", - urlConn.getResponseCode(), urlConn.getResponseMessage(), newUrl)); - urlConn.disconnect(); - return attemptDownload(newUrl, retryNumber + 1, errorList); - } else if (urlConn.getResponseCode() != HttpURLConnection.HTTP_OK) { - log.error( - String.format( - "HTTP error: %s %s", urlConn.getResponseCode(), urlConn.getResponseMessage())); - Thread.sleep(defaultDelay * 1000); - errorList.add( - String.format("%s %s", urlConn.getResponseCode(), urlConn.getResponseMessage())); - urlConn.disconnect(); - return attemptDownload(requestUrl, retryNumber + 1, errorList); - } else { - input = urlConn.getInputStream(); - responseType = urlConn.getContentType(); - return input; - } - } catch (final IOException e) { - log.error("error while retrieving from http-connection occured: " + requestUrl, e); - Thread.sleep(defaultDelay * 1000); - errorList.add(e.getMessage()); - return attemptDownload(requestUrl, retryNumber + 1, errorList); - } - } catch (final InterruptedException e) { - throw new DnetCollectorException(e); - } - } + final int retryAfter = obtainRetryAfter(urlConn.getHeaderFields()); + if (retryAfter > 0 && urlConn.getResponseCode() == HttpURLConnection.HTTP_UNAVAILABLE) { + log.warn("waiting and repeating request after " + retryAfter + " sec."); + Thread.sleep(retryAfter * 1000); + errorList.add("503 Service Unavailable"); + urlConn.disconnect(); + return attemptDownload(requestUrl, retryNumber + 1, errorList); + } else if (urlConn.getResponseCode() == HttpURLConnection.HTTP_MOVED_PERM + || urlConn.getResponseCode() == HttpURLConnection.HTTP_MOVED_TEMP) { + final String newUrl = obtainNewLocation(urlConn.getHeaderFields()); + log.debug("The requested url has been moved to " + newUrl); + errorList + .add( + String + .format( + "%s %s. Moved to: %s", + urlConn.getResponseCode(), urlConn.getResponseMessage(), newUrl)); + urlConn.disconnect(); + return attemptDownload(newUrl, retryNumber + 1, errorList); + } else if (urlConn.getResponseCode() != HttpURLConnection.HTTP_OK) { + log + .error( + String + .format( + "HTTP error: %s %s", urlConn.getResponseCode(), urlConn.getResponseMessage())); + Thread.sleep(defaultDelay * 1000); + errorList + .add( + String.format("%s %s", urlConn.getResponseCode(), urlConn.getResponseMessage())); + urlConn.disconnect(); + return attemptDownload(requestUrl, retryNumber + 1, errorList); + } else { + input = urlConn.getInputStream(); + responseType = urlConn.getContentType(); + return input; + } + } catch (final IOException e) { + log.error("error while retrieving from http-connection occured: " + requestUrl, e); + Thread.sleep(defaultDelay * 1000); + errorList.add(e.getMessage()); + return attemptDownload(requestUrl, retryNumber + 1, errorList); + } + } catch (final InterruptedException e) { + throw new DnetCollectorException(e); + } + } - private void logHeaderFields(final HttpURLConnection urlConn) throws IOException { - log.debug("StatusCode: " + urlConn.getResponseMessage()); + private void logHeaderFields(final HttpURLConnection urlConn) throws IOException { + log.debug("StatusCode: " + urlConn.getResponseMessage()); - for (final Map.Entry> e : urlConn.getHeaderFields().entrySet()) { - if (e.getKey() != null) { - for (final String v : e.getValue()) { - log.debug(" key: " + e.getKey() + " - value: " + v); - } - } - } - } + for (final Map.Entry> e : urlConn.getHeaderFields().entrySet()) { + if (e.getKey() != null) { + for (final String v : e.getValue()) { + log.debug(" key: " + e.getKey() + " - value: " + v); + } + } + } + } - private int obtainRetryAfter(final Map> headerMap) { - for (final String key : headerMap.keySet()) { - if (key != null - && key.toLowerCase().equals("retry-after") - && headerMap.get(key).size() > 0 - && NumberUtils.isNumber(headerMap.get(key).get(0))) { - return Integer.parseInt(headerMap.get(key).get(0)) + 10; - } - } - return -1; - } + private int obtainRetryAfter(final Map> headerMap) { + for (final String key : headerMap.keySet()) { + if (key != null + && key.toLowerCase().equals("retry-after") + && headerMap.get(key).size() > 0 + && NumberUtils.isNumber(headerMap.get(key).get(0))) { + return Integer.parseInt(headerMap.get(key).get(0)) + 10; + } + } + return -1; + } - private String obtainNewLocation(final Map> headerMap) - throws DnetCollectorException { - for (final String key : headerMap.keySet()) { - if (key != null && key.toLowerCase().equals("location") && headerMap.get(key).size() > 0) { - return headerMap.get(key).get(0); - } - } - throw new DnetCollectorException( - "The requested url has been MOVED, but 'location' param is MISSING"); - } + private String obtainNewLocation(final Map> headerMap) + throws DnetCollectorException { + for (final String key : headerMap.keySet()) { + if (key != null && key.toLowerCase().equals("location") && headerMap.get(key).size() > 0) { + return headerMap.get(key).get(0); + } + } + throw new DnetCollectorException( + "The requested url has been MOVED, but 'location' param is MISSING"); + } - /** - * register for https scheme; this is a workaround and not intended for the use in trusted - * environments - */ - public void initTrustManager() { - final X509TrustManager tm = - new X509TrustManager() { + /** + * register for https scheme; this is a workaround and not intended for the use in trusted environments + */ + public void initTrustManager() { + final X509TrustManager tm = new X509TrustManager() { - @Override - public void checkClientTrusted(final X509Certificate[] xcs, final String string) {} + @Override + public void checkClientTrusted(final X509Certificate[] xcs, final String string) { + } - @Override - public void checkServerTrusted(final X509Certificate[] xcs, final String string) {} + @Override + public void checkServerTrusted(final X509Certificate[] xcs, final String string) { + } - @Override - public X509Certificate[] getAcceptedIssuers() { - return null; - } - }; - try { - final SSLContext ctx = SSLContext.getInstance("TLS"); - ctx.init(null, new TrustManager[] {tm}, null); - HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory()); - } catch (final GeneralSecurityException e) { - log.fatal(e); - throw new IllegalStateException(e); - } - } + @Override + public X509Certificate[] getAcceptedIssuers() { + return null; + } + }; + try { + final SSLContext ctx = SSLContext.getInstance("TLS"); + ctx.init(null, new TrustManager[] { + tm + }, null); + HttpsURLConnection.setDefaultSSLSocketFactory(ctx.getSocketFactory()); + } catch (final GeneralSecurityException e) { + log.fatal(e); + throw new IllegalStateException(e); + } + } - public int getMaxNumberOfRetry() { - return maxNumberOfRetry; - } + public int getMaxNumberOfRetry() { + return maxNumberOfRetry; + } - public void setMaxNumberOfRetry(final int maxNumberOfRetry) { - this.maxNumberOfRetry = maxNumberOfRetry; - } + public void setMaxNumberOfRetry(final int maxNumberOfRetry) { + this.maxNumberOfRetry = maxNumberOfRetry; + } - public int getDefaultDelay() { - return defaultDelay; - } + public int getDefaultDelay() { + return defaultDelay; + } - public void setDefaultDelay(final int defaultDelay) { - this.defaultDelay = defaultDelay; - } + public void setDefaultDelay(final int defaultDelay) { + this.defaultDelay = defaultDelay; + } - public int getReadTimeOut() { - return readTimeOut; - } + public int getReadTimeOut() { + return readTimeOut; + } - public void setReadTimeOut(final int readTimeOut) { - this.readTimeOut = readTimeOut; - } + public void setReadTimeOut(final int readTimeOut) { + this.readTimeOut = readTimeOut; + } - public String getResponseType() { - return responseType; - } + public String getResponseType() { + return responseType; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/XmlCleaner.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/XmlCleaner.java index 6a9afd591..32eeeab4b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/XmlCleaner.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/worker/utils/XmlCleaner.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.collection.worker.utils; import java.util.HashMap; @@ -9,376 +10,374 @@ import java.util.regex.Pattern; /** @author jochen, Andreas Czerniak */ public class XmlCleaner { - /** Pattern for numeric entities. */ - private static Pattern validCharacterEntityPattern = - Pattern.compile("^&#x?\\d{2,4};"); // $NON-NLS-1$ - // private static Pattern validCharacterEntityPattern = Pattern.compile("^&#?\\d{2,4};"); - // //$NON-NLS-1$ + /** Pattern for numeric entities. */ + private static Pattern validCharacterEntityPattern = Pattern.compile("^&#x?\\d{2,4};"); // $NON-NLS-1$ + // private static Pattern validCharacterEntityPattern = Pattern.compile("^&#?\\d{2,4};"); + // //$NON-NLS-1$ - // see https://www.w3.org/TR/REC-xml/#charsets , not only limited to - private static Pattern invalidControlCharPattern = Pattern.compile("&#x?1[0-9a-fA-F];"); + // see https://www.w3.org/TR/REC-xml/#charsets , not only limited to + private static Pattern invalidControlCharPattern = Pattern.compile("&#x?1[0-9a-fA-F];"); - /** - * Pattern that negates the allowable XML 4 byte unicode characters. Valid are: #x9 | #xA | #xD | - * [#x20-#xD7FF] | [#xE000-#xFFFD] | [#x10000-#x10FFFF] - */ - private static Pattern invalidCharacterPattern = - Pattern.compile("[^\t\r\n\u0020-\uD7FF\uE000-\uFFFD]"); // $NON-NLS-1$ + /** + * Pattern that negates the allowable XML 4 byte unicode characters. Valid are: #x9 | #xA | #xD | [#x20-#xD7FF] | + * [#xE000-#xFFFD] | [#x10000-#x10FFFF] + */ + private static Pattern invalidCharacterPattern = Pattern.compile("[^\t\r\n\u0020-\uD7FF\uE000-\uFFFD]"); // $NON-NLS-1$ - // Map entities to their unicode equivalent - private static Set goodEntities = new HashSet<>(); - private static Map badEntities = new HashMap<>(); + // Map entities to their unicode equivalent + private static Set goodEntities = new HashSet<>(); + private static Map badEntities = new HashMap<>(); - static { - // pre-defined XML entities - goodEntities.add("""); // $NON-NLS-1$ // quotation mark - goodEntities.add("&"); // $NON-NLS-1$ // ampersand - goodEntities.add("<"); // $NON-NLS-1$ // less-than sign - goodEntities.add(">"); // $NON-NLS-1$ // greater-than sign - // control entities - // badEntities.put(" ", ""); - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("€", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("‚", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("ƒ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("„", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("…", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("†", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("‡", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("ˆ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("‰", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("Š", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("‹", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("Œ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("Ž", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("‘", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("’", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("“", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("”", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("•", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("–", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("—", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("˜", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("™", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("š", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("›", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("œ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("ž", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - badEntities.put("Ÿ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character - // misc entities - badEntities.put("€", "\u20AC"); // $NON-NLS-1$ //$NON-NLS-2$ // euro - badEntities.put("‘", "\u2018"); // $NON-NLS-1$ //$NON-NLS-2$ // left single quotation - // mark - badEntities.put("’", "\u2019"); // $NON-NLS-1$ //$NON-NLS-2$ // right single quotation - // mark - // Latin 1 entities - badEntities.put(" ", "\u00A0"); // $NON-NLS-1$ //$NON-NLS-2$ // no-break space - badEntities.put("¡", "\u00A1"); // $NON-NLS-1$ //$NON-NLS-2$ // inverted exclamation - // mark - badEntities.put("¢", "\u00A2"); // $NON-NLS-1$ //$NON-NLS-2$ // cent sign - badEntities.put("£", "\u00A3"); // $NON-NLS-1$ //$NON-NLS-2$ // pound sign - badEntities.put("¤", "\u00A4"); // $NON-NLS-1$ //$NON-NLS-2$ // currency sign - badEntities.put("¥", "\u00A5"); // $NON-NLS-1$ //$NON-NLS-2$ // yen sign - badEntities.put("¦", "\u00A6"); // $NON-NLS-1$ //$NON-NLS-2$ // broken vertical bar - badEntities.put("§", "\u00A7"); // $NON-NLS-1$ //$NON-NLS-2$ // section sign - badEntities.put("¨", "\u00A8"); // $NON-NLS-1$ //$NON-NLS-2$ // diaeresis - badEntities.put("©", "\u00A9"); // $NON-NLS-1$ //$NON-NLS-2$ // copyright sign - badEntities.put("ª", "\u00AA"); // $NON-NLS-1$ //$NON-NLS-2$ // feminine ordinal - // indicator - badEntities.put("«", "\u00AB"); // $NON-NLS-2$ // left-pointing double angle - // quotation mark - badEntities.put("¬", "\u00AC"); // $NON-NLS-1$ //$NON-NLS-2$ // not sign - badEntities.put("­", "\u00AD"); // $NON-NLS-1$ //$NON-NLS-2$ // soft hyphen - badEntities.put("®", "\u00AE"); // $NON-NLS-1$ //$NON-NLS-2$ // registered sign - badEntities.put("¯", "\u00AF"); // $NON-NLS-1$ //$NON-NLS-2$ // macron - badEntities.put("°", "\u00B0"); // $NON-NLS-1$ //$NON-NLS-2$ // degree sign - badEntities.put("±", "\u00B1"); // $NON-NLS-1$ //$NON-NLS-2$ // plus-minus sign - badEntities.put("²", "\u00B2"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript two - badEntities.put("³", "\u00B3"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript three - badEntities.put("´", "\u00B4"); // $NON-NLS-1$ //$NON-NLS-2$ // acute accent - badEntities.put("µ", "\u00B5"); // $NON-NLS-1$ //$NON-NLS-2$ // micro sign - badEntities.put("¶", "\u00B6"); // $NON-NLS-1$ //$NON-NLS-2$ // pilcrow sign - badEntities.put("·", "\u00B7"); // $NON-NLS-1$ //$NON-NLS-2$ // middle dot - badEntities.put("¸", "\u00B8"); // $NON-NLS-1$ //$NON-NLS-2$ // cedilla - badEntities.put("¹", "\u00B9"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript one - badEntities.put("º", "\u00BA"); // $NON-NLS-1$ //$NON-NLS-2$ // masculine ordinal - // indicator - badEntities.put("»", "\u00BB"); // $NON-NLS-2$ // right-pointing double - // angle quotation - // mark - badEntities.put("¼", "\u00BC"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction one - // quarter - badEntities.put("½", "\u00BD"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction one - // half - badEntities.put("¾", "\u00BE"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction three - // quarters - badEntities.put("¿", "\u00BF"); // $NON-NLS-1$ //$NON-NLS-2$ // inverted question - // mark - badEntities.put("À", "\u00C0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // A - // with grave - badEntities.put("Á", "\u00C1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // A - // with acute - badEntities.put("Â", "\u00C2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A - // with circumflex - badEntities.put("Ã", "\u00C3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // A - // with tilde - badEntities.put("Ä", "\u00C4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A - // with - // diaeresis - badEntities.put("Å", "\u00C5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A - // with ring above - badEntities.put("Æ", "\u00C6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // AE - badEntities.put("Ç", "\u00C7"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // C - // with cedilla - badEntities.put("È", "\u00C8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // E - // with grave - badEntities.put("É", "\u00C9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // E - // with acute - badEntities.put("Ê", "\u00CA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter E - // with circumflex - badEntities.put("Ë", "\u00CB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter E - // with - // diaeresis - badEntities.put("Ì", "\u00CC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // I - // with grave - badEntities.put("Í", "\u00CD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // I - // with acute - badEntities.put("Î", "\u00CE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter I - // with circumflex - badEntities.put("Ï", "\u00CF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter I - // with - // diaeresis - badEntities.put("Ð", "\u00D0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter ETH - badEntities.put("Ñ", "\u00D1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // N - // with tilde - badEntities.put("Ò", "\u00D2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // O - // with grave - badEntities.put("Ó", "\u00D3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // O - // with acute - badEntities.put("Ô", "\u00D4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter O - // with circumflex - badEntities.put("Õ", "\u00D5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // O - // with tilde - badEntities.put("Ö", "\u00D6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter O - // with - // diaeresis - badEntities.put("×", "\u00D7"); // $NON-NLS-1$ //$NON-NLS-2$ // multiplication sign - badEntities.put("Ø", "\u00D8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // O - // with stroke - badEntities.put("Ù", "\u00D9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // U - // with grave - badEntities.put("Ú", "\u00DA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // U - // with acute - badEntities.put("Û", "\u00DB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter U - // with circumflex - badEntities.put("Ü", "\u00DC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter U - // with - // diaeresis - badEntities.put("Ý", "\u00DD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // Y - // with acute - badEntities.put("Þ", "\u00DE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter - // THORN - badEntities.put("ß", "\u00DF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter - // sharp s - badEntities.put("à", "\u00E0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // grave - badEntities.put("á", "\u00E1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // acute - badEntities.put("â", "\u00E2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // circumflex - badEntities.put("ã", "\u00E3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // tilde - badEntities.put("ä", "\u00E4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // diaeresis - badEntities.put("å", "\u00E5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a - // with - // ring above - badEntities.put("æ", "\u00E6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter ae - badEntities.put("ç", "\u00E7"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter c - // with - // cedilla - badEntities.put("è", "\u00E8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e - // with - // grave - badEntities.put("é", "\u00E9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e - // with - // acute - badEntities.put("ê", "\u00EA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e - // with - // circumflex - badEntities.put("ë", "\u00EB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e - // with - // diaeresis - badEntities.put("ì", "\u00EC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i - // with - // grave - badEntities.put("í", "\u00ED"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i - // with - // acute - badEntities.put("î", "\u00EE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i - // with - // circumflex - badEntities.put("ï", "\u00EF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i - // with - // diaeresis - badEntities.put("ð", "\u00F0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter eth - badEntities.put("ñ", "\u00F1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter n - // with - // tilde - badEntities.put("ò", "\u00F2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // grave - badEntities.put("ó", "\u00F3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // acute - badEntities.put("ô", "\u00F4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // circumflex - badEntities.put("õ", "\u00F5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // tilde - badEntities.put("ö", "\u00F6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // diaeresis - badEntities.put("÷", "\u00F7"); // $NON-NLS-1$ //$NON-NLS-2$ // division sign - badEntities.put("ø", "\u00F8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o - // with - // stroke - badEntities.put("ù", "\u00F9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u - // with - // grave - badEntities.put("ú", "\u00FA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u - // with - // acute - badEntities.put("û", "\u00FB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u - // with - // circumflex - badEntities.put("ü", "\u00FC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u - // with - // diaeresis - badEntities.put("ý", "\u00FD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter y - // with - // acute - badEntities.put("þ", "\u00FE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter - // thorn - badEntities.put("ÿ", "\u00FF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter y - // with - // diaeresis - } + static { + // pre-defined XML entities + goodEntities.add("""); // $NON-NLS-1$ // quotation mark + goodEntities.add("&"); // $NON-NLS-1$ // ampersand + goodEntities.add("<"); // $NON-NLS-1$ // less-than sign + goodEntities.add(">"); // $NON-NLS-1$ // greater-than sign + // control entities + // badEntities.put(" ", ""); + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("€", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("‚", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("ƒ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("„", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("…", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("†", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("‡", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("ˆ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("‰", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("Š", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("‹", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("Œ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("Ž", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("‘", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("’", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("“", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("”", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("•", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("–", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("—", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("˜", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("™", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("š", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("›", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("œ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("ž", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + badEntities.put("Ÿ", " "); // $NON-NLS-1$ //$NON-NLS-2$ // illegal HTML character + // misc entities + badEntities.put("€", "\u20AC"); // $NON-NLS-1$ //$NON-NLS-2$ // euro + badEntities.put("‘", "\u2018"); // $NON-NLS-1$ //$NON-NLS-2$ // left single quotation + // mark + badEntities.put("’", "\u2019"); // $NON-NLS-1$ //$NON-NLS-2$ // right single quotation + // mark + // Latin 1 entities + badEntities.put(" ", "\u00A0"); // $NON-NLS-1$ //$NON-NLS-2$ // no-break space + badEntities.put("¡", "\u00A1"); // $NON-NLS-1$ //$NON-NLS-2$ // inverted exclamation + // mark + badEntities.put("¢", "\u00A2"); // $NON-NLS-1$ //$NON-NLS-2$ // cent sign + badEntities.put("£", "\u00A3"); // $NON-NLS-1$ //$NON-NLS-2$ // pound sign + badEntities.put("¤", "\u00A4"); // $NON-NLS-1$ //$NON-NLS-2$ // currency sign + badEntities.put("¥", "\u00A5"); // $NON-NLS-1$ //$NON-NLS-2$ // yen sign + badEntities.put("¦", "\u00A6"); // $NON-NLS-1$ //$NON-NLS-2$ // broken vertical bar + badEntities.put("§", "\u00A7"); // $NON-NLS-1$ //$NON-NLS-2$ // section sign + badEntities.put("¨", "\u00A8"); // $NON-NLS-1$ //$NON-NLS-2$ // diaeresis + badEntities.put("©", "\u00A9"); // $NON-NLS-1$ //$NON-NLS-2$ // copyright sign + badEntities.put("ª", "\u00AA"); // $NON-NLS-1$ //$NON-NLS-2$ // feminine ordinal + // indicator + badEntities.put("«", "\u00AB"); // $NON-NLS-2$ // left-pointing double angle + // quotation mark + badEntities.put("¬", "\u00AC"); // $NON-NLS-1$ //$NON-NLS-2$ // not sign + badEntities.put("­", "\u00AD"); // $NON-NLS-1$ //$NON-NLS-2$ // soft hyphen + badEntities.put("®", "\u00AE"); // $NON-NLS-1$ //$NON-NLS-2$ // registered sign + badEntities.put("¯", "\u00AF"); // $NON-NLS-1$ //$NON-NLS-2$ // macron + badEntities.put("°", "\u00B0"); // $NON-NLS-1$ //$NON-NLS-2$ // degree sign + badEntities.put("±", "\u00B1"); // $NON-NLS-1$ //$NON-NLS-2$ // plus-minus sign + badEntities.put("²", "\u00B2"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript two + badEntities.put("³", "\u00B3"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript three + badEntities.put("´", "\u00B4"); // $NON-NLS-1$ //$NON-NLS-2$ // acute accent + badEntities.put("µ", "\u00B5"); // $NON-NLS-1$ //$NON-NLS-2$ // micro sign + badEntities.put("¶", "\u00B6"); // $NON-NLS-1$ //$NON-NLS-2$ // pilcrow sign + badEntities.put("·", "\u00B7"); // $NON-NLS-1$ //$NON-NLS-2$ // middle dot + badEntities.put("¸", "\u00B8"); // $NON-NLS-1$ //$NON-NLS-2$ // cedilla + badEntities.put("¹", "\u00B9"); // $NON-NLS-1$ //$NON-NLS-2$ // superscript one + badEntities.put("º", "\u00BA"); // $NON-NLS-1$ //$NON-NLS-2$ // masculine ordinal + // indicator + badEntities.put("»", "\u00BB"); // $NON-NLS-2$ // right-pointing double + // angle quotation + // mark + badEntities.put("¼", "\u00BC"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction one + // quarter + badEntities.put("½", "\u00BD"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction one + // half + badEntities.put("¾", "\u00BE"); // $NON-NLS-1$ //$NON-NLS-2$ // vulgar fraction three + // quarters + badEntities.put("¿", "\u00BF"); // $NON-NLS-1$ //$NON-NLS-2$ // inverted question + // mark + badEntities.put("À", "\u00C0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // A + // with grave + badEntities.put("Á", "\u00C1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // A + // with acute + badEntities.put("Â", "\u00C2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A + // with circumflex + badEntities.put("Ã", "\u00C3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // A + // with tilde + badEntities.put("Ä", "\u00C4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A + // with + // diaeresis + badEntities.put("Å", "\u00C5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter A + // with ring above + badEntities.put("Æ", "\u00C6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // AE + badEntities.put("Ç", "\u00C7"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // C + // with cedilla + badEntities.put("È", "\u00C8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // E + // with grave + badEntities.put("É", "\u00C9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // E + // with acute + badEntities.put("Ê", "\u00CA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter E + // with circumflex + badEntities.put("Ë", "\u00CB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter E + // with + // diaeresis + badEntities.put("Ì", "\u00CC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // I + // with grave + badEntities.put("Í", "\u00CD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // I + // with acute + badEntities.put("Î", "\u00CE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter I + // with circumflex + badEntities.put("Ï", "\u00CF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter I + // with + // diaeresis + badEntities.put("Ð", "\u00D0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter ETH + badEntities.put("Ñ", "\u00D1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // N + // with tilde + badEntities.put("Ò", "\u00D2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // O + // with grave + badEntities.put("Ó", "\u00D3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // O + // with acute + badEntities.put("Ô", "\u00D4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter O + // with circumflex + badEntities.put("Õ", "\u00D5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // O + // with tilde + badEntities.put("Ö", "\u00D6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter O + // with + // diaeresis + badEntities.put("×", "\u00D7"); // $NON-NLS-1$ //$NON-NLS-2$ // multiplication sign + badEntities.put("Ø", "\u00D8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // O + // with stroke + badEntities.put("Ù", "\u00D9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // U + // with grave + badEntities.put("Ú", "\u00DA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // U + // with acute + badEntities.put("Û", "\u00DB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter U + // with circumflex + badEntities.put("Ü", "\u00DC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter U + // with + // diaeresis + badEntities.put("Ý", "\u00DD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // Y + // with acute + badEntities.put("Þ", "\u00DE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin capital letter + // THORN + badEntities.put("ß", "\u00DF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter + // sharp s + badEntities.put("à", "\u00E0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // grave + badEntities.put("á", "\u00E1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // acute + badEntities.put("â", "\u00E2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // circumflex + badEntities.put("ã", "\u00E3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // tilde + badEntities.put("ä", "\u00E4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // diaeresis + badEntities.put("å", "\u00E5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter a + // with + // ring above + badEntities.put("æ", "\u00E6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter ae + badEntities.put("ç", "\u00E7"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter c + // with + // cedilla + badEntities.put("è", "\u00E8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e + // with + // grave + badEntities.put("é", "\u00E9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e + // with + // acute + badEntities.put("ê", "\u00EA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e + // with + // circumflex + badEntities.put("ë", "\u00EB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter e + // with + // diaeresis + badEntities.put("ì", "\u00EC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i + // with + // grave + badEntities.put("í", "\u00ED"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i + // with + // acute + badEntities.put("î", "\u00EE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i + // with + // circumflex + badEntities.put("ï", "\u00EF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter i + // with + // diaeresis + badEntities.put("ð", "\u00F0"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter eth + badEntities.put("ñ", "\u00F1"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter n + // with + // tilde + badEntities.put("ò", "\u00F2"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // grave + badEntities.put("ó", "\u00F3"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // acute + badEntities.put("ô", "\u00F4"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // circumflex + badEntities.put("õ", "\u00F5"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // tilde + badEntities.put("ö", "\u00F6"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // diaeresis + badEntities.put("÷", "\u00F7"); // $NON-NLS-1$ //$NON-NLS-2$ // division sign + badEntities.put("ø", "\u00F8"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter o + // with + // stroke + badEntities.put("ù", "\u00F9"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u + // with + // grave + badEntities.put("ú", "\u00FA"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u + // with + // acute + badEntities.put("û", "\u00FB"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u + // with + // circumflex + badEntities.put("ü", "\u00FC"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter u + // with + // diaeresis + badEntities.put("ý", "\u00FD"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter y + // with + // acute + badEntities.put("þ", "\u00FE"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter + // thorn + badEntities.put("ÿ", "\u00FF"); // $NON-NLS-1$ //$NON-NLS-2$ // latin small letter y + // with + // diaeresis + } - /** - * For each entity in the input that is not allowed in XML, replace the entity with its unicode - * equivalent or remove it. For each instance of a bare {@literal &}, replace it with {@literal - * &
} XML only allows 4 entities: {@literal &amp;}, {@literal &quot;}, {@literal - * &lt;} and {@literal &gt;}. - * - * @param broken the string to handle entities - * @return the string with entities appropriately fixed up - */ - public static String cleanAllEntities(final String broken) { - if (broken == null) { - return null; - } + /** + * For each entity in the input that is not allowed in XML, replace the entity with its unicode equivalent or remove + * it. For each instance of a bare {@literal &}, replace it with {@literal + * &
+ } XML only allows 4 entities: {@literal &amp;}, {@literal &quot;}, {@literal + * &lt;} and {@literal &gt;}. + * + * @param broken the string to handle entities + * @return the string with entities appropriately fixed up + */ + public static String cleanAllEntities(final String broken) { + if (broken == null) { + return null; + } - String working = invalidControlCharPattern.matcher(broken).replaceAll(""); - working = invalidCharacterPattern.matcher(working).replaceAll(""); + String working = invalidControlCharPattern.matcher(broken).replaceAll(""); + working = invalidCharacterPattern.matcher(working).replaceAll(""); - int cleanfrom = 0; + int cleanfrom = 0; - while (true) { - int amp = working.indexOf('&', cleanfrom); - // If there are no more amps then we are done - if (amp == -1) { - break; - } - // Skip references of the kind &#ddd; - if (validCharacterEntityPattern.matcher(working.substring(amp)).find()) { - cleanfrom = working.indexOf(';', amp) + 1; - continue; - } - int i = amp + 1; - while (true) { - // if we are at the end of the string then just escape the '&'; - if (i >= working.length()) { - return working.substring(0, amp) + "&" + working.substring(amp + 1); // $NON-NLS-1$ - } - // if we have come to a ; then we have an entity - // If it is something that xml can't handle then replace it. - final char c = working.charAt(i); - if (c == ';') { - final String entity = working.substring(amp, i + 1); - final String replace = handleEntity(entity); - working = working.substring(0, amp) + replace + working.substring(i + 1); - break; - } - // Did we end an entity without finding a closing ; - // Then treat it as an '&' that needs to be replaced with & - if (!Character.isLetterOrDigit(c)) { - working = working.substring(0, amp) + "&" + working.substring(amp + 1); // $NON-NLS-1$ - amp = i + 4; // account for the 4 extra characters - break; - } - i++; - } - cleanfrom = amp + 1; - } + while (true) { + int amp = working.indexOf('&', cleanfrom); + // If there are no more amps then we are done + if (amp == -1) { + break; + } + // Skip references of the kind &#ddd; + if (validCharacterEntityPattern.matcher(working.substring(amp)).find()) { + cleanfrom = working.indexOf(';', amp) + 1; + continue; + } + int i = amp + 1; + while (true) { + // if we are at the end of the string then just escape the '&'; + if (i >= working.length()) { + return working.substring(0, amp) + "&" + working.substring(amp + 1); // $NON-NLS-1$ + } + // if we have come to a ; then we have an entity + // If it is something that xml can't handle then replace it. + final char c = working.charAt(i); + if (c == ';') { + final String entity = working.substring(amp, i + 1); + final String replace = handleEntity(entity); + working = working.substring(0, amp) + replace + working.substring(i + 1); + break; + } + // Did we end an entity without finding a closing ; + // Then treat it as an '&' that needs to be replaced with & + if (!Character.isLetterOrDigit(c)) { + working = working.substring(0, amp) + "&" + working.substring(amp + 1); // $NON-NLS-1$ + amp = i + 4; // account for the 4 extra characters + break; + } + i++; + } + cleanfrom = amp + 1; + } - if (Pattern.compile("<<").matcher(working).find()) { - working = working.replaceAll("<<", "<<"); - } + if (Pattern.compile("<<").matcher(working).find()) { + working = working.replaceAll("<<", "<<"); + } - if (Pattern.compile(">>").matcher(working).find()) { - working = working.replaceAll(">>", ">>"); - } + if (Pattern.compile(">>").matcher(working).find()) { + working = working.replaceAll(">>", ">>"); + } - return working; - } + return working; + } - /** - * Replace entity with its unicode equivalent, if it is not a valid XML entity. Otherwise strip it - * out. XML only allows 4 entities: &amp;, &quot;, &lt; and &gt;. - * - * @param entity the entity to be replaced - * @return the substitution for the entity, either itself, the unicode equivalent or an empty - * string. - */ - private static String handleEntity(final String entity) { - if (goodEntities.contains(entity)) { - return entity; - } + /** + * Replace entity with its unicode equivalent, if it is not a valid XML entity. Otherwise strip it out. XML only + * allows 4 entities: &amp;, &quot;, &lt; and &gt;. + * + * @param entity the entity to be replaced + * @return the substitution for the entity, either itself, the unicode equivalent or an empty string. + */ + private static String handleEntity(final String entity) { + if (goodEntities.contains(entity)) { + return entity; + } - final String replace = badEntities.get(entity); - if (replace != null) { - return replace; - } + final String replace = badEntities.get(entity); + if (replace != null) { + return replace; + } - return replace != null ? replace : ""; - } + return replace != null ? replace : ""; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java index c568714de..f4bf78e18 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformFunction.java @@ -1,71 +1,74 @@ + package eu.dnetlib.dhp.transformation; +import java.io.ByteArrayInputStream; +import java.io.StringWriter; +import java.util.Map; + +import javax.xml.transform.stream.StreamSource; + +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.util.LongAccumulator; + import eu.dnetlib.dhp.model.mdstore.MetadataRecord; import eu.dnetlib.dhp.transformation.functions.Cleaner; import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; -import java.io.ByteArrayInputStream; -import java.io.StringWriter; -import java.util.Map; -import javax.xml.transform.stream.StreamSource; import net.sf.saxon.s9api.*; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.util.LongAccumulator; public class TransformFunction implements MapFunction { - private final LongAccumulator totalItems; - private final LongAccumulator errorItems; - private final LongAccumulator transformedItems; - private final String transformationRule; - private final Cleaner cleanFunction; + private final LongAccumulator totalItems; + private final LongAccumulator errorItems; + private final LongAccumulator transformedItems; + private final String transformationRule; + private final Cleaner cleanFunction; - private final long dateOfTransformation; + private final long dateOfTransformation; - public TransformFunction( - LongAccumulator totalItems, - LongAccumulator errorItems, - LongAccumulator transformedItems, - final String transformationRule, - long dateOfTransformation, - final Map vocabularies) - throws Exception { - this.totalItems = totalItems; - this.errorItems = errorItems; - this.transformedItems = transformedItems; - this.transformationRule = transformationRule; - this.dateOfTransformation = dateOfTransformation; - cleanFunction = new Cleaner(vocabularies); - } + public TransformFunction( + LongAccumulator totalItems, + LongAccumulator errorItems, + LongAccumulator transformedItems, + final String transformationRule, + long dateOfTransformation, + final Map vocabularies) + throws Exception { + this.totalItems = totalItems; + this.errorItems = errorItems; + this.transformedItems = transformedItems; + this.transformationRule = transformationRule; + this.dateOfTransformation = dateOfTransformation; + cleanFunction = new Cleaner(vocabularies); + } - @Override - public MetadataRecord call(MetadataRecord value) { - totalItems.add(1); - try { - Processor processor = new Processor(false); - processor.registerExtensionFunction(cleanFunction); - final XsltCompiler comp = processor.newXsltCompiler(); - XsltExecutable xslt = - comp.compile(new StreamSource(new ByteArrayInputStream(transformationRule.getBytes()))); - XdmNode source = - processor - .newDocumentBuilder() - .build(new StreamSource(new ByteArrayInputStream(value.getBody().getBytes()))); - XsltTransformer trans = xslt.load(); - trans.setInitialContextNode(source); - final StringWriter output = new StringWriter(); - Serializer out = processor.newSerializer(output); - out.setOutputProperty(Serializer.Property.METHOD, "xml"); - out.setOutputProperty(Serializer.Property.INDENT, "yes"); - trans.setDestination(out); - trans.transform(); - final String xml = output.toString(); - value.setBody(xml); - value.setDateOfTransformation(dateOfTransformation); - transformedItems.add(1); - return value; - } catch (Throwable e) { - errorItems.add(1); - return null; - } - } + @Override + public MetadataRecord call(MetadataRecord value) { + totalItems.add(1); + try { + Processor processor = new Processor(false); + processor.registerExtensionFunction(cleanFunction); + final XsltCompiler comp = processor.newXsltCompiler(); + XsltExecutable xslt = comp + .compile(new StreamSource(new ByteArrayInputStream(transformationRule.getBytes()))); + XdmNode source = processor + .newDocumentBuilder() + .build(new StreamSource(new ByteArrayInputStream(value.getBody().getBytes()))); + XsltTransformer trans = xslt.load(); + trans.setInitialContextNode(source); + final StringWriter output = new StringWriter(); + Serializer out = processor.newSerializer(output); + out.setOutputProperty(Serializer.Property.METHOD, "xml"); + out.setOutputProperty(Serializer.Property.INDENT, "yes"); + trans.setDestination(out); + trans.transform(); + final String xml = output.toString(); + value.setBody(xml); + value.setDateOfTransformation(dateOfTransformation); + transformedItems.add(1); + return value; + } catch (Throwable e) { + errorItems.add(1); + return null; + } + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java index 550136247..5f39717d0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/TransformSparkJobNode.java @@ -1,17 +1,11 @@ + package eu.dnetlib.dhp.transformation; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; -import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; -import eu.dnetlib.dhp.utils.DHPUtils; -import eu.dnetlib.message.Message; -import eu.dnetlib.message.MessageManager; -import eu.dnetlib.message.MessageType; import java.io.ByteArrayInputStream; import java.util.HashMap; import java.util.Map; import java.util.Objects; + import org.apache.commons.cli.*; import org.apache.commons.io.IOUtils; import org.apache.spark.sql.Dataset; @@ -24,78 +18,87 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.model.mdstore.MetadataRecord; +import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; +import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; +import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.message.Message; +import eu.dnetlib.message.MessageManager; +import eu.dnetlib.message.MessageType; + public class TransformSparkJobNode { - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - TransformSparkJobNode.class.getResourceAsStream( - "/eu/dnetlib/dhp/transformation/transformation_input_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + TransformSparkJobNode.class + .getResourceAsStream( + "/eu/dnetlib/dhp/transformation/transformation_input_parameters.json"))); - parser.parseArgument(args); + parser.parseArgument(args); - final String inputPath = parser.get("input"); - final String outputPath = parser.get("output"); - final String workflowId = parser.get("workflowId"); - final String trasformationRule = - extractXSLTFromTR( - Objects.requireNonNull(DHPUtils.decompressString(parser.get("transformationRule")))); - final String master = parser.get("master"); - final String rabbitUser = parser.get("rabbitUser"); - final String rabbitPassword = parser.get("rabbitPassword"); - final String rabbitHost = parser.get("rabbitHost"); - final String rabbitReportQueue = parser.get("rabbitReportQueue"); - final long dateOfCollection = new Long(parser.get("dateOfCollection")); - final boolean test = - parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); + final String inputPath = parser.get("input"); + final String outputPath = parser.get("output"); + final String workflowId = parser.get("workflowId"); + final String trasformationRule = extractXSLTFromTR( + Objects.requireNonNull(DHPUtils.decompressString(parser.get("transformationRule")))); + final String master = parser.get("master"); + final String rabbitUser = parser.get("rabbitUser"); + final String rabbitPassword = parser.get("rabbitPassword"); + final String rabbitHost = parser.get("rabbitHost"); + final String rabbitReportQueue = parser.get("rabbitReportQueue"); + final long dateOfCollection = new Long(parser.get("dateOfCollection")); + final boolean test = parser.get("isTest") == null ? false : Boolean.valueOf(parser.get("isTest")); - final SparkSession spark = - SparkSession.builder().appName("TransformStoreSparkJob").master(master).getOrCreate(); + final SparkSession spark = SparkSession + .builder() + .appName("TransformStoreSparkJob") + .master(master) + .getOrCreate(); - final Encoder encoder = Encoders.bean(MetadataRecord.class); - final Dataset mdstoreInput = - spark.read().format("parquet").load(inputPath).as(encoder); - final LongAccumulator totalItems = spark.sparkContext().longAccumulator("TotalItems"); - final LongAccumulator errorItems = spark.sparkContext().longAccumulator("errorItems"); - final LongAccumulator transformedItems = - spark.sparkContext().longAccumulator("transformedItems"); - final Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); - final TransformFunction transformFunction = - new TransformFunction( - totalItems, - errorItems, - transformedItems, - trasformationRule, - dateOfCollection, - vocabularies); - mdstoreInput.map(transformFunction, encoder).write().format("parquet").save(outputPath); - if (rabbitHost != null) { - System.out.println("SEND FINAL REPORT"); - final Map reportMap = new HashMap<>(); - reportMap.put("inputItem", "" + totalItems.value()); - reportMap.put("invalidRecords", "" + errorItems.value()); - reportMap.put("mdStoreSize", "" + transformedItems.value()); - System.out.println(new Message(workflowId, "Transform", MessageType.REPORT, reportMap)); - if (!test) { - final MessageManager manager = - new MessageManager(rabbitHost, rabbitUser, rabbitPassword, false, false, null); - manager.sendMessage( - new Message(workflowId, "Transform", MessageType.REPORT, reportMap), - rabbitReportQueue, - true, - false); - manager.close(); - } - } - } + final Encoder encoder = Encoders.bean(MetadataRecord.class); + final Dataset mdstoreInput = spark.read().format("parquet").load(inputPath).as(encoder); + final LongAccumulator totalItems = spark.sparkContext().longAccumulator("TotalItems"); + final LongAccumulator errorItems = spark.sparkContext().longAccumulator("errorItems"); + final LongAccumulator transformedItems = spark.sparkContext().longAccumulator("transformedItems"); + final Map vocabularies = new HashMap<>(); + vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); + final TransformFunction transformFunction = new TransformFunction( + totalItems, + errorItems, + transformedItems, + trasformationRule, + dateOfCollection, + vocabularies); + mdstoreInput.map(transformFunction, encoder).write().format("parquet").save(outputPath); + if (rabbitHost != null) { + System.out.println("SEND FINAL REPORT"); + final Map reportMap = new HashMap<>(); + reportMap.put("inputItem", "" + totalItems.value()); + reportMap.put("invalidRecords", "" + errorItems.value()); + reportMap.put("mdStoreSize", "" + transformedItems.value()); + System.out.println(new Message(workflowId, "Transform", MessageType.REPORT, reportMap)); + if (!test) { + final MessageManager manager = new MessageManager(rabbitHost, rabbitUser, rabbitPassword, false, false, + null); + manager + .sendMessage( + new Message(workflowId, "Transform", MessageType.REPORT, reportMap), + rabbitReportQueue, + true, + false); + manager.close(); + } + } + } - private static String extractXSLTFromTR(final String tr) throws DocumentException { - SAXReader reader = new SAXReader(); - Document document = reader.read(new ByteArrayInputStream(tr.getBytes())); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - return node.asXML(); - } + private static String extractXSLTFromTR(final String tr) throws DocumentException { + SAXReader reader = new SAXReader(); + Document document = reader.read(new ByteArrayInputStream(tr.getBytes())); + Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); + return node.asXML(); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java index 09e77613c..7f9b6646c 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/functions/Cleaner.java @@ -1,48 +1,52 @@ + package eu.dnetlib.dhp.transformation.functions; +import java.util.Map; +import java.util.Optional; + import eu.dnetlib.dhp.transformation.vocabulary.Term; import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; -import java.util.Map; -import java.util.Optional; import net.sf.saxon.s9api.*; import scala.Serializable; public class Cleaner implements ExtensionFunction, Serializable { - private final Map vocabularies; + private final Map vocabularies; - public Cleaner(Map vocabularies) { - this.vocabularies = vocabularies; - } + public Cleaner(Map vocabularies) { + this.vocabularies = vocabularies; + } - @Override - public QName getName() { - return new QName("http://eu/dnetlib/trasform/extension", "clean"); - } + @Override + public QName getName() { + return new QName("http://eu/dnetlib/trasform/extension", "clean"); + } - @Override - public SequenceType getResultType() { - return SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE_OR_MORE); - } + @Override + public SequenceType getResultType() { + return SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE_OR_MORE); + } - @Override - public SequenceType[] getArgumentTypes() { - return new SequenceType[] { - SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE), - SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE) - }; - } + @Override + public SequenceType[] getArgumentTypes() { + return new SequenceType[] { + SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE), + SequenceType.makeSequenceType(ItemType.STRING, OccurrenceIndicator.ONE) + }; + } - @Override - public XdmValue call(XdmValue[] xdmValues) throws SaxonApiException { - final String currentValue = xdmValues[0].itemAt(0).getStringValue(); - final String vocabularyName = xdmValues[1].itemAt(0).getStringValue(); - Optional cleanedValue = - vocabularies.get(vocabularyName).getTerms().stream() - .filter(it -> it.getNativeName().equalsIgnoreCase(currentValue)) - .findAny(); + @Override + public XdmValue call(XdmValue[] xdmValues) throws SaxonApiException { + final String currentValue = xdmValues[0].itemAt(0).getStringValue(); + final String vocabularyName = xdmValues[1].itemAt(0).getStringValue(); + Optional cleanedValue = vocabularies + .get(vocabularyName) + .getTerms() + .stream() + .filter(it -> it.getNativeName().equalsIgnoreCase(currentValue)) + .findAny(); - return new XdmAtomicValue( - cleanedValue.isPresent() ? cleanedValue.get().getCode() : currentValue); - } + return new XdmAtomicValue( + cleanedValue.isPresent() ? cleanedValue.get().getCode() : currentValue); + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Term.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Term.java index 813a77941..b5ac18169 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Term.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Term.java @@ -1,52 +1,53 @@ + package eu.dnetlib.dhp.transformation.vocabulary; import java.io.Serializable; public class Term implements Serializable { - private String englishName; - private String nativeName; - private String encoding; - private String code; - private String synonyms; + private String englishName; + private String nativeName; + private String encoding; + private String code; + private String synonyms; - public String getEnglishName() { - return englishName; - } + public String getEnglishName() { + return englishName; + } - public void setEnglishName(String englishName) { - this.englishName = englishName; - } + public void setEnglishName(String englishName) { + this.englishName = englishName; + } - public String getNativeName() { - return nativeName; - } + public String getNativeName() { + return nativeName; + } - public void setNativeName(String nativeName) { - this.nativeName = nativeName; - } + public void setNativeName(String nativeName) { + this.nativeName = nativeName; + } - public String getEncoding() { - return encoding; - } + public String getEncoding() { + return encoding; + } - public void setEncoding(String encoding) { - this.encoding = encoding; - } + public void setEncoding(String encoding) { + this.encoding = encoding; + } - public String getCode() { - return code; - } + public String getCode() { + return code; + } - public void setCode(String code) { - this.code = code; - } + public void setCode(String code) { + this.code = code; + } - public String getSynonyms() { - return synonyms; - } + public String getSynonyms() { + return synonyms; + } - public void setSynonyms(String synonyms) { - this.synonyms = synonyms; - } + public void setSynonyms(String synonyms) { + this.synonyms = synonyms; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Vocabulary.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Vocabulary.java index 0579c8244..a9da6b725 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Vocabulary.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/Vocabulary.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.transformation.vocabulary; import java.io.Serializable; @@ -5,49 +6,49 @@ import java.util.List; public class Vocabulary implements Serializable { - private String id; - private String name; - private String description; - private String code; - private List terms; + private String id; + private String name; + private String description; + private String code; + private List terms; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getDescription() { - return description; - } + public String getDescription() { + return description; + } - public void setDescription(String description) { - this.description = description; - } + public void setDescription(String description) { + this.description = description; + } - public String getCode() { - return code; - } + public String getCode() { + return code; + } - public void setCode(String code) { - this.code = code; - } + public void setCode(String code) { + this.code = code; + } - public List getTerms() { - return terms; - } + public List getTerms() { + return terms; + } - public void setTerms(List terms) { - this.terms = terms; - } + public void setTerms(List terms) { + this.terms = terms; + } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyHelper.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyHelper.java index 349fc53de..10e959be0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyHelper.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyHelper.java @@ -1,21 +1,24 @@ + package eu.dnetlib.dhp.transformation.vocabulary; -import com.fasterxml.jackson.databind.ObjectMapper; import java.io.Serializable; import java.net.URL; import java.nio.charset.Charset; + import org.apache.commons.io.IOUtils; +import com.fasterxml.jackson.databind.ObjectMapper; + public class VocabularyHelper implements Serializable { - private static final String OPENAIRE_URL = "http://api.openaire.eu/vocabularies/%s.json"; + private static final String OPENAIRE_URL = "http://api.openaire.eu/vocabularies/%s.json"; - public static Vocabulary getVocabularyFromAPI(final String vocabularyName) throws Exception { - final URL url = new URL(String.format(OPENAIRE_URL, vocabularyName)); + public static Vocabulary getVocabularyFromAPI(final String vocabularyName) throws Exception { + final URL url = new URL(String.format(OPENAIRE_URL, vocabularyName)); - final String response = IOUtils.toString(url, Charset.defaultCharset()); - final ObjectMapper jsonMapper = new ObjectMapper(); - final Vocabulary vocabulary = jsonMapper.readValue(response, Vocabulary.class); - return vocabulary; - } + final String response = IOUtils.toString(url, Charset.defaultCharset()); + final ObjectMapper jsonMapper = new ObjectMapper(); + final Vocabulary vocabulary = jsonMapper.readValue(response, Vocabulary.class); + return vocabulary; + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java index cbf0cfd01..44364b30a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/CollectionJobTest.java @@ -1,116 +1,121 @@ + package eu.dnetlib.dhp.collection; import static org.junit.jupiter.api.Assertions.*; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.model.mdstore.Provenance; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.model.mdstore.MetadataRecord; +import eu.dnetlib.dhp.model.mdstore.Provenance; + public class CollectionJobTest { - private Path testDir; + private Path testDir; - @BeforeEach - public void setup() throws IOException { - testDir = Files.createTempDirectory("dhp-collection"); - } + @BeforeEach + public void setup() throws IOException { + testDir = Files.createTempDirectory("dhp-collection"); + } - @AfterEach - public void teadDown() throws IOException { - FileUtils.deleteDirectory(testDir.toFile()); - } + @AfterEach + public void teadDown() throws IOException { + FileUtils.deleteDirectory(testDir.toFile()); + } - @Test - public void tesCollection() throws Exception { - final Provenance provenance = new Provenance("pippo", "puppa", "ns_prefix"); - GenerateNativeStoreSparkJob.main( - new String[] { - "-mt", - "local", - "-w", - "wid", - "-e", - "XML", - "-d", - "" + System.currentTimeMillis(), - "-p", - new ObjectMapper().writeValueAsString(provenance), - "-x", - "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", - "-i", - this.getClass().getResource("/eu/dnetlib/dhp/collection/native.seq").toString(), - "-o", - testDir.toString() + "/store", - "-t", - "true", - "-ru", - "", - "-rp", - "", - "-rh", - "", - "-ro", - "", - "-rr", - "" - }); - System.out.println(new ObjectMapper().writeValueAsString(provenance)); - } + @Test + public void tesCollection() throws Exception { + final Provenance provenance = new Provenance("pippo", "puppa", "ns_prefix"); + GenerateNativeStoreSparkJob + .main( + new String[] { + "-mt", + "local", + "-w", + "wid", + "-e", + "XML", + "-d", + "" + System.currentTimeMillis(), + "-p", + new ObjectMapper().writeValueAsString(provenance), + "-x", + "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", + "-i", + this.getClass().getResource("/eu/dnetlib/dhp/collection/native.seq").toString(), + "-o", + testDir.toString() + "/store", + "-t", + "true", + "-ru", + "", + "-rp", + "", + "-rh", + "", + "-ro", + "", + "-rr", + "" + }); + System.out.println(new ObjectMapper().writeValueAsString(provenance)); + } - @Test - public void testGenerationMetadataRecord() throws Exception { + @Test + public void testGenerationMetadataRecord() throws Exception { - final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml")); + final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml")); - final MetadataRecord record = - GenerateNativeStoreSparkJob.parseRecord( - xml, - "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", - "XML", - new Provenance("foo", "bar", "ns_prefix"), - System.currentTimeMillis(), - null, - null); + final MetadataRecord record = GenerateNativeStoreSparkJob + .parseRecord( + xml, + "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", + "XML", + new Provenance("foo", "bar", "ns_prefix"), + System.currentTimeMillis(), + null, + null); - assert record != null; - System.out.println(record.getId()); - System.out.println(record.getOriginalId()); - } + assert record != null; + System.out.println(record.getId()); + System.out.println(record.getOriginalId()); + } - @Test - public void TestEquals() throws IOException { + @Test + public void TestEquals() throws IOException { - final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml")); - final MetadataRecord record = - GenerateNativeStoreSparkJob.parseRecord( - xml, - "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", - "XML", - new Provenance("foo", "bar", "ns_prefix"), - System.currentTimeMillis(), - null, - null); - final MetadataRecord record1 = - GenerateNativeStoreSparkJob.parseRecord( - xml, - "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", - "XML", - new Provenance("foo", "bar", "ns_prefix"), - System.currentTimeMillis(), - null, - null); - assert record != null; - record.setBody("ciao"); - assert record1 != null; - record1.setBody("mondo"); - assertEquals(record, record1); - } + final String xml = IOUtils.toString(this.getClass().getResourceAsStream("./record.xml")); + final MetadataRecord record = GenerateNativeStoreSparkJob + .parseRecord( + xml, + "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", + "XML", + new Provenance("foo", "bar", "ns_prefix"), + System.currentTimeMillis(), + null, + null); + final MetadataRecord record1 = GenerateNativeStoreSparkJob + .parseRecord( + xml, + "./*[local-name()='record']/*[local-name()='header']/*[local-name()='identifier']", + "XML", + new Provenance("foo", "bar", "ns_prefix"), + System.currentTimeMillis(), + null, + null); + assert record != null; + record.setBody("ciao"); + assert record1 != null; + record1.setBody("mondo"); + assertEquals(record, record1); + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java index a524d75e7..1a4fafb66 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collector/worker/DnetCollectorWorkerApplicationTests.java @@ -1,88 +1,92 @@ + package eu.dnetlib.dhp.collector.worker; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.*; +import java.io.File; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + import com.fasterxml.jackson.databind.ObjectMapper; + import eu.dnetlib.collector.worker.model.ApiDescriptor; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.collection.worker.DnetCollectorWorker; import eu.dnetlib.dhp.collection.worker.utils.CollectorPluginFactory; import eu.dnetlib.message.Message; import eu.dnetlib.message.MessageManager; -import java.io.File; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; public class DnetCollectorWorkerApplicationTests { - private ArgumentApplicationParser argumentParser = mock(ArgumentApplicationParser.class); - private MessageManager messageManager = mock(MessageManager.class); + private ArgumentApplicationParser argumentParser = mock(ArgumentApplicationParser.class); + private MessageManager messageManager = mock(MessageManager.class); - private DnetCollectorWorker worker; + private DnetCollectorWorker worker; - @BeforeEach - public void setup() throws Exception { - ObjectMapper mapper = new ObjectMapper(); - final String apiJson = mapper.writeValueAsString(getApi()); - when(argumentParser.get("apidescriptor")).thenReturn(apiJson); - when(argumentParser.get("namenode")).thenReturn("file://tmp/test.seq"); - when(argumentParser.get("hdfsPath")).thenReturn("/tmp/file.seq"); - when(argumentParser.get("userHDFS")).thenReturn("sandro"); - when(argumentParser.get("workflowId")).thenReturn("sandro"); - when(argumentParser.get("rabbitOngoingQueue")).thenReturn("sandro"); + @BeforeEach + public void setup() throws Exception { + ObjectMapper mapper = new ObjectMapper(); + final String apiJson = mapper.writeValueAsString(getApi()); + when(argumentParser.get("apidescriptor")).thenReturn(apiJson); + when(argumentParser.get("namenode")).thenReturn("file://tmp/test.seq"); + when(argumentParser.get("hdfsPath")).thenReturn("/tmp/file.seq"); + when(argumentParser.get("userHDFS")).thenReturn("sandro"); + when(argumentParser.get("workflowId")).thenReturn("sandro"); + when(argumentParser.get("rabbitOngoingQueue")).thenReturn("sandro"); - when(messageManager.sendMessage(any(Message.class), anyString(), anyBoolean(), anyBoolean())) - .thenAnswer( - a -> { - System.out.println("sent message: " + a.getArguments()[0]); - return true; - }); - when(messageManager.sendMessage(any(Message.class), anyString())) - .thenAnswer( - a -> { - System.out.println("Called"); - return true; - }); - worker = new DnetCollectorWorker(new CollectorPluginFactory(), argumentParser, messageManager); - } + when(messageManager.sendMessage(any(Message.class), anyString(), anyBoolean(), anyBoolean())) + .thenAnswer( + a -> { + System.out.println("sent message: " + a.getArguments()[0]); + return true; + }); + when(messageManager.sendMessage(any(Message.class), anyString())) + .thenAnswer( + a -> { + System.out.println("Called"); + return true; + }); + worker = new DnetCollectorWorker(new CollectorPluginFactory(), argumentParser, messageManager); + } - @AfterEach - public void dropDown() { - File f = new File("/tmp/file.seq"); - f.delete(); - } + @AfterEach + public void dropDown() { + File f = new File("/tmp/file.seq"); + f.delete(); + } - @Test - public void testFindPlugin() throws Exception { - final CollectorPluginFactory collectorPluginEnumerator = new CollectorPluginFactory(); - assertNotNull(collectorPluginEnumerator.getPluginByProtocol("oai")); - assertNotNull(collectorPluginEnumerator.getPluginByProtocol("OAI")); - } + @Test + public void testFindPlugin() throws Exception { + final CollectorPluginFactory collectorPluginEnumerator = new CollectorPluginFactory(); + assertNotNull(collectorPluginEnumerator.getPluginByProtocol("oai")); + assertNotNull(collectorPluginEnumerator.getPluginByProtocol("OAI")); + } - @Test - public void testCollectionOAI() throws Exception { - final ApiDescriptor api = new ApiDescriptor(); - api.setId("oai"); - api.setProtocol("oai"); - api.setBaseUrl("http://www.revista.vocesdelaeducacion.com.mx/index.php/index/oai"); - api.getParams().put("format", "oai_dc"); - ObjectMapper mapper = new ObjectMapper(); - assertNotNull(mapper.writeValueAsString(api)); - } + @Test + public void testCollectionOAI() throws Exception { + final ApiDescriptor api = new ApiDescriptor(); + api.setId("oai"); + api.setProtocol("oai"); + api.setBaseUrl("http://www.revista.vocesdelaeducacion.com.mx/index.php/index/oai"); + api.getParams().put("format", "oai_dc"); + ObjectMapper mapper = new ObjectMapper(); + assertNotNull(mapper.writeValueAsString(api)); + } - @Test - public void testFeeding() throws Exception { - worker.collect(); - } + @Test + public void testFeeding() throws Exception { + worker.collect(); + } - private ApiDescriptor getApi() { - final ApiDescriptor api = new ApiDescriptor(); - api.setId("oai"); - api.setProtocol("oai"); - api.setBaseUrl("http://www.revista.vocesdelaeducacion.com.mx/index.php/index/oai"); - api.getParams().put("format", "oai_dc"); - return api; - } + private ApiDescriptor getApi() { + final ApiDescriptor api = new ApiDescriptor(); + api.setId("oai"); + api.setProtocol("oai"); + api.setBaseUrl("http://www.revista.vocesdelaeducacion.com.mx/index.php/index/oai"); + api.getParams().put("format", "oai_dc"); + return api; + } } 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 12a89053e..01c9e3103 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 @@ -1,19 +1,16 @@ + package eu.dnetlib.dhp.transformation; import static org.junit.jupiter.api.Assertions.assertNotNull; -import eu.dnetlib.dhp.model.mdstore.MetadataRecord; -import eu.dnetlib.dhp.transformation.functions.Cleaner; -import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; -import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; -import eu.dnetlib.dhp.utils.DHPUtils; import java.io.StringWriter; import java.nio.file.Files; import java.nio.file.Path; import java.util.HashMap; import java.util.Map; + import javax.xml.transform.stream.StreamSource; -import net.sf.saxon.s9api.*; + import org.apache.commons.io.IOUtils; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; @@ -26,127 +23,133 @@ import org.junit.jupiter.api.io.TempDir; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import eu.dnetlib.dhp.model.mdstore.MetadataRecord; +import eu.dnetlib.dhp.transformation.functions.Cleaner; +import eu.dnetlib.dhp.transformation.vocabulary.Vocabulary; +import eu.dnetlib.dhp.transformation.vocabulary.VocabularyHelper; +import eu.dnetlib.dhp.utils.DHPUtils; +import net.sf.saxon.s9api.*; + @ExtendWith(MockitoExtension.class) public class TransformationJobTest { - @Mock private LongAccumulator accumulator; + @Mock + private LongAccumulator accumulator; - @Test - public void testTransformSaxonHE() throws Exception { + @Test + public void testTransformSaxonHE() throws Exception { - Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); - Cleaner cleanFunction = new Cleaner(vocabularies); - Processor proc = new Processor(false); - proc.registerExtensionFunction(cleanFunction); - final XsltCompiler comp = proc.newXsltCompiler(); - XsltExecutable exp = - comp.compile( - new StreamSource( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/ext_simple.xsl"))); - XdmNode source = - proc.newDocumentBuilder() - .build( - new StreamSource( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); - XsltTransformer trans = exp.load(); - trans.setInitialContextNode(source); - final StringWriter output = new StringWriter(); - Serializer out = proc.newSerializer(output); - out.setOutputProperty(Serializer.Property.METHOD, "xml"); - out.setOutputProperty(Serializer.Property.INDENT, "yes"); - trans.setDestination(out); - trans.transform(); - System.out.println(output.toString()); - } + Map vocabularies = new HashMap<>(); + vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); + Cleaner cleanFunction = new Cleaner(vocabularies); + Processor proc = new Processor(false); + proc.registerExtensionFunction(cleanFunction); + final XsltCompiler comp = proc.newXsltCompiler(); + XsltExecutable exp = comp + .compile( + new StreamSource( + this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/ext_simple.xsl"))); + XdmNode source = proc + .newDocumentBuilder() + .build( + new StreamSource( + this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); + XsltTransformer trans = exp.load(); + trans.setInitialContextNode(source); + final StringWriter output = new StringWriter(); + Serializer out = proc.newSerializer(output); + out.setOutputProperty(Serializer.Property.METHOD, "xml"); + out.setOutputProperty(Serializer.Property.INDENT, "yes"); + trans.setDestination(out); + trans.transform(); + System.out.println(output.toString()); + } - @DisplayName("Test TransformSparkJobNode.main") - @Test - public void transformTest(@TempDir Path testDir) throws Exception { - final String mdstore_input = - this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); - final String mdstore_output = testDir.toString() + "/version"; - final String xslt = - DHPUtils.compressString( - IOUtils.toString( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml"))); - TransformSparkJobNode.main( - new String[] { - "-mt", - "local", - "-i", - mdstore_input, - "-o", - mdstore_output, - "-d", - "1", - "-w", - "1", - "-tr", - xslt, - "-t", - "true", - "-ru", - "", - "-rp", - "", - "-rh", - "", - "-ro", - "", - "-rr", - "" - }); - } + @DisplayName("Test TransformSparkJobNode.main") + @Test + public void transformTest(@TempDir Path testDir) throws Exception { + final String mdstore_input = this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); + final String mdstore_output = testDir.toString() + "/version"; + final String xslt = DHPUtils + .compressString( + IOUtils + .toString( + this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml"))); + TransformSparkJobNode + .main( + new String[] { + "-mt", + "local", + "-i", + mdstore_input, + "-o", + mdstore_output, + "-d", + "1", + "-w", + "1", + "-tr", + xslt, + "-t", + "true", + "-ru", + "", + "-rp", + "", + "-rh", + "", + "-ro", + "", + "-rr", + "" + }); + } - @Test - public void tryLoadFolderOnCP() throws Exception { - final String path = - this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); - System.out.println("path = " + path); + @Test + public void tryLoadFolderOnCP() throws Exception { + final String path = this.getClass().getResource("/eu/dnetlib/dhp/transform/mdstorenative").getFile(); + System.out.println("path = " + path); - Path tempDirWithPrefix = Files.createTempDirectory("mdstore_output"); + Path tempDirWithPrefix = Files.createTempDirectory("mdstore_output"); - System.out.println(tempDirWithPrefix.toFile().getAbsolutePath()); + System.out.println(tempDirWithPrefix.toFile().getAbsolutePath()); - Files.deleteIfExists(tempDirWithPrefix); - } + Files.deleteIfExists(tempDirWithPrefix); + } - @Test - public void testTransformFunction() throws Exception { - SAXReader reader = new SAXReader(); - Document document = - reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - final String xslt = node.asXML(); - Map vocabularies = new HashMap<>(); - vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); + @Test + public void testTransformFunction() throws Exception { + SAXReader reader = new SAXReader(); + Document document = reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); + Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); + final String xslt = node.asXML(); + Map vocabularies = new HashMap<>(); + vocabularies.put("dnet:languages", VocabularyHelper.getVocabularyFromAPI("dnet:languages")); - TransformFunction tf = - new TransformFunction(accumulator, accumulator, accumulator, xslt, 1, vocabularies); + TransformFunction tf = new TransformFunction(accumulator, accumulator, accumulator, xslt, 1, vocabularies); - MetadataRecord record = new MetadataRecord(); - record.setBody( - IOUtils.toString( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); + MetadataRecord record = new MetadataRecord(); + record + .setBody( + IOUtils + .toString( + this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input.xml"))); - final MetadataRecord result = tf.call(record); - assertNotNull(result.getBody()); + final MetadataRecord result = tf.call(record); + assertNotNull(result.getBody()); - System.out.println(result.getBody()); - } + System.out.println(result.getBody()); + } - @Test - public void extractTr() throws Exception { + @Test + public void extractTr() throws Exception { - final String xmlTr = - IOUtils.toString(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); + final String xmlTr = IOUtils.toString(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); - SAXReader reader = new SAXReader(); - Document document = - reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); - Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); + SAXReader reader = new SAXReader(); + Document document = reader.read(this.getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/tr.xml")); + Node node = document.selectSingleNode("//CODE/*[local-name()='stylesheet']"); - System.out.println(node.asXML()); - } + System.out.println(node.asXML()); + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyTest.java index 3732c5e82..1ae942a6b 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/transformation/vocabulary/VocabularyTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.transformation.vocabulary; import static org.junit.jupiter.api.Assertions.*; @@ -6,10 +7,10 @@ import org.junit.jupiter.api.Test; public class VocabularyTest { - @Test - public void testLoadVocabulary() throws Exception { + @Test + public void testLoadVocabulary() throws Exception { - final Vocabulary vocabulary = VocabularyHelper.getVocabularyFromAPI("dnet:languages"); - assertEquals("dnet:languages", vocabulary.getName()); - } + final Vocabulary vocabulary = VocabularyHelper.getVocabularyFromAPI("dnet:languages"); + assertEquals("dnet:languages", vocabulary.getName()); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java index 89106560c..7a8ae0bd0 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java @@ -1,13 +1,10 @@ + package eu.dnetlib.dhp.bulktag; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.community.*; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -18,144 +15,152 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.community.*; +import eu.dnetlib.dhp.schema.oaf.*; + public class SparkBulkTagJob2 { - private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob2.class); + private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkBulkTagJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkBulkTagJob2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - Boolean isTest = - Optional.ofNullable(parser.get("isTest")) - .map(Boolean::valueOf) - .orElse(Boolean.FALSE); - log.info("isTest: {} ", isTest); + Boolean isTest = Optional + .ofNullable(parser.get("isTest")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); + log.info("isTest: {} ", isTest); - final String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - ProtoMap protoMappingParams = new Gson().fromJson(parser.get("protoMap"), ProtoMap.class); - ; - log.info("protoMap: {}", new Gson().toJson(protoMappingParams)); + ProtoMap protoMappingParams = new Gson().fromJson(parser.get("protoMap"), ProtoMap.class); + ; + log.info("protoMap: {}", new Gson().toJson(protoMappingParams)); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - CommunityConfiguration cc; + SparkConf conf = new SparkConf(); + CommunityConfiguration cc; - String taggingConf = parser.get("taggingConf"); + String taggingConf = parser.get("taggingConf"); - if (isTest) { - cc = CommunityConfigurationFactory.newInstance(taggingConf); - } else { - cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); - } + if (isTest) { + cc = CommunityConfigurationFactory.newInstance(taggingConf); + } else { + cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); + } - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - execBulkTag(spark, inputPath, outputPath, protoMappingParams, resultClazz, cc); - }); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + execBulkTag(spark, inputPath, outputPath, protoMappingParams, resultClazz, cc); + }); - // runWithSparkSession(conf, isSparkSessionManaged, - // spark -> { - // if(isTest(parser)) { - // removeOutputDir(spark, outputPath); - // } - // if(saveGraph) - // execPropagation(spark, possibleUpdates, inputPath, outputPath, - // resultClazz); - // }); - // - // - // - // - // - // - // sc.textFile(inputPath + "/publication") - // .map(item -> new ObjectMapper().readValue(item, Publication.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/publication"); - // sc.textFile(inputPath + "/dataset") - // .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/dataset"); - // sc.textFile(inputPath + "/software") - // .map(item -> new ObjectMapper().readValue(item, Software.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/software"); - // sc.textFile(inputPath + "/otherresearchproduct") - // .map(item -> new ObjectMapper().readValue(item, - // OtherResearchProduct.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/otherresearchproduct"); - // + // runWithSparkSession(conf, isSparkSessionManaged, + // spark -> { + // if(isTest(parser)) { + // removeOutputDir(spark, outputPath); + // } + // if(saveGraph) + // execPropagation(spark, possibleUpdates, inputPath, outputPath, + // resultClazz); + // }); + // + // + // + // + // + // + // sc.textFile(inputPath + "/publication") + // .map(item -> new ObjectMapper().readValue(item, Publication.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/publication"); + // sc.textFile(inputPath + "/dataset") + // .map(item -> new ObjectMapper().readValue(item, Dataset.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/dataset"); + // sc.textFile(inputPath + "/software") + // .map(item -> new ObjectMapper().readValue(item, Software.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/software"); + // sc.textFile(inputPath + "/otherresearchproduct") + // .map(item -> new ObjectMapper().readValue(item, + // OtherResearchProduct.class)) + // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) + // .map(p -> new ObjectMapper().writeValueAsString(p)) + // .saveAsTextFile(outputPath+"/otherresearchproduct"); + // - } + } - private static void execBulkTag( - SparkSession spark, - String inputPath, - String outputPath, - ProtoMap protoMappingParams, - Class resultClazz, - CommunityConfiguration communityConfiguration) { + private static void execBulkTag( + SparkSession spark, + String inputPath, + String outputPath, + ProtoMap protoMappingParams, + Class resultClazz, + CommunityConfiguration communityConfiguration) { - ResultTagger resultTagger = new ResultTagger(); - Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.map( - value -> - resultTagger.enrichContextCriteria( - value, communityConfiguration, protoMappingParams), - Encoders.bean(resultClazz)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + ResultTagger resultTagger = new ResultTagger(); + Dataset result = readPathEntity(spark, inputPath, resultClazz); + result + .map( + value -> resultTagger + .enrichContextCriteria( + value, communityConfiguration, protoMappingParams), + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } - private static org.apache.spark.sql.Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class resultClazz) { + private static org.apache.spark.sql.Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class resultClazz) { - return spark.read() - .textFile(inputEntityPath) - .map( - (MapFunction) - value -> OBJECT_MAPPER.readValue(value, resultClazz), - Encoders.bean(resultClazz)); - } + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), + Encoders.bean(resultClazz)); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java index b0c213c12..a73ff4d3e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java @@ -1,62 +1,65 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; import java.io.Serializable; import java.util.ArrayList; import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import com.google.gson.Gson; + /** Created by miriam on 01/08/2018. */ public class Community implements Serializable { - private static final Log log = LogFactory.getLog(Community.class); + private static final Log log = LogFactory.getLog(Community.class); - private String id; - private List subjects = new ArrayList<>(); - private List datasources = new ArrayList<>(); - private List zenodoCommunities = new ArrayList<>(); + private String id; + private List subjects = new ArrayList<>(); + private List datasources = new ArrayList<>(); + private List zenodoCommunities = new ArrayList<>(); - public String toJson() { - final Gson g = new Gson(); - return g.toJson(this); - } + public String toJson() { + final Gson g = new Gson(); + return g.toJson(this); + } - public boolean isValid() { - return !getSubjects().isEmpty() - || !getDatasources().isEmpty() - || !getZenodoCommunities().isEmpty(); - } + public boolean isValid() { + return !getSubjects().isEmpty() + || !getDatasources().isEmpty() + || !getZenodoCommunities().isEmpty(); + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public List getSubjects() { - return subjects; - } + public List getSubjects() { + return subjects; + } - public void setSubjects(List subjects) { - this.subjects = subjects; - } + public void setSubjects(List subjects) { + this.subjects = subjects; + } - public List getDatasources() { - return datasources; - } + public List getDatasources() { + return datasources; + } - public void setDatasources(List datasources) { - this.datasources = datasources; - } + public void setDatasources(List datasources) { + this.datasources = datasources; + } - public List getZenodoCommunities() { - return zenodoCommunities; - } + public List getZenodoCommunities() { + return zenodoCommunities; + } - public void setZenodoCommunities(List zenodoCommunities) { - this.zenodoCommunities = zenodoCommunities; - } + public void setZenodoCommunities(List zenodoCommunities) { + this.zenodoCommunities = zenodoCommunities; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java index 1fd5bedd4..c5bbb66eb 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java @@ -1,189 +1,196 @@ + package eu.dnetlib.dhp.community; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; -import eu.dnetlib.dhp.selectioncriteria.Selection; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; +import eu.dnetlib.dhp.selectioncriteria.Selection; + /** Created by miriam on 02/08/2018. */ public class CommunityConfiguration implements Serializable { - private static final Log log = LogFactory.getLog(CommunityConfiguration.class); + private static final Log log = LogFactory.getLog(CommunityConfiguration.class); - private Map communities; + private Map communities; - // map subject -> communityid - private Map>> subjectMap = new HashMap<>(); - // map datasourceid -> communityid - private Map>> datasourceMap = new HashMap<>(); - // map zenodocommunityid -> communityid - private Map>> zenodocommunityMap = - new HashMap<>(); + // map subject -> communityid + private Map>> subjectMap = new HashMap<>(); + // map datasourceid -> communityid + private Map>> datasourceMap = new HashMap<>(); + // map zenodocommunityid -> communityid + private Map>> zenodocommunityMap = new HashMap<>(); - public Map>> getSubjectMap() { - return subjectMap; - } + public Map>> getSubjectMap() { + return subjectMap; + } - public void setSubjectMap(Map>> subjectMap) { - this.subjectMap = subjectMap; - } + public void setSubjectMap(Map>> subjectMap) { + this.subjectMap = subjectMap; + } - public Map>> getDatasourceMap() { - return datasourceMap; - } + public Map>> getDatasourceMap() { + return datasourceMap; + } - public void setDatasourceMap( - Map>> datasourceMap) { - this.datasourceMap = datasourceMap; - } + public void setDatasourceMap( + Map>> datasourceMap) { + this.datasourceMap = datasourceMap; + } - public Map>> getZenodocommunityMap() { - return zenodocommunityMap; - } + public Map>> getZenodocommunityMap() { + return zenodocommunityMap; + } - public void setZenodocommunityMap( - Map>> zenodocommunityMap) { - this.zenodocommunityMap = zenodocommunityMap; - } + public void setZenodocommunityMap( + Map>> zenodocommunityMap) { + this.zenodocommunityMap = zenodocommunityMap; + } - CommunityConfiguration(final Map communities) { - this.communities = communities; - init(); - } + CommunityConfiguration(final Map communities) { + this.communities = communities; + init(); + } - void init() { + void init() { - if (subjectMap == null) { - subjectMap = Maps.newHashMap(); - } - if (datasourceMap == null) { - datasourceMap = Maps.newHashMap(); - } - if (zenodocommunityMap == null) { - zenodocommunityMap = Maps.newHashMap(); - } + if (subjectMap == null) { + subjectMap = Maps.newHashMap(); + } + if (datasourceMap == null) { + datasourceMap = Maps.newHashMap(); + } + if (zenodocommunityMap == null) { + zenodocommunityMap = Maps.newHashMap(); + } - for (Community c : getCommunities().values()) { - // get subjects - final String id = c.getId(); - for (String sbj : c.getSubjects()) { - Pair p = new Pair<>(id, new SelectionConstraints()); - add(sbj.toLowerCase().trim(), p, subjectMap); - } - // get datasources - for (Datasource d : c.getDatasources()) { + for (Community c : getCommunities().values()) { + // get subjects + final String id = c.getId(); + for (String sbj : c.getSubjects()) { + Pair p = new Pair<>(id, new SelectionConstraints()); + add(sbj.toLowerCase().trim(), p, subjectMap); + } + // get datasources + for (Datasource d : c.getDatasources()) { - add(d.getOpenaireId(), new Pair<>(id, d.getSelectionConstraints()), datasourceMap); - } - // get zenodo communities - for (ZenodoCommunity zc : c.getZenodoCommunities()) { - add( - zc.getZenodoCommunityId(), - new Pair<>(id, zc.getSelCriteria()), - zenodocommunityMap); - } - } - } + add(d.getOpenaireId(), new Pair<>(id, d.getSelectionConstraints()), datasourceMap); + } + // get zenodo communities + for (ZenodoCommunity zc : c.getZenodoCommunities()) { + add( + zc.getZenodoCommunityId(), + new Pair<>(id, zc.getSelCriteria()), + zenodocommunityMap); + } + } + } - private void add( - String key, - Pair value, - Map>> map) { - List> values = map.get(key); + private void add( + String key, + Pair value, + Map>> map) { + List> values = map.get(key); - if (values == null) { - values = new ArrayList<>(); - map.put(key, values); - } - values.add(value); - } + if (values == null) { + values = new ArrayList<>(); + map.put(key, values); + } + values.add(value); + } - public List> getCommunityForSubject(String sbj) { - return subjectMap.get(sbj); - } + public List> getCommunityForSubject(String sbj) { + return subjectMap.get(sbj); + } - public List> getCommunityForDatasource(String dts) { - return datasourceMap.get(dts); - } + public List> getCommunityForDatasource(String dts) { + return datasourceMap.get(dts); + } - public List getCommunityForDatasource( - final String dts, final Map> param) { - List> lp = datasourceMap.get(dts); - if (lp == null) return Lists.newArrayList(); + public List getCommunityForDatasource( + final String dts, final Map> param) { + List> lp = datasourceMap.get(dts); + if (lp == null) + return Lists.newArrayList(); - return lp.stream() - .map( - p -> { - if (p.getSnd() == null) return p.getFst(); - if (((SelectionConstraints) p.getSnd()).verifyCriteria(param)) - return p.getFst(); - else return null; - }) - .filter(st -> (st != null)) - .collect(Collectors.toList()); - } + return lp + .stream() + .map( + p -> { + if (p.getSnd() == null) + return p.getFst(); + if (((SelectionConstraints) p.getSnd()).verifyCriteria(param)) + return p.getFst(); + else + return null; + }) + .filter(st -> (st != null)) + .collect(Collectors.toList()); + } - public List> getCommunityForZenodoCommunity(String zc) { - return zenodocommunityMap.get(zc); - } + public List> getCommunityForZenodoCommunity(String zc) { + return zenodocommunityMap.get(zc); + } - public List getCommunityForSubjectValue(String value) { + public List getCommunityForSubjectValue(String value) { - return getContextIds(subjectMap.get(value)); - } + return getContextIds(subjectMap.get(value)); + } - public List getCommunityForDatasourceValue(String value) { + public List getCommunityForDatasourceValue(String value) { - return getContextIds(datasourceMap.get(value.toLowerCase())); - } + return getContextIds(datasourceMap.get(value.toLowerCase())); + } - public List getCommunityForZenodoCommunityValue(String value) { + public List getCommunityForZenodoCommunityValue(String value) { - return getContextIds(zenodocommunityMap.get(value.toLowerCase())); - } + return getContextIds(zenodocommunityMap.get(value.toLowerCase())); + } - private List getContextIds(List> list) { - if (list != null) { - return list.stream().map(p -> p.getFst()).collect(Collectors.toList()); - } - return Lists.newArrayList(); - } + private List getContextIds(List> list) { + if (list != null) { + return list.stream().map(p -> p.getFst()).collect(Collectors.toList()); + } + return Lists.newArrayList(); + } - public Map getCommunities() { - return communities; - } + public Map getCommunities() { + return communities; + } - public void setCommunities(Map communities) { - this.communities = communities; - } + public void setCommunities(Map communities) { + this.communities = communities; + } - public String toJson() { - GsonBuilder builder = new GsonBuilder(); - builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); - Gson gson = builder.create(); + public String toJson() { + GsonBuilder builder = new GsonBuilder(); + builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); + Gson gson = builder.create(); - return gson.toJson(this); - } + return gson.toJson(this); + } - public int size() { - return communities.keySet().size(); - } + public int size() { + return communities.keySet().size(); + } - public Community getCommunityById(String id) { - return communities.get(id); - } + public Community getCommunityById(String id) { + return communities.get(id); + } - public List getCommunityList() { - return Lists.newLinkedList(communities.values()); - } + public List getCommunityList() { + return Lists.newLinkedList(communities.values()); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java index 866ec28a1..508f0663d 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java @@ -1,17 +1,11 @@ + package eu.dnetlib.dhp.community; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; -import eu.dnetlib.dhp.selectioncriteria.Selection; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; -import eu.dnetlib.dhp.selectioncriteria.VerbResolverFactory; import java.io.StringReader; import java.util.ArrayList; import java.util.List; import java.util.Map; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -20,114 +14,125 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; +import eu.dnetlib.dhp.selectioncriteria.Selection; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import eu.dnetlib.dhp.selectioncriteria.VerbResolverFactory; + /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactory { - private static final Log log = LogFactory.getLog(CommunityConfigurationFactory.class); + private static final Log log = LogFactory.getLog(CommunityConfigurationFactory.class); - private static VerbResolver resolver = VerbResolverFactory.newInstance(); + private static VerbResolver resolver = VerbResolverFactory.newInstance(); - public static CommunityConfiguration newInstance(final String xml) throws DocumentException { + public static CommunityConfiguration newInstance(final String xml) throws DocumentException { - log.debug(String.format("parsing community configuration from:\n%s", xml)); + log.debug(String.format("parsing community configuration from:\n%s", xml)); - final Document doc = new SAXReader().read(new StringReader(xml)); + final Document doc = new SAXReader().read(new StringReader(xml)); - final Map communities = Maps.newHashMap(); + final Map communities = Maps.newHashMap(); - for (final Object o : doc.selectNodes("//community")) { + for (final Object o : doc.selectNodes("//community")) { - final Node node = (Node) o; + final Node node = (Node) o; - final Community community = parseCommunity(node); + final Community community = parseCommunity(node); - if (community.isValid()) { - communities.put(community.getId(), community); - } - } + if (community.isValid()) { + communities.put(community.getId(), community); + } + } - log.info(String.format("loaded %s community configuration profiles", communities.size())); - log.debug(String.format("loaded community configuration:\n%s", communities.toString())); + log.info(String.format("loaded %s community configuration profiles", communities.size())); + log.debug(String.format("loaded community configuration:\n%s", communities.toString())); - return new CommunityConfiguration(communities); - } + return new CommunityConfiguration(communities); + } - public static CommunityConfiguration fromJson(final String json) { - GsonBuilder builder = new GsonBuilder(); - builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); - Gson gson = builder.create(); - final CommunityConfiguration conf = gson.fromJson(json, CommunityConfiguration.class); - log.info(String.format("loaded %s community configuration profiles", conf.size())); - conf.init(); - log.info("created inverse maps"); + public static CommunityConfiguration fromJson(final String json) { + GsonBuilder builder = new GsonBuilder(); + builder.registerTypeAdapter(Selection.class, new InterfaceAdapter()); + Gson gson = builder.create(); + final CommunityConfiguration conf = gson.fromJson(json, CommunityConfiguration.class); + log.info(String.format("loaded %s community configuration profiles", conf.size())); + conf.init(); + log.info("created inverse maps"); - return conf; - } + return conf; + } - private static Community parseCommunity(final Node node) { + private static Community parseCommunity(final Node node) { - final Community c = new Community(); + final Community c = new Community(); - c.setId(node.valueOf("./@id")); + c.setId(node.valueOf("./@id")); - log.info(String.format("community id: %s", c.getId())); + log.info(String.format("community id: %s", c.getId())); - c.setSubjects(parseSubjects(node)); - c.setDatasources(parseDatasources(node)); - c.setZenodoCommunities(parseZenodoCommunities(node)); - return c; - } + c.setSubjects(parseSubjects(node)); + c.setDatasources(parseDatasources(node)); + c.setZenodoCommunities(parseZenodoCommunities(node)); + return c; + } - private static List parseSubjects(final Node node) { + private static List parseSubjects(final Node node) { - final List subjects = Lists.newArrayList(); + final List subjects = Lists.newArrayList(); - final List list = node.selectNodes("./subjects/subject"); + final List list = node.selectNodes("./subjects/subject"); - for (Node n : list) { - log.debug("text of the node " + n.getText()); - subjects.add(StringUtils.trim(n.getText())); - } - log.info("size of the subject list " + subjects.size()); - return subjects; - } + for (Node n : list) { + log.debug("text of the node " + n.getText()); + subjects.add(StringUtils.trim(n.getText())); + } + log.info("size of the subject list " + subjects.size()); + return subjects; + } - private static List parseDatasources(final Node node) { - final List list = node.selectNodes("./datasources/datasource"); - final List datasourceList = new ArrayList<>(); - for (Node n : list) { - Datasource d = new Datasource(); - d.setOpenaireId(n.selectSingleNode("./openaireId").getText()); - d.setSelCriteria(n.selectSingleNode("./selcriteria"), resolver); - datasourceList.add(d); - } - log.info("size of the datasource list " + datasourceList.size()); - return datasourceList; - } + private static List parseDatasources(final Node node) { + final List list = node.selectNodes("./datasources/datasource"); + final List datasourceList = new ArrayList<>(); + for (Node n : list) { + Datasource d = new Datasource(); + d.setOpenaireId(n.selectSingleNode("./openaireId").getText()); + d.setSelCriteria(n.selectSingleNode("./selcriteria"), resolver); + datasourceList.add(d); + } + log.info("size of the datasource list " + datasourceList.size()); + return datasourceList; + } - private static List parseZenodoCommunities(final Node node) { - final Node oacommunitynode = node.selectSingleNode("./oacommunity"); - String oacommunity = null; - if (oacommunitynode != null) { - String tmp = oacommunitynode.getText(); - if (StringUtils.isNotBlank(tmp)) oacommunity = tmp; - } + private static List parseZenodoCommunities(final Node node) { + final Node oacommunitynode = node.selectSingleNode("./oacommunity"); + String oacommunity = null; + if (oacommunitynode != null) { + String tmp = oacommunitynode.getText(); + if (StringUtils.isNotBlank(tmp)) + oacommunity = tmp; + } - final List list = node.selectNodes("./zenodocommunities/zenodocommunity"); - final List zenodoCommunityList = new ArrayList<>(); - for (Node n : list) { - ZenodoCommunity zc = new ZenodoCommunity(); - zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); - zc.setSelCriteria(n.selectSingleNode("./selcriteria")); + final List list = node.selectNodes("./zenodocommunities/zenodocommunity"); + final List zenodoCommunityList = new ArrayList<>(); + for (Node n : list) { + ZenodoCommunity zc = new ZenodoCommunity(); + zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); + zc.setSelCriteria(n.selectSingleNode("./selcriteria")); - zenodoCommunityList.add(zc); - } - if (oacommunity != null) { - ZenodoCommunity zc = new ZenodoCommunity(); - zc.setZenodoCommunityId(oacommunity); - zenodoCommunityList.add(zc); - } - log.info("size of the zenodo community list " + zenodoCommunityList.size()); - return zenodoCommunityList; - } + zenodoCommunityList.add(zc); + } + if (oacommunity != null) { + ZenodoCommunity zc = new ZenodoCommunity(); + zc.setZenodoCommunityId(oacommunity); + zenodoCommunityList.add(zc); + } + log.info("size of the zenodo community list " + zenodoCommunityList.size()); + return zenodoCommunityList; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java index 491f3de05..54f381d4a 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java @@ -1,53 +1,56 @@ + package eu.dnetlib.dhp.community; -import eu.dnetlib.dhp.selectioncriteria.Selection; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; +import eu.dnetlib.dhp.selectioncriteria.Selection; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + public class Constraint implements Serializable { - private String verb; - private String field; - private String value; - private Selection selection; + private String verb; + private String field; + private String value; + private Selection selection; - public Constraint() {} + public Constraint() { + } - public String getVerb() { - return verb; - } + public String getVerb() { + return verb; + } - public void setVerb(String verb) { - this.verb = verb; - } + public void setVerb(String verb) { + this.verb = verb; + } - public String getField() { - return field; - } + public String getField() { + return field; + } - public void setField(String field) { - this.field = field; - } + public void setField(String field) { + this.field = field; + } - public String getValue() { - return value; - } + public String getValue() { + return value; + } - public void setValue(String value) { - this.value = value; - } + public void setValue(String value) { + this.value = value; + } - public void setSelection(Selection sel) { - selection = sel; - } + public void setSelection(Selection sel) { + selection = sel; + } - public void setSelection(VerbResolver resolver) - throws InvocationTargetException, NoSuchMethodException, InstantiationException, - IllegalAccessException { - selection = resolver.getSelectionCriteria(verb, value); - } + public void setSelection(VerbResolver resolver) + throws InvocationTargetException, NoSuchMethodException, InstantiationException, + IllegalAccessException { + selection = resolver.getSelectionCriteria(verb, value); + } - public boolean verifyCriteria(String metadata) { - return selection.apply(metadata); - } + public boolean verifyCriteria(String metadata) { + return selection.apply(metadata); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java index 9b2974c27..af095c513 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java @@ -1,67 +1,74 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Type; import java.util.Collection; import java.util.List; import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; + +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + /** Created by miriam on 02/08/2018. */ public class Constraints implements Serializable { - private static final Log log = LogFactory.getLog(Constraints.class); - // private ConstraintEncapsulator ce; - private List constraint; + private static final Log log = LogFactory.getLog(Constraints.class); + // private ConstraintEncapsulator ce; + private List constraint; - public Constraints() {} + public Constraints() { + } - public List getConstraint() { - return constraint; - } + public List getConstraint() { + return constraint; + } - public void setConstraint(List constraint) { - this.constraint = constraint; - } + public void setConstraint(List constraint) { + this.constraint = constraint; + } - public void setSc(String json) { - Type collectionType = new TypeToken>() {}.getType(); - constraint = new Gson().fromJson(json, collectionType); - } + public void setSc(String json) { + Type collectionType = new TypeToken>() { + }.getType(); + constraint = new Gson().fromJson(json, collectionType); + } - void setSelection(VerbResolver resolver) { - for (Constraint st : constraint) { + void setSelection(VerbResolver resolver) { + for (Constraint st : constraint) { - try { - st.setSelection(resolver); - } catch (NoSuchMethodException e) { - log.error(e.getMessage()); - } catch (IllegalAccessException e) { - log.error(e.getMessage()); - } catch (InvocationTargetException e) { - log.error(e.getMessage()); - } catch (InstantiationException e) { - log.error(e.getMessage()); - } - } - } + try { + st.setSelection(resolver); + } catch (NoSuchMethodException e) { + log.error(e.getMessage()); + } catch (IllegalAccessException e) { + log.error(e.getMessage()); + } catch (InvocationTargetException e) { + log.error(e.getMessage()); + } catch (InstantiationException e) { + log.error(e.getMessage()); + } + } + } - // Constraint in and - public boolean verifyCriteria(final Map> param) { + // Constraint in and + public boolean verifyCriteria(final Map> param) { - for (Constraint sc : constraint) { - boolean verified = false; - for (String value : param.get(sc.getField())) { - if (sc.verifyCriteria(value.trim())) { - verified = true; - } - } - if (!verified) return verified; - } - return true; - } + for (Constraint sc : constraint) { + boolean verified = false; + for (String value : param.get(sc.getField())) { + if (sc.verifyCriteria(value.trim())) { + verified = true; + } + } + if (!verified) + return verified; + } + return true; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java index 5acba31d6..a3d343087 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java @@ -1,57 +1,61 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import java.io.Serializable; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.dom4j.Node; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + /** Created by miriam on 01/08/2018. */ public class Datasource implements Serializable { - private static final Log log = LogFactory.getLog(Datasource.class); + private static final Log log = LogFactory.getLog(Datasource.class); - private String openaireId; + private String openaireId; - private SelectionConstraints selectionConstraints; + private SelectionConstraints selectionConstraints; - public SelectionConstraints getSelCriteria() { - return selectionConstraints; - } + public SelectionConstraints getSelCriteria() { + return selectionConstraints; + } - public SelectionConstraints getSelectionConstraints() { - return selectionConstraints; - } + public SelectionConstraints getSelectionConstraints() { + return selectionConstraints; + } - public void setSelectionConstraints(SelectionConstraints selectionConstraints) { - this.selectionConstraints = selectionConstraints; - } + public void setSelectionConstraints(SelectionConstraints selectionConstraints) { + this.selectionConstraints = selectionConstraints; + } - public void setSelCriteria(SelectionConstraints selCriteria) { - this.selectionConstraints = selCriteria; - } + public void setSelCriteria(SelectionConstraints selCriteria) { + this.selectionConstraints = selCriteria; + } - public String getOpenaireId() { - return openaireId; - } + public String getOpenaireId() { + return openaireId; + } - public void setOpenaireId(String openaireId) { - this.openaireId = openaireId; - } + public void setOpenaireId(String openaireId) { + this.openaireId = openaireId; + } - private void setSelCriteria(String json, VerbResolver resolver) { - log.info("Selection constraints for datasource = " + json); - selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class); + private void setSelCriteria(String json, VerbResolver resolver) { + log.info("Selection constraints for datasource = " + json); + selectionConstraints = new Gson().fromJson(json, SelectionConstraints.class); - selectionConstraints.setSelection(resolver); - } + selectionConstraints.setSelection(resolver); + } - public void setSelCriteria(Node n, VerbResolver resolver) { - try { - setSelCriteria(n.getText(), resolver); - } catch (Exception e) { - log.info("not set selection criteria... "); - selectionConstraints = null; - } - } + public void setSelCriteria(Node n, VerbResolver resolver) { + try { + setSelCriteria(n.getText(), resolver); + } catch (Exception e) { + log.info("not set selection criteria... "); + selectionConstraints = null; + } + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java index 78ffe860d..01cd3ce22 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java @@ -1,37 +1,39 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; import java.io.Serializable; +import com.google.gson.Gson; + /** Created by miriam on 03/08/2018. */ public class Pair implements Serializable { - private A fst; - private B snd; + private A fst; + private B snd; - public A getFst() { - return fst; - } + public A getFst() { + return fst; + } - public Pair setFst(A fst) { - this.fst = fst; - return this; - } + public Pair setFst(A fst) { + this.fst = fst; + return this; + } - public B getSnd() { - return snd; - } + public B getSnd() { + return snd; + } - public Pair setSnd(B snd) { - this.snd = snd; - return this; - } + public Pair setSnd(B snd) { + this.snd = snd; + return this; + } - public Pair(A a, B b) { - fst = a; - snd = b; - } + public Pair(A a, B b) { + fst = a; + snd = b; + } - public String toJson() { - return new Gson().toJson(this); - } + public String toJson() { + return new Gson().toJson(this); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java index 773955d4a..d48dce2c6 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.community; import java.io.Serializable; @@ -5,7 +6,7 @@ import java.util.HashMap; public class ProtoMap extends HashMap implements Serializable { - public ProtoMap() { - super(); - } + public ProtoMap() { + super(); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java index 7dd9339ef..2c18392c7 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java @@ -1,62 +1,65 @@ + package eu.dnetlib.dhp.community; +import java.util.List; + +import org.dom4j.DocumentException; + import com.google.common.base.Joiner; + import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import java.util.List; -import org.dom4j.DocumentException; public class QueryInformationSystem { - private static final String XQUERY = - "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " - + " let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() " - + " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " - + " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " - + " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " - + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " - + " return " - + " " - + " { $x//CONFIGURATION/context/@id} " - + " " - + " {for $y in tokenize($subj,',') " - + " return " - + " {$y}} " - + " " - + " " - + " {for $d in $datasources " - + " where $d/param[./@name='enabled']/text()='true' " - + " return " - + " " - + " " - + " {$d//param[./@name='openaireId']/text()} " - + " " - + " " - + " {$d/param[./@name='selcriteria']/text()} " - + " " - + " } " - + " " - + " " - + " {for $zc in $communities " - + " return " - + " " - + " " - + " {$zc/param[./@name='zenodoid']/text()} " - + " " - + " " - + " {$zc/param[./@name='selcriteria']/text()} " - + " " - + " } " - + " " - + " "; + private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " + + " let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() " + + " let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept " + + " let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept " + + " let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept " + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] " + + " return " + + " " + + " { $x//CONFIGURATION/context/@id} " + + " " + + " {for $y in tokenize($subj,',') " + + " return " + + " {$y}} " + + " " + + " " + + " {for $d in $datasources " + + " where $d/param[./@name='enabled']/text()='true' " + + " return " + + " " + + " " + + " {$d//param[./@name='openaireId']/text()} " + + " " + + " " + + " {$d/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " " + + " {for $zc in $communities " + + " return " + + " " + + " " + + " {$zc/param[./@name='zenodoid']/text()} " + + " " + + " " + + " {$zc/param[./@name='selcriteria']/text()} " + + " " + + " } " + + " " + + " "; - public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) - throws ISLookUpException, DocumentException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - final List res = isLookUp.quickSearchProfile(XQUERY); + public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) + throws ISLookUpException, DocumentException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + final List res = isLookUp.quickSearchProfile(XQUERY); - final String xmlConf = "" + Joiner.on(" ").join(res) + ""; + final String xmlConf = "" + Joiner.on(" ").join(res) + ""; - return CommunityConfigurationFactory.newInstance(xmlConf); - } + return CommunityConfigurationFactory.newInstance(xmlConf); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java index 8752a4c57..eb531c6b1 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java @@ -1,224 +1,246 @@ + package eu.dnetlib.dhp.community; import static eu.dnetlib.dhp.community.TagginConstants.*; -import com.google.gson.Gson; -import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; + import org.apache.commons.lang3.StringUtils; +import com.google.gson.Gson; +import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.JsonPath; + +import eu.dnetlib.dhp.schema.oaf.*; + /** Created by miriam on 02/08/2018. */ public class ResultTagger implements Serializable { - private String trust = "0.8"; + private String trust = "0.8"; - private boolean clearContext(Result result) { - int tmp = result.getContext().size(); - List clist = - result.getContext().stream() - .filter(c -> (!c.getId().contains(ZENODO_COMMUNITY_INDICATOR))) - .collect(Collectors.toList()); - result.setContext(clist); - return (tmp != clist.size()); - } + private boolean clearContext(Result result) { + int tmp = result.getContext().size(); + List clist = result + .getContext() + .stream() + .filter(c -> (!c.getId().contains(ZENODO_COMMUNITY_INDICATOR))) + .collect(Collectors.toList()); + result.setContext(clist); + return (tmp != clist.size()); + } - private Map> getParamMap(final Result result, Map params) { - Map> param = new HashMap<>(); - String json = new Gson().toJson(result, Result.class); - DocumentContext jsonContext = JsonPath.parse(json); - if (params == null) { - params = new HashMap<>(); - } - for (String key : params.keySet()) { - try { - param.put(key, jsonContext.read(params.get(key))); - } catch (com.jayway.jsonpath.PathNotFoundException e) { - param.put(key, new ArrayList<>()); - // throw e; - } - } - return param; - } + private Map> getParamMap(final Result result, Map params) { + Map> param = new HashMap<>(); + String json = new Gson().toJson(result, Result.class); + DocumentContext jsonContext = JsonPath.parse(json); + if (params == null) { + params = new HashMap<>(); + } + for (String key : params.keySet()) { + try { + param.put(key, jsonContext.read(params.get(key))); + } catch (com.jayway.jsonpath.PathNotFoundException e) { + param.put(key, new ArrayList<>()); + // throw e; + } + } + return param; + } - public R enrichContextCriteria( - final R result, final CommunityConfiguration conf, final Map criteria) { + public R enrichContextCriteria( + final R result, final CommunityConfiguration conf, final Map criteria) { - // } - // public Result enrichContextCriteria(final Result result, final CommunityConfiguration - // conf, final Map criteria) { - final Map> param = getParamMap(result, criteria); + // } + // public Result enrichContextCriteria(final Result result, final CommunityConfiguration + // conf, final Map criteria) { + final Map> param = getParamMap(result, criteria); - // Verify if the entity is deletedbyinference. In case verify if to clean the context list - // from all the zenodo communities - if (result.getDataInfo().getDeletedbyinference()) { - clearContext(result); - return result; - } + // Verify if the entity is deletedbyinference. In case verify if to clean the context list + // from all the zenodo communities + if (result.getDataInfo().getDeletedbyinference()) { + clearContext(result); + return result; + } - // communities contains all the communities to be added as context for the result - final Set communities = new HashSet<>(); + // communities contains all the communities to be added as context for the result + final Set communities = new HashSet<>(); - // tagging for Subject - final Set subjects = new HashSet<>(); - Optional> oresultsubj = Optional.ofNullable(result.getSubject()); - if (oresultsubj.isPresent()) { - oresultsubj.get().stream() - .map(subject -> subject.getValue()) - .filter(StringUtils::isNotBlank) - .map(String::toLowerCase) - .map(String::trim) - .collect(Collectors.toCollection(HashSet::new)) - .forEach(s -> subjects.addAll(conf.getCommunityForSubjectValue(s))); - } + // tagging for Subject + final Set subjects = new HashSet<>(); + Optional> oresultsubj = Optional.ofNullable(result.getSubject()); + if (oresultsubj.isPresent()) { + oresultsubj + .get() + .stream() + .map(subject -> subject.getValue()) + .filter(StringUtils::isNotBlank) + .map(String::toLowerCase) + .map(String::trim) + .collect(Collectors.toCollection(HashSet::new)) + .forEach(s -> subjects.addAll(conf.getCommunityForSubjectValue(s))); + } - communities.addAll(subjects); + communities.addAll(subjects); - // Tagging for datasource - final Set datasources = new HashSet<>(); - final Set tmp = new HashSet<>(); + // Tagging for datasource + final Set datasources = new HashSet<>(); + final Set tmp = new HashSet<>(); - Optional> oresultinstance = Optional.ofNullable(result.getInstance()); - if (oresultinstance.isPresent()) { - for (Instance i : oresultinstance.get()) { - tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); - tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); - } + Optional> oresultinstance = Optional.ofNullable(result.getInstance()); + if (oresultinstance.isPresent()) { + for (Instance i : oresultinstance.get()) { + tmp.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); + tmp.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); + } - oresultinstance.get().stream() - .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) - .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) - .map(s -> StringUtils.substringAfter(s, "|")) - .collect(Collectors.toCollection(HashSet::new)) - .forEach( - dsId -> - datasources.addAll( - conf.getCommunityForDatasource(dsId, param))); - } + oresultinstance + .get() + .stream() + .map(i -> new Pair<>(i.getCollectedfrom().getKey(), i.getHostedby().getKey())) + .flatMap(p -> Stream.of(p.getFst(), p.getSnd())) + .map(s -> StringUtils.substringAfter(s, "|")) + .collect(Collectors.toCollection(HashSet::new)) + .forEach( + dsId -> datasources + .addAll( + conf.getCommunityForDatasource(dsId, param))); + } - communities.addAll(datasources); + communities.addAll(datasources); - /*Tagging for Zenodo Communities*/ - final Set czenodo = new HashSet<>(); + /* Tagging for Zenodo Communities */ + final Set czenodo = new HashSet<>(); - Optional> oresultcontext = Optional.ofNullable(result.getContext()); - if (oresultcontext.isPresent()) { - oresultcontext.get().stream() - .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) - .collect(Collectors.toList()) - .forEach( - c -> - czenodo.addAll( - conf.getCommunityForZenodoCommunityValue( - c.getId() - .substring( - c.getId().lastIndexOf("/") + 1) - .trim()))); - } + Optional> oresultcontext = Optional.ofNullable(result.getContext()); + if (oresultcontext.isPresent()) { + oresultcontext + .get() + .stream() + .filter(c -> c.getId().contains(ZENODO_COMMUNITY_INDICATOR)) + .collect(Collectors.toList()) + .forEach( + c -> czenodo + .addAll( + conf + .getCommunityForZenodoCommunityValue( + c + .getId() + .substring( + c.getId().lastIndexOf("/") + 1) + .trim()))); + } - communities.addAll(czenodo); + communities.addAll(czenodo); - clearContext(result); + clearContext(result); - /*Verify if there is something to bulktag*/ - if (communities.isEmpty()) { - return result; - } + /* Verify if there is something to bulktag */ + if (communities.isEmpty()) { + return result; + } - result.getContext().stream() - .map( - c -> { - if (communities.contains(c.getId())) { - Optional> opt_dataInfoList = - Optional.ofNullable(c.getDataInfo()); - List dataInfoList; - if (opt_dataInfoList.isPresent()) - dataInfoList = opt_dataInfoList.get(); - else { - dataInfoList = new ArrayList<>(); - c.setDataInfo(dataInfoList); - } - if (subjects.contains(c.getId())) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_SUBJECT, - CLASS_NAME_BULKTAG_SUBJECT)); - if (datasources.contains(c.getId())) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_DATASOURCE, - CLASS_NAME_BULKTAG_DATASOURCE)); - if (czenodo.contains(c.getId())) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_CZENODO, - CLASS_NAME_BULKTAG_ZENODO)); - } - return c; - }) - .collect(Collectors.toList()); + result + .getContext() + .stream() + .map( + c -> { + if (communities.contains(c.getId())) { + Optional> opt_dataInfoList = Optional.ofNullable(c.getDataInfo()); + List dataInfoList; + if (opt_dataInfoList.isPresent()) + dataInfoList = opt_dataInfoList.get(); + else { + dataInfoList = new ArrayList<>(); + c.setDataInfo(dataInfoList); + } + if (subjects.contains(c.getId())) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_SUBJECT, + CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c.getId())) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_DATASOURCE, + CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c.getId())) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_CZENODO, + CLASS_NAME_BULKTAG_ZENODO)); + } + return c; + }) + .collect(Collectors.toList()); - communities.removeAll( - result.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet())); + communities + .removeAll( + result.getContext().stream().map(c -> c.getId()).collect(Collectors.toSet())); - if (communities.isEmpty()) return result; + if (communities.isEmpty()) + return result; - List toaddcontext = - communities.stream() - .map( - c -> { - Context context = new Context(); - context.setId(c); - List dataInfoList = new ArrayList<>(); - if (subjects.contains(c)) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_SUBJECT, - CLASS_NAME_BULKTAG_SUBJECT)); - if (datasources.contains(c)) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_DATASOURCE, - CLASS_NAME_BULKTAG_DATASOURCE)); - if (czenodo.contains(c)) - dataInfoList.add( - getDataInfo( - BULKTAG_DATA_INFO_TYPE, - CLASS_ID_CZENODO, - CLASS_NAME_BULKTAG_ZENODO)); - context.setDataInfo(dataInfoList); - return context; - }) - .collect(Collectors.toList()); + List toaddcontext = communities + .stream() + .map( + c -> { + Context context = new Context(); + context.setId(c); + List dataInfoList = new ArrayList<>(); + if (subjects.contains(c)) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_SUBJECT, + CLASS_NAME_BULKTAG_SUBJECT)); + if (datasources.contains(c)) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_DATASOURCE, + CLASS_NAME_BULKTAG_DATASOURCE)); + if (czenodo.contains(c)) + dataInfoList + .add( + getDataInfo( + BULKTAG_DATA_INFO_TYPE, + CLASS_ID_CZENODO, + CLASS_NAME_BULKTAG_ZENODO)); + context.setDataInfo(dataInfoList); + return context; + }) + .collect(Collectors.toList()); - result.getContext().addAll(toaddcontext); - return result; - } + result.getContext().addAll(toaddcontext); + return result; + } - public static DataInfo getDataInfo( - String inference_provenance, String inference_class_id, String inference_class_name) { - DataInfo di = new DataInfo(); - di.setInferred(true); - di.setInferenceprovenance(inference_provenance); - di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); - return di; - } + public static DataInfo getDataInfo( + String inference_provenance, String inference_class_id, String inference_class_name) { + DataInfo di = new DataInfo(); + di.setInferred(true); + di.setInferenceprovenance(inference_provenance); + di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); + return di; + } - public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { - Qualifier pa = new Qualifier(); - pa.setClassid(inference_class_id); - pa.setClassname(inference_class_name); - pa.setSchemeid(DNET_SCHEMA_ID); - pa.setSchemename(DNET_SCHEMA_NAME); - return pa; - } + public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { + Qualifier pa = new Qualifier(); + pa.setClassid(inference_class_id); + pa.setClassname(inference_class_name); + pa.setSchemeid(DNET_SCHEMA_ID); + pa.setSchemename(DNET_SCHEMA_NAME); + return pa; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java index 530861425..802e2f5d6 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java @@ -1,46 +1,51 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import java.io.Serializable; import java.lang.reflect.Type; import java.util.Collection; import java.util.List; import java.util.Map; +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; + +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + public class SelectionConstraints implements Serializable { - private List criteria; + private List criteria; - public SelectionConstraints() {} + public SelectionConstraints() { + } - public List getCriteria() { - return criteria; - } + public List getCriteria() { + return criteria; + } - public void setCriteria(List criteria) { - this.criteria = criteria; - } + public void setCriteria(List criteria) { + this.criteria = criteria; + } - public void setSc(String json) { - Type collectionType = new TypeToken>() {}.getType(); - criteria = new Gson().fromJson(json, collectionType); - } + public void setSc(String json) { + Type collectionType = new TypeToken>() { + }.getType(); + criteria = new Gson().fromJson(json, collectionType); + } - // Constraints in or - public boolean verifyCriteria(final Map> param) { - for (Constraints selc : criteria) { - if (selc.verifyCriteria(param)) { - return true; - } - } - return false; - } + // Constraints in or + public boolean verifyCriteria(final Map> param) { + for (Constraints selc : criteria) { + if (selc.verifyCriteria(param)) { + return true; + } + } + return false; + } - public void setSelection(VerbResolver resolver) { + public void setSelection(VerbResolver resolver) { - for (Constraints cs : criteria) { - cs.setSelection(resolver); - } - } + for (Constraints cs : criteria) { + cs.setSelection(resolver); + } + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java index 9f681472a..92d37d089 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java @@ -1,23 +1,23 @@ + package eu.dnetlib.dhp.community; public class TagginConstants { - public static final String BULKTAG_DATA_INFO_TYPE = "bulktagging"; + public static final String BULKTAG_DATA_INFO_TYPE = "bulktagging"; - public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; - public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; + public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; - public static final String CLASS_ID_SUBJECT = "community:subject"; - public static final String CLASS_ID_DATASOURCE = "community:datasource"; - public static final String CLASS_ID_CZENODO = "community:zenodocommunity"; + public static final String CLASS_ID_SUBJECT = "community:subject"; + public static final String CLASS_ID_DATASOURCE = "community:datasource"; + public static final String CLASS_ID_CZENODO = "community:zenodocommunity"; - public static final String SCHEMA_ID = "dnet:provenanceActions"; - public static final String COUNTER_GROUP = "Bulk Tagging"; + public static final String SCHEMA_ID = "dnet:provenanceActions"; + public static final String COUNTER_GROUP = "Bulk Tagging"; - public static final String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; + public static final String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; - public static final String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; - public static final String CLASS_NAME_BULKTAG_DATASOURCE = - "Bulktagging for Community - Datasource"; - public static final String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; + public static final String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; + public static final String CLASS_NAME_BULKTAG_DATASOURCE = "Bulktagging for Community - Datasource"; + public static final String CLASS_NAME_BULKTAG_ZENODO = "Bulktagging for Community - Zenodo"; } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java index 19d97d221..e1492f6a5 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java @@ -1,42 +1,45 @@ + package eu.dnetlib.dhp.community; -import com.google.gson.Gson; import java.io.Serializable; + import org.dom4j.Node; +import com.google.gson.Gson; + /** Created by miriam on 01/08/2018. */ public class ZenodoCommunity implements Serializable { - private String zenodoCommunityId; + private String zenodoCommunityId; - private SelectionConstraints selCriteria; + private SelectionConstraints selCriteria; - public String getZenodoCommunityId() { - return zenodoCommunityId; - } + public String getZenodoCommunityId() { + return zenodoCommunityId; + } - public void setZenodoCommunityId(String zenodoCommunityId) { - this.zenodoCommunityId = zenodoCommunityId; - } + public void setZenodoCommunityId(String zenodoCommunityId) { + this.zenodoCommunityId = zenodoCommunityId; + } - public SelectionConstraints getSelCriteria() { - return selCriteria; - } + public SelectionConstraints getSelCriteria() { + return selCriteria; + } - public void setSelCriteria(SelectionConstraints selCriteria) { - this.selCriteria = selCriteria; - } + public void setSelCriteria(SelectionConstraints selCriteria) { + this.selCriteria = selCriteria; + } - private void setSelCriteria(String json) { - // Type collectionType = new TypeToken>(){}.getType(); - selCriteria = new Gson().fromJson(json, SelectionConstraints.class); - } + private void setSelCriteria(String json) { + // Type collectionType = new TypeToken>(){}.getType(); + selCriteria = new Gson().fromJson(json, SelectionConstraints.class); + } - public void setSelCriteria(Node n) { - if (n == null) { - selCriteria = null; - } else { - setSelCriteria(n.getText()); - } - } + public void setSelCriteria(Node n) { + if (n == null) { + selCriteria = null; + } else { + setSelCriteria(n.getText()); + } + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java index fc6456a8c..a6ef2d908 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("contains") public class ContainsVerb implements Selection, Serializable { - private String param; + private String param; - public ContainsVerb() {} + public ContainsVerb() { + } - public ContainsVerb(final String param) { - this.param = param; - } + public ContainsVerb(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return value.contains(param); - } + @Override + public boolean apply(String value) { + return value.contains(param); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java index d5651e5b8..b8b0262e9 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("contains_ignorecase") public class ContainsVerbIgnoreCase implements Selection, Serializable { - private String param; + private String param; - public ContainsVerbIgnoreCase() {} + public ContainsVerbIgnoreCase() { + } - public ContainsVerbIgnoreCase(final String param) { - this.param = param; - } + public ContainsVerbIgnoreCase(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return value.toLowerCase().contains(param.toLowerCase()); - } + @Override + public boolean apply(String value) { + return value.toLowerCase().contains(param.toLowerCase()); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java index cc793f8f8..3f17a6bb3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("equals") public class EqualVerb implements Selection, Serializable { - private String param; + private String param; - public EqualVerb() {} + public EqualVerb() { + } - public EqualVerb(final String param) { - this.param = param; - } + public EqualVerb(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return value.equals(param); - } + @Override + public boolean apply(String value) { + return value.equals(param); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java index 26c636090..934406859 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("equals_ignorecase") public class EqualVerbIgnoreCase implements Selection, Serializable { - private String param; + private String param; - public EqualVerbIgnoreCase() {} + public EqualVerbIgnoreCase() { + } - public EqualVerbIgnoreCase(final String param) { - this.param = param; - } + public EqualVerbIgnoreCase(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return value.equalsIgnoreCase(param); - } + @Override + public boolean apply(String value) { + return value.equalsIgnoreCase(param); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java index a4a1494b3..9ef3bd60c 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java @@ -1,40 +1,43 @@ + package eu.dnetlib.dhp.selectioncriteria; -import com.google.gson.*; import java.lang.reflect.Type; +import com.google.gson.*; + public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { - private static final String CLASSNAME = "CLASSNAME"; - private static final String DATA = "DATA"; + private static final String CLASSNAME = "CLASSNAME"; + private static final String DATA = "DATA"; - public Object deserialize( - JsonElement jsonElement, - Type type, - JsonDeserializationContext jsonDeserializationContext) - throws JsonParseException { + public Object deserialize( + JsonElement jsonElement, + Type type, + JsonDeserializationContext jsonDeserializationContext) + throws JsonParseException { - JsonObject jsonObject = jsonElement.getAsJsonObject(); - JsonPrimitive prim = (JsonPrimitive) jsonObject.get(CLASSNAME); - String className = prim.getAsString(); - Class klass = getObjectClass(className); - return jsonDeserializationContext.deserialize(jsonObject.get(DATA), klass); - } + JsonObject jsonObject = jsonElement.getAsJsonObject(); + JsonPrimitive prim = (JsonPrimitive) jsonObject.get(CLASSNAME); + String className = prim.getAsString(); + Class klass = getObjectClass(className); + return jsonDeserializationContext.deserialize(jsonObject.get(DATA), klass); + } - public JsonElement serialize( - Object jsonElement, Type type, JsonSerializationContext jsonSerializationContext) { - JsonObject jsonObject = new JsonObject(); - jsonObject.addProperty(CLASSNAME, jsonElement.getClass().getName()); - jsonObject.add(DATA, jsonSerializationContext.serialize(jsonElement)); - return jsonObject; - } - /** **** Helper method to get the className of the object to be deserialized **** */ - public Class getObjectClass(String className) { - try { - return Class.forName(className); - } catch (ClassNotFoundException e) { - // e.printStackTrace(); - throw new JsonParseException(e.getMessage()); - } - } + public JsonElement serialize( + Object jsonElement, Type type, JsonSerializationContext jsonSerializationContext) { + JsonObject jsonObject = new JsonObject(); + jsonObject.addProperty(CLASSNAME, jsonElement.getClass().getName()); + jsonObject.add(DATA, jsonSerializationContext.serialize(jsonElement)); + return jsonObject; + } + + /** **** Helper method to get the className of the object to be deserialized **** */ + public Class getObjectClass(String className) { + try { + return Class.forName(className); + } catch (ClassNotFoundException e) { + // e.printStackTrace(); + throw new JsonParseException(e.getMessage()); + } + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java index f07540d35..eb83b256e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("not_contains") public class NotContainsVerb implements Selection, Serializable { - private String param; + private String param; - public NotContainsVerb() {} + public NotContainsVerb() { + } - public NotContainsVerb(final String param) { - this.param = param; - } + public NotContainsVerb(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return !value.contains(param); - } + @Override + public boolean apply(String value) { + return !value.contains(param); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java index e26b4a1a3..fab3efef3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("not_contains_ignorecase") public class NotContainsVerbIgnoreCase implements Selection, Serializable { - private String param; + private String param; - public NotContainsVerbIgnoreCase() {} + public NotContainsVerbIgnoreCase() { + } - public NotContainsVerbIgnoreCase(final String param) { - this.param = param; - } + public NotContainsVerbIgnoreCase(final String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return !(value.toLowerCase().contains(param.toLowerCase())); - } + @Override + public boolean apply(String value) { + return !(value.toLowerCase().contains(param.toLowerCase())); + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java index 1946812f5..2311c2987 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("not_equals") public class NotEqualVerb implements Selection, Serializable { - private String param; + private String param; - public NotEqualVerb(final String param) { - this.param = param; - } + public NotEqualVerb(final String param) { + this.param = param; + } - public NotEqualVerb() {} + public NotEqualVerb() { + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return !value.equals(param); - } + @Override + public boolean apply(String value) { + return !value.equals(param); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java index d669fa6ac..de2f682a5 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.io.Serializable; @@ -5,24 +6,25 @@ import java.io.Serializable; @VerbClass("not_equals_ignorecase") public class NotEqualVerbIgnoreCase implements Selection, Serializable { - private String param; + private String param; - public NotEqualVerbIgnoreCase(final String param) { - this.param = param; - } + public NotEqualVerbIgnoreCase(final String param) { + this.param = param; + } - public NotEqualVerbIgnoreCase() {} + public NotEqualVerbIgnoreCase() { + } - public String getParam() { - return param; - } + public String getParam() { + return param; + } - public void setParam(String param) { - this.param = param; - } + public void setParam(String param) { + this.param = param; + } - @Override - public boolean apply(String value) { - return !value.equalsIgnoreCase(param); - } + @Override + public boolean apply(String value) { + return !value.equalsIgnoreCase(param); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java index c9b30790f..b488bda01 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java @@ -1,6 +1,7 @@ + package eu.dnetlib.dhp.selectioncriteria; public interface Selection { - boolean apply(String value); + boolean apply(String value); } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java index 9a5fe4e8a..d467f934f 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.selectioncriteria; import java.lang.annotation.ElementType; @@ -9,5 +10,5 @@ import java.lang.annotation.Target; @Target(ElementType.TYPE) @interface VerbClass { - String value(); + String value(); } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java index c2a9b4544..6a8ceebc3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java @@ -1,57 +1,56 @@ + package eu.dnetlib.dhp.selectioncriteria; -import io.github.classgraph.ClassGraph; -import io.github.classgraph.ClassInfo; -import io.github.classgraph.ClassInfoList; -import io.github.classgraph.ScanResult; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.util.Map; import java.util.stream.Collectors; +import io.github.classgraph.ClassGraph; +import io.github.classgraph.ClassInfo; +import io.github.classgraph.ClassInfoList; +import io.github.classgraph.ScanResult; + public class VerbResolver implements Serializable { - private Map> map = null; // = new HashMap<>(); - private final ClassGraph classgraph = new ClassGraph(); + private Map> map = null; // = new HashMap<>(); + private final ClassGraph classgraph = new ClassGraph(); - public VerbResolver() { + public VerbResolver() { - try (ScanResult scanResult = // Assign scanResult in try-with-resources - classgraph // Create a new ClassGraph instance - .verbose() // If you want to enable logging to stderr - .enableAllInfo() // Scan classes, methods, fields, annotations - .whitelistPackages( - "eu.dnetlib.dhp.selectioncriteria") // Scan com.xyz and subpackages - .scan()) { // Perform the scan and return a ScanResult + try (ScanResult scanResult = // Assign scanResult in try-with-resources + classgraph // Create a new ClassGraph instance + .verbose() // If you want to enable logging to stderr + .enableAllInfo() // Scan classes, methods, fields, annotations + .whitelistPackages( + "eu.dnetlib.dhp.selectioncriteria") // Scan com.xyz and subpackages + .scan()) { // Perform the scan and return a ScanResult - ClassInfoList routeClassInfoList = - scanResult.getClassesWithAnnotation( - "eu.dnetlib.dhp.selectioncriteria.VerbClass"); + ClassInfoList routeClassInfoList = scanResult + .getClassesWithAnnotation( + "eu.dnetlib.dhp.selectioncriteria.VerbClass"); - this.map = - routeClassInfoList.stream() - .collect( - Collectors.toMap( - value -> - (String) - ((ClassInfo) value) - .getAnnotationInfo() - .get(0) - .getParameterValues() - .get(0) - .getValue(), - value -> - (Class) - ((ClassInfo) value).loadClass())); - } catch (Exception e) { - e.printStackTrace(); - } - } + this.map = routeClassInfoList + .stream() + .collect( + Collectors + .toMap( + value -> (String) ((ClassInfo) value) + .getAnnotationInfo() + .get(0) + .getParameterValues() + .get(0) + .getValue(), + value -> (Class) ((ClassInfo) value).loadClass())); + } catch (Exception e) { + e.printStackTrace(); + } + } - public Selection getSelectionCriteria(String name, String param) - throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, - InstantiationException { + public Selection getSelectionCriteria(String name, String param) + throws NoSuchMethodException, IllegalAccessException, InvocationTargetException, + InstantiationException { - // return Class.forName(tmp_map.get(name)). - return map.get(name).getDeclaredConstructor((String.class)).newInstance(param); - } + // return Class.forName(tmp_map.get(name)). + return map.get(name).getDeclaredConstructor((String.class)).newInstance(param); + } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java index c878c7f58..58bf60d42 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java @@ -1,9 +1,10 @@ + package eu.dnetlib.dhp.selectioncriteria; public class VerbResolverFactory { - public static VerbResolver newInstance() { + public static VerbResolver newInstance() { - return new VerbResolver(); - } + return new VerbResolver(); + } } diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index 04c8816df..6873f2df9 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -1,16 +1,12 @@ + package eu.dnetlib.dhp; import static eu.dnetlib.dhp.community.TagginConstants.ZENODO_COMMUNITY_INDICATOR; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.bulktag.SparkBulkTagJob2; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Software; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -26,778 +22,832 @@ import org.mortbay.util.IO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.bulktag.SparkBulkTagJob2; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Software; + public class BulkTagJobTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private static final ClassLoader cl = eu.dnetlib.dhp.BulkTagJobTest.class.getClassLoader(); - - private static SparkSession spark; - - private static Path workingDir; - private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.BulkTagJobTest.class); - - private static String taggingConf = ""; - - static { - try { - taggingConf = - IO.toString( - BulkTagJobTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); - } catch (IOException e) { - e.printStackTrace(); - } - } - - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); - - SparkConf conf = new SparkConf(); - conf.setAppName(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); - - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - - spark = - SparkSession.builder() - .appName(BulkTagJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } - - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } - - @Test - public void noUpdatesTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/sample/dataset/no_updates").getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - // "-preparedInfoPath", - // getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - - verificationDataset.createOrReplaceTempView("dataset"); - - String query = - "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - Assertions.assertEquals(0, spark.sql(query).count()); - } - - @Test - public void bulktagBySubjectNoPreviousContextTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - - verificationDataset.createOrReplaceTempView("dataset"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - Assertions.assertEquals(5, spark.sql(query).count()); - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - Assertions.assertEquals( - 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); - Assertions.assertEquals( - 5, - idExplodeCommunity.filter("name = 'Bulktagging for Community - Subject'").count()); - - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); - - Assertions.assertEquals( - 1, - idExplodeCommunity - .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") - .count()); - Assertions.assertEquals( - 1, - idExplodeCommunity - .filter( - "community = 'covid-19' and id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") - .count()); - - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter("id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") - .count()); - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter( - "(community = 'covid-19' or community = 'aginfra') and id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") - .count()); - - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter("id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") - .count()); - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter( - "(community = 'mes' or community = 'fam') and id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") - .count()); - } - - @Test - public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - - verificationDataset.createOrReplaceTempView("dataset"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyT.id = 'covid-19' "; - - Assertions.assertEquals(3, spark.sql(query).count()); - - org.apache.spark.sql.Dataset communityContext = spark.sql(query); - - Assertions.assertEquals( - 2, - communityContext - .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") - .count()); - Assertions.assertEquals( - 1, - communityContext - .filter( - "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'community:subject'") - .count()); - Assertions.assertEquals( - 1, - communityContext - .filter( - "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'propagation:community:productsthroughsemrel'") - .count()); - - query = - "select id, MyT.id community, size(MyT.datainfo) datainfosize " - + "from dataset " - + "lateral view explode (context) as MyT " - + "where size(MyT.datainfo) > 0"; - - Assertions.assertEquals( - 2, - spark.sql(query) - .select("datainfosize") - .where( - "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' a" - + "nd community = 'covid-19'") - .collectAsList() - .get(0) - .getInt(0)); - } - - @Test - public void bulktagByDatasourceTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/publication/update_datasource") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Publication", - "-outputPath", - workingDir.toString() + "/publication", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/publication") - .map(item -> OBJECT_MAPPER.readValue(item, Publication.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Publication.class)); - - verificationDataset.createOrReplaceTempView("publication"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from publication " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - - Assertions.assertEquals(5, idExplodeCommunity.count()); - Assertions.assertEquals( - 5, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); - Assertions.assertEquals( - 5, - idExplodeCommunity - .filter("name = 'Bulktagging for Community - Datasource'") - .count()); - - Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'fam'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); - - Assertions.assertEquals( - 3, - idExplodeCommunity - .filter( - "community = 'fam' and (id = '50|ec_fp7health::000085c89f4b96dc2269bd37edb35306' " - + "or id = '50|ec_fp7health::000b9e61f83f5a4b0c35777b7bccdf38' " - + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") - .count()); - - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter( - "community = 'aginfra' and (id = '50|ec_fp7health::000c8195edd542e4e64ebb32172cbf89' " - + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") - .count()); - } - - @Test - public void bulktagByZenodoCommunityTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", - workingDir.toString() + "/orp", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/orp") - .map(item -> OBJECT_MAPPER.readValue(item, OtherResearchProduct.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(OtherResearchProduct.class)); - - verificationDataset.createOrReplaceTempView("orp"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from orp " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - Assertions.assertEquals(8, idExplodeCommunity.count()); - - Assertions.assertEquals( - 8, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); - Assertions.assertEquals( - 8, - idExplodeCommunity.filter("name = 'Bulktagging for Community - Zenodo'").count()); - - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'covid-19'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'beopen'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'mes'").count()); - - Assertions.assertEquals( - 1, - idExplodeCommunity - .filter( - "id = '50|od______2017::0750a4d0782265873d669520f5e33c07' " - + "and community = 'covid-19'") - .count()); - Assertions.assertEquals( - 3, - idExplodeCommunity - .filter( - "id = '50|od______2017::1bd97baef19dbd2db3203b112bb83bc5' and " - + "(community = 'aginfra' or community = 'mes' or community = 'fam')") - .count()); - Assertions.assertEquals( - 1, - idExplodeCommunity - .filter( - "id = '50|od______2017::1e400f1747487fd15998735c41a55c72' " - + "and community = 'beopen'") - .count()); - Assertions.assertEquals( - 3, - idExplodeCommunity - .filter( - "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' and " - + "(community = 'beopen' or community = 'fam' or community = 'mes')") - .count()); - - query = - "select id, MyT.id community, size(MyT.datainfo) datainfosize " - + "from orp " - + "lateral view explode (context) as MyT " - + "where size(MyT.datainfo) > 0"; - - Assertions.assertEquals( - 2, - spark.sql(query) - .select("datainfosize") - .where( - "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' a" - + "nd community = 'beopen'") - .collectAsList() - .get(0) - .getInt(0)); - - // verify the zenodo community context is not present anymore in the records - query = - "select id, MyT.id community " - + "from orp " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD "; - - org.apache.spark.sql.Dataset tmp2 = spark.sql(query); - - Assertions.assertEquals( - 0, - tmp2.select("community") - .where(tmp2.col("community").contains(ZENODO_COMMUNITY_INDICATOR)) - .count()); - } - - @Test - public void bulktagBySubjectDatasourceTest() throws Exception { - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject_datasource") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - - verificationDataset.createOrReplaceTempView("dataset"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - Assertions.assertEquals(7, idExplodeCommunity.count()); - - Assertions.assertEquals( - 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); - Assertions.assertEquals( - 2, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); - Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); - - query = - "select id, MyT.id community, size(MyT.datainfo) datainfosize " - + "from dataset " - + "lateral view explode (context) as MyT " - + "where size(MyT.datainfo) > 0"; - - org.apache.spark.sql.Dataset tmp2 = spark.sql(query); - - Assertions.assertEquals( - 2, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " - + "community = 'aginfra'") - .collectAsList() - .get(0) - .getInt(0)); - - Assertions.assertEquals( - 1, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " - + "community = 'covid-19'") - .collectAsList() - .get(0) - .getInt(0)); - - Assertions.assertEquals( - 2, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " - + "community = 'fam'") - .collectAsList() - .get(0) - .getInt(0)); - Assertions.assertEquals( - 2, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " - + "community = 'covid-19'") - .collectAsList() - .get(0) - .getInt(0)); - - Assertions.assertEquals( - 1, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " - + "community = 'fam'") - .collectAsList() - .get(0) - .getInt(0)); - Assertions.assertEquals( - 1, - tmp2.select("datainfosize") - .where( - "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " - + "community = 'mes'") - .collectAsList() - .get(0) - .getInt(0)); - } - - @Test - public void bulktagBySubjectDatasourceZenodoCommunityTest() throws Exception { - - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/sample/software/").getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", - workingDir.toString() + "/software", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/software") - .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); - - verificationDataset.createOrReplaceTempView("software"); - - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from software " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - Assertions.assertEquals(10, idExplodeCommunity.count()); - - idExplodeCommunity.show(false); - Assertions.assertEquals( - 3, idExplodeCommunity.filter("provenance = 'community:subject'").count()); - Assertions.assertEquals( - 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); - Assertions.assertEquals( - 4, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); - - Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'covid-19'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dh-ch'").count()); - Assertions.assertEquals(4, idExplodeCommunity.filter("community = 'aginfra'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dariah'").count()); - Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); - - Assertions.assertEquals( - 2, - idExplodeCommunity - .filter( - "provenance = 'community:zenodocommunity' and " - + "id = '50|od______1582::4132f5ec9496f0d6adc7b00a50a56ff4' and (" - + "community = 'dh-ch' or community = 'dariah')") - .count()); - - query = - "select id, MyT.id community, size(MyT.datainfo) datainfosize " - + "from software " - + "lateral view explode (context) as MyT " - + "where size(MyT.datainfo) > 0"; - - org.apache.spark.sql.Dataset tmp2 = spark.sql(query); - - Assertions.assertEquals( - 2, - tmp2.select("datainfosize") - .where( - "id = '50|od______1582::501b25d420f808c8eddcd9b16e917f11' and " - + "community = 'covid-19'") - .collectAsList() - .get(0) - .getInt(0)); - - Assertions.assertEquals( - 3, - tmp2.select("datainfosize") - .where( - "id = '50|od______1582::581621232a561b7e8b4952b18b8b0e56' and " - + "community = 'aginfra'") - .collectAsList() - .get(0) - .getInt(0)); - } - - @Test - public void bulktagDatasourcewithConstraintsTest() throws Exception { - - SparkBulkTagJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookupUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - }); - - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - - verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; - - org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); - - idExplodeCommunity.show(false); - Assertions.assertEquals(3, idExplodeCommunity.count()); - - Assertions.assertEquals( - 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); - } + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = eu.dnetlib.dhp.BulkTagJobTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.BulkTagJobTest.class); + + private static String taggingConf = ""; + + static { + try { + taggingConf = IO + .toString( + BulkTagJobTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); + } catch (IOException e) { + e.printStackTrace(); + } + } + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(BulkTagJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void noUpdatesTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/sample/dataset/no_updates").getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + // "-preparedInfoPath", + // getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + Assertions.assertEquals(0, spark.sql(query).count()); + } + + @Test + public void bulktagBySubjectNoPreviousContextTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + Assertions.assertEquals(5, spark.sql(query).count()); + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions + .assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions + .assertEquals( + 5, + idExplodeCommunity.filter("name = 'Bulktagging for Community - Subject'").count()); + + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); + + Assertions + .assertEquals( + 1, + idExplodeCommunity + .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + Assertions + .assertEquals( + 1, + idExplodeCommunity + .filter( + "community = 'covid-19' and id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter("id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") + .count()); + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter( + "(community = 'covid-19' or community = 'aginfra') and id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b'") + .count()); + + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter("id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") + .count()); + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter( + "(community = 'mes' or community = 'fam') and id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62'") + .count()); + } + + @Test + public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyT.id = 'covid-19' "; + + Assertions.assertEquals(3, spark.sql(query).count()); + + org.apache.spark.sql.Dataset communityContext = spark.sql(query); + + Assertions + .assertEquals( + 2, + communityContext + .filter("id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529'") + .count()); + Assertions + .assertEquals( + 1, + communityContext + .filter( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'community:subject'") + .count()); + Assertions + .assertEquals( + 1, + communityContext + .filter( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and provenance = 'propagation:community:productsthroughsemrel'") + .count()); + + query = "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from dataset " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + Assertions + .assertEquals( + 2, + spark + .sql(query) + .select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' a" + + "nd community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagByDatasourceTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/publication/update_datasource") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", + workingDir.toString() + "/publication", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/publication") + .map(item -> OBJECT_MAPPER.readValue(item, Publication.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Publication.class)); + + verificationDataset.createOrReplaceTempView("publication"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from publication " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + Assertions.assertEquals(5, idExplodeCommunity.count()); + Assertions + .assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions + .assertEquals( + 5, + idExplodeCommunity + .filter("name = 'Bulktagging for Community - Datasource'") + .count()); + + Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); + + Assertions + .assertEquals( + 3, + idExplodeCommunity + .filter( + "community = 'fam' and (id = '50|ec_fp7health::000085c89f4b96dc2269bd37edb35306' " + + "or id = '50|ec_fp7health::000b9e61f83f5a4b0c35777b7bccdf38' " + + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") + .count()); + + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter( + "community = 'aginfra' and (id = '50|ec_fp7health::000c8195edd542e4e64ebb32172cbf89' " + + "or id = '50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798')") + .count()); + } + + @Test + public void bulktagByZenodoCommunityTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", + "-outputPath", + workingDir.toString() + "/orp", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/orp") + .map(item -> OBJECT_MAPPER.readValue(item, OtherResearchProduct.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(OtherResearchProduct.class)); + + verificationDataset.createOrReplaceTempView("orp"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from orp " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(8, idExplodeCommunity.count()); + + Assertions + .assertEquals( + 8, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); + Assertions + .assertEquals( + 8, + idExplodeCommunity.filter("name = 'Bulktagging for Community - Zenodo'").count()); + + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'beopen'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'mes'").count()); + + Assertions + .assertEquals( + 1, + idExplodeCommunity + .filter( + "id = '50|od______2017::0750a4d0782265873d669520f5e33c07' " + + "and community = 'covid-19'") + .count()); + Assertions + .assertEquals( + 3, + idExplodeCommunity + .filter( + "id = '50|od______2017::1bd97baef19dbd2db3203b112bb83bc5' and " + + "(community = 'aginfra' or community = 'mes' or community = 'fam')") + .count()); + Assertions + .assertEquals( + 1, + idExplodeCommunity + .filter( + "id = '50|od______2017::1e400f1747487fd15998735c41a55c72' " + + "and community = 'beopen'") + .count()); + Assertions + .assertEquals( + 3, + idExplodeCommunity + .filter( + "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' and " + + "(community = 'beopen' or community = 'fam' or community = 'mes')") + .count()); + + query = "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from orp " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + Assertions + .assertEquals( + 2, + spark + .sql(query) + .select("datainfosize") + .where( + "id = '50|od______2017::210281c5bc1c739a11ccceeeca806396' a" + + "nd community = 'beopen'") + .collectAsList() + .get(0) + .getInt(0)); + + // verify the zenodo community context is not present anymore in the records + query = "select id, MyT.id community " + + "from orp " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions + .assertEquals( + 0, + tmp2 + .select("community") + .where(tmp2.col("community").contains(ZENODO_COMMUNITY_INDICATOR)) + .count()); + } + + @Test + public void bulktagBySubjectDatasourceTest() throws Exception { + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject_datasource") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(7, idExplodeCommunity.count()); + + Assertions + .assertEquals( + 5, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions + .assertEquals( + 2, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'fam'").count()); + Assertions.assertEquals(2, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'mes'").count()); + + query = "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from dataset " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions + .assertEquals( + 2, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " + + "community = 'aginfra'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions + .assertEquals( + 1, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions + .assertEquals( + 2, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " + + "community = 'fam'") + .collectAsList() + .get(0) + .getInt(0)); + Assertions + .assertEquals( + 2, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions + .assertEquals( + 1, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " + + "community = 'fam'") + .collectAsList() + .get(0) + .getInt(0)); + Assertions + .assertEquals( + 1, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______3989::0f89464c4ac4c398fe0c71433b175a62' and " + + "community = 'mes'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagBySubjectDatasourceZenodoCommunityTest() throws Exception { + + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/sample/software/").getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath", + workingDir.toString() + "/software", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/software") + .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Software.class)); + + verificationDataset.createOrReplaceTempView("software"); + + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from software " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(10, idExplodeCommunity.count()); + + idExplodeCommunity.show(false); + Assertions + .assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:subject'").count()); + Assertions + .assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + Assertions + .assertEquals( + 4, idExplodeCommunity.filter("provenance = 'community:zenodocommunity'").count()); + + Assertions.assertEquals(3, idExplodeCommunity.filter("community = 'covid-19'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dh-ch'").count()); + Assertions.assertEquals(4, idExplodeCommunity.filter("community = 'aginfra'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'dariah'").count()); + Assertions.assertEquals(1, idExplodeCommunity.filter("community = 'fam'").count()); + + Assertions + .assertEquals( + 2, + idExplodeCommunity + .filter( + "provenance = 'community:zenodocommunity' and " + + "id = '50|od______1582::4132f5ec9496f0d6adc7b00a50a56ff4' and (" + + "community = 'dh-ch' or community = 'dariah')") + .count()); + + query = "select id, MyT.id community, size(MyT.datainfo) datainfosize " + + "from software " + + "lateral view explode (context) as MyT " + + "where size(MyT.datainfo) > 0"; + + org.apache.spark.sql.Dataset tmp2 = spark.sql(query); + + Assertions + .assertEquals( + 2, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______1582::501b25d420f808c8eddcd9b16e917f11' and " + + "community = 'covid-19'") + .collectAsList() + .get(0) + .getInt(0)); + + Assertions + .assertEquals( + 3, + tmp2 + .select("datainfosize") + .where( + "id = '50|od______1582::581621232a561b7e8b4952b18b8b0e56' and " + + "community = 'aginfra'") + .collectAsList() + .get(0) + .getInt(0)); + } + + @Test + public void bulktagDatasourcewithConstraintsTest() throws Exception { + + SparkBulkTagJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints") + .getPath(), + "-taggingConf", + taggingConf, + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-isLookupUrl", + "http://beta.services.openaire.eu:8280/is/services/isLookUp", + "-protoMap", + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}" + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + String query = "select id, MyT.id community, MyD.provenanceaction.classid provenance, MyD.provenanceaction.classname name " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + + idExplodeCommunity.show(false); + Assertions.assertEquals(3, idExplodeCommunity.count()); + + Assertions + .assertEquals( + 3, idExplodeCommunity.filter("provenance = 'community:datasource'").count()); + } } diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java index a04395337..3aae9ebee 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java @@ -1,155 +1,166 @@ + package eu.dnetlib.dhp; -import com.google.gson.Gson; -import eu.dnetlib.dhp.community.CommunityConfiguration; -import eu.dnetlib.dhp.community.CommunityConfigurationFactory; -import eu.dnetlib.dhp.community.Constraint; -import eu.dnetlib.dhp.community.SelectionConstraints; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.dom4j.DocumentException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.community.CommunityConfiguration; +import eu.dnetlib.dhp.community.CommunityConfigurationFactory; +import eu.dnetlib.dhp.community.Constraint; +import eu.dnetlib.dhp.community.SelectionConstraints; +import eu.dnetlib.dhp.selectioncriteria.VerbResolver; + /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactoryTest { - private final VerbResolver resolver = new VerbResolver(); + private final VerbResolver resolver = new VerbResolver(); - @Test - public void parseTest() throws DocumentException, IOException { - String xml = - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml")); - final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); - Assertions.assertEquals(5, cc.size()); - cc.getCommunityList() - .forEach(c -> Assertions.assertTrue(StringUtils.isNoneBlank(c.getId()))); - } + @Test + public void parseTest() throws DocumentException, IOException { + String xml = IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + Assertions.assertEquals(5, cc.size()); + cc + .getCommunityList() + .forEach(c -> Assertions.assertTrue(StringUtils.isNoneBlank(c.getId()))); + } - @Test - public void applyVerb() - throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, - InstantiationException { - Constraint sc = new Constraint(); - sc.setVerb("not_contains"); - sc.setField("contributor"); - sc.setValue("DARIAH"); - sc.setSelection(resolver.getSelectionCriteria(sc.getVerb(), sc.getValue())); - String metadata = "This work has been partially supported by DARIAH-EU infrastructure"; - Assertions.assertFalse(sc.verifyCriteria(metadata)); - } + @Test + public void applyVerb() + throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, + InstantiationException { + Constraint sc = new Constraint(); + sc.setVerb("not_contains"); + sc.setField("contributor"); + sc.setValue("DARIAH"); + sc.setSelection(resolver.getSelectionCriteria(sc.getVerb(), sc.getValue())); + String metadata = "This work has been partially supported by DARIAH-EU infrastructure"; + Assertions.assertFalse(sc.verifyCriteria(metadata)); + } - @Test - public void loadSelCriteriaTest() throws DocumentException, IOException { - String xml = - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml")); - final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); - Map> param = new HashMap<>(); - param.put("author", new ArrayList<>(Collections.singletonList("Pippo Pippi"))); - param.put( - "description", - new ArrayList<>( - Collections.singletonList( - "This work has been partially supported by DARIAH-EU infrastructure"))); - param.put( - "contributor", - new ArrayList<>( - Collections.singletonList( - "Pallino ha aiutato a scrivere il paper. Pallino lavora per DARIAH"))); - List comm = - cc.getCommunityForDatasource( - "openaire____::1cfdb2e14977f31a98e0118283401f32", param); - Assertions.assertEquals(1, comm.size()); - Assertions.assertEquals("dariah", comm.get(0)); - } + @Test + public void loadSelCriteriaTest() throws DocumentException, IOException { + String xml = IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + Map> param = new HashMap<>(); + param.put("author", new ArrayList<>(Collections.singletonList("Pippo Pippi"))); + param + .put( + "description", + new ArrayList<>( + Collections + .singletonList( + "This work has been partially supported by DARIAH-EU infrastructure"))); + param + .put( + "contributor", + new ArrayList<>( + Collections + .singletonList( + "Pallino ha aiutato a scrivere il paper. Pallino lavora per DARIAH"))); + List comm = cc + .getCommunityForDatasource( + "openaire____::1cfdb2e14977f31a98e0118283401f32", param); + Assertions.assertEquals(1, comm.size()); + Assertions.assertEquals("dariah", comm.get(0)); + } - @Test - public void test4() throws DocumentException, IOException { - final CommunityConfiguration cc = - CommunityConfigurationFactory.fromJson( - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json"))); - cc.toString(); - } + @Test + public void test4() throws DocumentException, IOException { + final CommunityConfiguration cc = CommunityConfigurationFactory + .fromJson( + IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json"))); + cc.toString(); + } - @Test - public void test5() throws IOException, DocumentException { + @Test + public void test5() throws IOException, DocumentException { - // final CommunityConfiguration cc = - // CommunityConfigurationFactory.newInstance(IOUtils.toString(getClass().getResourceAsStream("test.xml"))); - final CommunityConfiguration cc = - CommunityConfigurationFactory.fromJson( - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.json"))); + // final CommunityConfiguration cc = + // CommunityConfigurationFactory.newInstance(IOUtils.toString(getClass().getResourceAsStream("test.xml"))); + final CommunityConfiguration cc = CommunityConfigurationFactory + .fromJson( + IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/community_configuration.json"))); - System.out.println(cc.toJson()); - } + System.out.println(cc.toJson()); + } - @Test - public void test6() { - String json = - "{\"criteria\":[{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}]}"; + @Test + public void test6() { + String json = "{\"criteria\":[{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}]}"; - String step1 = "{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}"; + String step1 = "{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}"; - Constraint c = new Gson().fromJson(step1, Constraint.class); - // - // String step2 = - // "{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}"; - // - // ConstraintEncapsulator ce = new - // Gson().fromJson(step2,ConstraintEncapsulator.class); - // - // - // String step3 = - // "{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}"; - // - // Constraints cons = new Gson().fromJson(step3,Constraints.class); - // - // String step4 = - // "{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}"; - // - // ConstraintsList cl = new Gson().fromJson(step4,ConstraintsList.class); - // - // String step5 = - // "{\"cl\":{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}}"; - SelectionConstraints sl = new Gson().fromJson(json, SelectionConstraints.class); - } + Constraint c = new Gson().fromJson(step1, Constraint.class); + // + // String step2 = + // "{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}"; + // + // ConstraintEncapsulator ce = new + // Gson().fromJson(step2,ConstraintEncapsulator.class); + // + // + // String step3 = + // "{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}"; + // + // Constraints cons = new Gson().fromJson(step3,Constraints.class); + // + // String step4 = + // "{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}"; + // + // ConstraintsList cl = new Gson().fromJson(step4,ConstraintsList.class); + // + // String step5 = + // "{\"cl\":{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}}"; + SelectionConstraints sl = new Gson().fromJson(json, SelectionConstraints.class); + } - @Test - public void test7() throws IOException { - final CommunityConfiguration cc = - CommunityConfigurationFactory.fromJson( - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json"))); + @Test + public void test7() throws IOException { + final CommunityConfiguration cc = CommunityConfigurationFactory + .fromJson( + IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json"))); - System.out.println(cc.toJson()); - } + System.out.println(cc.toJson()); + } - @Test - public void temporaneo() throws Exception { - String xml = - IOUtils.toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); - final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); - System.out.println(cc.toJson()); - } + @Test + public void temporaneo() throws Exception { + String xml = IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); + final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); + System.out.println(cc.toJson()); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java index 0c4a77be9..2120da080 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java @@ -1,18 +1,12 @@ + package eu.dnetlib.dhp.oa.dedup; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; import java.io.IOException; import java.io.Serializable; import java.io.StringReader; import java.util.ArrayList; import java.util.List; + import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SaveMode; @@ -22,72 +16,82 @@ import org.dom4j.DocumentException; import org.dom4j.Element; import org.dom4j.io.SAXReader; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; + abstract class AbstractSparkAction implements Serializable { - protected static final ObjectMapper OBJECT_MAPPER = - new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - public ArgumentApplicationParser parser; // parameters for the spark action - public SparkSession spark; // the spark session + public ArgumentApplicationParser parser; // parameters for the spark action + public SparkSession spark; // the spark session - public AbstractSparkAction(ArgumentApplicationParser parser, SparkSession spark) { + public AbstractSparkAction(ArgumentApplicationParser parser, SparkSession spark) { - this.parser = parser; - this.spark = spark; - } + this.parser = parser; + this.spark = spark; + } - public List getConfigurations(ISLookUpService isLookUpService, String orchestrator) - throws ISLookUpException, DocumentException, IOException { + public List getConfigurations(ISLookUpService isLookUpService, String orchestrator) + throws ISLookUpException, DocumentException, IOException { - final String xquery = - String.format("/RESOURCE_PROFILE[.//DEDUPLICATION/ACTION_SET/@id = '%s']", orchestrator); + final String xquery = String.format("/RESOURCE_PROFILE[.//DEDUPLICATION/ACTION_SET/@id = '%s']", orchestrator); - String orchestratorProfile = isLookUpService.getResourceProfileByQuery(xquery); + String orchestratorProfile = isLookUpService.getResourceProfileByQuery(xquery); - final Document doc = new SAXReader().read(new StringReader(orchestratorProfile)); + final Document doc = new SAXReader().read(new StringReader(orchestratorProfile)); - final String actionSetId = doc.valueOf("//DEDUPLICATION/ACTION_SET/@id"); + final String actionSetId = doc.valueOf("//DEDUPLICATION/ACTION_SET/@id"); - final List configurations = new ArrayList<>(); + final List configurations = new ArrayList<>(); - for (final Object o : doc.selectNodes("//SCAN_SEQUENCE/SCAN")) { - configurations.add(loadConfig(isLookUpService, actionSetId, o)); - } + for (final Object o : doc.selectNodes("//SCAN_SEQUENCE/SCAN")) { + configurations.add(loadConfig(isLookUpService, actionSetId, o)); + } - return configurations; - } + return configurations; + } - private DedupConfig loadConfig( - final ISLookUpService isLookUpService, final String actionSetId, final Object o) - throws ISLookUpException, IOException { - final Element s = (Element) o; - final String configProfileId = s.attributeValue("id"); - final String conf = - isLookUpService.getResourceProfileByQuery( - String.format( - "for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", - configProfileId)); + private DedupConfig loadConfig( + final ISLookUpService isLookUpService, final String actionSetId, final Object o) + throws ISLookUpException, IOException { + final Element s = (Element) o; + final String configProfileId = s.attributeValue("id"); + final String conf = isLookUpService + .getResourceProfileByQuery( + String + .format( + "for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", + configProfileId)); - DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class); - dedupConfig.getPace().initModel(); - dedupConfig.getPace().initTranslationMap(); - dedupConfig.getWf().setConfigurationId(actionSetId); + DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class); + dedupConfig.getPace().initModel(); + dedupConfig.getPace().initTranslationMap(); + dedupConfig.getWf().setConfigurationId(actionSetId); - return dedupConfig; - } + return dedupConfig; + } - abstract void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException; + abstract void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException; - protected static SparkSession getSparkSession(SparkConf conf) { - return SparkSession.builder().config(conf).getOrCreate(); - } + protected static SparkSession getSparkSession(SparkConf conf) { + return SparkSession.builder().config(conf).getOrCreate(); + } - protected static void save(Dataset dataset, String outPath, SaveMode mode) { - dataset.write().option("compression", "gzip").mode(mode).json(outPath); - } + protected static void save(Dataset dataset, String outPath, SaveMode mode) { + dataset.write().option("compression", "gzip").mode(mode).json(outPath); + } - protected static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + protected static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DatePicker.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DatePicker.java index 098d024f4..70fb2cc5b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DatePicker.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DatePicker.java @@ -1,113 +1,121 @@ + package eu.dnetlib.dhp.oa.dedup; import static java.util.Collections.reverseOrder; import static java.util.Map.Entry.comparingByValue; import static java.util.stream.Collectors.toMap; + import static org.apache.commons.lang.StringUtils.endsWith; import static org.apache.commons.lang.StringUtils.substringBefore; -import eu.dnetlib.dhp.schema.oaf.Field; import java.time.Year; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.lang.StringUtils; +import eu.dnetlib.dhp.schema.oaf.Field; + public class DatePicker { - private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}"; - private static final String DATE_DEFAULT_SUFFIX = "01-01"; - private static final int YEAR_LB = 1300; - private static final int YEAR_UB = Year.now().getValue() + 5; + private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}"; + private static final String DATE_DEFAULT_SUFFIX = "01-01"; + private static final int YEAR_LB = 1300; + private static final int YEAR_UB = Year.now().getValue() + 5; - public static Field pick(final Collection dateofacceptance) { + public static Field pick(final Collection dateofacceptance) { - final Map frequencies = - dateofacceptance - .parallelStream() - .filter(StringUtils::isNotBlank) - .collect(Collectors.toConcurrentMap(w -> w, w -> 1, Integer::sum)); + final Map frequencies = dateofacceptance + .parallelStream() + .filter(StringUtils::isNotBlank) + .collect(Collectors.toConcurrentMap(w -> w, w -> 1, Integer::sum)); - if (frequencies.isEmpty()) { - return new Field<>(); - } + if (frequencies.isEmpty()) { + return new Field<>(); + } - final Field date = new Field<>(); - date.setValue(frequencies.keySet().iterator().next()); + final Field date = new Field<>(); + date.setValue(frequencies.keySet().iterator().next()); - // let's sort this map by values first, filtering out invalid dates - final Map sorted = - frequencies.entrySet().stream() - .filter(d -> StringUtils.isNotBlank(d.getKey())) - .filter(d -> d.getKey().matches(DATE_PATTERN)) - .filter(d -> inRange(d.getKey())) - .sorted(reverseOrder(comparingByValue())) - .collect( - toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2, LinkedHashMap::new)); + // let's sort this map by values first, filtering out invalid dates + final Map sorted = frequencies + .entrySet() + .stream() + .filter(d -> StringUtils.isNotBlank(d.getKey())) + .filter(d -> d.getKey().matches(DATE_PATTERN)) + .filter(d -> inRange(d.getKey())) + .sorted(reverseOrder(comparingByValue())) + .collect( + toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2, LinkedHashMap::new)); - // shortcut - if (sorted.size() == 0) { - return date; - } + // shortcut + if (sorted.size() == 0) { + return date; + } - // voting method (1/3 + 1) wins - if (sorted.size() >= 3) { - final int acceptThreshold = (sorted.size() / 3) + 1; - final List accepted = - sorted.entrySet().stream() - .filter(e -> e.getValue() >= acceptThreshold) - .map(e -> e.getKey()) - .collect(Collectors.toList()); + // voting method (1/3 + 1) wins + if (sorted.size() >= 3) { + final int acceptThreshold = (sorted.size() / 3) + 1; + final List accepted = sorted + .entrySet() + .stream() + .filter(e -> e.getValue() >= acceptThreshold) + .map(e -> e.getKey()) + .collect(Collectors.toList()); - // cannot find strong majority - if (accepted.isEmpty()) { - final int max = sorted.values().iterator().next(); - Optional first = - sorted.entrySet().stream() - .filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) - .map(Map.Entry::getKey) - .findFirst(); - if (first.isPresent()) { - date.setValue(first.get()); - return date; - } + // cannot find strong majority + if (accepted.isEmpty()) { + final int max = sorted.values().iterator().next(); + Optional first = sorted + .entrySet() + .stream() + .filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) + .map(Map.Entry::getKey) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } - date.setValue(sorted.keySet().iterator().next()); - return date; - } + date.setValue(sorted.keySet().iterator().next()); + return date; + } - if (accepted.size() == 1) { - date.setValue(accepted.get(0)); - return date; - } else { - final Optional first = - accepted.stream().filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX)).findFirst(); - if (first.isPresent()) { - date.setValue(first.get()); - return date; - } + if (accepted.size() == 1) { + date.setValue(accepted.get(0)); + return date; + } else { + final Optional first = accepted + .stream() + .filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX)) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } - return date; - } + return date; + } - // 1st non YYYY-01-01 is returned - } else { - if (sorted.size() == 2) { - for (Map.Entry e : sorted.entrySet()) { - if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) { - date.setValue(e.getKey()); - return date; - } - } - } + // 1st non YYYY-01-01 is returned + } else { + if (sorted.size() == 2) { + for (Map.Entry e : sorted.entrySet()) { + if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) { + date.setValue(e.getKey()); + return date; + } + } + } - // none of the dates seems good enough, return the 1st one - date.setValue(sorted.keySet().iterator().next()); - return date; - } - } + // none of the dates seems good enough, return the 1st one + date.setValue(sorted.keySet().iterator().next()); + return date; + } + } - private static boolean inRange(final String date) { - final int year = Integer.parseInt(substringBefore(date, "-")); - return year >= YEAR_LB && year <= YEAR_UB; - } + private static boolean inRange(final String date) { + final int year = Integer.parseInt(substringBefore(date, "-")); + return year >= YEAR_LB && year <= YEAR_UB; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index 5a806c0a0..fa06424d7 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -1,12 +1,9 @@ + package eu.dnetlib.dhp.oa.dedup; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Lists; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Collection; import java.util.Iterator; + import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; @@ -14,92 +11,96 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; public class DedupRecordFactory { - private static final Logger log = LoggerFactory.getLogger(DedupRecordFactory.class); + private static final Logger log = LoggerFactory.getLogger(DedupRecordFactory.class); - protected static final ObjectMapper OBJECT_MAPPER = - new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - public static Dataset createDedupRecord( - final SparkSession spark, - final DataInfo dataInfo, - final String mergeRelsInputPath, - final String entitiesInputPath, - final Class clazz) { + public static Dataset createDedupRecord( + final SparkSession spark, + final DataInfo dataInfo, + final String mergeRelsInputPath, + final String entitiesInputPath, + final Class clazz) { - long ts = System.currentTimeMillis(); + long ts = System.currentTimeMillis(); - // - Dataset> entities = - spark - .read() - .textFile(entitiesInputPath) - .map( - (MapFunction>) - it -> { - T entity = OBJECT_MAPPER.readValue(it, clazz); - return new Tuple2<>(entity.getId(), entity); - }, - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + // + Dataset> entities = spark + .read() + .textFile(entitiesInputPath) + .map( + (MapFunction>) it -> { + T entity = OBJECT_MAPPER.readValue(it, clazz); + return new Tuple2<>(entity.getId(), entity); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); - // : source is the dedup_id, target is the id of the mergedIn - Dataset> mergeRels = - spark - .read() - .load(mergeRelsInputPath) - .as(Encoders.bean(Relation.class)) - .where("relClass == 'merges'") - .map( - (MapFunction>) - r -> new Tuple2<>(r.getSource(), r.getTarget()), - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + // : source is the dedup_id, target is the id of the mergedIn + Dataset> mergeRels = spark + .read() + .load(mergeRelsInputPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'") + .map( + (MapFunction>) r -> new Tuple2<>(r.getSource(), r.getTarget()), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - return mergeRels - .joinWith(entities, mergeRels.col("_2").equalTo(entities.col("_1")), "inner") - .map( - (MapFunction, Tuple2>, Tuple2>) - value -> new Tuple2<>(value._1()._1(), value._2()._2()), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))) - .groupByKey( - (MapFunction, String>) entity -> entity._1(), Encoders.STRING()) - .mapGroups( - (MapGroupsFunction, T>) - (key, values) -> entityMerger(key, values, ts, dataInfo), - Encoders.bean(clazz)); - } + return mergeRels + .joinWith(entities, mergeRels.col("_2").equalTo(entities.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, Tuple2>) value -> new Tuple2<>( + value._1()._1(), value._2()._2()), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))) + .groupByKey( + (MapFunction, String>) entity -> entity._1(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction, T>) (key, + values) -> entityMerger(key, values, ts, dataInfo), + Encoders.bean(clazz)); + } - private static T entityMerger( - String id, Iterator> entities, long ts, DataInfo dataInfo) { + private static T entityMerger( + String id, Iterator> entities, long ts, DataInfo dataInfo) { - T entity = entities.next()._2(); + T entity = entities.next()._2(); - final Collection dates = Lists.newArrayList(); - entities.forEachRemaining( - t -> { - T duplicate = t._2(); - entity.mergeFrom(duplicate); - if (ModelSupport.isSubClass(duplicate, Result.class)) { - Result r1 = (Result) duplicate; - Result er = (Result) entity; - er.setAuthor(DedupUtility.mergeAuthor(er.getAuthor(), r1.getAuthor())); + final Collection dates = Lists.newArrayList(); + entities + .forEachRemaining( + t -> { + T duplicate = t._2(); + entity.mergeFrom(duplicate); + if (ModelSupport.isSubClass(duplicate, Result.class)) { + Result r1 = (Result) duplicate; + Result er = (Result) entity; + er.setAuthor(DedupUtility.mergeAuthor(er.getAuthor(), r1.getAuthor())); - if (r1.getDateofacceptance() != null) { - dates.add(r1.getDateofacceptance().getValue()); - } - } - }); + if (r1.getDateofacceptance() != null) { + dates.add(r1.getDateofacceptance().getValue()); + } + } + }); - if (ModelSupport.isSubClass(entity, Result.class)) { - ((Result) entity).setDateofacceptance(DatePicker.pick(dates)); - } + if (ModelSupport.isSubClass(entity, Result.class)) { + ((Result) entity).setDateofacceptance(DatePicker.pick(dates)); + } - entity.setId(id); - entity.setLastupdatetimestamp(ts); - entity.setDataInfo(dataInfo); + entity.setId(id); + entity.setLastupdatetimestamp(ts); + entity.setDataInfo(dataInfo); - return entity; - } + return entity; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java index f239e072f..4f797f7f7 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java @@ -1,7 +1,24 @@ + package eu.dnetlib.dhp.oa.dedup; +import java.io.StringReader; +import java.security.MessageDigest; +import java.text.Normalizer; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.SparkContext; +import org.apache.spark.util.LongAccumulator; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.Element; +import org.dom4j.io.SAXReader; + import com.google.common.collect.Sets; import com.wcohen.ss.JaroWinkler; + import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.utils.ISLookupClientFactory; @@ -11,230 +28,222 @@ import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.model.Person; -import java.io.StringReader; -import java.security.MessageDigest; -import java.text.Normalizer; -import java.util.*; -import java.util.stream.Collectors; -import org.apache.commons.codec.binary.Hex; -import org.apache.commons.lang3.StringUtils; -import org.apache.spark.SparkContext; -import org.apache.spark.util.LongAccumulator; -import org.dom4j.Document; -import org.dom4j.DocumentException; -import org.dom4j.Element; -import org.dom4j.io.SAXReader; import scala.Tuple2; public class DedupUtility { - private static final Double THRESHOLD = 0.95; + private static final Double THRESHOLD = 0.95; - public static Map constructAccumulator( - final DedupConfig dedupConf, final SparkContext context) { + public static Map constructAccumulator( + final DedupConfig dedupConf, final SparkContext context) { - Map accumulators = new HashMap<>(); + 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)); + 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; - } + return accumulators; + } - static Set getGroupingKeys(DedupConfig conf, MapDocument doc) { - return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf)); - } + 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; - } - } + 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; + } + } - public static List mergeAuthor(final List a, final List b) { - int pa = countAuthorsPids(a); - int pb = countAuthorsPids(b); - List base, enrich; - int sa = authorsSize(a); - int sb = authorsSize(b); + public static List mergeAuthor(final List a, final List b) { + int pa = countAuthorsPids(a); + int pb = countAuthorsPids(b); + List base, enrich; + int sa = authorsSize(a); + int sb = authorsSize(b); - if (pa == pb) { - base = sa > sb ? a : b; - enrich = sa > sb ? b : a; - } else { - base = pa > pb ? a : b; - enrich = pa > pb ? b : a; - } - enrichPidFromList(base, enrich); - return base; - } + if (pa == pb) { + base = sa > sb ? a : b; + enrich = sa > sb ? b : a; + } else { + base = pa > pb ? a : b; + enrich = pa > pb ? b : a; + } + enrichPidFromList(base, enrich); + return base; + } - private static void enrichPidFromList(List base, List enrich) { - if (base == null || enrich == null) return; - final Map basePidAuthorMap = - base.stream() - .filter(a -> a.getPid() != null && a.getPid().size() > 0) - .flatMap(a -> a.getPid().stream().map(p -> new Tuple2<>(p.toComparableString(), a))) - .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); + private static void enrichPidFromList(List base, List enrich) { + if (base == null || enrich == null) + return; + final Map basePidAuthorMap = base + .stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap(a -> a.getPid().stream().map(p -> new Tuple2<>(p.toComparableString(), a))) + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); - final List> pidToEnrich = - enrich.stream() - .filter(a -> a.getPid() != null && a.getPid().size() > 0) - .flatMap( - a -> - a.getPid().stream() - .filter(p -> !basePidAuthorMap.containsKey(p.toComparableString())) - .map(p -> new Tuple2<>(p, a))) - .collect(Collectors.toList()); + final List> pidToEnrich = enrich + .stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap( + a -> a + .getPid() + .stream() + .filter(p -> !basePidAuthorMap.containsKey(p.toComparableString())) + .map(p -> new Tuple2<>(p, a))) + .collect(Collectors.toList()); - pidToEnrich.forEach( - a -> { - Optional> simAuhtor = - base.stream() - .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) - .max(Comparator.comparing(Tuple2::_1)); - if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) { - Author r = simAuhtor.get()._2(); - r.getPid().add(a._1()); - } - }); - } + pidToEnrich + .forEach( + a -> { + Optional> simAuhtor = base + .stream() + .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) + .max(Comparator.comparing(Tuple2::_1)); + if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) { + Author r = simAuhtor.get()._2(); + r.getPid().add(a._1()); + } + }); + } - public static String createDedupRecordPath( - final String basePath, final String actionSetId, final String entityType) { - return String.format("%s/%s/%s_deduprecord", basePath, actionSetId, entityType); - } + public static String createDedupRecordPath( + final String basePath, final String actionSetId, final String entityType) { + return String.format("%s/%s/%s_deduprecord", basePath, actionSetId, entityType); + } - public static String createEntityPath(final String basePath, final String entityType) { - return String.format("%s/%s", basePath, entityType); - } + public static String createEntityPath(final String basePath, final String entityType) { + return String.format("%s/%s", basePath, entityType); + } - public static String createSimRelPath( - final String basePath, final String actionSetId, final String entityType) { - return String.format("%s/%s/%s_simrel", basePath, actionSetId, entityType); - } + public static String createSimRelPath( + final String basePath, final String actionSetId, final String entityType) { + return String.format("%s/%s/%s_simrel", basePath, actionSetId, entityType); + } - public static String createMergeRelPath( - final String basePath, final String actionSetId, final String entityType) { - return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType); - } + public static String createMergeRelPath( + final String basePath, final String actionSetId, final String entityType) { + return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType); + } - private static Double sim(Author a, Author b) { + private static Double sim(Author a, Author b) { - final Person pa = parse(a); - final Person pb = parse(b); + final Person pa = parse(a); + final Person pb = parse(b); - if (pa.isAccurate() & pb.isAccurate()) { - return new JaroWinkler() - .score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())); - } else { - return new JaroWinkler() - .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); - } - } + if (pa.isAccurate() & pb.isAccurate()) { + return new JaroWinkler() + .score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())); + } else { + return new JaroWinkler() + .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); + } + } - private static String normalize(final String s) { - return nfd(s) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") - .trim(); - } + private static String normalize(final String s) { + return nfd(s) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim(); + } - private static String nfd(final String s) { - return Normalizer.normalize(s, Normalizer.Form.NFD); - } + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } - private static Person parse(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return new Person(author.getSurname() + ", " + author.getName(), false); - } else { - return new Person(author.getFullname(), false); - } - } + private static Person parse(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return new Person(author.getSurname() + ", " + author.getName(), false); + } else { + return new Person(author.getFullname(), false); + } + } - private static int countAuthorsPids(List authors) { - if (authors == null) return 0; + private static int countAuthorsPids(List authors) { + if (authors == null) + return 0; - return (int) authors.stream().filter(DedupUtility::hasPid).count(); - } + return (int) authors.stream().filter(DedupUtility::hasPid).count(); + } - private static int authorsSize(List authors) { - if (authors == null) return 0; - return authors.size(); - } + private static int authorsSize(List authors) { + if (authors == null) + return 0; + return authors.size(); + } - private static boolean hasPid(Author a) { - if (a == null || a.getPid() == null || a.getPid().size() == 0) return false; - return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); - } + private static boolean hasPid(Author a) { + if (a == null || a.getPid() == null || a.getPid().size() == 0) + return false; + return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); + } - public static List getConfigurations(String isLookUpUrl, String orchestrator) - throws ISLookUpException, DocumentException { - final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookUpUrl); + public static List getConfigurations(String isLookUpUrl, String orchestrator) + throws ISLookUpException, DocumentException { + final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookUpUrl); - final String xquery = - String.format("/RESOURCE_PROFILE[.//DEDUPLICATION/ACTION_SET/@id = '%s']", orchestrator); + final String xquery = String.format("/RESOURCE_PROFILE[.//DEDUPLICATION/ACTION_SET/@id = '%s']", orchestrator); - String orchestratorProfile = isLookUpService.getResourceProfileByQuery(xquery); + String orchestratorProfile = isLookUpService.getResourceProfileByQuery(xquery); - final Document doc = new SAXReader().read(new StringReader(orchestratorProfile)); + final Document doc = new SAXReader().read(new StringReader(orchestratorProfile)); - final String actionSetId = doc.valueOf("//DEDUPLICATION/ACTION_SET/@id"); - final List configurations = new ArrayList<>(); + final String actionSetId = doc.valueOf("//DEDUPLICATION/ACTION_SET/@id"); + final List configurations = new ArrayList<>(); - for (final Object o : doc.selectNodes("//SCAN_SEQUENCE/SCAN")) { - configurations.add(loadConfig(isLookUpService, actionSetId, o)); - } + for (final Object o : doc.selectNodes("//SCAN_SEQUENCE/SCAN")) { + configurations.add(loadConfig(isLookUpService, actionSetId, o)); + } - return configurations; - } + return configurations; + } - private static DedupConfig loadConfig( - final ISLookUpService isLookUpService, final String actionSetId, final Object o) - throws ISLookUpException { - final Element s = (Element) o; - final String configProfileId = s.attributeValue("id"); - final String conf = - isLookUpService.getResourceProfileByQuery( - String.format( - "for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", - configProfileId)); - final DedupConfig dedupConfig = DedupConfig.load(conf); - dedupConfig.getWf().setConfigurationId(actionSetId); - return dedupConfig; - } + private static DedupConfig loadConfig( + final ISLookUpService isLookUpService, final String actionSetId, final Object o) + throws ISLookUpException { + final Element s = (Element) o; + final String configProfileId = s.attributeValue("id"); + final String conf = isLookUpService + .getResourceProfileByQuery( + String + .format( + "for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", + configProfileId)); + final DedupConfig dedupConfig = DedupConfig.load(conf); + dedupConfig.getWf().setConfigurationId(actionSetId); + return dedupConfig; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java index a7515d575..c72940deb 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java @@ -1,54 +1,57 @@ + package eu.dnetlib.dhp.oa.dedup; +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.util.LongAccumulator; + import eu.dnetlib.dhp.oa.dedup.model.Block; import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.util.BlockProcessor; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.util.LongAccumulator; import scala.Serializable; import scala.Tuple2; public class Deduper implements Serializable { - public static JavaPairRDD computeRelations( - JavaSparkContext context, JavaPairRDD blocks, DedupConfig config) { - Map accumulators = - DedupUtility.constructAccumulator(config, context.sc()); + public static JavaPairRDD computeRelations( + JavaSparkContext context, JavaPairRDD blocks, DedupConfig config) { + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); - return blocks - .flatMapToPair( - it -> { - final SparkReporter reporter = new SparkReporter(accumulators); - new BlockProcessor(config) - .processSortedBlock(it._1(), it._2().getDocuments(), reporter); - return reporter.getRelations().iterator(); - }) - .mapToPair(it -> new Tuple2<>(it._1() + it._2(), it)) - .reduceByKey((a, b) -> a) - .mapToPair(Tuple2::_2); - } + return blocks + .flatMapToPair( + it -> { + final SparkReporter reporter = new SparkReporter(accumulators); + new BlockProcessor(config) + .processSortedBlock(it._1(), it._2().getDocuments(), reporter); + return reporter.getRelations().iterator(); + }) + .mapToPair(it -> new Tuple2<>(it._1() + it._2(), it)) + .reduceByKey((a, b) -> a) + .mapToPair(Tuple2::_2); + } - public static JavaPairRDD createSortedBlocks( - JavaPairRDD mapDocs, DedupConfig config) { - final String of = config.getWf().getOrderField(); - final int maxQueueSize = config.getWf().getGroupMaxSize(); + public static JavaPairRDD createSortedBlocks( + 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 - .flatMap( - a -> - DedupUtility.getGroupingKeys(config, a).stream() - .map(it -> Block.from(it, a)) - .collect(Collectors.toList()) - .iterator()) - .mapToPair(block -> new Tuple2<>(block.getKey(), block)) - .reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize)); - } + 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 + .flatMap( + a -> DedupUtility + .getGroupingKeys(config, a) + .stream() + .map(it -> Block.from(it, a)) + .collect(Collectors.toList()) + .iterator()) + .mapToPair(block -> new Tuple2<>(block.getKey(), block)) + .reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize)); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java index b47b880e9..d870f6256 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateDedupRecord.java @@ -1,5 +1,16 @@ + package eu.dnetlib.dhp.oa.dedup; +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; @@ -10,92 +21,84 @@ import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.config.DedupConfig; -import java.io.IOException; -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.dom4j.DocumentException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SparkCreateDedupRecord extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class); + private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class); - public static final String ROOT_TRUST = "0.8"; - public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; - public static final String PROVENANCE_ACTIONS = "dnet:provenanceActions"; + public static final String ROOT_TRUST = "0.8"; + public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; + public static final String PROVENANCE_ACTIONS = "dnet:provenanceActions"; - public SparkCreateDedupRecord(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkCreateDedupRecord(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateSimRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - new SparkCreateDedupRecord(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + new SparkCreateDedupRecord(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) - throws ISLookUpException, DocumentException, IOException { + @Override + public void run(ISLookUpService isLookUpService) + throws ISLookUpException, DocumentException, IOException { - final String graphBasePath = parser.get("graphBasePath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Creating deduprecords for: '{}'", subEntity); + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Creating deduprecords for: '{}'", subEntity); - final String outputPath = - DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); + final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); - final String mergeRelPath = - DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); - final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); + final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); - final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); - final DataInfo dataInfo = getDataInfo(dedupConf); - DedupRecordFactory.createDedupRecord(spark, dataInfo, mergeRelPath, entityPath, clazz) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); - } - } + final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); + final DataInfo dataInfo = getDataInfo(dedupConf); + DedupRecordFactory + .createDedupRecord(spark, dataInfo, mergeRelPath, entityPath, clazz) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } + } - private static DataInfo getDataInfo(DedupConfig dedupConf) { - DataInfo info = new DataInfo(); - info.setDeletedbyinference(false); - info.setInferred(true); - info.setInvisible(false); - info.setTrust(ROOT_TRUST); - info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); - Qualifier provenance = new Qualifier(); - provenance.setClassid(PROVENANCE_ACTION_CLASS); - provenance.setClassname(PROVENANCE_ACTION_CLASS); - provenance.setSchemeid(PROVENANCE_ACTIONS); - provenance.setSchemename(PROVENANCE_ACTIONS); - info.setProvenanceaction(provenance); - return info; - } + private static DataInfo getDataInfo(DedupConfig dedupConf) { + DataInfo info = new DataInfo(); + info.setDeletedbyinference(false); + info.setInferred(true); + info.setInvisible(false); + info.setTrust(ROOT_TRUST); + info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); + Qualifier provenance = new Qualifier(); + provenance.setClassid(PROVENANCE_ACTION_CLASS); + provenance.setClassname(PROVENANCE_ACTION_CLASS); + provenance.setSchemeid(PROVENANCE_ACTIONS); + provenance.setSchemename(PROVENANCE_ACTIONS); + info.setProvenanceaction(provenance); + return info; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java index 229379a53..a44650823 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java @@ -1,22 +1,11 @@ + package eu.dnetlib.dhp.oa.dedup; -import com.google.common.hash.Hashing; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; -import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.util.MapDocumentUtil; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; @@ -31,132 +20,149 @@ import org.apache.spark.sql.SparkSession; import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.google.common.hash.Hashing; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; +import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Qualifier; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; public class SparkCreateMergeRels extends AbstractSparkAction { - public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; - private static final Logger log = LoggerFactory.getLogger(SparkCreateMergeRels.class); - public static final String DNET_PROVENANCE_ACTIONS = "dnet:provenanceActions"; + public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; + private static final Logger log = LoggerFactory.getLogger(SparkCreateMergeRels.class); + public static final String DNET_PROVENANCE_ACTIONS = "dnet:provenanceActions"; - public SparkCreateMergeRels(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkCreateMergeRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateSimRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + parser.parseArgument(args); - final String isLookUpUrl = parser.get("isLookUpUrl"); - log.info("isLookupUrl {}", isLookUpUrl); + final String isLookUpUrl = parser.get("isLookUpUrl"); + log.info("isLookupUrl {}", isLookUpUrl); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - new SparkCreateMergeRels(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(isLookUpUrl)); - } + new SparkCreateMergeRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(isLookUpUrl)); + } - @Override - public void run(ISLookUpService isLookUpService) - throws ISLookUpException, DocumentException, IOException { + @Override + public void run(ISLookUpService isLookUpService) + throws ISLookUpException, DocumentException, IOException { - final String graphBasePath = parser.get("graphBasePath"); - final String workingPath = parser.get("workingPath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); + final String graphBasePath = parser.get("graphBasePath"); + final String workingPath = parser.get("workingPath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - final String subEntity = dedupConf.getWf().getSubEntityValue(); + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + final String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Creating mergerels for: '{}'", subEntity); + log.info("Creating mergerels for: '{}'", subEntity); - final int maxIterations = dedupConf.getWf().getMaxIterations(); - log.info("Max iterations {}", maxIterations); + final int maxIterations = dedupConf.getWf().getMaxIterations(); + log.info("Max iterations {}", maxIterations); - final String mergeRelPath = - DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); + final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); - final JavaPairRDD vertexes = - sc.textFile(graphBasePath + "/" + subEntity) - .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) - .mapToPair((PairFunction) s -> new Tuple2<>(hash(s), s)); + final JavaPairRDD vertexes = sc + .textFile(graphBasePath + "/" + subEntity) + .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) + .mapToPair((PairFunction) s -> new Tuple2<>(hash(s), s)); - final RDD> edgeRdd = - spark - .read() - .load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity)) - .as(Encoders.bean(Relation.class)) - .javaRDD() - .map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass())) - .rdd(); + final RDD> edgeRdd = spark + .read() + .load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity)) + .as(Encoders.bean(Relation.class)) + .javaRDD() + .map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass())) + .rdd(); - final Dataset mergeRels = - spark.createDataset( - GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, maxIterations) - .toJavaRDD() - .filter(k -> k.getDocIds().size() > 1) - .flatMap(cc -> ccToMergeRel(cc, dedupConf)) - .rdd(), - Encoders.bean(Relation.class)); + final Dataset mergeRels = spark + .createDataset( + GraphProcessor + .findCCs(vertexes.rdd(), edgeRdd, maxIterations) + .toJavaRDD() + .filter(k -> k.getDocIds().size() > 1) + .flatMap(cc -> ccToMergeRel(cc, dedupConf)) + .rdd(), + Encoders.bean(Relation.class)); - mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath); - } - } + mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath); + } + } - public Iterator ccToMergeRel(ConnectedComponent cc, DedupConfig dedupConf) { - return cc.getDocIds().stream() - .flatMap( - id -> { - List tmp = new ArrayList<>(); + public Iterator ccToMergeRel(ConnectedComponent cc, DedupConfig dedupConf) { + return cc + .getDocIds() + .stream() + .flatMap( + id -> { + List tmp = new ArrayList<>(); - tmp.add(rel(cc.getCcId(), id, "merges", dedupConf)); - tmp.add(rel(id, cc.getCcId(), "isMergedIn", dedupConf)); + tmp.add(rel(cc.getCcId(), id, "merges", dedupConf)); + tmp.add(rel(id, cc.getCcId(), "isMergedIn", dedupConf)); - return tmp.stream(); - }) - .iterator(); - } + return tmp.stream(); + }) + .iterator(); + } - private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { - Relation r = new Relation(); - r.setSource(source); - r.setTarget(target); - r.setRelClass(relClass); - r.setSubRelType("dedup"); + private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(relClass); + r.setSubRelType("dedup"); - DataInfo info = new DataInfo(); - info.setDeletedbyinference(false); - info.setInferred(true); - info.setInvisible(false); - info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); - Qualifier provenanceAction = new Qualifier(); - provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); - provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); - provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); - provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); - info.setProvenanceaction(provenanceAction); + DataInfo info = new DataInfo(); + info.setDeletedbyinference(false); + info.setInferred(true); + info.setInvisible(false); + info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); + Qualifier provenanceAction = new Qualifier(); + provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); + provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); + provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); + provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); + info.setProvenanceaction(provenanceAction); - // TODO calculate the trust value based on the similarity score of the elements in the CC - // info.setTrust(); + // TODO calculate the trust value based on the similarity score of the elements in the CC + // info.setTrust(); - r.setDataInfo(info); - return r; - } + r.setDataInfo(info); + return r; + } - public static long hash(final String id) { - return Hashing.murmur3_128().hashString(id).asLong(); - } + public static long hash(final String id) { + return Hashing.murmur3_128().hashString(id).asLong(); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java index c5a1d768c..2cfe2e080 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java @@ -1,5 +1,21 @@ + package eu.dnetlib.dhp.oa.dedup; +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.dedup.model.Block; import eu.dnetlib.dhp.schema.oaf.DataInfo; @@ -12,117 +28,107 @@ import eu.dnetlib.pace.model.FieldListImpl; import eu.dnetlib.pace.model.FieldValueImpl; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.util.MapDocumentUtil; -import java.io.IOException; -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.dom4j.DocumentException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import scala.Tuple2; public class SparkCreateSimRels extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); - public SparkCreateSimRels(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkCreateSimRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateSimRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses( - new Class[] {MapDocument.class, FieldListImpl.class, FieldValueImpl.class, Block.class}); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf + .registerKryoClasses( + new Class[] { + MapDocument.class, FieldListImpl.class, FieldValueImpl.class, Block.class + }); - new SparkCreateSimRels(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + new SparkCreateSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException { + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { - // read oozie parameters - final String graphBasePath = parser.get("graphBasePath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - // for each dedup configuration - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + // for each dedup configuration + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - final String entity = dedupConf.getWf().getEntityType(); - final String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Creating simrels for: '{}'", subEntity); + final String entity = dedupConf.getWf().getEntityType(); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Creating simrels for: '{}'", subEntity); - final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); + final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaPairRDD mapDocuments = - sc.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) - .mapToPair( - (PairFunction) - s -> { - MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); - return new Tuple2<>(d.getIdentifier(), d); - }); + JavaPairRDD mapDocuments = sc + .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) + .mapToPair( + (PairFunction) s -> { + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); + return new Tuple2<>(d.getIdentifier(), d); + }); - // create blocks for deduplication - JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); + // create blocks for deduplication + JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); - // create relations by comparing only elements in the same group - JavaRDD relations = - Deduper.computeRelations(sc, blocks, dedupConf) - .map(t -> createSimRel(t._1(), t._2(), entity)); + // create relations by comparing only elements in the same group + JavaRDD relations = Deduper + .computeRelations(sc, blocks, dedupConf) + .map(t -> createSimRel(t._1(), t._2(), entity)); - // save the simrel in the workingdir - spark - .createDataset(relations.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Append) - .save(outputPath); - } - } + // save the simrel in the workingdir + spark + .createDataset(relations.rdd(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Append) + .save(outputPath); + } + } - private Relation createSimRel(String source, String target, String entity) { - final Relation r = new Relation(); - r.setSource(source); - r.setTarget(target); - r.setSubRelType("dedupSimilarity"); - r.setRelClass("isSimilarTo"); - r.setDataInfo(new DataInfo()); + private Relation createSimRel(String source, String target, String entity) { + final Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setSubRelType("dedupSimilarity"); + r.setRelClass("isSimilarTo"); + r.setDataInfo(new DataInfo()); - switch (entity) { - case "result": - r.setRelType("resultResult"); - break; - case "organization": - r.setRelType("organizationOrganization"); - break; - default: - throw new IllegalArgumentException("unmanaged entity type: " + entity); - } - return r; - } + switch (entity) { + case "result": + r.setRelType("resultResult"); + break; + case "organization": + r.setRelType("organizationOrganization"); + break; + default: + throw new IllegalArgumentException("unmanaged entity type: " + entity); + } + return r; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java index d829a9a03..34611db8e 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java @@ -1,183 +1,178 @@ + package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import scala.Tuple2; public class SparkPropagateRelation extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class); + private static final Logger log = LoggerFactory.getLogger(SparkPropagateRelation.class); - enum FieldType { - SOURCE, - TARGET - } + enum FieldType { + SOURCE, TARGET + } - public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) - throws Exception { - super(parser, spark); - } + public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) + throws Exception { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateSimRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"))); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"))); - parser.parseArgument(args); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - new SparkPropagateRelation(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + new SparkPropagateRelation(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) { + @Override + public void run(ISLookUpService isLookUpService) { - final String graphBasePath = parser.get("graphBasePath"); - final String workingPath = parser.get("workingPath"); - final String dedupGraphPath = parser.get("dedupGraphPath"); + final String graphBasePath = parser.get("graphBasePath"); + final String workingPath = parser.get("workingPath"); + final String dedupGraphPath = parser.get("dedupGraphPath"); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("workingPath: '{}'", workingPath); - log.info("dedupGraphPath: '{}'", dedupGraphPath); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("workingPath: '{}'", workingPath); + log.info("dedupGraphPath: '{}'", dedupGraphPath); - final String outputRelationPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); - removeOutputDir(spark, outputRelationPath); + final String outputRelationPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); + removeOutputDir(spark, outputRelationPath); - Dataset mergeRels = - spark - .read() - .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) - .as(Encoders.bean(Relation.class)); + Dataset mergeRels = spark + .read() + .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) + .as(Encoders.bean(Relation.class)); - Dataset> mergedIds = - mergeRels - .where(col("relClass").equalTo("merges")) - .select(col("source"), col("target")) - .distinct() - .map( - (MapFunction>) - r -> new Tuple2<>(r.getString(1), r.getString(0)), - Encoders.tuple(Encoders.STRING(), Encoders.STRING())) - .cache(); + Dataset> mergedIds = mergeRels + .where(col("relClass").equalTo("merges")) + .select(col("source"), col("target")) + .distinct() + .map( + (MapFunction>) r -> new Tuple2<>(r.getString(1), r.getString(0)), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .cache(); - final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); - Dataset rels = - spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)); + Dataset rels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)); - Dataset newRels = - processDataset( - processDataset(rels, mergedIds, FieldType.SOURCE, getFixRelFn(FieldType.SOURCE)), - mergedIds, - FieldType.TARGET, - getFixRelFn(FieldType.TARGET)) - .filter(SparkPropagateRelation::containsDedup); + Dataset newRels = processDataset( + processDataset(rels, mergedIds, FieldType.SOURCE, getFixRelFn(FieldType.SOURCE)), + mergedIds, + FieldType.TARGET, + getFixRelFn(FieldType.TARGET)) + .filter(SparkPropagateRelation::containsDedup); - Dataset updated = - processDataset( - processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()), - mergedIds, - FieldType.TARGET, - getDeletedFn()); + Dataset updated = processDataset( + processDataset(rels, mergedIds, FieldType.SOURCE, getDeletedFn()), + mergedIds, + FieldType.TARGET, + getDeletedFn()); - save(newRels.union(updated), outputRelationPath, SaveMode.Overwrite); - } + save(newRels.union(updated), outputRelationPath, SaveMode.Overwrite); + } - private static Dataset processDataset( - Dataset rels, - Dataset> mergedIds, - FieldType type, - MapFunction, Tuple2>, Relation> mapFn) { - final Dataset> mapped = - rels.map( - (MapFunction>) r -> new Tuple2<>(getId(r, type), r), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class))); - return mapped - .joinWith(mergedIds, mapped.col("_1").equalTo(mergedIds.col("_1")), "left_outer") - .map(mapFn, Encoders.bean(Relation.class)); - } + private static Dataset processDataset( + Dataset rels, + Dataset> mergedIds, + FieldType type, + MapFunction, Tuple2>, Relation> mapFn) { + final Dataset> mapped = rels + .map( + (MapFunction>) r -> new Tuple2<>(getId(r, type), r), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class))); + return mapped + .joinWith(mergedIds, mapped.col("_1").equalTo(mergedIds.col("_1")), "left_outer") + .map(mapFn, Encoders.bean(Relation.class)); + } - private static MapFunction patchRelFn() { - return value -> { - final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } - return rel; - }; - } + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } - private static String getId(Relation r, FieldType type) { - switch (type) { - case SOURCE: - return r.getSource(); - case TARGET: - return r.getTarget(); - default: - throw new IllegalArgumentException(""); - } - } + private static String getId(Relation r, FieldType type) { + switch (type) { + case SOURCE: + return r.getSource(); + case TARGET: + return r.getTarget(); + default: + throw new IllegalArgumentException(""); + } + } - private static MapFunction, Tuple2>, Relation> - getFixRelFn(FieldType type) { - return value -> { - if (value._2() != null) { - Relation r = value._1()._2(); - String id = value._2()._2(); - if (r.getDataInfo() == null) { - r.setDataInfo(new DataInfo()); - } - r.getDataInfo().setDeletedbyinference(false); - switch (type) { - case SOURCE: - r.setSource(id); - return r; - case TARGET: - r.setTarget(id); - return r; - default: - throw new IllegalArgumentException(""); - } - } - return value._1()._2(); - }; - } + private static MapFunction, Tuple2>, Relation> getFixRelFn( + FieldType type) { + return value -> { + if (value._2() != null) { + Relation r = value._1()._2(); + String id = value._2()._2(); + if (r.getDataInfo() == null) { + r.setDataInfo(new DataInfo()); + } + r.getDataInfo().setDeletedbyinference(false); + switch (type) { + case SOURCE: + r.setSource(id); + return r; + case TARGET: + r.setTarget(id); + return r; + default: + throw new IllegalArgumentException(""); + } + } + return value._1()._2(); + }; + } - private static MapFunction, Tuple2>, Relation> - getDeletedFn() { - return value -> { - if (value._2() != null) { - Relation r = value._1()._2(); - if (r.getDataInfo() == null) { - r.setDataInfo(new DataInfo()); - } - r.getDataInfo().setDeletedbyinference(true); - return r; - } - return value._1()._2(); - }; - } + private static MapFunction, Tuple2>, Relation> getDeletedFn() { + return value -> { + if (value._2() != null) { + Relation r = value._1()._2(); + if (r.getDataInfo() == null) { + r.setDataInfo(new DataInfo()); + } + r.getDataInfo().setDeletedbyinference(true); + return r; + } + return value._1()._2(); + }; + } - private static boolean containsDedup(final Relation r) { - return r.getSource().toLowerCase().contains("dedup") - || r.getTarget().toLowerCase().contains("dedup"); - } + private static boolean containsDedup(final Relation r) { + return r.getSource().toLowerCase().contains("dedup") + || r.getTarget().toLowerCase().contains("dedup"); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkReporter.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkReporter.java index 2ffd982b1..7100c9037 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkReporter.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkReporter.java @@ -1,47 +1,50 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.pace.util.Reporter; import java.util.ArrayList; import java.util.List; import java.util.Map; + import org.apache.spark.util.LongAccumulator; + +import eu.dnetlib.pace.util.Reporter; import scala.Serializable; import scala.Tuple2; public class SparkReporter implements Serializable, Reporter { - private final List> relations = new ArrayList<>(); + private final List> relations = new ArrayList<>(); - private Map accumulators; + private Map accumulators; - public SparkReporter(Map accumulators) { - this.accumulators = accumulators; - } + public SparkReporter(Map accumulators) { + this.accumulators = accumulators; + } - public void incrementCounter( - String counterGroup, - String counterName, - long delta, - Map 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); - } - } + 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) { + @Override + public void incrementCounter(String counterGroup, String counterName, long delta) { - incrementCounter(counterGroup, counterName, delta, accumulators); - } + incrementCounter(counterGroup, counterName, delta, accumulators); + } - @Override - public void emit(String type, String from, String to) { - relations.add(new Tuple2<>(from, to)); - } + @Override + public void emit(String type, String from, String to) { + relations.add(new Tuple2<>(from, to)); + } - public List> getRelations() { - return relations; - } + public List> getRelations() { + return relations; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java index 56dec79cf..779fb91d6 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java @@ -1,15 +1,8 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.util.MapDocumentUtil; import java.io.IOException; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -27,123 +20,133 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; public class SparkUpdateEntity extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); + private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); - private static final String IDJSONPATH = "$.id"; + private static final String IDJSONPATH = "$.id"; - public SparkUpdateEntity(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkUpdateEntity(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkUpdateEntity.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkUpdateEntity.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - new SparkUpdateEntity(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + new SparkUpdateEntity(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - public void run(ISLookUpService isLookUpService) throws IOException { + public void run(ISLookUpService isLookUpService) throws IOException { - final String graphBasePath = parser.get("graphBasePath"); - final String workingPath = parser.get("workingPath"); - final String dedupGraphPath = parser.get("dedupGraphPath"); + final String graphBasePath = parser.get("graphBasePath"); + final String workingPath = parser.get("workingPath"); + final String dedupGraphPath = parser.get("dedupGraphPath"); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("workingPath: '{}'", workingPath); - log.info("dedupGraphPath: '{}'", dedupGraphPath); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("workingPath: '{}'", workingPath); + log.info("dedupGraphPath: '{}'", dedupGraphPath); - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - // for each entity - ModelSupport.entityTypes.forEach( - (type, clazz) -> { - final String outputPath = dedupGraphPath + "/" + type; - removeOutputDir(spark, outputPath); + // for each entity + ModelSupport.entityTypes + .forEach( + (type, clazz) -> { + final String outputPath = dedupGraphPath + "/" + type; + removeOutputDir(spark, outputPath); - JavaRDD sourceEntity = - sc.textFile(DedupUtility.createEntityPath(graphBasePath, type.toString())); + JavaRDD sourceEntity = sc + .textFile(DedupUtility.createEntityPath(graphBasePath, type.toString())); - if (mergeRelExists(workingPath, type.toString())) { + if (mergeRelExists(workingPath, type.toString())) { - final String mergeRelPath = - DedupUtility.createMergeRelPath(workingPath, "*", type.toString()); - final String dedupRecordPath = - DedupUtility.createDedupRecordPath(workingPath, "*", type.toString()); + final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, "*", type.toString()); + final String dedupRecordPath = DedupUtility + .createDedupRecordPath(workingPath, "*", type.toString()); - final Dataset rel = - spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class)); + final Dataset rel = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class)); - final JavaPairRDD mergedIds = - rel.where("relClass == 'merges'") - .select(rel.col("target")) - .distinct() - .toJavaRDD() - .mapToPair( - (PairFunction) r -> new Tuple2<>(r.getString(0), "d")); + final JavaPairRDD mergedIds = rel + .where("relClass == 'merges'") + .select(rel.col("target")) + .distinct() + .toJavaRDD() + .mapToPair( + (PairFunction) r -> new Tuple2<>(r.getString(0), "d")); - JavaPairRDD entitiesWithId = - sourceEntity.mapToPair( - (PairFunction) - s -> new Tuple2<>(MapDocumentUtil.getJPathString(IDJSONPATH, s), s)); - JavaRDD map = - entitiesWithId - .leftOuterJoin(mergedIds) - .map( - k -> - k._2()._2().isPresent() - ? updateDeletedByInference(k._2()._1(), clazz) - : k._2()._1()); + JavaPairRDD entitiesWithId = sourceEntity + .mapToPair( + (PairFunction) s -> new Tuple2<>( + MapDocumentUtil.getJPathString(IDJSONPATH, s), s)); + JavaRDD map = entitiesWithId + .leftOuterJoin(mergedIds) + .map( + k -> k._2()._2().isPresent() + ? updateDeletedByInference(k._2()._1(), clazz) + : k._2()._1()); - sourceEntity = map.union(sc.textFile(dedupRecordPath)); - } + sourceEntity = map.union(sc.textFile(dedupRecordPath)); + } - sourceEntity.saveAsTextFile(outputPath, GzipCodec.class); - }); - } + sourceEntity.saveAsTextFile(outputPath, GzipCodec.class); + }); + } - public boolean mergeRelExists(String basePath, String entity) { + public boolean mergeRelExists(String basePath, String entity) { - boolean result = false; - try { - FileSystem fileSystem = FileSystem.get(new Configuration()); + boolean result = false; + try { + FileSystem fileSystem = FileSystem.get(new Configuration()); - FileStatus[] fileStatuses = fileSystem.listStatus(new Path(basePath)); + FileStatus[] fileStatuses = fileSystem.listStatus(new Path(basePath)); - for (FileStatus fs : fileStatuses) { - if (fs.isDirectory()) - if (fileSystem.exists( - new Path(DedupUtility.createMergeRelPath(basePath, fs.getPath().getName(), entity)))) - result = true; - } + for (FileStatus fs : fileStatuses) { + if (fs.isDirectory()) + if (fileSystem + .exists( + new Path(DedupUtility.createMergeRelPath(basePath, fs.getPath().getName(), entity)))) + result = true; + } - return result; - } catch (IOException e) { - throw new RuntimeException(e); - } - } + return result; + } catch (IOException e) { + throw new RuntimeException(e); + } + } - private static String updateDeletedByInference( - final String json, final Class clazz) { - try { - Oaf entity = OBJECT_MAPPER.readValue(json, clazz); - if (entity.getDataInfo() == null) entity.setDataInfo(new DataInfo()); - entity.getDataInfo().setDeletedbyinference(true); - return OBJECT_MAPPER.writeValueAsString(entity); - } catch (IOException e) { - throw new RuntimeException("Unable to convert json", e); - } - } + private static String updateDeletedByInference( + final String json, final Class clazz) { + try { + Oaf entity = OBJECT_MAPPER.readValue(json, clazz); + if (entity.getDataInfo() == null) + entity.setDataInfo(new DataInfo()); + entity.getDataInfo().setDeletedbyinference(true); + return OBJECT_MAPPER.writeValueAsString(entity); + } catch (IOException e) { + throw new RuntimeException("Unable to convert json", e); + } + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/ConnectedComponent.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/ConnectedComponent.java index f4370a79c..bfd2c25e2 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/ConnectedComponent.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/ConnectedComponent.java @@ -1,78 +1,84 @@ + package eu.dnetlib.dhp.oa.dedup.graph; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.oa.dedup.DedupUtility; -import eu.dnetlib.pace.util.PaceException; import java.io.IOException; import java.io.Serializable; import java.util.Set; + import org.apache.commons.lang.StringUtils; import org.codehaus.jackson.annotate.JsonIgnore; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.oa.dedup.DedupUtility; +import eu.dnetlib.pace.util.PaceException; + public class ConnectedComponent implements Serializable { - private Set docIds; - private String ccId; + private Set docIds; + private String ccId; - public ConnectedComponent() {} + public ConnectedComponent() { + } - public ConnectedComponent(Set docIds) { - this.docIds = docIds; - createID(); - } + 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_wf_001::" + DedupUtility.md5(s); - return ccId; - } else { - return docIds.iterator().next(); - } - } + public String createID() { + if (docIds.size() > 1) { + final String s = getMin(); + String prefix = s.split("\\|")[0]; + ccId = prefix + "|dedup_wf_001::" + DedupUtility.md5(s); + return ccId; + } else { + return docIds.iterator().next(); + } + } - @JsonIgnore - public String getMin() { + @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(); - } + 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); - } - } + @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 Set getDocIds() { + return docIds; + } - public void setDocIds(Set docIds) { - this.docIds = docIds; - } + public void setDocIds(Set docIds) { + this.docIds = docIds; + } - public String getCcId() { - return ccId; - } + public String getCcId() { + return ccId; + } - public void setCcId(String ccId) { - this.ccId = ccId; - } + public void setCcId(String ccId) { + this.ccId = ccId; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Block.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Block.java index 10b622497..4f0d95c8f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Block.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Block.java @@ -1,7 +1,6 @@ + package eu.dnetlib.dhp.oa.dedup.model; -import com.google.common.collect.Lists; -import eu.dnetlib.pace.model.MapDocument; import java.io.Serializable; import java.util.ArrayList; import java.util.Comparator; @@ -11,63 +10,71 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; +import com.google.common.collect.Lists; + +import eu.dnetlib.pace.model.MapDocument; + public class Block implements Serializable { - private String key; + private String key; - private List documents; + private List documents; - public Block() { - super(); - } + public Block() { + super(); + } - public static Block from(String key, MapDocument doc) { - Block block = new Block(); - block.setKey(key); - block.setDocuments(Lists.newArrayList(doc)); - return block; - } + public static Block from(String key, MapDocument doc) { + Block block = new Block(); + block.setKey(key); + block.setDocuments(Lists.newArrayList(doc)); + return block; + } - public static Block from(String key, Iterator blocks, String orderField, int maxSize) { - Block block = new Block(); - block.setKey(key); + public static Block from(String key, Iterator blocks, String orderField, int maxSize) { + Block block = new Block(); + block.setKey(key); - Iterable it = () -> blocks; + Iterable it = () -> blocks; - block.setDocuments( - StreamSupport.stream(it.spliterator(), false) - .flatMap(b -> b.getDocuments().stream()) - .sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue())) - .limit(maxSize) - .collect(Collectors.toCollection(ArrayList::new))); - return block; - } + block + .setDocuments( + StreamSupport + .stream(it.spliterator(), false) + .flatMap(b -> b.getDocuments().stream()) + .sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue())) + .limit(maxSize) + .collect(Collectors.toCollection(ArrayList::new))); + return block; + } - public static Block from(Block b1, Block b2, String orderField, int maxSize) { - Block block = new Block(); - block.setKey(b1.getKey()); - block.setDocuments( - Stream.concat(b1.getDocuments().stream(), b2.getDocuments().stream()) - .sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue())) - .limit(maxSize) - .collect(Collectors.toCollection(ArrayList::new))); + public static Block from(Block b1, Block b2, String orderField, int maxSize) { + Block block = new Block(); + block.setKey(b1.getKey()); + block + .setDocuments( + Stream + .concat(b1.getDocuments().stream(), b2.getDocuments().stream()) + .sorted(Comparator.comparing(a -> a.getFieldMap().get(orderField).stringValue())) + .limit(maxSize) + .collect(Collectors.toCollection(ArrayList::new))); - return block; - } + return block; + } - public String getKey() { - return key; - } + public String getKey() { + return key; + } - public void setKey(String key) { - this.key = key; - } + public void setKey(String key) { + this.key = key; + } - public List getDocuments() { - return documents; - } + public List getDocuments() { + return documents; + } - public void setDocuments(List documents) { - this.documents = documents; - } + public void setDocuments(List documents) { + this.documents = documents; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/MergeAuthorTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/MergeAuthorTest.java index 4236f32e3..a217a2657 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/MergeAuthorTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/MergeAuthorTest.java @@ -1,49 +1,54 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.schema.oaf.Publication; import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.codehaus.jackson.map.ObjectMapper; import org.junit.jupiter.api.BeforeEach; +import eu.dnetlib.dhp.schema.oaf.Publication; + public class MergeAuthorTest { - private List publicationsToMerge; - private final ObjectMapper mapper = new ObjectMapper(); + private List publicationsToMerge; + private final ObjectMapper mapper = new ObjectMapper(); - @BeforeEach - public void setUp() throws Exception { - final String json = - IOUtils.toString( - this.getClass().getResourceAsStream("/eu/dnetlib/dhp/dedup/json/authors_merge.json")); + @BeforeEach + public void setUp() throws Exception { + final String json = IOUtils + .toString( + this.getClass().getResourceAsStream("/eu/dnetlib/dhp/dedup/json/authors_merge.json")); - publicationsToMerge = - Arrays.asList(json.split("\n")).stream() - .map( - s -> { - try { - return mapper.readValue(s, Publication.class); - } catch (IOException e) { - throw new RuntimeException(e); - } - }) - .collect(Collectors.toList()); - } + publicationsToMerge = Arrays + .asList(json.split("\n")) + .stream() + .map( + s -> { + try { + return mapper.readValue(s, Publication.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + } - // FIX ME Michele DB this tests doesn't work - // @Test - public void test() throws Exception { - Publication dedup = new Publication(); + // FIX ME Michele DB this tests doesn't work + // @Test + public void test() throws Exception { + Publication dedup = new Publication(); - publicationsToMerge.forEach( - p -> { - dedup.mergeFrom(p); - dedup.setAuthor(DedupUtility.mergeAuthor(dedup.getAuthor(), p.getAuthor())); - }); + publicationsToMerge + .forEach( + p -> { + dedup.mergeFrom(p); + dedup.setAuthor(DedupUtility.mergeAuthor(dedup.getAuthor(), p.getAuthor())); + }); - System.out.println(mapper.writeValueAsString(dedup)); - } + System.out.println(mapper.writeValueAsString(dedup)); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java index 838e7188d..a0ae7bc3c 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java @@ -1,19 +1,17 @@ + package eu.dnetlib.dhp.oa.dedup; import static java.nio.file.Files.createTempDirectory; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.lenient; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.util.MapDocumentUtil; import java.io.File; import java.io.IOException; import java.io.Serializable; import java.net.URISyntaxException; import java.nio.file.Paths; + import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -30,424 +28,435 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; @ExtendWith(MockitoExtension.class) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) public class SparkDedupTest implements Serializable { - @Mock(serializable = true) - ISLookUpService isLookUpService; + @Mock(serializable = true) + ISLookUpService isLookUpService; - private static SparkSession spark; - private static JavaSparkContext jsc; + private static SparkSession spark; + private static JavaSparkContext jsc; - private static String testGraphBasePath; - private static String testOutputBasePath; - private static String testDedupGraphBasePath; - private static final String testActionSetId = "test-orchestrator"; + private static String testGraphBasePath; + private static String testOutputBasePath; + private static String testDedupGraphBasePath; + private static final String testActionSetId = "test-orchestrator"; - @BeforeAll - public static void cleanUp() throws IOException, URISyntaxException { + @BeforeAll + public static void cleanUp() throws IOException, URISyntaxException { - testGraphBasePath = - Paths.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) - .toFile() - .getAbsolutePath(); - testOutputBasePath = - createTempDirectory(SparkDedupTest.class.getSimpleName() + "-").toAbsolutePath().toString(); - testDedupGraphBasePath = - createTempDirectory(SparkDedupTest.class.getSimpleName() + "-").toAbsolutePath().toString(); + testGraphBasePath = Paths + .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) + .toFile() + .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + testDedupGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); - FileUtils.deleteDirectory(new File(testOutputBasePath)); - FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); - spark = - SparkSession.builder() - .appName(SparkDedupTest.class.getSimpleName()) - .master("local[*]") - .config(new SparkConf()) - .getOrCreate(); + spark = SparkSession + .builder() + .appName(SparkDedupTest.class.getSimpleName()) + .master("local[*]") + .config(new SparkConf()) + .getOrCreate(); - jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - } + jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } - @BeforeEach - public void setUp() throws IOException, ISLookUpException { + @BeforeEach + public void setUp() throws IOException, ISLookUpException { - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator.xml"))); + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator.xml"))); - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization"))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication"))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json"))); + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json"))); - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("software"))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json"))); + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("software"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json"))); - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("dataset"))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json"))); + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("dataset"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json"))); - lenient() - .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("otherresearchproduct"))) - .thenReturn( - IOUtils.toString( - SparkDedupTest.class.getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json"))); - } + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("otherresearchproduct"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json"))); + } - @Test - @Order(1) - public void createSimRelsTest() throws Exception { + @Test + @Order(1) + public void createSimRelsTest() throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateSimRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); - parser.parseArgument( - new String[] { - "-i", - testGraphBasePath, - "-asi", - testActionSetId, - "-la", - "lookupurl", - "-w", - testOutputBasePath - }); + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath + }); - new SparkCreateSimRels(parser, spark).run(isLookUpService); + new SparkCreateSimRels(parser, spark).run(isLookUpService); - long orgs_simrel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") - .count(); - long pubs_simrel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel") - .count(); - long sw_simrel = - spark.read().load(testOutputBasePath + "/" + testActionSetId + "/software_simrel").count(); + long orgs_simrel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") + .count(); + long pubs_simrel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel") + .count(); + long sw_simrel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/software_simrel").count(); - long ds_simrel = - spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel").count(); + long ds_simrel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel").count(); - long orp_simrel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") - .count(); + long orp_simrel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") + .count(); - assertEquals(3432, orgs_simrel); - assertEquals(7152, pubs_simrel); - assertEquals(344, sw_simrel); - assertEquals(458, ds_simrel); - assertEquals(6750, orp_simrel); - } + assertEquals(3432, orgs_simrel); + assertEquals(7152, pubs_simrel); + assertEquals(344, sw_simrel); + assertEquals(458, ds_simrel); + assertEquals(6750, orp_simrel); + } - @Test - @Order(2) - public void createMergeRelsTest() throws Exception { + @Test + @Order(2) + public void createMergeRelsTest() throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateMergeRels.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); - parser.parseArgument( - new String[] { - "-i", - testGraphBasePath, - "-asi", - testActionSetId, - "-la", - "lookupurl", - "-w", - testOutputBasePath - }); + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateMergeRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath + }); - new SparkCreateMergeRels(parser, spark).run(isLookUpService); + new SparkCreateMergeRels(parser, spark).run(isLookUpService); - long orgs_mergerel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") - .count(); - long pubs_mergerel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") - .count(); - long sw_mergerel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") - .count(); + long orgs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .count(); + long pubs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .count(); + long sw_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") + .count(); - long ds_mergerel = - spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel").count(); + long ds_mergerel = spark.read().load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel").count(); - long orp_mergerel = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") - .count(); + long orp_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") + .count(); - assertEquals(1276, orgs_mergerel); - assertEquals(1442, pubs_mergerel); - assertEquals(288, sw_mergerel); - assertEquals(472, ds_mergerel); - assertEquals(718, orp_mergerel); - } + assertEquals(1276, orgs_mergerel); + assertEquals(1442, pubs_mergerel); + assertEquals(288, sw_mergerel); + assertEquals(472, ds_mergerel); + assertEquals(718, orp_mergerel); + } - @Test - @Order(3) - public void createDedupRecordTest() throws Exception { + @Test + @Order(3) + public void createDedupRecordTest() throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateDedupRecord.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json"))); - parser.parseArgument( - new String[] { - "-i", - testGraphBasePath, - "-asi", - testActionSetId, - "-la", - "lookupurl", - "-w", - testOutputBasePath - }); + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateDedupRecord.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath + }); - new SparkCreateDedupRecord(parser, spark).run(isLookUpService); + new SparkCreateDedupRecord(parser, spark).run(isLookUpService); - long orgs_deduprecord = - jsc.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord") - .count(); - long pubs_deduprecord = - jsc.textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") - .count(); - long sw_deduprecord = - jsc.textFile(testOutputBasePath + "/" + testActionSetId + "/software_deduprecord").count(); - long ds_deduprecord = - jsc.textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_deduprecord").count(); - long orp_deduprecord = - jsc.textFile( - testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_deduprecord") - .count(); + long orgs_deduprecord = jsc + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord") + .count(); + long pubs_deduprecord = jsc + .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") + .count(); + long sw_deduprecord = jsc + .textFile(testOutputBasePath + "/" + testActionSetId + "/software_deduprecord") + .count(); + long ds_deduprecord = jsc.textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_deduprecord").count(); + long orp_deduprecord = jsc + .textFile( + testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_deduprecord") + .count(); - assertEquals(82, orgs_deduprecord); - assertEquals(66, pubs_deduprecord); - assertEquals(51, sw_deduprecord); - assertEquals(96, ds_deduprecord); - assertEquals(89, orp_deduprecord); - } + assertEquals(82, orgs_deduprecord); + assertEquals(66, pubs_deduprecord); + assertEquals(51, sw_deduprecord); + assertEquals(96, ds_deduprecord); + assertEquals(89, orp_deduprecord); + } - @Test - @Order(4) - public void updateEntityTest() throws Exception { + @Test + @Order(4) + public void updateEntityTest() throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkUpdateEntity.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); - parser.parseArgument( - new String[] { - "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath - }); + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkUpdateEntity.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath + }); - new SparkUpdateEntity(parser, spark).run(isLookUpService); + new SparkUpdateEntity(parser, spark).run(isLookUpService); - long organizations = jsc.textFile(testDedupGraphBasePath + "/organization").count(); - long publications = jsc.textFile(testDedupGraphBasePath + "/publication").count(); - long projects = jsc.textFile(testDedupGraphBasePath + "/project").count(); - long datasource = jsc.textFile(testDedupGraphBasePath + "/datasource").count(); - long softwares = jsc.textFile(testDedupGraphBasePath + "/software").count(); - long dataset = jsc.textFile(testDedupGraphBasePath + "/dataset").count(); - long otherresearchproduct = - jsc.textFile(testDedupGraphBasePath + "/otherresearchproduct").count(); + long organizations = jsc.textFile(testDedupGraphBasePath + "/organization").count(); + long publications = jsc.textFile(testDedupGraphBasePath + "/publication").count(); + long projects = jsc.textFile(testDedupGraphBasePath + "/project").count(); + long datasource = jsc.textFile(testDedupGraphBasePath + "/datasource").count(); + long softwares = jsc.textFile(testDedupGraphBasePath + "/software").count(); + long dataset = jsc.textFile(testDedupGraphBasePath + "/dataset").count(); + long otherresearchproduct = jsc.textFile(testDedupGraphBasePath + "/otherresearchproduct").count(); - long mergedOrgs = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + long mergedOrgs = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); - long mergedPubs = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + long mergedPubs = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); - long mergedSw = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + long mergedSw = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); - long mergedDs = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel") - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + long mergedDs = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); - long mergedOrp = - spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + long mergedOrp = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); - assertEquals(897, publications); - assertEquals(835, organizations); - assertEquals(100, projects); - assertEquals(100, datasource); - assertEquals(200, softwares); - assertEquals(388, dataset); - assertEquals(517, otherresearchproduct); + assertEquals(897, publications); + assertEquals(835, organizations); + assertEquals(100, projects); + assertEquals(100, datasource); + assertEquals(200, softwares); + assertEquals(388, dataset); + assertEquals(517, otherresearchproduct); - long deletedOrgs = - jsc.textFile(testDedupGraphBasePath + "/organization") - .filter(this::isDeletedByInference) - .count(); + long deletedOrgs = jsc + .textFile(testDedupGraphBasePath + "/organization") + .filter(this::isDeletedByInference) + .count(); - long deletedPubs = - jsc.textFile(testDedupGraphBasePath + "/publication") - .filter(this::isDeletedByInference) - .count(); + long deletedPubs = jsc + .textFile(testDedupGraphBasePath + "/publication") + .filter(this::isDeletedByInference) + .count(); - long deletedSw = - jsc.textFile(testDedupGraphBasePath + "/software") - .filter(this::isDeletedByInference) - .count(); + long deletedSw = jsc + .textFile(testDedupGraphBasePath + "/software") + .filter(this::isDeletedByInference) + .count(); - long deletedDs = - jsc.textFile(testDedupGraphBasePath + "/dataset") - .filter(this::isDeletedByInference) - .count(); + long deletedDs = jsc + .textFile(testDedupGraphBasePath + "/dataset") + .filter(this::isDeletedByInference) + .count(); - long deletedOrp = - jsc.textFile(testDedupGraphBasePath + "/otherresearchproduct") - .filter(this::isDeletedByInference) - .count(); + long deletedOrp = jsc + .textFile(testDedupGraphBasePath + "/otherresearchproduct") + .filter(this::isDeletedByInference) + .count(); - assertEquals(mergedOrgs, deletedOrgs); - assertEquals(mergedPubs, deletedPubs); - assertEquals(mergedSw, deletedSw); - assertEquals(mergedDs, deletedDs); - assertEquals(mergedOrp, deletedOrp); - } + assertEquals(mergedOrgs, deletedOrgs); + assertEquals(mergedPubs, deletedPubs); + assertEquals(mergedSw, deletedSw); + assertEquals(mergedDs, deletedDs); + assertEquals(mergedOrp, deletedOrp); + } - @Test - @Order(5) - public void propagateRelationTest() throws Exception { + @Test + @Order(5) + public void propagateRelationTest() throws Exception { - ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkPropagateRelation.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"))); - parser.parseArgument( - new String[] { - "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath - }); + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkPropagateRelation.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath + }); - new SparkPropagateRelation(parser, spark).run(isLookUpService); + new SparkPropagateRelation(parser, spark).run(isLookUpService); - long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); + long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); - assertEquals(826, relations); + assertEquals(826, relations); - // check deletedbyinference - final Dataset mergeRels = - spark - .read() - .load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*")) - .as(Encoders.bean(Relation.class)); - final JavaPairRDD mergedIds = - mergeRels - .where("relClass == 'merges'") - .select(mergeRels.col("target")) - .distinct() - .toJavaRDD() - .mapToPair( - (PairFunction) - r -> new Tuple2(r.getString(0), "d")); + // check deletedbyinference + final Dataset mergeRels = spark + .read() + .load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*")) + .as(Encoders.bean(Relation.class)); + final JavaPairRDD mergedIds = mergeRels + .where("relClass == 'merges'") + .select(mergeRels.col("target")) + .distinct() + .toJavaRDD() + .mapToPair( + (PairFunction) r -> new Tuple2(r.getString(0), "d")); - JavaRDD toCheck = - jsc.textFile(testDedupGraphBasePath + "/relation") - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()) - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()); + JavaRDD toCheck = jsc + .textFile(testDedupGraphBasePath + "/relation") + .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json)) + .join(mergedIds) + .map(t -> t._2()._1()) + .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json)) + .join(mergedIds) + .map(t -> t._2()._1()); - long deletedbyinference = toCheck.filter(this::isDeletedByInference).count(); - long updated = toCheck.count(); + long deletedbyinference = toCheck.filter(this::isDeletedByInference).count(); + long updated = toCheck.count(); - assertEquals(updated, deletedbyinference); - } + assertEquals(updated, deletedbyinference); + } - @AfterAll - public static void finalCleanUp() throws IOException { - FileUtils.deleteDirectory(new File(testOutputBasePath)); - FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); - } + @AfterAll + public static void finalCleanUp() throws IOException { + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + } - public boolean isDeletedByInference(String s) { - return s.contains("\"deletedbyinference\":true"); - } + public boolean isDeletedByInference(String s) { + return s.contains("\"deletedbyinference\":true"); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java index 7c8d937ce..9518efdb5 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/jpath/JsonPathTest.java @@ -1,291 +1,292 @@ + package eu.dnetlib.dhp.oa.dedup.jpath; +import org.junit.jupiter.api.Test; + import eu.dnetlib.pace.config.DedupConfig; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.util.MapDocumentUtil; -import org.junit.jupiter.api.Test; public class JsonPathTest { - String json = - "{\t\"dataInfo\":{\t\t\"invisible\":false,\t\t\"inferred\":false,\t\t\"deletedbyinference\":false,\t\t\"trust\":\"0.810000002384185791\",\t\t\"inferenceprovenance\":\"\",\t\t\"provenanceaction\":{\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t}\t},\t\"lastupdatetimestamp\":1584960968152,\t\"id\":\"20|corda__h2020::9faf23721249f26ac2c16eb857ea1fb9\",\t\"originalId\":[\t\t\"corda__h2020::927957582\"\t],\t\"collectedfrom\":[\t\t{\t\t\t\"key\":\"openaire____::corda_h2020\",\t\t\t\"value\":\"CORDA - COmmon Research DAta Warehouse - Horizon 2020\",\t\t\t\"dataInfo\":null\t\t}\t],\t\"pid\":[\t],\t\"dateofcollection\":\"2016-06-05\",\t\"dateoftransformation\":\"2019-11-19\",\t\"extraInfo\":[\t],\t\"oaiprovenance\":null,\t\"legalshortname\":{\t\t\"value\":\"Comentor AB\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"legalname\":{\t\t\"value\":\"Comentor AB\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"alternativeNames\":[\t],\t\"websiteurl\":{\t\t\"value\":\"http://www.comentor.se\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"logourl\":null,\t\"eclegalbody\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"eclegalperson\":{\t\t\"value\":\"true\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecnonprofit\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecresearchorganization\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"echighereducation\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecinternationalorganizationeurinterests\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecinternationalorganization\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecenterprise\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecsmevalidated\":{\t\t\"value\":\"true\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecnutscode\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"country\":null}"; - DedupConfig conf = - DedupConfig.load( - "{\n" - + " \"wf\" : {\n" - + " \"threshold\" : \"0.99\",\n" - + " \"dedupRun\" : \"001\",\n" - + " \"entityType\" : \"organization\",\n" - + " \"subEntityValue\": \"organization\",\n" - + " \"orderField\" : \"legalname\",\n" - + " \"queueMaxSize\" : \"2000\",\n" - + " \"groupMaxSize\" : \"50\",\n" - + " \"slidingWindowSize\" : \"200\",\n" - + " \"idPath\":\"$.id\",\n" - + " \"rootBuilder\" : [ \"organization\", \"projectOrganization_participation_isParticipant\", \"datasourceOrganization_provision_isProvidedBy\" ],\n" - + " \"includeChildren\" : \"true\",\n" - + " \"maxIterations\": \"20\"\n" - + " },\n" - + " \"pace\" : {\n" - + " \"clustering\" : [\n" - + " { \"name\" : \"sortedngrampairs\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\" : 2, \"ngramLen\" : \"3\"} },\n" - + " { \"name\" : \"suffixprefix\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\" : 1, \"len\" : \"3\" } },\n" - + " { \"name\" : \"urlclustering\", \"fields\" : [ \"websiteurl\" ], \"params\" : { } },\n" - + " { \"name\" : \"keywordsclustering\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\": 2, \"windowSize\": 4} }\n" - + " ],\n" - + " \"decisionTree\" : {\n" - + " \"start\": {\n" - + " \"fields\": [\n" - + " {\n" - + " \"field\": \"gridid\",\n" - + " \"comparator\": \"exactMatch\",\n" - + " \"weight\": 1,\n" - + " \"countIfUndefined\": \"false\",\n" - + " \"params\": {}\n" - + " }\n" - + " ],\n" - + " \"threshold\": 1,\n" - + " \"aggregation\": \"AVG\",\n" - + " \"positive\": \"MATCH\",\n" - + " \"negative\": \"NO_MATCH\",\n" - + " \"undefined\": \"layer2\",\n" - + " \"ignoreUndefined\": \"false\"\n" - + " },\n" - + " \"layer2\": {\n" - + " \"fields\": [\n" - + " {\n" - + " \"field\": \"websiteurl\",\n" - + " \"comparator\": \"domainExactMatch\",\n" - + " \"weight\": 1,\n" - + " \"countIfUndefined\": \"false\",\n" - + " \"params\": {}\n" - + " },\n" - + " {\n" - + " \"field\": \"country\",\n" - + " \"comparator\": \"exactMatch\",\n" - + " \"weight\": 1,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {}\n" - + " },\n" - + " {\n" - + " \"field\": \"legalname\",\n" - + " \"comparator\": \"numbersMatch\",\n" - + " \"weight\": 1,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {}\n" - + " },\n" - + " {\n" - + " \"field\": \"legalname\",\n" - + " \"comparator\": \"romansMatch\",\n" - + " \"weight\": 1,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {}\n" - + " }\n" - + " ],\n" - + " \"threshold\": 1,\n" - + " \"aggregation\": \"AND\",\n" - + " \"positive\": \"layer3\",\n" - + " \"negative\": \"NO_MATCH\",\n" - + " \"undefined\": \"layer3\",\n" - + " \"ignoreUndefined\": \"true\"\n" - + " },\n" - + " \"layer3\": {\n" - + " \"fields\": [\n" - + " {\n" - + " \"field\": \"legalname\",\n" - + " \"comparator\": \"cityMatch\",\n" - + " \"weight\": 1.0,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {\n" - + " \"windowSize\": \"4\"\n" - + " }\n" - + " }\n" - + " ],\n" - + " \"threshold\": 0.1,\n" - + " \"aggregation\": \"AVG\",\n" - + " \"positive\": \"layer4\",\n" - + " \"negative\": \"NO_MATCH\",\n" - + " \"undefined\": \"NO_MATCH\",\n" - + " \"ignoreUndefined\": \"true\"\n" - + " },\n" - + " \"layer4\": {\n" - + " \"fields\": [\n" - + " {\n" - + " \"field\": \"legalname\",\n" - + " \"comparator\": \"keywordMatch\",\n" - + " \"weight\": 1.0,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {\n" - + " \"windowSize\": \"4\"\n" - + " }\n" - + " }\n" - + " ],\n" - + " \"threshold\": 0.7,\n" - + " \"aggregation\": \"AVG\",\n" - + " \"positive\": \"layer5\",\n" - + " \"negative\": \"NO_MATCH\",\n" - + " \"undefined\": \"layer5\",\n" - + " \"ignoreUndefined\": \"true\"\n" - + " },\n" - + " \"layer5\": {\n" - + " \"fields\": [\n" - + " {\n" - + " \"field\": \"legalname\",\n" - + " \"comparator\": \"jaroWinklerNormalizedName\",\n" - + " \"weight\": 0.9,\n" - + " \"countIfUndefined\": \"true\",\n" - + " \"params\": {\n" - + " \"windowSize\": \"4\"\n" - + " }\n" - + " },\n" - + " {\n" - + " \"field\": \"legalshortname\",\n" - + " \"comparator\": \"jaroWinklerNormalizedName\",\n" - + " \"weight\": 0.1,\n" - + " \"countIfUndefined\": \"false\",\n" - + " \"params\": {\n" - + " \"windowSize\": 4\n" - + " }\n" - + " }\n" - + " ],\n" - + " \"threshold\": 0.9,\n" - + " \"aggregation\": \"W_MEAN\",\n" - + " \"positive\": \"MATCH\",\n" - + " \"negative\": \"NO_MATCH\",\n" - + " \"undefined\": \"NO_MATCH\",\n" - + " \"ignoreUndefined\": \"true\"\n" - + " }\n" - + " },\n" - + " \"model\" : [\n" - + " { \"name\" : \"country\", \"type\" : \"String\", \"path\" : \"$.country.classid\"},\n" - + " { \"name\" : \"legalshortname\", \"type\" : \"String\", \"path\" : \"$.legalshortname.value\"},\n" - + " { \"name\" : \"legalname\", \"type\" : \"String\", \"path\" : \"$.legalname.value\" },\n" - + " { \"name\" : \"websiteurl\", \"type\" : \"URL\", \"path\" : \"$.websiteurl.value\" },\n" - + " { \"name\" : \"gridid\", \"type\" : \"String\", \"path\" : \"$.pid[?(@.qualifier.classid =='grid')].value\"},\n" - + " { \"name\" : \"originalId\", \"type\" : \"String\", \"path\" : \"$.id\" }\n" - + " ],\n" - + " \"blacklists\" : {\n" - + " \"legalname\" : []\n" - + " },\n" - + " \"synonyms\": {\n" - + " \"key::1\": [\"university\",\"università\", \"universitas\", \"università studi\",\"universitario\",\"universitaria\",\"université\", \"universite\", \"universitaire\",\"universitaires\",\"universidad\",\"universitade\",\"Universität\",\"universitaet\",\"Uniwersytet\",\"университет\",\"universiteit\",\"πανεπιστήμιο\",\"universitesi\",\"universiteti\", \"universiti\"],\n" - + " \"key::2\": [\"studies\",\"studi\",\"études\",\"estudios\",\"estudos\",\"Studien\",\"studia\",\"исследования\",\"studies\",\"σπουδές\"],\n" - + " \"key::3\": [\"advanced\",\"superiore\",\"supérieur\",\"supérieure\",\"supérieurs\",\"supérieures\",\"avancado\",\"avancados\",\"fortgeschrittene\",\"fortgeschritten\",\"zaawansowany\",\"передовой\",\"gevorderd\",\"gevorderde\",\"προχωρημένος\",\"προχωρημένη\",\"προχωρημένο\",\"προχωρημένες\",\"προχωρημένα\",\"wyzsza\"],\n" - + " \"key::4\": [\"institute\",\"istituto\",\"institut\",\"instituto\",\"instituto\",\"Institut\",\"instytut\",\"институт\",\"instituut\",\"ινστιτούτο\"],\n" - + " \"key::5\": [\"hospital\",\"ospedale\",\"hôpital\",\"hospital\",\"hospital\",\"Krankenhaus\",\"szpital\",\"больница\",\"ziekenhuis\",\"νοσοκομείο\"],\n" - + " \"key::6\": [\"research\",\"ricerca\",\"recherche\",\"investigacion\",\"pesquisa\",\"Forschung\",\"badania\",\"исследования\",\"onderzoek\",\"έρευνα\",\"erevna\",\"erevnas\"],\n" - + " \"key::7\": [\"college\",\"collegio\",\"colegio\",\"faculdade\",\"Hochschule\",\"Szkoła Wyższa\",\"Высшая школа\",\"κολλέγιο\"],\n" - + " \"key::8\": [\"foundation\",\"fondazione\",\"fondation\",\"fundación\",\"fundação\",\"Stiftung\",\"Fundacja\",\"фонд\",\"stichting\",\"ίδρυμα\",\"idryma\"],\n" - + " \"key::9\": [\"center\",\"centro\",\"centre\",\"centro\",\"centro\",\"zentrum\",\"centrum\",\"центр\",\"centrum\",\"κέντρο\"],\n" - + " \"key::10\": [\"national\",\"nazionale\",\"national\",\"nationale\",\"nationaux\",\"nationales\",\"nacional\",\"nacional\",\"national\",\"krajowy\",\"национальный\",\"nationaal\",\"nationale\",\"εθνικό\"],\n" - + " \"key::11\": [\"association\",\"associazione\",\"association\",\"asociación\",\"associação\",\"Verein\",\"verband\",\"stowarzyszenie\",\"ассоциация\",\"associatie\"],\n" - + " \"key::12\": [\"society\",\"societa\",\"société\",\"sociedad\",\"sociedade\",\"gesellschaft\",\"społeczeństwo\",\"общество\",\"maatschappij\",\"κοινωνία\"],\n" - + " \"key::13\": [\"international\",\"internazionale\",\"international\",\"internacional\",\"internacional\",\"international\",\"międzynarodowy\",\"Международный\",\"internationaal\",\"internationale\",\"διεθνής\",\"διεθνή\",\"διεθνές\"],\n" - + " \"key::14\": [\"community\",\"comunita\",\"communauté\",\"comunidad\",\"comunidade\",\"Gemeinschaft\",\"społeczność\",\"сообщество\",\"gemeenschap\",\"κοινότητα\"],\n" - + " \"key::15\": [\"school\",\"scuola\",\"école\",\"escuela\",\"escola\",\"schule\",\"Szkoła\",\"школа\",\"school\",\"σχολείο\"],\n" - + " \"key::16\": [\"education\",\"educazione\",\"éducation\",\"educacion\",\"Educação\",\"Bildung\",\"Edukacja\",\"образование\",\"opleiding\",\"εκπαίδευση\"],\n" - + " \"key::17\": [\"academy\",\"accademia\",\"académie\",\"academia\",\"academia\",\"Akademie\",\"akademie\",\"академия\",\"academie\",\"ακαδημία\"],\n" - + " \"key::18\": [\"public\",\"pubblico\",\"public\",\"publique\",\"publics\",\"publiques\",\"publico\",\"publico\",\"Öffentlichkeit\",\"publiczny\",\"публичный\",\"publiek\",\"publieke\",\"δημόσιος\",\"δημόσια\",\"δημόσιο\"],\n" - + " \"key::19\": [\"museum\",\"museo\",\"musée\",\"mueso\",\"museu\",\"museum\",\"muzeum\",\"музей\",\"museum\",\"μουσείο\"],\n" - + " \"key::20\": [\"group\",\"gruppo\",\"groupe\",\"grupo\",\"grupo\",\"gruppe\",\"grupa\",\"группа\",\"groep\",\"ομάδα\",\"όμιλος\"],\n" - + " \"key::21\": [\"department\",\"dipartimento\",\"département\",\"departamento\",\"departamento\",\"abteilung\",\"departament\",\"отдел\",\"afdeling\",\"τμήμα\"],\n" - + " \"key::22\": [\"council\",\"consiglio\",\"conseil\",\"Consejo\",\"conselho\",\"gesellschaft\",\"rada\",\"совет\",\"raad\",\"συμβούλιο\"],\n" - + " \"key::23\": [\"library\",\"biblioteca\",\"bibliothèque\",\"biblioteca\",\"biblioteca\",\"Bibliothek\",\"biblioteka\",\"библиотека\",\"bibliotheek\",\"βιβλιοθήκη\"],\n" - + " \"key::24\": [\"ministry\",\"ministero\",\"ministère\",\"ministerio\",\"ministério\",\"Ministerium\",\"ministerstwo\",\"министерство\",\"ministerie\",\"υπουργείο\"],\n" - + " \"key::25\": [\"services\",\"servizi\",\"services\",\"servicios\",\"Serviços\",\"Dienstleistungen\",\"usługi\",\"услуги\",\"diensten\",\"υπηρεσίες\"],\n" - + " \"key::26\": [\"central\",\"centrale\",\"central\",\"centrale\",\"centrales\",\"central\",\"central\",\"zentral\",\"centralny\",\"цетральный\",\"centraal\",\"κεντρικός\",\"κεντρική\",\"κεντρικό\",\"κεντρικά\"],\n" - + " \"key::27\": [\"general\",\"generale\",\"général\",\"générale\",\"généraux\",\"générales\",\"general\",\"geral\",\"general\",\"Allgemeines\",\"general\",\"общий\",\"algemeen\",\"algemene\",\"γενικός\",\"γενική\",\"γενικό\",\"γενικά\"],\n" - + " \"key::28\": [\"applied\",\"applicati\",\"appliqué\",\"appliquée\",\"appliqués\",\"appliquées\",\"aplicado\",\"aplicada\",\"angewendet\",\"stosowany\",\"прикладной\",\"toegepast\",\"toegepaste\",\"εφαρμοσμένος\",\"εφαρμοσμένη\",\"εφαρμοσμένο\",\"εφαρμοσμένα\"],\n" - + " \"key::29\": [\"european\",\"europee\",\"europea\",\"européen\",\"européenne\",\"européens\",\"européennes\",\"europeo\",\"europeu\",\"europäisch\",\"europejski\",\"европейский\",\"Europees\",\"Europese\",\"ευρωπαϊκός\",\"ευρωπαϊκή\",\"ευρωπαϊκό\",\"ευρωπαϊκά\"],\n" - + " \"key::30\": [\"agency\",\"agenzia\",\"agence\",\"agencia\",\"agencia\",\"agentur\",\"agencja\",\"агенция\",\"agentschap\",\"πρακτορείο\"],\n" - + " \"key::31\": [\"laboratory\",\"laboratorio\",\"laboratoire\",\"laboratorio\",\"laboratorio\",\"labor\",\"laboratorium\",\"лаборатория\",\"laboratorium\",\"εργαστήριο\"],\n" - + " \"key::32\": [\"industry\",\"industria\",\"industrie\",\"индустрия\",\"industrie\",\"βιομηχανία\"],\n" - + " \"key::33\": [\"industrial\",\"industriale\",\"industriel\",\"industrielle\",\"industriels\",\"industrielles\",\"индустриальный\",\"industrieel\",\"βιομηχανικός\",\"βιομηχανική\",\"βιομηχανικό\",\"βιομηχανικά\",\"βιομηχανικές\"],\n" - + " \"key::34\": [\"consortium\",\"consorzio\",\"consortium\",\"консорциум\",\"consortium\",\"κοινοπραξία\"],\n" - + " \"key::35\": [\"organization\",\"organizzazione\",\"organisation\",\"organización\",\"organização\",\"organizacja\",\"организация\",\"organisatie\",\"οργανισμός\"],\n" - + " \"key::36\": [\"authority\",\"autorità\",\"autorité\",\"авторитет\",\"autoriteit\"],\n" - + " \"key::37\": [\"federation\",\"federazione\",\"fédération\",\"федерация\",\"federatie\",\"ομοσπονδία\"],\n" - + " \"key::38\": [\"observatory\",\"osservatorio\",\"observatoire\",\"обсерватория\",\"observatorium\",\"αστεροσκοπείο\"],\n" - + " \"key::39\": [\"bureau\",\"ufficio\",\"bureau\",\"офис\",\"bureau\",\"γραφείο\"],\n" - + " \"key::40\": [\"company\",\"impresa\",\"compagnie\",\"société\",\"компания\",\"bedrijf\",\"εταιρία\"],\n" - + " \"key::41\": [\"polytechnic\",\"politecnico\",\"polytechnique\",\"политехника\",\"polytechnisch\",\"πολυτεχνείο\",\"universita politecnica\",\"polytechnic university\",\"universidad politecnica\",\"universitat politecnica\",\"politechnika\",\"politechniki\",\"university technology\",\"university science technology\"],\n" - + " \"key::42\": [\"coalition\",\"coalizione\",\"coalition\",\"коалиция\",\"coalitie\",\"συνασπισμός\"],\n" - + " \"key::43\": [\"initiative\",\"iniziativa\",\"initiative\",\"инициатива\",\"initiatief\",\"πρωτοβουλία\"],\n" - + " \"key::44\": [\"academic\",\"accademico\",\"académique\",\"universitaire\",\"акадеческий academisch\",\"ακαδημαϊκός\",\"ακαδημαϊκή\",\"ακαδημαϊκό\",\"ακαδημαϊκές\",\"ακαδημαϊκοί\"],\n" - + " \"key::45\": [\"institution\",\"istituzione\",\"institution\",\"институциональный\",\"instelling\",\"ινστιτούτο\"],\n" - + " \"key::46\": [\"division\",\"divisione\",\"division\",\"отделение\",\"divisie\",\"τμήμα\"],\n" - + " \"key::47\": [\"committee\",\"comitato\",\"comité\",\"комитет\",\"commissie\",\"επιτροπή\"],\n" - + " \"key::48\": [\"promotion\",\"promozione\",\"продвижение\",\"proothisis\",\"forderung\"],\n" - + " \"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\"],\n" - + " \"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\"],\n" - + " \"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\",\"\"],\n" - + " \"key::52\": [\"engineering\",\"ingegneria\",\"ingeniería\",\"engenharia\",\"Ingenieurwissenschaft\",\"ingenieurswetenschappen\",\"bouwkunde\",\"μηχανικός\",\"μηχανική\",\"μηχανικό\",\"mühendislik\",\"mérnöki\",\"Inženirstvo\",\"inseneeria\",\"inseneri\",\"\"],\n" - + " \"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\",\"\"],\n" - + " \"key::54\": [\"energy\",\"energia\",\"energía\",\"energia\",\"Energie\",\"energie\",\"ενέργεια\",\"enerji\",\"energia\",\"energija\",\"energia\",\"\"],\n" - + " \"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\",\"\"],\n" - + " \"key::56\": [\"information\",\"informazione\",\"información\",\"informação\",\"Information\",\"informatie\",\"πληροφορία\",\"bilgi\",\"információ\",\"informacija\",\"informatsioon\",\"informatycznych\",\"\"],\n" - + " \"key::57\": [\"social\",\"sociali\",\"social\",\"social\",\"Sozial\",\"sociaal\",\"maatschappelijk\",\"κοινωνικός\",\"κοινωνική\",\"κοινωνικό\",\"κοινωνικά\",\"sosyal\",\"szociális\",\"družbeni\",\"sotsiaal\",\"sotsiaalne\",\"\"],\n" - + " \"key::58\": [\"environmental\",\"ambiente\",\"medioambiental\",\"ambiente\",\"medioambiente\",\"meioambiente\",\"Umwelt\",\"milieu\",\"milieuwetenschap\",\"milieukunde\",\"περιβαλλοντικός\",\"περιβαλλοντική\",\"περιβαλλοντικό\",\"περιβαλλοντικά\",\"çevre\",\"környezeti\",\"okoliški\",\"keskonna\",\"\"],\n" - + " \"key::59\": [\"business\",\"economia\",\"economiche\",\"economica\",\"negocio\",\"empresa\",\"negócio\",\"Unternehmen\",\"bedrijf\",\"bedrijfskunde\",\"επιχείρηση\",\"iş\",\"üzleti\",\"posel\",\"ettevõte/äri\",\"\"],\n" - + " \"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\",\"\"],\n" - + " \"key::61\": [\"healthcare\",\"health services\",\"salute\",\"atenciónmédica\",\"cuidadodelasalud\",\"cuidadoscomasaúde\",\"Gesundheitswesen\",\"gezondheidszorg\",\"ιατροφαρμακευτικήπερίθαλψη\",\"sağlıkhizmeti\",\"egészségügy\",\"zdravstvo\",\"tervishoid\",\"tervishoiu\",\"\"],\n" - + " \"key::62\": [\"history\",\"storia\",\"historia\",\"história\",\"Geschichte\",\"geschiedenis\",\"geschiedkunde\",\"ιστορία\",\"tarih\",\"történelem\",\"zgodovina\",\"ajalugu\",\"\"],\n" - + " \"key::63\": [\"materials\",\"materiali\",\"materia\",\"materiales\",\"materiais\",\"materialen\",\"υλικά\",\"τεκμήρια\",\"malzemeler\",\"anyagok\",\"materiali\",\"materjalid\",\"vahendid\",\"\"],\n" - + " \"key::64\": [\"economics\",\"economia\",\"economiche\",\"economica\",\"economía\",\"economia\",\"Wirtschaft\",\"economie\",\"οικονομικά\",\"οικονομικέςεπιστήμες\",\"ekonomi\",\"közgazdaságtan\",\"gospodarstvo\",\"ekonomija\",\"majanduslik\",\"majandus\",\"\"],\n" - + " \"key::65\": [\"therapeutics\",\"terapeutica\",\"terapéutica\",\"terapêutica\",\"therapie\",\"θεραπευτική\",\"tedavibilimi\",\"gyógykezelés\",\"terapevtika\",\"terapeutiline\",\"ravi\",\"\"],\n" - + " \"key::66\": [\"oncology\",\"oncologia\",\"oncologico\",\"oncología\",\"oncologia\",\"Onkologie\",\"oncologie\",\"ογκολογία\",\"onkoloji\",\"onkológia\",\"onkologija\",\"onkoloogia\",\"\"],\n" - + " \"key::67\": [\"natural\",\"naturali\",\"naturale\",\"natural\",\"natural\",\"natürlich\",\"natuurlijk\",\"φυσικός\",\"φυσική\",\"φυσικό\",\"φυσικά\",\"doğal\",\"természetes\",\"naraven\",\"loodus\",\"\"],\n" - + " \"key::68\": [\"educational\",\"educazione\",\"pedagogia\",\"educacional\",\"educativo\",\"educacional\",\"pädagogisch\",\"educatief\",\"εκπαιδευτικός\",\"εκπαιδευτική\",\"εκπαιδευτικό\",\"εκπαιδευτικά\",\"eğitimsel\",\"oktatási\",\"izobraževalen\",\"haridus\",\"hariduslik\",\"\"],\n" - + " \"key::69\": [\"biomedical\",\"biomedica\",\"biomédico\",\"biomédico\",\"biomedizinisch\",\"biomedisch\",\"βιοιατρικός\",\"βιοιατρική\",\"βιοιατρικό\",\"βιοιατρικά\",\"biyomedikal\",\"orvosbiológiai\",\"biomedicinski\",\"biomeditsiiniline\",\"\"],\n" - + " \"key::70\": [\"veterinary\",\"veterinaria\",\"veterinarie\",\"veterinaria\",\"veterinária\",\"tierärtzlich\",\"veterinair\",\"veeartsenijlkunde\",\"κτηνιατρικός\",\"κτηνιατρική\",\"κτηνιατρικό\",\"κτηνιατρικά\",\"veteriner\",\"állatorvosi\",\"veterinar\",\"veterinarski\",\"veterinaaria\",\"\"],\n" - + " \"key::71\": [\"chemistry\",\"chimica\",\"química\",\"química\",\"Chemie\",\"chemie\",\"scheikunde\",\"χημεία\",\"kimya\",\"kémia\",\"kemija\",\"keemia\",\"\"],\n" - + " \"key::72\": [\"security\",\"sicurezza\",\"seguridad\",\"segurança\",\"Sicherheit\",\"veiligheid\",\"ασφάλεια\",\"güvenlik\",\"biztonsági\",\"varnost\",\"turvalisus\",\"julgeolek\",\"\"],\n" - + " \"key::73\": [\"biotechnology\",\"biotecnologia\",\"biotecnologie\",\"biotecnología\",\"biotecnologia\",\"Biotechnologie\",\"biotechnologie\",\"βιοτεχνολογία\",\"biyoteknoloji\",\"biotechnológia\",\"biotehnologija\",\"biotehnoloogia\",\"\"],\n" - + " \"key::74\": [\"military\",\"militare\",\"militari\",\"militar\",\"militar\",\"Militär\",\"militair\",\"leger\",\"στρατιωτικός\",\"στρατιωτική\",\"στρατιωτικό\",\"στρατιωτικά\",\"askeri\",\"katonai\",\"vojaški\",\"vojni\",\"militaar\",\"wojskowa\",\"\"],\n" - + " \"key::75\": [\"theological\",\"teologia\",\"teologico\",\"teológico\",\"tecnológica\",\"theologisch\",\"theologisch\",\"θεολογικός\",\"θεολογική\",\"θεολογικό\",\"θεολογικά\",\"teolojik\",\"technológiai\",\"teološki\",\"teoloogia\",\"usuteadus\",\"teoloogiline\",\"\"],\n" - + " \"key::76\": [\"electronics\",\"elettronica\",\"electrónica\",\"eletrônicos\",\"Elektronik\",\"elektronica\",\"ηλεκτρονική\",\"elektronik\",\"elektronika\",\"elektronika\",\"elektroonika\",\"\"],\n" - + " \"key::77\": [\"forestry\",\"forestale\",\"forestali\",\"silvicultura\",\"forestal\",\"floresta\",\"Forstwirtschaft\",\"bosbouw\",\"δασοκομία\",\"δασολογία\",\"ormancılık\",\"erdészet\",\"gozdarstvo\",\"metsandus\",\"\"],\n" - + " \"key::78\": [\"maritime\",\"marittima\",\"marittime\",\"marittimo\",\"marítimo\",\"marítimo\",\"maritiem\",\"ναυτικός\",\"ναυτική\",\"ναυτικό\",\"ναυτικά\",\"ναυτιλιακός\",\"ναυτιλιακή\",\"ναυτιλιακό\",\"ναυτιλιακά\",\"θαλάσσιος\",\"θαλάσσια\",\"θαλάσσιο\",\"denizcilik\",\"tengeri\",\"morski\",\"mere\",\"merendus\",\"\"],\n" - + " \"key::79\": [\"sports\",\"sport\",\"deportes\",\"esportes\",\"Sport\",\"sport\",\"sportwetenschappen\",\"άθληση\",\"γυμναστικήδραστηριότητα\",\"spor\",\"sport\",\"šport\",\"sport\",\"spordi\",\"\"],\n" - + " \"key::80\": [\"surgery\",\"chirurgia\",\"chirurgiche\",\"cirugía\",\"cirurgia\",\"Chirurgie\",\"chirurgie\",\"heelkunde\",\"εγχείρηση\",\"επέμβαση\",\"χειρουργικήεπέμβαση\",\"cerrahi\",\"sebészet\",\"kirurgija\",\"kirurgia\",\"\"],\n" - + " \"key::81\": [\"cultural\",\"culturale\",\"culturali\",\"cultura\",\"cultural\",\"cultural\",\"kulturell\",\"cultureel\",\"πολιτιστικός\",\"πολιτιστική\",\"πολιτιστικό\",\"πολιτισμικός\",\"πολιτισμική\",\"πολιτισμικό\",\"kültürel\",\"kultúrális\",\"kulturni\",\"kultuuri\",\"kultuuriline\",\"\"],\n" - + " \"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\",\"\"],\n" - + " \"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\",\"\"],\n" - + " \"key::84\": [\"communication\",\"comunicazione\",\"comuniciación\",\"comunicação\",\"Kommunikation\",\"communication\",\"επικοινωνία\",\"iletişim\",\"kommunikáció\",\"komuniciranje\",\"kommunikatsioon\",\"\"],\n" - + " \"key::85\": [\"justice\",\"giustizia\",\"justicia\",\"justiça\",\"Recht\",\"Justiz\",\"justitie\",\"gerechtigheid\",\"δικαιοσύνη\",\"υπουργείοδικαιοσύνης\",\"δίκαιο\",\"adalet\",\"igazságügy\",\"pravo\",\"õigus\",\"\"],\n" - + " \"key::86\": [\"aerospace\",\"aerospaziale\",\"aerospaziali\",\"aeroespacio\",\"aeroespaço\",\"Luftfahrt\",\"luchtvaart\",\"ruimtevaart\",\"αεροπορικός\",\"αεροπορική\",\"αεροπορικό\",\"αεροναυπηγικός\",\"αεροναυπηγική\",\"αεροναυπηγικό\",\"αεροναυπηγικά\",\"havacılıkveuzay\",\"légtér\",\"zrakoplovstvo\",\"atmosfäär\",\"kosmos\",\"\"],\n" - + " \"key::87\": [\"dermatology\",\"dermatologia\",\"dermatología\",\"dermatologia\",\"Dermatologie\",\"dermatologie\",\"δρματολογία\",\"dermatoloji\",\"bőrgyógyászat\",\"dermatológia\",\"dermatologija\",\"dermatoloogia\",\"\"],\n" - + " \"key::88\": [\"architecture\",\"architettura\",\"arquitectura\",\"arquitetura\",\"Architektur\",\"architectuur\",\"αρχιτεκτονική\",\"mimarlık\",\"építészet\",\"arhitektura\",\"arhitektuur\",\"\"],\n" - + " \"key::89\": [\"mathematics\",\"matematica\",\"matematiche\",\"matemáticas\",\"matemáticas\",\"Mathematik\",\"wiskunde\",\"mathematica\",\"μαθηματικά\",\"matematik\",\"matematika\",\"matematika\",\"matemaatika\",\"\"],\n" - + " \"key::90\": [\"language\",\"lingue\",\"linguistica\",\"linguistiche\",\"lenguaje\",\"idioma\",\"língua\",\"idioma\",\"Sprache\",\"taal\",\"taalkunde\",\"γλώσσα\",\"dil\",\"nyelv\",\"jezik\",\"keel\",\"\"],\n" - + " \"key::91\": [\"neuroscience\",\"neuroscienza\",\"neurociencia\",\"neurociência\",\"Neurowissenschaft\",\"neurowetenschappen\",\"νευροεπιστήμη\",\"nörobilim\",\"idegtudomány\",\"nevroznanost\",\"neuroteadused\",\"\"],\n" - + " \"key::92\": [\"automation\",\"automazione\",\"automatización\",\"automação\",\"Automatisierung\",\"automatisering\",\"αυτοματοποίηση\",\"otomasyon\",\"automatizálás\",\"avtomatizacija\",\"automatiseeritud\",\"\"],\n" - + " \"key::93\": [\"pediatric\",\"pediatria\",\"pediatriche\",\"pediatrico\",\"pediátrico\",\"pediatría\",\"pediátrico\",\"pediatria\",\"pädiatrisch\",\"pediatrische\",\"παιδιατρική\",\"pediatrik\",\"gyermekgyógyászat\",\"pediatrija\",\"pediaatria\",\"\"],\n" - + " \"key::94\": [\"photonics\",\"fotonica\",\"fotoniche\",\"fotónica\",\"fotônica\",\"Photonik\",\"fotonica\",\"φωτονική\",\"fotonik\",\"fotonika\",\"fotonika\",\"fotoonika\",\"\"],\n" - + " \"key::95\": [\"mechanics\", \"mechanical\", \"meccanica\",\"meccaniche\",\"mecánica\",\"mecânica\",\"Mechanik\",\"Maschinenbau\",\"mechanica\",\"werktuigkunde\",\"μηχανικής\",\"mekanik\",\"gépészet\",\"mehanika\",\"mehaanika\",\"\"],\n" - + " \"key::96\": [\"psychiatrics\",\"psichiatria\",\"psichiatrica\",\"psichiatriche\",\"psiquiatría\",\"psiquiatria\",\"Psychiatrie\",\"psychiatrie\",\"ψυχιατρική\",\"psikiyatrik\",\"pszihiátria\",\"psihiatrija\",\"psühhaatria\",\"\"],\n" - + " \"key::97\": [\"psychology\",\"fisiologia\",\"psicología\",\"psicologia\",\"Psychologie\",\"psychologie\",\"ψυχολογία\",\"psikoloji\",\"pszihológia\",\"psihologija\",\"psühholoogia\",\"\"],\n" - + " \"key::98\": [\"automotive\",\"industriaautomobilistica\",\"industriadelautomóvil\",\"automotriz\",\"industriaautomotriz\",\"automotivo\",\"Automobilindustrie\",\"autoindustrie\",\"αυτοκίνητος\",\"αυτοκίνητη\",\"αυτοκίνητο\",\"αυτοκινούμενος\",\"αυτοκινούμενη\",\"αυτοκινούμενο\",\"αυτοκινητιστικός\",\"αυτοκινητιστική\",\"αυτοκινητιστικό\",\"otomotiv\",\"autóipari\",\"samogiben\",\"avtomobilskaindustrija\",\"auto-\",\"\"],\n" - + " \"key::99\": [\"neurology\",\"neurologia\",\"neurologiche\",\"neurología\",\"neurologia\",\"Neurologie\",\"neurologie\",\"zenuwleer\",\"νευρολογία\",\"nöroloji\",\"neurológia\",\"ideggyógyászat\",\"nevrologija\",\"neuroloogia\",\"\"],\n" - + " \"key::100\": [\"geology\",\"geologia\",\"geologiche\",\"geología\",\"geologia\",\"Geologie\",\"geologie\",\"aardkunde\",\"γεωλογία\",\"jeoloji\",\"geológia\",\"földtudomány\",\"geologija\",\"geoloogia\",\"\"],\n" - + " \"key::101\": [\"microbiology\",\"microbiologia\",\"micro-biologia\",\"microbiologiche\",\"microbiología\",\"microbiologia\",\"Mikrobiologie\",\"microbiologie\",\"μικροβιολογία\",\"mikrobiyoloji\",\"mikrobiológia\",\"mikrobiologija\",\"mikrobioloogia\",\"\"],\n" - + " \"key::102\": [\"informatics\",\"informatica\",\"informática\",\"informática\",\"informatica\",\"\"],\n" - + " \"key::103\": [\"forschungsgemeinschaft\",\"comunita ricerca\",\"research community\",\"research foundation\",\"research association\"],\n" - + " \"key::104\": [\"commerce\",\"ticaret\",\"ticarət\",\"commercio\",\"trade\",\"handel\",\"comercio\"],\n" - + " \"key::105\" : [\"state\", \"stato\", \"etade\", \"estado\", \"statale\", \"etat\", \"zustand\", \"estado\"],\n" - + " \"key::106\" : [\"seminary\", \"seminario\", \"seminaire\", \"seminar\"],\n" - + " \"key::107\" : [\"agricultural forestry\", \"af\", \"a f\"],\n" - + " \"key::108\" : [\"agricultural mechanical\", \"am\", \"a m\"],\n" - + " \"key::109\" : [\"catholic\", \"catholique\", \"katholische\", \"catolica\", \"cattolica\", \"catolico\"]\n" - + " }\n" - + " }\n" - + "}"); + String json = "{\t\"dataInfo\":{\t\t\"invisible\":false,\t\t\"inferred\":false,\t\t\"deletedbyinference\":false,\t\t\"trust\":\"0.810000002384185791\",\t\t\"inferenceprovenance\":\"\",\t\t\"provenanceaction\":{\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t}\t},\t\"lastupdatetimestamp\":1584960968152,\t\"id\":\"20|corda__h2020::9faf23721249f26ac2c16eb857ea1fb9\",\t\"originalId\":[\t\t\"corda__h2020::927957582\"\t],\t\"collectedfrom\":[\t\t{\t\t\t\"key\":\"openaire____::corda_h2020\",\t\t\t\"value\":\"CORDA - COmmon Research DAta Warehouse - Horizon 2020\",\t\t\t\"dataInfo\":null\t\t}\t],\t\"pid\":[\t],\t\"dateofcollection\":\"2016-06-05\",\t\"dateoftransformation\":\"2019-11-19\",\t\"extraInfo\":[\t],\t\"oaiprovenance\":null,\t\"legalshortname\":{\t\t\"value\":\"Comentor AB\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"legalname\":{\t\t\"value\":\"Comentor AB\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"alternativeNames\":[\t],\t\"websiteurl\":{\t\t\"value\":\"http://www.comentor.se\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"logourl\":null,\t\"eclegalbody\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"eclegalperson\":{\t\t\"value\":\"true\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecnonprofit\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecresearchorganization\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"echighereducation\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecinternationalorganizationeurinterests\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecinternationalorganization\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecenterprise\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecsmevalidated\":{\t\t\"value\":\"true\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"ecnutscode\":{\t\t\"value\":\"false\",\t\t\"dataInfo\":{\t\t\t\"invisible\":false,\t\t\t\"inferred\":false,\t\t\t\"deletedbyinference\":false,\t\t\t\"trust\":\"0.810000002384185791\",\t\t\t\"inferenceprovenance\":\"\",\t\t\t\"provenanceaction\":{\t\t\t\t\"classid\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"classname\":\"sysimport:crosswalk:entityregistry\",\t\t\t\t\"schemeid\":\"dnet:provenance_actions\",\t\t\t\t\"schemename\":\"dnet:provenance_actions\"\t\t\t}\t\t}\t},\t\"country\":null}"; + DedupConfig conf = DedupConfig + .load( + "{\n" + + " \"wf\" : {\n" + + " \"threshold\" : \"0.99\",\n" + + " \"dedupRun\" : \"001\",\n" + + " \"entityType\" : \"organization\",\n" + + " \"subEntityValue\": \"organization\",\n" + + " \"orderField\" : \"legalname\",\n" + + " \"queueMaxSize\" : \"2000\",\n" + + " \"groupMaxSize\" : \"50\",\n" + + " \"slidingWindowSize\" : \"200\",\n" + + " \"idPath\":\"$.id\",\n" + + " \"rootBuilder\" : [ \"organization\", \"projectOrganization_participation_isParticipant\", \"datasourceOrganization_provision_isProvidedBy\" ],\n" + + " \"includeChildren\" : \"true\",\n" + + " \"maxIterations\": \"20\"\n" + + " },\n" + + " \"pace\" : {\n" + + " \"clustering\" : [\n" + + " { \"name\" : \"sortedngrampairs\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\" : 2, \"ngramLen\" : \"3\"} },\n" + + " { \"name\" : \"suffixprefix\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\" : 1, \"len\" : \"3\" } },\n" + + " { \"name\" : \"urlclustering\", \"fields\" : [ \"websiteurl\" ], \"params\" : { } },\n" + + " { \"name\" : \"keywordsclustering\", \"fields\" : [ \"legalname\" ], \"params\" : { \"max\": 2, \"windowSize\": 4} }\n" + + " ],\n" + + " \"decisionTree\" : {\n" + + " \"start\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"field\": \"gridid\",\n" + + " \"comparator\": \"exactMatch\",\n" + + " \"weight\": 1,\n" + + " \"countIfUndefined\": \"false\",\n" + + " \"params\": {}\n" + + " }\n" + + " ],\n" + + " \"threshold\": 1,\n" + + " \"aggregation\": \"AVG\",\n" + + " \"positive\": \"MATCH\",\n" + + " \"negative\": \"NO_MATCH\",\n" + + " \"undefined\": \"layer2\",\n" + + " \"ignoreUndefined\": \"false\"\n" + + " },\n" + + " \"layer2\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"field\": \"websiteurl\",\n" + + " \"comparator\": \"domainExactMatch\",\n" + + " \"weight\": 1,\n" + + " \"countIfUndefined\": \"false\",\n" + + " \"params\": {}\n" + + " },\n" + + " {\n" + + " \"field\": \"country\",\n" + + " \"comparator\": \"exactMatch\",\n" + + " \"weight\": 1,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {}\n" + + " },\n" + + " {\n" + + " \"field\": \"legalname\",\n" + + " \"comparator\": \"numbersMatch\",\n" + + " \"weight\": 1,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {}\n" + + " },\n" + + " {\n" + + " \"field\": \"legalname\",\n" + + " \"comparator\": \"romansMatch\",\n" + + " \"weight\": 1,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {}\n" + + " }\n" + + " ],\n" + + " \"threshold\": 1,\n" + + " \"aggregation\": \"AND\",\n" + + " \"positive\": \"layer3\",\n" + + " \"negative\": \"NO_MATCH\",\n" + + " \"undefined\": \"layer3\",\n" + + " \"ignoreUndefined\": \"true\"\n" + + " },\n" + + " \"layer3\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"field\": \"legalname\",\n" + + " \"comparator\": \"cityMatch\",\n" + + " \"weight\": 1.0,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {\n" + + " \"windowSize\": \"4\"\n" + + " }\n" + + " }\n" + + " ],\n" + + " \"threshold\": 0.1,\n" + + " \"aggregation\": \"AVG\",\n" + + " \"positive\": \"layer4\",\n" + + " \"negative\": \"NO_MATCH\",\n" + + " \"undefined\": \"NO_MATCH\",\n" + + " \"ignoreUndefined\": \"true\"\n" + + " },\n" + + " \"layer4\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"field\": \"legalname\",\n" + + " \"comparator\": \"keywordMatch\",\n" + + " \"weight\": 1.0,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {\n" + + " \"windowSize\": \"4\"\n" + + " }\n" + + " }\n" + + " ],\n" + + " \"threshold\": 0.7,\n" + + " \"aggregation\": \"AVG\",\n" + + " \"positive\": \"layer5\",\n" + + " \"negative\": \"NO_MATCH\",\n" + + " \"undefined\": \"layer5\",\n" + + " \"ignoreUndefined\": \"true\"\n" + + " },\n" + + " \"layer5\": {\n" + + " \"fields\": [\n" + + " {\n" + + " \"field\": \"legalname\",\n" + + " \"comparator\": \"jaroWinklerNormalizedName\",\n" + + " \"weight\": 0.9,\n" + + " \"countIfUndefined\": \"true\",\n" + + " \"params\": {\n" + + " \"windowSize\": \"4\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"field\": \"legalshortname\",\n" + + " \"comparator\": \"jaroWinklerNormalizedName\",\n" + + " \"weight\": 0.1,\n" + + " \"countIfUndefined\": \"false\",\n" + + " \"params\": {\n" + + " \"windowSize\": 4\n" + + " }\n" + + " }\n" + + " ],\n" + + " \"threshold\": 0.9,\n" + + " \"aggregation\": \"W_MEAN\",\n" + + " \"positive\": \"MATCH\",\n" + + " \"negative\": \"NO_MATCH\",\n" + + " \"undefined\": \"NO_MATCH\",\n" + + " \"ignoreUndefined\": \"true\"\n" + + " }\n" + + " },\n" + + " \"model\" : [\n" + + " { \"name\" : \"country\", \"type\" : \"String\", \"path\" : \"$.country.classid\"},\n" + + " { \"name\" : \"legalshortname\", \"type\" : \"String\", \"path\" : \"$.legalshortname.value\"},\n" + + " { \"name\" : \"legalname\", \"type\" : \"String\", \"path\" : \"$.legalname.value\" },\n" + + " { \"name\" : \"websiteurl\", \"type\" : \"URL\", \"path\" : \"$.websiteurl.value\" },\n" + + " { \"name\" : \"gridid\", \"type\" : \"String\", \"path\" : \"$.pid[?(@.qualifier.classid =='grid')].value\"},\n" + + " { \"name\" : \"originalId\", \"type\" : \"String\", \"path\" : \"$.id\" }\n" + + " ],\n" + + " \"blacklists\" : {\n" + + " \"legalname\" : []\n" + + " },\n" + + " \"synonyms\": {\n" + + " \"key::1\": [\"university\",\"università\", \"universitas\", \"università studi\",\"universitario\",\"universitaria\",\"université\", \"universite\", \"universitaire\",\"universitaires\",\"universidad\",\"universitade\",\"Universität\",\"universitaet\",\"Uniwersytet\",\"университет\",\"universiteit\",\"πανεπιστήμιο\",\"universitesi\",\"universiteti\", \"universiti\"],\n" + + " \"key::2\": [\"studies\",\"studi\",\"études\",\"estudios\",\"estudos\",\"Studien\",\"studia\",\"исследования\",\"studies\",\"σπουδές\"],\n" + + " \"key::3\": [\"advanced\",\"superiore\",\"supérieur\",\"supérieure\",\"supérieurs\",\"supérieures\",\"avancado\",\"avancados\",\"fortgeschrittene\",\"fortgeschritten\",\"zaawansowany\",\"передовой\",\"gevorderd\",\"gevorderde\",\"προχωρημένος\",\"προχωρημένη\",\"προχωρημένο\",\"προχωρημένες\",\"προχωρημένα\",\"wyzsza\"],\n" + + " \"key::4\": [\"institute\",\"istituto\",\"institut\",\"instituto\",\"instituto\",\"Institut\",\"instytut\",\"институт\",\"instituut\",\"ινστιτούτο\"],\n" + + " \"key::5\": [\"hospital\",\"ospedale\",\"hôpital\",\"hospital\",\"hospital\",\"Krankenhaus\",\"szpital\",\"больница\",\"ziekenhuis\",\"νοσοκομείο\"],\n" + + " \"key::6\": [\"research\",\"ricerca\",\"recherche\",\"investigacion\",\"pesquisa\",\"Forschung\",\"badania\",\"исследования\",\"onderzoek\",\"έρευνα\",\"erevna\",\"erevnas\"],\n" + + " \"key::7\": [\"college\",\"collegio\",\"colegio\",\"faculdade\",\"Hochschule\",\"Szkoła Wyższa\",\"Высшая школа\",\"κολλέγιο\"],\n" + + " \"key::8\": [\"foundation\",\"fondazione\",\"fondation\",\"fundación\",\"fundação\",\"Stiftung\",\"Fundacja\",\"фонд\",\"stichting\",\"ίδρυμα\",\"idryma\"],\n" + + " \"key::9\": [\"center\",\"centro\",\"centre\",\"centro\",\"centro\",\"zentrum\",\"centrum\",\"центр\",\"centrum\",\"κέντρο\"],\n" + + " \"key::10\": [\"national\",\"nazionale\",\"national\",\"nationale\",\"nationaux\",\"nationales\",\"nacional\",\"nacional\",\"national\",\"krajowy\",\"национальный\",\"nationaal\",\"nationale\",\"εθνικό\"],\n" + + " \"key::11\": [\"association\",\"associazione\",\"association\",\"asociación\",\"associação\",\"Verein\",\"verband\",\"stowarzyszenie\",\"ассоциация\",\"associatie\"],\n" + + " \"key::12\": [\"society\",\"societa\",\"société\",\"sociedad\",\"sociedade\",\"gesellschaft\",\"społeczeństwo\",\"общество\",\"maatschappij\",\"κοινωνία\"],\n" + + " \"key::13\": [\"international\",\"internazionale\",\"international\",\"internacional\",\"internacional\",\"international\",\"międzynarodowy\",\"Международный\",\"internationaal\",\"internationale\",\"διεθνής\",\"διεθνή\",\"διεθνές\"],\n" + + " \"key::14\": [\"community\",\"comunita\",\"communauté\",\"comunidad\",\"comunidade\",\"Gemeinschaft\",\"społeczność\",\"сообщество\",\"gemeenschap\",\"κοινότητα\"],\n" + + " \"key::15\": [\"school\",\"scuola\",\"école\",\"escuela\",\"escola\",\"schule\",\"Szkoła\",\"школа\",\"school\",\"σχολείο\"],\n" + + " \"key::16\": [\"education\",\"educazione\",\"éducation\",\"educacion\",\"Educação\",\"Bildung\",\"Edukacja\",\"образование\",\"opleiding\",\"εκπαίδευση\"],\n" + + " \"key::17\": [\"academy\",\"accademia\",\"académie\",\"academia\",\"academia\",\"Akademie\",\"akademie\",\"академия\",\"academie\",\"ακαδημία\"],\n" + + " \"key::18\": [\"public\",\"pubblico\",\"public\",\"publique\",\"publics\",\"publiques\",\"publico\",\"publico\",\"Öffentlichkeit\",\"publiczny\",\"публичный\",\"publiek\",\"publieke\",\"δημόσιος\",\"δημόσια\",\"δημόσιο\"],\n" + + " \"key::19\": [\"museum\",\"museo\",\"musée\",\"mueso\",\"museu\",\"museum\",\"muzeum\",\"музей\",\"museum\",\"μουσείο\"],\n" + + " \"key::20\": [\"group\",\"gruppo\",\"groupe\",\"grupo\",\"grupo\",\"gruppe\",\"grupa\",\"группа\",\"groep\",\"ομάδα\",\"όμιλος\"],\n" + + " \"key::21\": [\"department\",\"dipartimento\",\"département\",\"departamento\",\"departamento\",\"abteilung\",\"departament\",\"отдел\",\"afdeling\",\"τμήμα\"],\n" + + " \"key::22\": [\"council\",\"consiglio\",\"conseil\",\"Consejo\",\"conselho\",\"gesellschaft\",\"rada\",\"совет\",\"raad\",\"συμβούλιο\"],\n" + + " \"key::23\": [\"library\",\"biblioteca\",\"bibliothèque\",\"biblioteca\",\"biblioteca\",\"Bibliothek\",\"biblioteka\",\"библиотека\",\"bibliotheek\",\"βιβλιοθήκη\"],\n" + + " \"key::24\": [\"ministry\",\"ministero\",\"ministère\",\"ministerio\",\"ministério\",\"Ministerium\",\"ministerstwo\",\"министерство\",\"ministerie\",\"υπουργείο\"],\n" + + " \"key::25\": [\"services\",\"servizi\",\"services\",\"servicios\",\"Serviços\",\"Dienstleistungen\",\"usługi\",\"услуги\",\"diensten\",\"υπηρεσίες\"],\n" + + " \"key::26\": [\"central\",\"centrale\",\"central\",\"centrale\",\"centrales\",\"central\",\"central\",\"zentral\",\"centralny\",\"цетральный\",\"centraal\",\"κεντρικός\",\"κεντρική\",\"κεντρικό\",\"κεντρικά\"],\n" + + " \"key::27\": [\"general\",\"generale\",\"général\",\"générale\",\"généraux\",\"générales\",\"general\",\"geral\",\"general\",\"Allgemeines\",\"general\",\"общий\",\"algemeen\",\"algemene\",\"γενικός\",\"γενική\",\"γενικό\",\"γενικά\"],\n" + + " \"key::28\": [\"applied\",\"applicati\",\"appliqué\",\"appliquée\",\"appliqués\",\"appliquées\",\"aplicado\",\"aplicada\",\"angewendet\",\"stosowany\",\"прикладной\",\"toegepast\",\"toegepaste\",\"εφαρμοσμένος\",\"εφαρμοσμένη\",\"εφαρμοσμένο\",\"εφαρμοσμένα\"],\n" + + " \"key::29\": [\"european\",\"europee\",\"europea\",\"européen\",\"européenne\",\"européens\",\"européennes\",\"europeo\",\"europeu\",\"europäisch\",\"europejski\",\"европейский\",\"Europees\",\"Europese\",\"ευρωπαϊκός\",\"ευρωπαϊκή\",\"ευρωπαϊκό\",\"ευρωπαϊκά\"],\n" + + " \"key::30\": [\"agency\",\"agenzia\",\"agence\",\"agencia\",\"agencia\",\"agentur\",\"agencja\",\"агенция\",\"agentschap\",\"πρακτορείο\"],\n" + + " \"key::31\": [\"laboratory\",\"laboratorio\",\"laboratoire\",\"laboratorio\",\"laboratorio\",\"labor\",\"laboratorium\",\"лаборатория\",\"laboratorium\",\"εργαστήριο\"],\n" + + " \"key::32\": [\"industry\",\"industria\",\"industrie\",\"индустрия\",\"industrie\",\"βιομηχανία\"],\n" + + " \"key::33\": [\"industrial\",\"industriale\",\"industriel\",\"industrielle\",\"industriels\",\"industrielles\",\"индустриальный\",\"industrieel\",\"βιομηχανικός\",\"βιομηχανική\",\"βιομηχανικό\",\"βιομηχανικά\",\"βιομηχανικές\"],\n" + + " \"key::34\": [\"consortium\",\"consorzio\",\"consortium\",\"консорциум\",\"consortium\",\"κοινοπραξία\"],\n" + + " \"key::35\": [\"organization\",\"organizzazione\",\"organisation\",\"organización\",\"organização\",\"organizacja\",\"организация\",\"organisatie\",\"οργανισμός\"],\n" + + " \"key::36\": [\"authority\",\"autorità\",\"autorité\",\"авторитет\",\"autoriteit\"],\n" + + " \"key::37\": [\"federation\",\"federazione\",\"fédération\",\"федерация\",\"federatie\",\"ομοσπονδία\"],\n" + + " \"key::38\": [\"observatory\",\"osservatorio\",\"observatoire\",\"обсерватория\",\"observatorium\",\"αστεροσκοπείο\"],\n" + + " \"key::39\": [\"bureau\",\"ufficio\",\"bureau\",\"офис\",\"bureau\",\"γραφείο\"],\n" + + " \"key::40\": [\"company\",\"impresa\",\"compagnie\",\"société\",\"компания\",\"bedrijf\",\"εταιρία\"],\n" + + " \"key::41\": [\"polytechnic\",\"politecnico\",\"polytechnique\",\"политехника\",\"polytechnisch\",\"πολυτεχνείο\",\"universita politecnica\",\"polytechnic university\",\"universidad politecnica\",\"universitat politecnica\",\"politechnika\",\"politechniki\",\"university technology\",\"university science technology\"],\n" + + " \"key::42\": [\"coalition\",\"coalizione\",\"coalition\",\"коалиция\",\"coalitie\",\"συνασπισμός\"],\n" + + " \"key::43\": [\"initiative\",\"iniziativa\",\"initiative\",\"инициатива\",\"initiatief\",\"πρωτοβουλία\"],\n" + + " \"key::44\": [\"academic\",\"accademico\",\"académique\",\"universitaire\",\"акадеческий academisch\",\"ακαδημαϊκός\",\"ακαδημαϊκή\",\"ακαδημαϊκό\",\"ακαδημαϊκές\",\"ακαδημαϊκοί\"],\n" + + " \"key::45\": [\"institution\",\"istituzione\",\"institution\",\"институциональный\",\"instelling\",\"ινστιτούτο\"],\n" + + " \"key::46\": [\"division\",\"divisione\",\"division\",\"отделение\",\"divisie\",\"τμήμα\"],\n" + + " \"key::47\": [\"committee\",\"comitato\",\"comité\",\"комитет\",\"commissie\",\"επιτροπή\"],\n" + + " \"key::48\": [\"promotion\",\"promozione\",\"продвижение\",\"proothisis\",\"forderung\"],\n" + + " \"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\"],\n" + + " \"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\"],\n" + + " \"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\",\"\"],\n" + + " \"key::52\": [\"engineering\",\"ingegneria\",\"ingeniería\",\"engenharia\",\"Ingenieurwissenschaft\",\"ingenieurswetenschappen\",\"bouwkunde\",\"μηχανικός\",\"μηχανική\",\"μηχανικό\",\"mühendislik\",\"mérnöki\",\"Inženirstvo\",\"inseneeria\",\"inseneri\",\"\"],\n" + + " \"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\",\"\"],\n" + + " \"key::54\": [\"energy\",\"energia\",\"energía\",\"energia\",\"Energie\",\"energie\",\"ενέργεια\",\"enerji\",\"energia\",\"energija\",\"energia\",\"\"],\n" + + " \"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\",\"\"],\n" + + " \"key::56\": [\"information\",\"informazione\",\"información\",\"informação\",\"Information\",\"informatie\",\"πληροφορία\",\"bilgi\",\"információ\",\"informacija\",\"informatsioon\",\"informatycznych\",\"\"],\n" + + " \"key::57\": [\"social\",\"sociali\",\"social\",\"social\",\"Sozial\",\"sociaal\",\"maatschappelijk\",\"κοινωνικός\",\"κοινωνική\",\"κοινωνικό\",\"κοινωνικά\",\"sosyal\",\"szociális\",\"družbeni\",\"sotsiaal\",\"sotsiaalne\",\"\"],\n" + + " \"key::58\": [\"environmental\",\"ambiente\",\"medioambiental\",\"ambiente\",\"medioambiente\",\"meioambiente\",\"Umwelt\",\"milieu\",\"milieuwetenschap\",\"milieukunde\",\"περιβαλλοντικός\",\"περιβαλλοντική\",\"περιβαλλοντικό\",\"περιβαλλοντικά\",\"çevre\",\"környezeti\",\"okoliški\",\"keskonna\",\"\"],\n" + + " \"key::59\": [\"business\",\"economia\",\"economiche\",\"economica\",\"negocio\",\"empresa\",\"negócio\",\"Unternehmen\",\"bedrijf\",\"bedrijfskunde\",\"επιχείρηση\",\"iş\",\"üzleti\",\"posel\",\"ettevõte/äri\",\"\"],\n" + + " \"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\",\"\"],\n" + + " \"key::61\": [\"healthcare\",\"health services\",\"salute\",\"atenciónmédica\",\"cuidadodelasalud\",\"cuidadoscomasaúde\",\"Gesundheitswesen\",\"gezondheidszorg\",\"ιατροφαρμακευτικήπερίθαλψη\",\"sağlıkhizmeti\",\"egészségügy\",\"zdravstvo\",\"tervishoid\",\"tervishoiu\",\"\"],\n" + + " \"key::62\": [\"history\",\"storia\",\"historia\",\"história\",\"Geschichte\",\"geschiedenis\",\"geschiedkunde\",\"ιστορία\",\"tarih\",\"történelem\",\"zgodovina\",\"ajalugu\",\"\"],\n" + + " \"key::63\": [\"materials\",\"materiali\",\"materia\",\"materiales\",\"materiais\",\"materialen\",\"υλικά\",\"τεκμήρια\",\"malzemeler\",\"anyagok\",\"materiali\",\"materjalid\",\"vahendid\",\"\"],\n" + + " \"key::64\": [\"economics\",\"economia\",\"economiche\",\"economica\",\"economía\",\"economia\",\"Wirtschaft\",\"economie\",\"οικονομικά\",\"οικονομικέςεπιστήμες\",\"ekonomi\",\"közgazdaságtan\",\"gospodarstvo\",\"ekonomija\",\"majanduslik\",\"majandus\",\"\"],\n" + + " \"key::65\": [\"therapeutics\",\"terapeutica\",\"terapéutica\",\"terapêutica\",\"therapie\",\"θεραπευτική\",\"tedavibilimi\",\"gyógykezelés\",\"terapevtika\",\"terapeutiline\",\"ravi\",\"\"],\n" + + " \"key::66\": [\"oncology\",\"oncologia\",\"oncologico\",\"oncología\",\"oncologia\",\"Onkologie\",\"oncologie\",\"ογκολογία\",\"onkoloji\",\"onkológia\",\"onkologija\",\"onkoloogia\",\"\"],\n" + + " \"key::67\": [\"natural\",\"naturali\",\"naturale\",\"natural\",\"natural\",\"natürlich\",\"natuurlijk\",\"φυσικός\",\"φυσική\",\"φυσικό\",\"φυσικά\",\"doğal\",\"természetes\",\"naraven\",\"loodus\",\"\"],\n" + + " \"key::68\": [\"educational\",\"educazione\",\"pedagogia\",\"educacional\",\"educativo\",\"educacional\",\"pädagogisch\",\"educatief\",\"εκπαιδευτικός\",\"εκπαιδευτική\",\"εκπαιδευτικό\",\"εκπαιδευτικά\",\"eğitimsel\",\"oktatási\",\"izobraževalen\",\"haridus\",\"hariduslik\",\"\"],\n" + + " \"key::69\": [\"biomedical\",\"biomedica\",\"biomédico\",\"biomédico\",\"biomedizinisch\",\"biomedisch\",\"βιοιατρικός\",\"βιοιατρική\",\"βιοιατρικό\",\"βιοιατρικά\",\"biyomedikal\",\"orvosbiológiai\",\"biomedicinski\",\"biomeditsiiniline\",\"\"],\n" + + " \"key::70\": [\"veterinary\",\"veterinaria\",\"veterinarie\",\"veterinaria\",\"veterinária\",\"tierärtzlich\",\"veterinair\",\"veeartsenijlkunde\",\"κτηνιατρικός\",\"κτηνιατρική\",\"κτηνιατρικό\",\"κτηνιατρικά\",\"veteriner\",\"állatorvosi\",\"veterinar\",\"veterinarski\",\"veterinaaria\",\"\"],\n" + + " \"key::71\": [\"chemistry\",\"chimica\",\"química\",\"química\",\"Chemie\",\"chemie\",\"scheikunde\",\"χημεία\",\"kimya\",\"kémia\",\"kemija\",\"keemia\",\"\"],\n" + + " \"key::72\": [\"security\",\"sicurezza\",\"seguridad\",\"segurança\",\"Sicherheit\",\"veiligheid\",\"ασφάλεια\",\"güvenlik\",\"biztonsági\",\"varnost\",\"turvalisus\",\"julgeolek\",\"\"],\n" + + " \"key::73\": [\"biotechnology\",\"biotecnologia\",\"biotecnologie\",\"biotecnología\",\"biotecnologia\",\"Biotechnologie\",\"biotechnologie\",\"βιοτεχνολογία\",\"biyoteknoloji\",\"biotechnológia\",\"biotehnologija\",\"biotehnoloogia\",\"\"],\n" + + " \"key::74\": [\"military\",\"militare\",\"militari\",\"militar\",\"militar\",\"Militär\",\"militair\",\"leger\",\"στρατιωτικός\",\"στρατιωτική\",\"στρατιωτικό\",\"στρατιωτικά\",\"askeri\",\"katonai\",\"vojaški\",\"vojni\",\"militaar\",\"wojskowa\",\"\"],\n" + + " \"key::75\": [\"theological\",\"teologia\",\"teologico\",\"teológico\",\"tecnológica\",\"theologisch\",\"theologisch\",\"θεολογικός\",\"θεολογική\",\"θεολογικό\",\"θεολογικά\",\"teolojik\",\"technológiai\",\"teološki\",\"teoloogia\",\"usuteadus\",\"teoloogiline\",\"\"],\n" + + " \"key::76\": [\"electronics\",\"elettronica\",\"electrónica\",\"eletrônicos\",\"Elektronik\",\"elektronica\",\"ηλεκτρονική\",\"elektronik\",\"elektronika\",\"elektronika\",\"elektroonika\",\"\"],\n" + + " \"key::77\": [\"forestry\",\"forestale\",\"forestali\",\"silvicultura\",\"forestal\",\"floresta\",\"Forstwirtschaft\",\"bosbouw\",\"δασοκομία\",\"δασολογία\",\"ormancılık\",\"erdészet\",\"gozdarstvo\",\"metsandus\",\"\"],\n" + + " \"key::78\": [\"maritime\",\"marittima\",\"marittime\",\"marittimo\",\"marítimo\",\"marítimo\",\"maritiem\",\"ναυτικός\",\"ναυτική\",\"ναυτικό\",\"ναυτικά\",\"ναυτιλιακός\",\"ναυτιλιακή\",\"ναυτιλιακό\",\"ναυτιλιακά\",\"θαλάσσιος\",\"θαλάσσια\",\"θαλάσσιο\",\"denizcilik\",\"tengeri\",\"morski\",\"mere\",\"merendus\",\"\"],\n" + + " \"key::79\": [\"sports\",\"sport\",\"deportes\",\"esportes\",\"Sport\",\"sport\",\"sportwetenschappen\",\"άθληση\",\"γυμναστικήδραστηριότητα\",\"spor\",\"sport\",\"šport\",\"sport\",\"spordi\",\"\"],\n" + + " \"key::80\": [\"surgery\",\"chirurgia\",\"chirurgiche\",\"cirugía\",\"cirurgia\",\"Chirurgie\",\"chirurgie\",\"heelkunde\",\"εγχείρηση\",\"επέμβαση\",\"χειρουργικήεπέμβαση\",\"cerrahi\",\"sebészet\",\"kirurgija\",\"kirurgia\",\"\"],\n" + + " \"key::81\": [\"cultural\",\"culturale\",\"culturali\",\"cultura\",\"cultural\",\"cultural\",\"kulturell\",\"cultureel\",\"πολιτιστικός\",\"πολιτιστική\",\"πολιτιστικό\",\"πολιτισμικός\",\"πολιτισμική\",\"πολιτισμικό\",\"kültürel\",\"kultúrális\",\"kulturni\",\"kultuuri\",\"kultuuriline\",\"\"],\n" + + " \"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\",\"\"],\n" + + " \"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\",\"\"],\n" + + " \"key::84\": [\"communication\",\"comunicazione\",\"comuniciación\",\"comunicação\",\"Kommunikation\",\"communication\",\"επικοινωνία\",\"iletişim\",\"kommunikáció\",\"komuniciranje\",\"kommunikatsioon\",\"\"],\n" + + " \"key::85\": [\"justice\",\"giustizia\",\"justicia\",\"justiça\",\"Recht\",\"Justiz\",\"justitie\",\"gerechtigheid\",\"δικαιοσύνη\",\"υπουργείοδικαιοσύνης\",\"δίκαιο\",\"adalet\",\"igazságügy\",\"pravo\",\"õigus\",\"\"],\n" + + " \"key::86\": [\"aerospace\",\"aerospaziale\",\"aerospaziali\",\"aeroespacio\",\"aeroespaço\",\"Luftfahrt\",\"luchtvaart\",\"ruimtevaart\",\"αεροπορικός\",\"αεροπορική\",\"αεροπορικό\",\"αεροναυπηγικός\",\"αεροναυπηγική\",\"αεροναυπηγικό\",\"αεροναυπηγικά\",\"havacılıkveuzay\",\"légtér\",\"zrakoplovstvo\",\"atmosfäär\",\"kosmos\",\"\"],\n" + + " \"key::87\": [\"dermatology\",\"dermatologia\",\"dermatología\",\"dermatologia\",\"Dermatologie\",\"dermatologie\",\"δρματολογία\",\"dermatoloji\",\"bőrgyógyászat\",\"dermatológia\",\"dermatologija\",\"dermatoloogia\",\"\"],\n" + + " \"key::88\": [\"architecture\",\"architettura\",\"arquitectura\",\"arquitetura\",\"Architektur\",\"architectuur\",\"αρχιτεκτονική\",\"mimarlık\",\"építészet\",\"arhitektura\",\"arhitektuur\",\"\"],\n" + + " \"key::89\": [\"mathematics\",\"matematica\",\"matematiche\",\"matemáticas\",\"matemáticas\",\"Mathematik\",\"wiskunde\",\"mathematica\",\"μαθηματικά\",\"matematik\",\"matematika\",\"matematika\",\"matemaatika\",\"\"],\n" + + " \"key::90\": [\"language\",\"lingue\",\"linguistica\",\"linguistiche\",\"lenguaje\",\"idioma\",\"língua\",\"idioma\",\"Sprache\",\"taal\",\"taalkunde\",\"γλώσσα\",\"dil\",\"nyelv\",\"jezik\",\"keel\",\"\"],\n" + + " \"key::91\": [\"neuroscience\",\"neuroscienza\",\"neurociencia\",\"neurociência\",\"Neurowissenschaft\",\"neurowetenschappen\",\"νευροεπιστήμη\",\"nörobilim\",\"idegtudomány\",\"nevroznanost\",\"neuroteadused\",\"\"],\n" + + " \"key::92\": [\"automation\",\"automazione\",\"automatización\",\"automação\",\"Automatisierung\",\"automatisering\",\"αυτοματοποίηση\",\"otomasyon\",\"automatizálás\",\"avtomatizacija\",\"automatiseeritud\",\"\"],\n" + + " \"key::93\": [\"pediatric\",\"pediatria\",\"pediatriche\",\"pediatrico\",\"pediátrico\",\"pediatría\",\"pediátrico\",\"pediatria\",\"pädiatrisch\",\"pediatrische\",\"παιδιατρική\",\"pediatrik\",\"gyermekgyógyászat\",\"pediatrija\",\"pediaatria\",\"\"],\n" + + " \"key::94\": [\"photonics\",\"fotonica\",\"fotoniche\",\"fotónica\",\"fotônica\",\"Photonik\",\"fotonica\",\"φωτονική\",\"fotonik\",\"fotonika\",\"fotonika\",\"fotoonika\",\"\"],\n" + + " \"key::95\": [\"mechanics\", \"mechanical\", \"meccanica\",\"meccaniche\",\"mecánica\",\"mecânica\",\"Mechanik\",\"Maschinenbau\",\"mechanica\",\"werktuigkunde\",\"μηχανικής\",\"mekanik\",\"gépészet\",\"mehanika\",\"mehaanika\",\"\"],\n" + + " \"key::96\": [\"psychiatrics\",\"psichiatria\",\"psichiatrica\",\"psichiatriche\",\"psiquiatría\",\"psiquiatria\",\"Psychiatrie\",\"psychiatrie\",\"ψυχιατρική\",\"psikiyatrik\",\"pszihiátria\",\"psihiatrija\",\"psühhaatria\",\"\"],\n" + + " \"key::97\": [\"psychology\",\"fisiologia\",\"psicología\",\"psicologia\",\"Psychologie\",\"psychologie\",\"ψυχολογία\",\"psikoloji\",\"pszihológia\",\"psihologija\",\"psühholoogia\",\"\"],\n" + + " \"key::98\": [\"automotive\",\"industriaautomobilistica\",\"industriadelautomóvil\",\"automotriz\",\"industriaautomotriz\",\"automotivo\",\"Automobilindustrie\",\"autoindustrie\",\"αυτοκίνητος\",\"αυτοκίνητη\",\"αυτοκίνητο\",\"αυτοκινούμενος\",\"αυτοκινούμενη\",\"αυτοκινούμενο\",\"αυτοκινητιστικός\",\"αυτοκινητιστική\",\"αυτοκινητιστικό\",\"otomotiv\",\"autóipari\",\"samogiben\",\"avtomobilskaindustrija\",\"auto-\",\"\"],\n" + + " \"key::99\": [\"neurology\",\"neurologia\",\"neurologiche\",\"neurología\",\"neurologia\",\"Neurologie\",\"neurologie\",\"zenuwleer\",\"νευρολογία\",\"nöroloji\",\"neurológia\",\"ideggyógyászat\",\"nevrologija\",\"neuroloogia\",\"\"],\n" + + " \"key::100\": [\"geology\",\"geologia\",\"geologiche\",\"geología\",\"geologia\",\"Geologie\",\"geologie\",\"aardkunde\",\"γεωλογία\",\"jeoloji\",\"geológia\",\"földtudomány\",\"geologija\",\"geoloogia\",\"\"],\n" + + " \"key::101\": [\"microbiology\",\"microbiologia\",\"micro-biologia\",\"microbiologiche\",\"microbiología\",\"microbiologia\",\"Mikrobiologie\",\"microbiologie\",\"μικροβιολογία\",\"mikrobiyoloji\",\"mikrobiológia\",\"mikrobiologija\",\"mikrobioloogia\",\"\"],\n" + + " \"key::102\": [\"informatics\",\"informatica\",\"informática\",\"informática\",\"informatica\",\"\"],\n" + + " \"key::103\": [\"forschungsgemeinschaft\",\"comunita ricerca\",\"research community\",\"research foundation\",\"research association\"],\n" + + " \"key::104\": [\"commerce\",\"ticaret\",\"ticarət\",\"commercio\",\"trade\",\"handel\",\"comercio\"],\n" + + " \"key::105\" : [\"state\", \"stato\", \"etade\", \"estado\", \"statale\", \"etat\", \"zustand\", \"estado\"],\n" + + " \"key::106\" : [\"seminary\", \"seminario\", \"seminaire\", \"seminar\"],\n" + + " \"key::107\" : [\"agricultural forestry\", \"af\", \"a f\"],\n" + + " \"key::108\" : [\"agricultural mechanical\", \"am\", \"a m\"],\n" + + " \"key::109\" : [\"catholic\", \"catholique\", \"katholische\", \"catolica\", \"cattolica\", \"catolico\"]\n" + + " }\n" + + " }\n" + + "}"); - @Test - public void testJPath() throws Exception { + @Test + public void testJPath() throws Exception { - MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(conf, json); + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(conf, json); - System.out.println("d = " + d); - } + System.out.println("d = " + d); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DatePicker.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DatePicker.java index 8a6cdf7dc..db55434d8 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DatePicker.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DatePicker.java @@ -1,113 +1,121 @@ + package eu.dnetlib.dedup; import static java.util.Collections.reverseOrder; import static java.util.Map.Entry.comparingByValue; import static java.util.stream.Collectors.toMap; + import static org.apache.commons.lang.StringUtils.endsWith; import static org.apache.commons.lang.StringUtils.substringBefore; -import eu.dnetlib.dhp.schema.oaf.Field; import java.time.Year; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.lang.StringUtils; +import eu.dnetlib.dhp.schema.oaf.Field; + public class DatePicker { - private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}"; - private static final String DATE_DEFAULT_SUFFIX = "01-01"; - private static final int YEAR_LB = 1300; - private static final int YEAR_UB = Year.now().getValue() + 5; + private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}"; + private static final String DATE_DEFAULT_SUFFIX = "01-01"; + private static final int YEAR_LB = 1300; + private static final int YEAR_UB = Year.now().getValue() + 5; - public static Field pick(final Collection dateofacceptance) { + public static Field pick(final Collection dateofacceptance) { - final Map frequencies = - dateofacceptance - .parallelStream() - .filter(StringUtils::isNotBlank) - .collect(Collectors.toConcurrentMap(w -> w, w -> 1, Integer::sum)); + final Map frequencies = dateofacceptance + .parallelStream() + .filter(StringUtils::isNotBlank) + .collect(Collectors.toConcurrentMap(w -> w, w -> 1, Integer::sum)); - if (frequencies.isEmpty()) { - return new Field<>(); - } + if (frequencies.isEmpty()) { + return new Field<>(); + } - final Field date = new Field<>(); - date.setValue(frequencies.keySet().iterator().next()); + final Field date = new Field<>(); + date.setValue(frequencies.keySet().iterator().next()); - // let's sort this map by values first, filtering out invalid dates - final Map sorted = - frequencies.entrySet().stream() - .filter(d -> StringUtils.isNotBlank(d.getKey())) - .filter(d -> d.getKey().matches(DATE_PATTERN)) - .filter(d -> inRange(d.getKey())) - .sorted(reverseOrder(comparingByValue())) - .collect( - toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2, LinkedHashMap::new)); + // let's sort this map by values first, filtering out invalid dates + final Map sorted = frequencies + .entrySet() + .stream() + .filter(d -> StringUtils.isNotBlank(d.getKey())) + .filter(d -> d.getKey().matches(DATE_PATTERN)) + .filter(d -> inRange(d.getKey())) + .sorted(reverseOrder(comparingByValue())) + .collect( + toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2, LinkedHashMap::new)); - // shortcut - if (sorted.size() == 0) { - return date; - } + // shortcut + if (sorted.size() == 0) { + return date; + } - // voting method (1/3 + 1) wins - if (sorted.size() >= 3) { - final int acceptThreshold = (sorted.size() / 3) + 1; - final List accepted = - sorted.entrySet().stream() - .filter(e -> e.getValue() >= acceptThreshold) - .map(e -> e.getKey()) - .collect(Collectors.toList()); + // voting method (1/3 + 1) wins + if (sorted.size() >= 3) { + final int acceptThreshold = (sorted.size() / 3) + 1; + final List accepted = sorted + .entrySet() + .stream() + .filter(e -> e.getValue() >= acceptThreshold) + .map(e -> e.getKey()) + .collect(Collectors.toList()); - // cannot find strong majority - if (accepted.isEmpty()) { - final int max = sorted.values().iterator().next(); - Optional first = - sorted.entrySet().stream() - .filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) - .map(Map.Entry::getKey) - .findFirst(); - if (first.isPresent()) { - date.setValue(first.get()); - return date; - } + // cannot find strong majority + if (accepted.isEmpty()) { + final int max = sorted.values().iterator().next(); + Optional first = sorted + .entrySet() + .stream() + .filter(e -> e.getValue() == max && !endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) + .map(Map.Entry::getKey) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } - date.setValue(sorted.keySet().iterator().next()); - return date; - } + date.setValue(sorted.keySet().iterator().next()); + return date; + } - if (accepted.size() == 1) { - date.setValue(accepted.get(0)); - return date; - } else { - final Optional first = - accepted.stream().filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX)).findFirst(); - if (first.isPresent()) { - date.setValue(first.get()); - return date; - } + if (accepted.size() == 1) { + date.setValue(accepted.get(0)); + return date; + } else { + final Optional first = accepted + .stream() + .filter(d -> !endsWith(d, DATE_DEFAULT_SUFFIX)) + .findFirst(); + if (first.isPresent()) { + date.setValue(first.get()); + return date; + } - return date; - } + return date; + } - // 1st non YYYY-01-01 is returned - } else { - if (sorted.size() == 2) { - for (Map.Entry e : sorted.entrySet()) { - if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) { - date.setValue(e.getKey()); - return date; - } - } - } + // 1st non YYYY-01-01 is returned + } else { + if (sorted.size() == 2) { + for (Map.Entry e : sorted.entrySet()) { + if (!endsWith(e.getKey(), DATE_DEFAULT_SUFFIX)) { + date.setValue(e.getKey()); + return date; + } + } + } - // none of the dates seems good enough, return the 1st one - date.setValue(sorted.keySet().iterator().next()); - return date; - } - } + // none of the dates seems good enough, return the 1st one + date.setValue(sorted.keySet().iterator().next()); + return date; + } + } - private static boolean inRange(final String date) { - final int year = Integer.parseInt(substringBefore(date, "-")); - return year >= YEAR_LB && year <= YEAR_UB; - } + private static boolean inRange(final String date) { + final int year = Integer.parseInt(substringBefore(date, "-")); + return year >= YEAR_LB && year <= YEAR_UB; + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java index 782aa174f..d03cc2589 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupRecordFactory.java @@ -1,304 +1,318 @@ + package eu.dnetlib.dedup; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Lists; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.util.MapDocumentUtil; import java.util.Collection; + 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.sql.Encoders; import org.apache.spark.sql.SparkSession; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; public class DedupRecordFactory { - public static JavaRDD createDedupRecord( - final JavaSparkContext sc, - final SparkSession spark, - final String mergeRelsInputPath, - final String entitiesInputPath, - final OafEntityType entityType, - final DedupConfig dedupConf) { - long ts = System.currentTimeMillis(); - // - final JavaPairRDD inputJsonEntities = - sc.textFile(entitiesInputPath) - .mapToPair( - (PairFunction) - it -> - new Tuple2( - MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it)); + public static JavaRDD createDedupRecord( + final JavaSparkContext sc, + final SparkSession spark, + final String mergeRelsInputPath, + final String entitiesInputPath, + final OafEntityType entityType, + final DedupConfig dedupConf) { + long ts = System.currentTimeMillis(); + // + final JavaPairRDD inputJsonEntities = sc + .textFile(entitiesInputPath) + .mapToPair( + (PairFunction) it -> new Tuple2( + MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), it), it)); - // : source is the dedup_id, target is the id of the mergedIn - JavaPairRDD mergeRels = - spark - .read() - .load(mergeRelsInputPath) - .as(Encoders.bean(Relation.class)) - .where("relClass=='merges'") - .javaRDD() - .mapToPair( - (PairFunction) - r -> new Tuple2(r.getTarget(), r.getSource())); + // : source is the dedup_id, target is the id of the mergedIn + JavaPairRDD mergeRels = spark + .read() + .load(mergeRelsInputPath) + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .mapToPair( + (PairFunction) r -> new Tuple2(r.getTarget(), r.getSource())); - // - final JavaPairRDD joinResult = - mergeRels - .join(inputJsonEntities) - .mapToPair( - (PairFunction>, String, String>) Tuple2::_2); + // + final JavaPairRDD joinResult = mergeRels + .join(inputJsonEntities) + .mapToPair( + (PairFunction>, String, String>) Tuple2::_2); - JavaPairRDD> sortedJoinResult = joinResult.groupByKey(); + JavaPairRDD> sortedJoinResult = joinResult.groupByKey(); - switch (entityType) { - case publication: - return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts)); - case dataset: - return sortedJoinResult.map(d -> DedupRecordFactory.datasetMerger(d, ts)); - case project: - return sortedJoinResult.map(p -> DedupRecordFactory.projectMerger(p, ts)); - case software: - return sortedJoinResult.map(s -> DedupRecordFactory.softwareMerger(s, ts)); - case datasource: - return sortedJoinResult.map(d -> DedupRecordFactory.datasourceMerger(d, ts)); - case organization: - return sortedJoinResult.map(o -> DedupRecordFactory.organizationMerger(o, ts)); - case otherresearchproduct: - return sortedJoinResult.map(o -> DedupRecordFactory.otherresearchproductMerger(o, ts)); - default: - return null; - } - } + switch (entityType) { + case publication: + return sortedJoinResult.map(p -> DedupRecordFactory.publicationMerger(p, ts)); + case dataset: + return sortedJoinResult.map(d -> DedupRecordFactory.datasetMerger(d, ts)); + case project: + return sortedJoinResult.map(p -> DedupRecordFactory.projectMerger(p, ts)); + case software: + return sortedJoinResult.map(s -> DedupRecordFactory.softwareMerger(s, ts)); + case datasource: + return sortedJoinResult.map(d -> DedupRecordFactory.datasourceMerger(d, ts)); + case organization: + return sortedJoinResult.map(o -> DedupRecordFactory.organizationMerger(o, ts)); + case otherresearchproduct: + return sortedJoinResult.map(o -> DedupRecordFactory.otherresearchproductMerger(o, ts)); + default: + return null; + } + } - private static Publication publicationMerger(Tuple2> e, final long ts) { + private static Publication publicationMerger(Tuple2> e, final long ts) { - Publication p = new Publication(); // the result of the merge, to be returned at the end + Publication p = new Publication(); // the result of the merge, to be returned at the end - p.setId(e._1()); + p.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - final Collection dateofacceptance = Lists.newArrayList(); + final Collection dateofacceptance = Lists.newArrayList(); - if (e._2() != null) - e._2() - .forEach( - pub -> { - try { - Publication publication = mapper.readValue(pub, Publication.class); + if (e._2() != null) + e + ._2() + .forEach( + pub -> { + try { + Publication publication = mapper.readValue(pub, Publication.class); - p.mergeFrom(publication); - p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); - // add to the list if they are not null - if (publication.getDateofacceptance() != null) - dateofacceptance.add(publication.getDateofacceptance().getValue()); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - p.setDateofacceptance(DatePicker.pick(dateofacceptance)); - if (p.getDataInfo() == null) p.setDataInfo(new DataInfo()); - p.getDataInfo().setTrust("0.9"); - p.setLastupdatetimestamp(ts); - return p; - } + p.mergeFrom(publication); + p.setAuthor(DedupUtility.mergeAuthor(p.getAuthor(), publication.getAuthor())); + // add to the list if they are not null + if (publication.getDateofacceptance() != null) + dateofacceptance.add(publication.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + p.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (p.getDataInfo() == null) + p.setDataInfo(new DataInfo()); + p.getDataInfo().setTrust("0.9"); + p.setLastupdatetimestamp(ts); + return p; + } - private static Dataset datasetMerger(Tuple2> e, final long ts) { + private static Dataset datasetMerger(Tuple2> e, final long ts) { - Dataset d = new Dataset(); // the result of the merge, to be returned at the end + Dataset d = new Dataset(); // the result of the merge, to be returned at the end - d.setId(e._1()); + d.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - final Collection dateofacceptance = Lists.newArrayList(); + final Collection dateofacceptance = Lists.newArrayList(); - if (e._2() != null) - e._2() - .forEach( - dat -> { - try { - Dataset dataset = mapper.readValue(dat, Dataset.class); + if (e._2() != null) + e + ._2() + .forEach( + dat -> { + try { + Dataset dataset = mapper.readValue(dat, Dataset.class); - d.mergeFrom(dataset); - d.setAuthor(DedupUtility.mergeAuthor(d.getAuthor(), dataset.getAuthor())); - // add to the list if they are not null - if (dataset.getDateofacceptance() != null) - dateofacceptance.add(dataset.getDateofacceptance().getValue()); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - d.setDateofacceptance(DatePicker.pick(dateofacceptance)); - if (d.getDataInfo() == null) d.setDataInfo(new DataInfo()); - d.getDataInfo().setTrust("0.9"); - d.setLastupdatetimestamp(ts); - return d; - } + d.mergeFrom(dataset); + d.setAuthor(DedupUtility.mergeAuthor(d.getAuthor(), dataset.getAuthor())); + // add to the list if they are not null + if (dataset.getDateofacceptance() != null) + dateofacceptance.add(dataset.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + d.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (d.getDataInfo() == null) + d.setDataInfo(new DataInfo()); + d.getDataInfo().setTrust("0.9"); + d.setLastupdatetimestamp(ts); + return d; + } - private static Project projectMerger(Tuple2> e, final long ts) { + private static Project projectMerger(Tuple2> e, final long ts) { - Project p = new Project(); // the result of the merge, to be returned at the end + Project p = new Project(); // the result of the merge, to be returned at the end - p.setId(e._1()); + p.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - if (e._2() != null) - e._2() - .forEach( - proj -> { - try { - Project project = mapper.readValue(proj, Project.class); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + if (e._2() != null) + e + ._2() + .forEach( + proj -> { + try { + Project project = mapper.readValue(proj, Project.class); - p.mergeFrom(project); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - if (p.getDataInfo() == null) p.setDataInfo(new DataInfo()); - p.getDataInfo().setTrust("0.9"); - p.setLastupdatetimestamp(ts); - return p; - } + p.mergeFrom(project); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + if (p.getDataInfo() == null) + p.setDataInfo(new DataInfo()); + p.getDataInfo().setTrust("0.9"); + p.setLastupdatetimestamp(ts); + return p; + } - private static Software softwareMerger(Tuple2> e, final long ts) { + private static Software softwareMerger(Tuple2> e, final long ts) { - Software s = new Software(); // the result of the merge, to be returned at the end + Software s = new Software(); // the result of the merge, to be returned at the end - s.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - final Collection dateofacceptance = Lists.newArrayList(); - if (e._2() != null) - e._2() - .forEach( - soft -> { - try { - Software software = mapper.readValue(soft, Software.class); + s.setId(e._1()); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + final Collection dateofacceptance = Lists.newArrayList(); + if (e._2() != null) + e + ._2() + .forEach( + soft -> { + try { + Software software = mapper.readValue(soft, Software.class); - s.mergeFrom(software); - s.setAuthor(DedupUtility.mergeAuthor(s.getAuthor(), software.getAuthor())); - // add to the list if they are not null - if (software.getDateofacceptance() != null) - dateofacceptance.add(software.getDateofacceptance().getValue()); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - s.setDateofacceptance(DatePicker.pick(dateofacceptance)); - if (s.getDataInfo() == null) s.setDataInfo(new DataInfo()); - s.getDataInfo().setTrust("0.9"); - s.setLastupdatetimestamp(ts); - return s; - } + s.mergeFrom(software); + s.setAuthor(DedupUtility.mergeAuthor(s.getAuthor(), software.getAuthor())); + // add to the list if they are not null + if (software.getDateofacceptance() != null) + dateofacceptance.add(software.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + s.setDateofacceptance(DatePicker.pick(dateofacceptance)); + if (s.getDataInfo() == null) + s.setDataInfo(new DataInfo()); + s.getDataInfo().setTrust("0.9"); + s.setLastupdatetimestamp(ts); + return s; + } - private static Datasource datasourceMerger(Tuple2> e, final long ts) { - Datasource d = new Datasource(); // the result of the merge, to be returned at the end - d.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - if (e._2() != null) - e._2() - .forEach( - dat -> { - try { - Datasource datasource = mapper.readValue(dat, Datasource.class); + private static Datasource datasourceMerger(Tuple2> e, final long ts) { + Datasource d = new Datasource(); // the result of the merge, to be returned at the end + d.setId(e._1()); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + if (e._2() != null) + e + ._2() + .forEach( + dat -> { + try { + Datasource datasource = mapper.readValue(dat, Datasource.class); - d.mergeFrom(datasource); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - if (d.getDataInfo() == null) d.setDataInfo(new DataInfo()); - d.getDataInfo().setTrust("0.9"); - d.setLastupdatetimestamp(ts); - return d; - } + d.mergeFrom(datasource); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + if (d.getDataInfo() == null) + d.setDataInfo(new DataInfo()); + d.getDataInfo().setTrust("0.9"); + d.setLastupdatetimestamp(ts); + return d; + } - private static Organization organizationMerger( - Tuple2> e, final long ts) { + private static Organization organizationMerger( + Tuple2> e, final long ts) { - Organization o = new Organization(); // the result of the merge, to be returned at the end + Organization o = new Organization(); // the result of the merge, to be returned at the end - o.setId(e._1()); + o.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - StringBuilder trust = new StringBuilder("0.0"); + StringBuilder trust = new StringBuilder("0.0"); - if (e._2() != null) - e._2() - .forEach( - pub -> { - try { - Organization organization = mapper.readValue(pub, Organization.class); + if (e._2() != null) + e + ._2() + .forEach( + pub -> { + try { + Organization organization = mapper.readValue(pub, Organization.class); - final String currentTrust = organization.getDataInfo().getTrust(); - if (!"1.0".equals(currentTrust)) { - trust.setLength(0); - trust.append(currentTrust); - } - o.mergeFrom(organization); + final String currentTrust = organization.getDataInfo().getTrust(); + if (!"1.0".equals(currentTrust)) { + trust.setLength(0); + trust.append(currentTrust); + } + o.mergeFrom(organization); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); - if (o.getDataInfo() == null) { - o.setDataInfo(new DataInfo()); - } - if (o.getDataInfo() == null) o.setDataInfo(new DataInfo()); - o.getDataInfo().setTrust("0.9"); - o.setLastupdatetimestamp(ts); + if (o.getDataInfo() == null) { + o.setDataInfo(new DataInfo()); + } + if (o.getDataInfo() == null) + o.setDataInfo(new DataInfo()); + o.getDataInfo().setTrust("0.9"); + o.setLastupdatetimestamp(ts); - return o; - } + return o; + } - private static OtherResearchProduct otherresearchproductMerger( - Tuple2> e, final long ts) { + private static OtherResearchProduct otherresearchproductMerger( + Tuple2> e, final long ts) { - OtherResearchProduct o = new OtherResearchProduct(); // the result of the merge, to be - // returned at the end + OtherResearchProduct o = new OtherResearchProduct(); // the result of the merge, to be + // returned at the end - o.setId(e._1()); + o.setId(e._1()); - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - final Collection dateofacceptance = Lists.newArrayList(); + final Collection dateofacceptance = Lists.newArrayList(); - if (e._2() != null) - e._2() - .forEach( - orp -> { - try { - OtherResearchProduct otherResearchProduct = - mapper.readValue(orp, OtherResearchProduct.class); + if (e._2() != null) + e + ._2() + .forEach( + orp -> { + try { + OtherResearchProduct otherResearchProduct = mapper + .readValue(orp, OtherResearchProduct.class); - o.mergeFrom(otherResearchProduct); - o.setAuthor( - DedupUtility.mergeAuthor(o.getAuthor(), otherResearchProduct.getAuthor())); - // add to the list if they are not null - if (otherResearchProduct.getDateofacceptance() != null) - dateofacceptance.add(otherResearchProduct.getDateofacceptance().getValue()); - } catch (Exception exc) { - throw new RuntimeException(exc); - } - }); - if (o.getDataInfo() == null) o.setDataInfo(new DataInfo()); - o.setDateofacceptance(DatePicker.pick(dateofacceptance)); - o.getDataInfo().setTrust("0.9"); - o.setLastupdatetimestamp(ts); - return o; - } + o.mergeFrom(otherResearchProduct); + o + .setAuthor( + DedupUtility.mergeAuthor(o.getAuthor(), otherResearchProduct.getAuthor())); + // add to the list if they are not null + if (otherResearchProduct.getDateofacceptance() != null) + dateofacceptance.add(otherResearchProduct.getDateofacceptance().getValue()); + } catch (Exception exc) { + throw new RuntimeException(exc); + } + }); + if (o.getDataInfo() == null) + o.setDataInfo(new DataInfo()); + o.setDateofacceptance(DatePicker.pick(dateofacceptance)); + o.getDataInfo().setTrust("0.9"); + o.setLastupdatetimestamp(ts); + return o; + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupUtility.java index 60f0a50f7..70a2e3591 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupUtility.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/DedupUtility.java @@ -1,13 +1,6 @@ + package eu.dnetlib.dedup; -import com.google.common.collect.Sets; -import com.wcohen.ss.JaroWinkler; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.model.Person; import java.io.IOException; import java.io.StringWriter; import java.nio.charset.StandardCharsets; @@ -15,6 +8,7 @@ import java.security.MessageDigest; import java.text.Normalizer; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.codec.binary.Hex; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; @@ -26,205 +20,220 @@ 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 com.google.common.collect.Sets; +import com.wcohen.ss.JaroWinkler; + +import eu.dnetlib.dhp.schema.oaf.Author; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import eu.dnetlib.pace.clustering.BlacklistAwareClusteringCombiner; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.model.Person; import scala.Tuple2; public class DedupUtility { - private static final Double THRESHOLD = 0.95; + private static final Double THRESHOLD = 0.95; - public static Map constructAccumulator( - final DedupConfig dedupConf, final SparkContext context) { + public static Map constructAccumulator( + final DedupConfig dedupConf, final SparkContext context) { - Map accumulators = new HashMap<>(); + 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)); + 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; - } + return accumulators; + } - public static JavaRDD loadDataFromHDFS(String path, JavaSparkContext context) { - return context.textFile(path); - } + 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 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 { + 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))); + 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())); - } + 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 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)); - } + 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; - } - } + 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; + } + } - public static List mergeAuthor(final List a, final List b) { - int pa = countAuthorsPids(a); - int pb = countAuthorsPids(b); - List base, enrich; - int sa = authorsSize(a); - int sb = authorsSize(b); + public static List mergeAuthor(final List a, final List b) { + int pa = countAuthorsPids(a); + int pb = countAuthorsPids(b); + List base, enrich; + int sa = authorsSize(a); + int sb = authorsSize(b); - if (pa == pb) { - base = sa > sb ? a : b; - enrich = sa > sb ? b : a; - } else { - base = pa > pb ? a : b; - enrich = pa > pb ? b : a; - } - enrichPidFromList(base, enrich); - return base; - } + if (pa == pb) { + base = sa > sb ? a : b; + enrich = sa > sb ? b : a; + } else { + base = pa > pb ? a : b; + enrich = pa > pb ? b : a; + } + enrichPidFromList(base, enrich); + return base; + } - private static void enrichPidFromList(List base, List enrich) { - if (base == null || enrich == null) return; - final Map basePidAuthorMap = - base.stream() - .filter(a -> a.getPid() != null && a.getPid().size() > 0) - .flatMap(a -> a.getPid().stream().map(p -> new Tuple2<>(p.toComparableString(), a))) - .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); + private static void enrichPidFromList(List base, List enrich) { + if (base == null || enrich == null) + return; + final Map basePidAuthorMap = base + .stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap(a -> a.getPid().stream().map(p -> new Tuple2<>(p.toComparableString(), a))) + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); - final List> pidToEnrich = - enrich.stream() - .filter(a -> a.getPid() != null && a.getPid().size() > 0) - .flatMap( - a -> - a.getPid().stream() - .filter(p -> !basePidAuthorMap.containsKey(p.toComparableString())) - .map(p -> new Tuple2<>(p, a))) - .collect(Collectors.toList()); + final List> pidToEnrich = enrich + .stream() + .filter(a -> a.getPid() != null && a.getPid().size() > 0) + .flatMap( + a -> a + .getPid() + .stream() + .filter(p -> !basePidAuthorMap.containsKey(p.toComparableString())) + .map(p -> new Tuple2<>(p, a))) + .collect(Collectors.toList()); - pidToEnrich.forEach( - a -> { - Optional> simAuhtor = - base.stream() - .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) - .max(Comparator.comparing(Tuple2::_1)); - if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) { - Author r = simAuhtor.get()._2(); - r.getPid().add(a._1()); - } - }); - } + pidToEnrich + .forEach( + a -> { + Optional> simAuhtor = base + .stream() + .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) + .max(Comparator.comparing(Tuple2::_1)); + if (simAuhtor.isPresent() && simAuhtor.get()._1() > THRESHOLD) { + Author r = simAuhtor.get()._2(); + r.getPid().add(a._1()); + } + }); + } - public static String createEntityPath(final String basePath, final String entityType) { - return String.format("%s/%s", basePath, entityType); - } + public static String createEntityPath(final String basePath, final String entityType) { + return String.format("%s/%s", basePath, entityType); + } - public static String createSimRelPath(final String basePath, final String entityType) { - return String.format("%s/%s/simRel", basePath, entityType); - } + public static String createSimRelPath(final String basePath, final String entityType) { + return String.format("%s/%s/simRel", basePath, entityType); + } - public static String createMergeRelPath(final String basePath, final String entityType) { - return String.format("%s/%s/mergeRel", basePath, entityType); - } + public static String createMergeRelPath(final String basePath, final String entityType) { + return String.format("%s/%s/mergeRel", basePath, entityType); + } - private static Double sim(Author a, Author b) { + private static Double sim(Author a, Author b) { - final Person pa = parse(a); - final Person pb = parse(b); + final Person pa = parse(a); + final Person pb = parse(b); - if (pa.isAccurate() & pb.isAccurate()) { - return new JaroWinkler() - .score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())); - } else { - return new JaroWinkler() - .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); - } - } + if (pa.isAccurate() & pb.isAccurate()) { + return new JaroWinkler() + .score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())); + } else { + return new JaroWinkler() + .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); + } + } - private static String normalize(final String s) { - return nfd(s) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") - .trim(); - } + private static String normalize(final String s) { + return nfd(s) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim(); + } - private static String nfd(final String s) { - return Normalizer.normalize(s, Normalizer.Form.NFD); - } + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } - private static Person parse(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return new Person(author.getSurname() + ", " + author.getName(), false); - } else { - return new Person(author.getFullname(), false); - } - } + private static Person parse(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return new Person(author.getSurname() + ", " + author.getName(), false); + } else { + return new Person(author.getFullname(), false); + } + } - private static int countAuthorsPids(List authors) { - if (authors == null) return 0; + private static int countAuthorsPids(List authors) { + if (authors == null) + return 0; - return (int) authors.stream().filter(DedupUtility::hasPid).count(); - } + return (int) authors.stream().filter(DedupUtility::hasPid).count(); + } - private static int authorsSize(List authors) { - if (authors == null) return 0; - return authors.size(); - } + private static int authorsSize(List authors) { + if (authors == null) + return 0; + return authors.size(); + } - private static boolean hasPid(Author a) { - if (a == null || a.getPid() == null || a.getPid().size() == 0) return false; - return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); - } + private static boolean hasPid(Author a) { + if (a == null || a.getPid() == null || a.getPid().size() == 0) + return false; + return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/Deduper.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/Deduper.java index 681a4168a..e7d49be98 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/Deduper.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/Deduper.java @@ -1,11 +1,9 @@ + package eu.dnetlib.dedup; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.util.BlockProcessor; -import eu.dnetlib.pace.util.MapDocumentUtil; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.spark.api.java.JavaPairRDD; @@ -15,170 +13,170 @@ 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 eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.BlockProcessor; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Serializable; import scala.Tuple2; public class Deduper implements Serializable { - private static final Log log = LogFactory.getLog(Deduper.class); + 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) { + /** + * @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()); + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); - // create vertexes of the graph: - JavaPairRDD mapDocs = mapToVertexes(context, entities, config); + // create vertexes of the graph: + JavaPairRDD mapDocs = mapToVertexes(context, entities, config); - // create blocks for deduplication - JavaPairRDD> blocks = createBlocks(context, mapDocs, 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); + // 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(); - } + // 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) { + /** + * @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()); + 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 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(); - } + /** + * @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; - }); - } + 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 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); - }); - } + 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()); + public static JavaPairRDD computeRelations2( + JavaSparkContext context, JavaPairRDD> blocks, DedupConfig config) { + Map accumulators = DedupUtility.constructAccumulator(config, context.sc()); - return blocks - .flatMapToPair( - (PairFlatMapFunction>, String, String>) - it -> { - try { - final SparkReporter reporter = new SparkReporter(accumulators); - new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter); - return reporter.getRelations().iterator(); - } catch (Exception e) { - throw new RuntimeException(it._2().get(0).getIdentifier(), e); - } - }) - .mapToPair( - (PairFunction, String, Tuple2>) - item -> new Tuple2>(item._1() + item._2(), item)) - .reduceByKey((a, b) -> a) - .mapToPair( - (PairFunction>, String, String>) Tuple2::_2); - } + return blocks + .flatMapToPair( + (PairFlatMapFunction>, String, String>) it -> { + try { + final SparkReporter reporter = new SparkReporter(accumulators); + new BlockProcessor(config).processSortedBlock(it._1(), it._2(), reporter); + return reporter.getRelations().iterator(); + } catch (Exception e) { + throw new RuntimeException(it._2().get(0).getIdentifier(), e); + } + }) + .mapToPair( + (PairFunction, String, Tuple2>) item -> new Tuple2>( + item._1() + item._2(), item)) + .reduceByKey((a, b) -> a) + .mapToPair( + (PairFunction>, String, String>) Tuple2::_2); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/OafEntityType.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/OafEntityType.java index 72c771a13..bc9948190 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/OafEntityType.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/OafEntityType.java @@ -1,11 +1,6 @@ + package eu.dnetlib.dedup; public enum OafEntityType { - datasource, - organization, - project, - dataset, - otherresearchproduct, - software, - publication + datasource, organization, project, dataset, otherresearchproduct, software, publication } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java index 1039b8636..f86410d29 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateConnectedComponent.java @@ -1,14 +1,9 @@ + package eu.dnetlib.dedup; -import com.google.common.hash.Hashing; -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 java.util.ArrayList; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -20,86 +15,93 @@ 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 com.google.common.hash.Hashing; + +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 scala.Tuple2; public class SparkCreateConnectedComponent { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateConnectedComponent.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/dedup/dedup_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkCreateConnectedComponent.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateConnectedComponent.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/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 DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); + 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 DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); - final JavaPairRDD vertexes = - sc.textFile(inputPath + "/" + entity) - .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) - .mapToPair( - (PairFunction) - s -> new Tuple2(getHashcode(s), s)); + final JavaPairRDD vertexes = sc + .textFile(inputPath + "/" + entity) + .map(s -> MapDocumentUtil.getJPathString(dedupConf.getWf().getIdPath(), s)) + .mapToPair( + (PairFunction) s -> new Tuple2(getHashcode(s), s)); - final Dataset similarityRelations = - spark - .read() - .load(DedupUtility.createSimRelPath(targetPath, entity)) - .as(Encoders.bean(Relation.class)); - final RDD> edgeRdd = - similarityRelations - .javaRDD() - .map( - it -> - new Edge<>( - getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())) - .rdd(); - final JavaRDD cc = - GraphProcessor.findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()) - .toJavaRDD(); - final Dataset mergeRelation = - spark.createDataset( - cc.filter(k -> k.getDocIds().size() > 1) - .flatMap( - (FlatMapFunction) - c -> - c.getDocIds().stream() - .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(DedupUtility.createMergeRelPath(targetPath, entity)); - } + final Dataset similarityRelations = spark + .read() + .load(DedupUtility.createSimRelPath(targetPath, entity)) + .as(Encoders.bean(Relation.class)); + final RDD> edgeRdd = similarityRelations + .javaRDD() + .map( + it -> new Edge<>( + getHashcode(it.getSource()), getHashcode(it.getTarget()), it.getRelClass())) + .rdd(); + final JavaRDD cc = GraphProcessor + .findCCs(vertexes.rdd(), edgeRdd, dedupConf.getWf().getMaxIterations()) + .toJavaRDD(); + final 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(DedupUtility.createMergeRelPath(targetPath, entity)); + } - public static long getHashcode(final String id) { - return Hashing.murmur3_128().hashString(id).asLong(); - } + public static long getHashcode(final String id) { + return Hashing.murmur3_128().hashString(id).asLong(); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java index dbc97466d..d87269f03 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateDedupRecord.java @@ -1,48 +1,52 @@ + package eu.dnetlib.dedup; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.pace.config.DedupConfig; import org.apache.commons.io.IOUtils; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.pace.config.DedupConfig; + public class SparkCreateDedupRecord { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkCreateDedupRecord.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/dedup/dedupRecord_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkCreateDedupRecord.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateDedupRecord.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/dedup/dedupRecord_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 sourcePath = parser.get("sourcePath"); - final String entity = parser.get("entity"); - final String dedupPath = parser.get("dedupPath"); - final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String sourcePath = parser.get("sourcePath"); + final String entity = parser.get("entity"); + final String dedupPath = parser.get("dedupPath"); + final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); - final JavaRDD dedupRecord = - DedupRecordFactory.createDedupRecord( - sc, - spark, - DedupUtility.createMergeRelPath(dedupPath, entity), - DedupUtility.createEntityPath(sourcePath, entity), - OafEntityType.valueOf(entity), - dedupConf); - dedupRecord - .map( - r -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(r); - }) - .saveAsTextFile(dedupPath + "/" + entity + "/dedup_records"); - } + final JavaRDD dedupRecord = DedupRecordFactory + .createDedupRecord( + sc, + spark, + DedupUtility.createMergeRelPath(dedupPath, entity), + DedupUtility.createEntityPath(sourcePath, entity), + OafEntityType.valueOf(entity), + dedupConf); + dedupRecord + .map( + r -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(r); + }) + .saveAsTextFile(dedupPath + "/" + entity + "/dedup_records"); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java index b8df49af3..41fe911e7 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkCreateSimRels.java @@ -1,80 +1,83 @@ + package eu.dnetlib.dedup; -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 java.util.List; + 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.sql.Encoders; import org.apache.spark.sql.SparkSession; + +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 scala.Tuple2; /** * This Spark class creates similarity relations between entities, saving result - * - *

param request: sourcePath entityType target Path + *

+ * 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/sx/dedup/dedup_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkCreateSimRels.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/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.conf.json"))); - final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); + 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.conf.json"))); + final DedupConfig dedupConf = DedupConfig.load(parser.get("dedupConf")); - JavaPairRDD mapDocument = - sc.textFile(inputPath + "/" + entity) - .mapToPair( - s -> { - MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); - return new Tuple2<>(d.getIdentifier(), d); - }); + JavaPairRDD mapDocument = sc + .textFile(inputPath + "/" + entity) + .mapToPair( + s -> { + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); + return new Tuple2<>(d.getIdentifier(), d); + }); - // create blocks for deduplication - JavaPairRDD> blocks = - Deduper.createsortedBlocks(sc, mapDocument, dedupConf); - // JavaPairRDD> blocks = Deduper.createBlocks(sc, - // mapDocument, dedupConf); + // create blocks for deduplication + JavaPairRDD> blocks = Deduper.createsortedBlocks(sc, mapDocument, dedupConf); + // JavaPairRDD> blocks = Deduper.createBlocks(sc, + // mapDocument, dedupConf); - // create relations by comparing only elements in the same group - final JavaPairRDD dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf); - // final JavaPairRDD dedupRels = Deduper.computeRelations(sc, blocks, - // dedupConf); + // create relations by comparing only elements in the same group + final JavaPairRDD dedupRels = Deduper.computeRelations2(sc, blocks, dedupConf); + // final JavaPairRDD dedupRels = Deduper.computeRelations(sc, blocks, + // dedupConf); - final JavaRDD isSimilarToRDD = - dedupRels.map( - simRel -> { - final Relation r = new Relation(); - r.setSource(simRel._1()); - r.setTarget(simRel._2()); - r.setRelClass("isSimilarTo"); - return r; - }); + 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(DedupUtility.createSimRelPath(targetPath, entity)); - } + spark + .createDataset(isSimilarToRDD.rdd(), Encoders.bean(Relation.class)) + .write() + .mode("overwrite") + .save(DedupUtility.createSimRelPath(targetPath, entity)); + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java index d92eef2d4..21e72b5b8 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/SparkReporter.java @@ -1,49 +1,52 @@ + package eu.dnetlib.dedup; -import eu.dnetlib.pace.util.Reporter; import java.util.ArrayList; import java.util.List; import java.util.Map; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.spark.util.LongAccumulator; + +import eu.dnetlib.pace.util.Reporter; import scala.Serializable; import scala.Tuple2; public class SparkReporter implements Serializable, Reporter { - final List> relations = new ArrayList<>(); - private static final Log log = LogFactory.getLog(SparkReporter.class); - Map accumulators; + final List> relations = new ArrayList<>(); + private static final Log log = LogFactory.getLog(SparkReporter.class); + Map accumulators; - public SparkReporter(Map accumulators) { - this.accumulators = accumulators; - } + public SparkReporter(Map accumulators) { + this.accumulators = accumulators; + } - public void incrementCounter( - String counterGroup, - String counterName, - long delta, - Map 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); - } - } + 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) { + @Override + public void incrementCounter(String counterGroup, String counterName, long delta) { - incrementCounter(counterGroup, counterName, delta, accumulators); - } + incrementCounter(counterGroup, counterName, delta, accumulators); + } - @Override - public void emit(String type, String from, String to) { - relations.add(new Tuple2<>(from, to)); - } + @Override + public void emit(String type, String from, String to) { + relations.add(new Tuple2<>(from, to)); + } - public List> getRelations() { - return relations; - } + public List> getRelations() { + return relations; + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java index 444d987d8..79a3114fd 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/graph/ConnectedComponent.java @@ -1,78 +1,84 @@ + package eu.dnetlib.dedup.graph; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dedup.DedupUtility; -import eu.dnetlib.pace.util.PaceException; import java.io.IOException; import java.io.Serializable; import java.util.Set; + import org.apache.commons.lang.StringUtils; import org.codehaus.jackson.annotate.JsonIgnore; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dedup.DedupUtility; +import eu.dnetlib.pace.util.PaceException; + public class ConnectedComponent implements Serializable { - private Set docIds; - private String ccId; + private Set docIds; + private String ccId; - public ConnectedComponent() {} + public ConnectedComponent() { + } - public ConnectedComponent(Set docIds) { - this.docIds = docIds; - createID(); - } + 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_wf_001::" + DedupUtility.md5(s); - return ccId; - } else { - return docIds.iterator().next(); - } - } + public String createID() { + if (docIds.size() > 1) { + final String s = getMin(); + String prefix = s.split("\\|")[0]; + ccId = prefix + "|dedup_wf_001::" + DedupUtility.md5(s); + return ccId; + } else { + return docIds.iterator().next(); + } + } - @JsonIgnore - public String getMin() { + @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(); - } + 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); - } - } + @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 Set getDocIds() { + return docIds; + } - public void setDocIds(Set docIds) { - this.docIds = docIds; - } + public void setDocIds(Set docIds) { + this.docIds = docIds; + } - public String getCcId() { - return ccId; - } + public String getCcId() { + return ccId; + } - public void setCcId(String ccId) { - this.ccId = ccId; - } + public void setCcId(String ccId) { + this.ccId = ccId; + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java index 54a2d5dba..e3d4fdbe3 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkPropagateRelationsJob.java @@ -1,110 +1,112 @@ + package eu.dnetlib.dedup.sx; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.DHPUtils; import java.io.IOException; + import org.apache.commons.io.IOUtils; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; public class SparkPropagateRelationsJob { - enum FieldType { - SOURCE, - TARGET - } + enum FieldType { + SOURCE, TARGET + } - static final String SOURCEJSONPATH = "$.source"; - static final String TARGETJSONPATH = "$.target"; + static final String SOURCEJSONPATH = "$.source"; + static final String TARGETJSONPATH = "$.target"; - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkPropagateRelationsJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkUpdateEntityJob.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkPropagateRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/dedup/dedup_propagate_relation_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkUpdateEntityJob.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String relationPath = parser.get("relationPath"); - final String mergeRelPath = parser.get("mergeRelPath"); - final String targetRelPath = parser.get("targetRelPath"); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String relationPath = parser.get("relationPath"); + final String mergeRelPath = parser.get("mergeRelPath"); + final String targetRelPath = parser.get("targetRelPath"); - final Dataset merge = - spark - .read() - .load(mergeRelPath) - .as(Encoders.bean(Relation.class)) - .where("relClass == 'merges'"); + final Dataset merge = spark + .read() + .load(mergeRelPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'"); - final Dataset rels = - spark.read().load(relationPath).as(Encoders.bean(Relation.class)); + final Dataset rels = spark.read().load(relationPath).as(Encoders.bean(Relation.class)); - final Dataset firstJoin = - rels.joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer") - .map( - (MapFunction, Relation>) - r -> { - final Relation mergeRelation = r._2(); - final Relation relation = r._1(); + final Dataset firstJoin = rels + .joinWith(merge, merge.col("target").equalTo(rels.col("source")), "left_outer") + .map( + (MapFunction, Relation>) r -> { + final Relation mergeRelation = r._2(); + final Relation relation = r._1(); - if (mergeRelation != null) relation.setSource(mergeRelation.getSource()); - return relation; - }, - Encoders.bean(Relation.class)); + if (mergeRelation != null) + relation.setSource(mergeRelation.getSource()); + return relation; + }, + Encoders.bean(Relation.class)); - final Dataset secondJoin = - firstJoin - .joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer") - .map( - (MapFunction, Relation>) - r -> { - final Relation mergeRelation = r._2(); - final Relation relation = r._1(); - if (mergeRelation != null) relation.setTarget(mergeRelation.getSource()); - return relation; - }, - Encoders.bean(Relation.class)); + final Dataset secondJoin = firstJoin + .joinWith(merge, merge.col("target").equalTo(firstJoin.col("target")), "left_outer") + .map( + (MapFunction, Relation>) r -> { + final Relation mergeRelation = r._2(); + final Relation relation = r._1(); + if (mergeRelation != null) + relation.setTarget(mergeRelation.getSource()); + return relation; + }, + Encoders.bean(Relation.class)); - secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath); - } + secondJoin.write().mode(SaveMode.Overwrite).save(targetRelPath); + } - private static boolean containsDedup(final String json) { - final String source = DHPUtils.getJPathString(SOURCEJSONPATH, json); - final String target = DHPUtils.getJPathString(TARGETJSONPATH, json); + private static boolean containsDedup(final String json) { + final String source = DHPUtils.getJPathString(SOURCEJSONPATH, json); + final String target = DHPUtils.getJPathString(TARGETJSONPATH, json); - return source.toLowerCase().contains("dedup") || target.toLowerCase().contains("dedup"); - } + return source.toLowerCase().contains("dedup") || target.toLowerCase().contains("dedup"); + } - private static String replaceField(final String json, final String id, final FieldType type) { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - try { - Relation relation = mapper.readValue(json, Relation.class); - if (relation.getDataInfo() == null) relation.setDataInfo(new DataInfo()); - relation.getDataInfo().setDeletedbyinference(false); - switch (type) { - case SOURCE: - relation.setSource(id); - return mapper.writeValueAsString(relation); - case TARGET: - relation.setTarget(id); - return mapper.writeValueAsString(relation); - default: - throw new IllegalArgumentException(""); - } - } catch (IOException e) { - throw new RuntimeException("unable to deserialize json relation: " + json, e); - } - } + private static String replaceField(final String json, final String id, final FieldType type) { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + try { + Relation relation = mapper.readValue(json, Relation.class); + if (relation.getDataInfo() == null) + relation.setDataInfo(new DataInfo()); + relation.getDataInfo().setDeletedbyinference(false); + switch (type) { + case SOURCE: + relation.setSource(id); + return mapper.writeValueAsString(relation); + case TARGET: + relation.setTarget(id); + return mapper.writeValueAsString(relation); + default: + throw new IllegalArgumentException(""); + } + } catch (IOException e) { + throw new RuntimeException("unable to deserialize json relation: " + json, e); + } + } } diff --git a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java index 6ebdb0572..a847ad612 100644 --- a/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java +++ b/dhp-workflows/dhp-dedup-scholexplorer/src/main/java/eu/dnetlib/dedup/sx/SparkUpdateEntityJob.java @@ -1,7 +1,19 @@ + package eu.dnetlib.dedup.sx; +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; +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.sql.*; + import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Oaf; @@ -10,90 +22,81 @@ import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; import eu.dnetlib.dhp.utils.DHPUtils; -import java.io.IOException; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.compress.GzipCodec; -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.sql.*; import scala.Tuple2; public class SparkUpdateEntityJob { - static final String IDJSONPATH = "$.id"; + static final String IDJSONPATH = "$.id"; - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkUpdateEntityJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkUpdateEntityJob.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkUpdateEntityJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/dedup/dedup_delete_by_inference_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkUpdateEntityJob.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String entityPath = parser.get("entityPath"); - final String mergeRelPath = parser.get("mergeRelPath"); - final String dedupRecordPath = parser.get("dedupRecordPath"); - final String entity = parser.get("entity"); - final String destination = parser.get("targetPath"); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String entityPath = parser.get("entityPath"); + final String mergeRelPath = parser.get("mergeRelPath"); + final String dedupRecordPath = parser.get("dedupRecordPath"); + final String entity = parser.get("entity"); + final String destination = parser.get("targetPath"); - final Dataset df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class)); - final JavaPairRDD mergedIds = - df.where("relClass == 'merges'") - .select(df.col("target")) - .distinct() - .toJavaRDD() - .mapToPair((PairFunction) r -> new Tuple2<>(r.getString(0), "d")); - final JavaRDD sourceEntity = sc.textFile(entityPath); + final Dataset df = spark.read().load(mergeRelPath).as(Encoders.bean(Relation.class)); + final JavaPairRDD mergedIds = df + .where("relClass == 'merges'") + .select(df.col("target")) + .distinct() + .toJavaRDD() + .mapToPair((PairFunction) r -> new Tuple2<>(r.getString(0), "d")); + final JavaRDD sourceEntity = sc.textFile(entityPath); - final JavaRDD dedupEntity = sc.textFile(dedupRecordPath); - JavaPairRDD entitiesWithId = - sourceEntity.mapToPair( - (PairFunction) - s -> new Tuple2<>(DHPUtils.getJPathString(IDJSONPATH, s), s)); - Class mainClass; - switch (entity) { - case "publication": - mainClass = DLIPublication.class; - break; - case "dataset": - mainClass = DLIDataset.class; - break; - case "unknown": - mainClass = DLIUnknown.class; - break; - default: - throw new IllegalArgumentException("Illegal type " + entity); - } - JavaRDD map = - entitiesWithId - .leftOuterJoin(mergedIds) - .map( - k -> - k._2()._2().isPresent() - ? updateDeletedByInference(k._2()._1(), mainClass) - : k._2()._1()); - map.union(dedupEntity).saveAsTextFile(destination, GzipCodec.class); - } + final JavaRDD dedupEntity = sc.textFile(dedupRecordPath); + JavaPairRDD entitiesWithId = sourceEntity + .mapToPair( + (PairFunction) s -> new Tuple2<>(DHPUtils.getJPathString(IDJSONPATH, s), s)); + Class mainClass; + switch (entity) { + case "publication": + mainClass = DLIPublication.class; + break; + case "dataset": + mainClass = DLIDataset.class; + break; + case "unknown": + mainClass = DLIUnknown.class; + break; + default: + throw new IllegalArgumentException("Illegal type " + entity); + } + JavaRDD map = entitiesWithId + .leftOuterJoin(mergedIds) + .map( + k -> k._2()._2().isPresent() + ? updateDeletedByInference(k._2()._1(), mainClass) + : k._2()._1()); + map.union(dedupEntity).saveAsTextFile(destination, GzipCodec.class); + } - private static String updateDeletedByInference( - final String json, final Class clazz) { - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - try { - Oaf entity = mapper.readValue(json, clazz); - if (entity.getDataInfo() == null) entity.setDataInfo(new DataInfo()); - entity.getDataInfo().setDeletedbyinference(true); - return mapper.writeValueAsString(entity); - } catch (IOException e) { - throw new RuntimeException("Unable to convert json", e); - } - } + private static String updateDeletedByInference( + final String json, final Class clazz) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + try { + Oaf entity = mapper.readValue(json, clazz); + if (entity.getDataInfo() == null) + entity.setDataInfo(new DataInfo()); + entity.getDataInfo().setDeletedbyinference(true); + return mapper.writeValueAsString(entity); + } catch (IOException e) { + throw new RuntimeException("Unable to convert json", e); + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveImporterJob.java index 1046df609..0f74c6343 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveImporterJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveImporterJob.java @@ -1,11 +1,10 @@ + package eu.dnetlib.dhp.oa.graph.hive; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelSupport; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; @@ -15,61 +14,68 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelSupport; + public class GraphHiveImporterJob { - private static final Logger log = LoggerFactory.getLogger(GraphHiveImporterJob.class); + private static final Logger log = LoggerFactory.getLogger(GraphHiveImporterJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - GraphHiveImporterJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/input_graph_hive_parameters.json"))); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + GraphHiveImporterJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/input_graph_hive_parameters.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); - String hiveDbName = parser.get("hiveDbName"); - log.info("hiveDbName: {}", hiveDbName); + String hiveDbName = parser.get("hiveDbName"); + log.info("hiveDbName: {}", hiveDbName); - String hiveMetastoreUris = parser.get("hiveMetastoreUris"); - log.info("hiveMetastoreUris: {}", hiveMetastoreUris); + String hiveMetastoreUris = parser.get("hiveMetastoreUris"); + log.info("hiveMetastoreUris: {}", hiveMetastoreUris); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", hiveMetastoreUris); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", hiveMetastoreUris); - runWithSparkHiveSession( - conf, isSparkSessionManaged, spark -> loadGraphAsHiveDB(spark, inputPath, hiveDbName)); - } + runWithSparkHiveSession( + conf, isSparkSessionManaged, spark -> loadGraphAsHiveDB(spark, inputPath, hiveDbName)); + } - // protected for testing - private static void loadGraphAsHiveDB(SparkSession spark, String inputPath, String hiveDbName) { + // protected for testing + private static void loadGraphAsHiveDB(SparkSession spark, String inputPath, String hiveDbName) { - spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", hiveDbName)); - spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", hiveDbName)); + spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", hiveDbName)); + spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", hiveDbName)); - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - // Read the input file and convert it into RDD of serializable object - ModelSupport.oafTypes.forEach( - (name, clazz) -> - spark - .createDataset( - sc.textFile(inputPath + "/" + name) - .map(s -> OBJECT_MAPPER.readValue(s, clazz)) - .rdd(), - Encoders.bean(clazz)) - .write() - .mode(SaveMode.Overwrite) - .saveAsTable(hiveDbName + "." + name)); - } + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + // Read the input file and convert it into RDD of serializable object + ModelSupport.oafTypes + .forEach( + (name, clazz) -> spark + .createDataset( + sc + .textFile(inputPath + "/" + name) + .map(s -> OBJECT_MAPPER.readValue(s, clazz)) + .rdd(), + Encoders.bean(clazz)) + .write() + .mode(SaveMode.Overwrite) + .saveAsTable(hiveDbName + "." + name)); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 69cd0001f..e20d1eb79 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.createOpenaireId; @@ -10,6 +11,19 @@ import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.oaiIProvenance; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.qualifier; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.structuredProperty; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.DocumentFactory; +import org.dom4j.DocumentHelper; +import org.dom4j.Node; + import eu.dnetlib.dhp.oa.graph.raw.common.MigrationConstants; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Context; @@ -29,440 +43,429 @@ import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.Software; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.DocumentFactory; -import org.dom4j.DocumentHelper; -import org.dom4j.Node; public abstract class AbstractMdRecordToOafMapper { - protected final Map code2name; - - protected static final Qualifier MAIN_TITLE_QUALIFIER = - qualifier("main title", "main title", "dnet:dataCite_title", "dnet:dataCite_title"); - - protected AbstractMdRecordToOafMapper(final Map code2name) { - this.code2name = code2name; - } - - public List processMdRecord(final String xml) { - try { - final Map nsContext = new HashMap<>(); - nsContext.put("dr", "http://www.driver-repository.eu/namespace/dr"); - nsContext.put("dri", "http://www.driver-repository.eu/namespace/dri"); - nsContext.put("oaf", "http://namespace.openaire.eu/oaf"); - nsContext.put("oai", "http://www.openarchives.org/OAI/2.0/"); - nsContext.put("prov", "http://www.openarchives.org/OAI/2.0/provenance"); - nsContext.put("dc", "http://purl.org/dc/elements/1.1/"); - nsContext.put("datacite", "http://datacite.org/schema/kernel-3"); - DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext); - - final Document doc = - DocumentHelper.parseText( - xml.replaceAll( - "http://datacite.org/schema/kernel-4", "http://datacite.org/schema/kernel-3")); - - final String type = doc.valueOf("//dr:CobjCategory/@type"); - final KeyValue collectedFrom = - keyValue( - createOpenaireId(10, doc.valueOf("//oaf:collectedFrom/@id"), true), - doc.valueOf("//oaf:collectedFrom/@name")); - final KeyValue hostedBy = - StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id")) - ? collectedFrom - : keyValue( - createOpenaireId(10, doc.valueOf("//oaf:hostedBy/@id"), true), - doc.valueOf("//oaf:hostedBy/@name")); - - final DataInfo info = prepareDataInfo(doc); - final long lastUpdateTimestamp = new Date().getTime(); - - return createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp); - } catch (final Exception e) { - throw new RuntimeException(e); - } - } - - protected List createOafs( - final Document doc, - final String type, - final KeyValue collectedFrom, - final KeyValue hostedBy, - final DataInfo info, - final long lastUpdateTimestamp) { - - final List oafs = new ArrayList<>(); - - switch (type.toLowerCase()) { - case "": - case "publication": - final Publication p = new Publication(); - populateResultFields(p, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); - p.setResulttype(MigrationConstants.PUBLICATION_RESULTTYPE_QUALIFIER); - p.setJournal(prepareJournal(doc, info)); - oafs.add(p); - break; - case "dataset": - final Dataset d = new Dataset(); - populateResultFields(d, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); - d.setResulttype(MigrationConstants.DATASET_RESULTTYPE_QUALIFIER); - d.setStoragedate(prepareDatasetStorageDate(doc, info)); - d.setDevice(prepareDatasetDevice(doc, info)); - d.setSize(prepareDatasetSize(doc, info)); - d.setVersion(prepareDatasetVersion(doc, info)); - d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info)); - d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info)); - d.setGeolocation(prepareDatasetGeoLocations(doc, info)); - oafs.add(d); - break; - case "software": - final Software s = new Software(); - populateResultFields(s, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); - s.setResulttype(MigrationConstants.SOFTWARE_RESULTTYPE_QUALIFIER); - s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); - s.setLicense(prepareSoftwareLicenses(doc, info)); - s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info)); - s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info)); - oafs.add(s); - break; - case "otherresearchproducts": - default: - final OtherResearchProduct o = new OtherResearchProduct(); - populateResultFields(o, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); - o.setResulttype(MigrationConstants.OTHER_RESULTTYPE_QUALIFIER); - o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); - o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); - o.setTool(prepareOtherResearchProductTools(doc, info)); - oafs.add(o); - break; - } - - if (!oafs.isEmpty()) { - oafs.addAll(addProjectRels(doc, collectedFrom, info, lastUpdateTimestamp)); - oafs.addAll(addOtherResultRels(doc, collectedFrom, info, lastUpdateTimestamp)); - } - - return oafs; - } - - private List addProjectRels( - final Document doc, - final KeyValue collectedFrom, - final DataInfo info, - final long lastUpdateTimestamp) { - - final List res = new ArrayList<>(); - - final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); - - for (final Object o : doc.selectNodes("//oaf:projectid")) { - - final String originalId = ((Node) o).getText(); - - if (StringUtils.isNotBlank(originalId)) { - final String projectId = createOpenaireId(40, originalId, true); - - final Relation r1 = new Relation(); - r1.setRelType("resultProject"); - r1.setSubRelType("outcome"); - r1.setRelClass("isProducedBy"); - r1.setSource(docId); - r1.setTarget(projectId); - r1.setCollectedfrom(Arrays.asList(collectedFrom)); - r1.setDataInfo(info); - r1.setLastupdatetimestamp(lastUpdateTimestamp); - res.add(r1); - - final Relation r2 = new Relation(); - r2.setRelType("resultProject"); - r2.setSubRelType("outcome"); - r2.setRelClass("produces"); - r2.setSource(projectId); - r2.setTarget(docId); - r2.setCollectedfrom(Arrays.asList(collectedFrom)); - r2.setDataInfo(info); - r2.setLastupdatetimestamp(lastUpdateTimestamp); - res.add(r2); - } - } - - return res; - } - - protected abstract List addOtherResultRels( - final Document doc, - final KeyValue collectedFrom, - final DataInfo info, - final long lastUpdateTimestamp); - - private void populateResultFields( - final Result r, - final Document doc, - final KeyValue collectedFrom, - final KeyValue hostedBy, - final DataInfo info, - final long lastUpdateTimestamp) { - r.setDataInfo(info); - r.setLastupdatetimestamp(lastUpdateTimestamp); - r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false)); - r.setOriginalId(Arrays.asList(doc.valueOf("//dri:objIdentifier"))); - r.setCollectedfrom(Arrays.asList(collectedFrom)); - r.setPid( - prepareListStructProps( - doc, "//oaf:identifier", "@identifierType", "dnet:pid_types", "dnet:pid_types", info)); - r.setDateofcollection(doc.valueOf("//dr:dateOfCollection")); - r.setDateoftransformation(doc.valueOf("//dr:dateOfTransformation")); - r.setExtraInfo(new ArrayList<>()); // NOT PRESENT IN MDSTORES - r.setOaiprovenance(prepareOAIprovenance(doc)); - r.setAuthor(prepareAuthors(doc, info)); - r.setLanguage(prepareLanguages(doc)); - r.setCountry(new ArrayList<>()); // NOT PRESENT IN MDSTORES - r.setSubject(prepareSubjects(doc, info)); - r.setTitle(prepareTitles(doc, info)); - r.setRelevantdate(prepareRelevantDates(doc, info)); - r.setDescription(prepareDescriptions(doc, info)); - r.setDateofacceptance(prepareField(doc, "//oaf:dateAccepted", info)); - r.setPublisher(preparePublisher(doc, info)); - r.setEmbargoenddate(prepareField(doc, "//oaf:embargoenddate", info)); - r.setSource(prepareSources(doc, info)); - r.setFulltext(new ArrayList<>()); // NOT PRESENT IN MDSTORES - r.setFormat(prepareFormats(doc, info)); - r.setContributor(prepareContributors(doc, info)); - r.setResourcetype(prepareResourceType(doc, info)); - r.setCoverage(prepareCoverages(doc, info)); - r.setContext(prepareContexts(doc, info)); - r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES - r.setInstance(prepareInstances(doc, info, collectedFrom, hostedBy)); - } - - private List prepareContexts(final Document doc, final DataInfo info) { - final List list = new ArrayList<>(); - for (final Object o : doc.selectNodes("//oaf:concept")) { - final String cid = ((Node) o).valueOf("@id"); - if (StringUtils.isNotBlank(cid)) { - final Context c = new Context(); - c.setId(cid); - c.setDataInfo(Arrays.asList(info)); - list.add(c); - } - } - return list; - } - - protected abstract Qualifier prepareResourceType(Document doc, DataInfo info); - - protected abstract List prepareInstances( - Document doc, DataInfo info, KeyValue collectedfrom, KeyValue hostedby); - - protected abstract List> prepareSources(Document doc, DataInfo info); - - protected abstract List prepareRelevantDates(Document doc, DataInfo info); - - protected abstract List> prepareCoverages(Document doc, DataInfo info); - - protected abstract List> prepareContributors(Document doc, DataInfo info); - - protected abstract List> prepareFormats(Document doc, DataInfo info); - - protected abstract Field preparePublisher(Document doc, DataInfo info); - - protected abstract List> prepareDescriptions(Document doc, DataInfo info); - - protected abstract List prepareTitles(Document doc, DataInfo info); - - protected abstract List prepareSubjects(Document doc, DataInfo info); - - protected abstract Qualifier prepareLanguages(Document doc); - - protected abstract List prepareAuthors(Document doc, DataInfo info); - - protected abstract List> prepareOtherResearchProductTools( - Document doc, DataInfo info); - - protected abstract List> prepareOtherResearchProductContactGroups( - Document doc, DataInfo info); - - protected abstract List> prepareOtherResearchProductContactPersons( - Document doc, DataInfo info); - - protected abstract Qualifier prepareSoftwareProgrammingLanguage(Document doc, DataInfo info); - - protected abstract Field prepareSoftwareCodeRepositoryUrl(Document doc, DataInfo info); - - protected abstract List prepareSoftwareLicenses(Document doc, DataInfo info); - - protected abstract List> prepareSoftwareDocumentationUrls( - Document doc, DataInfo info); - - protected abstract List prepareDatasetGeoLocations(Document doc, DataInfo info); - - protected abstract Field prepareDatasetMetadataVersionNumber(Document doc, DataInfo info); - - protected abstract Field prepareDatasetLastMetadataUpdate(Document doc, DataInfo info); - - protected abstract Field prepareDatasetVersion(Document doc, DataInfo info); - - protected abstract Field prepareDatasetSize(Document doc, DataInfo info); - - protected abstract Field prepareDatasetDevice(Document doc, DataInfo info); - - protected abstract Field prepareDatasetStorageDate(Document doc, DataInfo info); - - private Journal prepareJournal(final Document doc, final DataInfo info) { - final Node n = doc.selectSingleNode("//oaf:journal"); - if (n != null) { - final String name = n.getText(); - final String issnPrinted = n.valueOf("@issn"); - final String issnOnline = n.valueOf("@eissn"); - final String issnLinking = n.valueOf("@lissn"); - final String ep = n.valueOf("@ep"); - final String iss = n.valueOf("@iss"); - final String sp = n.valueOf("@sp"); - final String vol = n.valueOf("@vol"); - final String edition = n.valueOf("@edition"); - if (StringUtils.isNotBlank(name)) { - return journal( - name, - issnPrinted, - issnOnline, - issnLinking, - ep, - iss, - sp, - vol, - edition, - null, - null, - info); - } - } - return null; - } - - protected Qualifier prepareQualifier( - final Node node, final String xpath, final String schemeId, final String schemeName) { - final String classId = node.valueOf(xpath); - final String className = code2name.get(classId); - return qualifier(classId, className, schemeId, schemeName); - } - - protected List prepareListStructProps( - final Node node, - final String xpath, - final String xpathClassId, - final String schemeId, - final String schemeName, - final DataInfo info) { - final List res = new ArrayList<>(); - for (final Object o : node.selectNodes(xpath)) { - final Node n = (Node) o; - final String classId = n.valueOf(xpathClassId); - final String className = code2name.get(classId); - res.add(structuredProperty(n.getText(), classId, className, schemeId, schemeName, info)); - } - return res; - } - - protected List prepareListStructProps( - final Node node, final String xpath, final Qualifier qualifier, final DataInfo info) { - final List res = new ArrayList<>(); - for (final Object o : node.selectNodes(xpath)) { - final Node n = (Node) o; - res.add(structuredProperty(n.getText(), qualifier, info)); - } - return res; - } - - protected List prepareListStructProps( - final Node node, final String xpath, final DataInfo info) { - final List res = new ArrayList<>(); - for (final Object o : node.selectNodes(xpath)) { - final Node n = (Node) o; - res.add( - structuredProperty( - n.getText(), - n.valueOf("@classid"), - n.valueOf("@classname"), - n.valueOf("@schemeid"), - n.valueOf("@schemename"), - info)); - } - return res; - } - - protected OAIProvenance prepareOAIprovenance(final Document doc) { - final Node n = - doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']"); - - if (n == null) { - return null; - } - - final String identifier = n.valueOf("./*[local-name()='identifier']"); - final String baseURL = n.valueOf("./*[local-name()='baseURL']"); - ; - final String metadataNamespace = n.valueOf("./*[local-name()='metadataNamespace']"); - ; - final boolean altered = n.valueOf("@altered").equalsIgnoreCase("true"); - final String datestamp = n.valueOf("./*[local-name()='datestamp']"); - ; - final String harvestDate = n.valueOf("@harvestDate"); - ; - - return oaiIProvenance(identifier, baseURL, metadataNamespace, altered, datestamp, harvestDate); - } - - protected DataInfo prepareDataInfo(final Document doc) { - final Node n = doc.selectSingleNode("//oaf:datainfo"); - - if (n == null) { - return dataInfo( - false, null, false, false, MigrationConstants.REPOSITORY_PROVENANCE_ACTIONS, "0.9"); - } - - final String paClassId = n.valueOf("./oaf:provenanceaction/@classid"); - final String paClassName = n.valueOf("./oaf:provenanceaction/@classname"); - final String paSchemeId = n.valueOf("./oaf:provenanceaction/@schemeid"); - final String paSchemeName = n.valueOf("./oaf:provenanceaction/@schemename"); - - final boolean deletedbyinference = Boolean.parseBoolean(n.valueOf("./oaf:deletedbyinference")); - final String inferenceprovenance = n.valueOf("./oaf:inferenceprovenance"); - final Boolean inferred = Boolean.parseBoolean(n.valueOf("./oaf:inferred")); - final String trust = n.valueOf("./oaf:trust"); - - return dataInfo( - deletedbyinference, - inferenceprovenance, - inferred, - false, - qualifier(paClassId, paClassName, paSchemeId, paSchemeName), - trust); - } - - protected Field prepareField(final Node node, final String xpath, final DataInfo info) { - return field(node.valueOf(xpath), info); - } - - protected List> prepareListFields( - final Node node, final String xpath, final DataInfo info) { - return listFields(info, prepareListString(node, xpath)); - } - - protected List prepareListString(final Node node, final String xpath) { - final List res = new ArrayList<>(); - for (final Object o : node.selectNodes(xpath)) { - final String s = ((Node) o).getText().trim(); - if (StringUtils.isNotBlank(s)) { - res.add(s); - } - } - return res; - } + protected final Map code2name; + + protected static final Qualifier MAIN_TITLE_QUALIFIER = qualifier( + "main title", "main title", "dnet:dataCite_title", "dnet:dataCite_title"); + + protected AbstractMdRecordToOafMapper(final Map code2name) { + this.code2name = code2name; + } + + public List processMdRecord(final String xml) { + try { + final Map nsContext = new HashMap<>(); + nsContext.put("dr", "http://www.driver-repository.eu/namespace/dr"); + nsContext.put("dri", "http://www.driver-repository.eu/namespace/dri"); + nsContext.put("oaf", "http://namespace.openaire.eu/oaf"); + nsContext.put("oai", "http://www.openarchives.org/OAI/2.0/"); + nsContext.put("prov", "http://www.openarchives.org/OAI/2.0/provenance"); + nsContext.put("dc", "http://purl.org/dc/elements/1.1/"); + nsContext.put("datacite", "http://datacite.org/schema/kernel-3"); + DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext); + + final Document doc = DocumentHelper + .parseText( + xml + .replaceAll( + "http://datacite.org/schema/kernel-4", "http://datacite.org/schema/kernel-3")); + + final String type = doc.valueOf("//dr:CobjCategory/@type"); + final KeyValue collectedFrom = keyValue( + createOpenaireId(10, doc.valueOf("//oaf:collectedFrom/@id"), true), + doc.valueOf("//oaf:collectedFrom/@name")); + final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id")) + ? collectedFrom + : keyValue( + createOpenaireId(10, doc.valueOf("//oaf:hostedBy/@id"), true), + doc.valueOf("//oaf:hostedBy/@name")); + + final DataInfo info = prepareDataInfo(doc); + final long lastUpdateTimestamp = new Date().getTime(); + + return createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + protected List createOafs( + final Document doc, + final String type, + final KeyValue collectedFrom, + final KeyValue hostedBy, + final DataInfo info, + final long lastUpdateTimestamp) { + + final List oafs = new ArrayList<>(); + + switch (type.toLowerCase()) { + case "": + case "publication": + final Publication p = new Publication(); + populateResultFields(p, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + p.setResulttype(MigrationConstants.PUBLICATION_RESULTTYPE_QUALIFIER); + p.setJournal(prepareJournal(doc, info)); + oafs.add(p); + break; + case "dataset": + final Dataset d = new Dataset(); + populateResultFields(d, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + d.setResulttype(MigrationConstants.DATASET_RESULTTYPE_QUALIFIER); + d.setStoragedate(prepareDatasetStorageDate(doc, info)); + d.setDevice(prepareDatasetDevice(doc, info)); + d.setSize(prepareDatasetSize(doc, info)); + d.setVersion(prepareDatasetVersion(doc, info)); + d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info)); + d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info)); + d.setGeolocation(prepareDatasetGeoLocations(doc, info)); + oafs.add(d); + break; + case "software": + final Software s = new Software(); + populateResultFields(s, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + s.setResulttype(MigrationConstants.SOFTWARE_RESULTTYPE_QUALIFIER); + s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); + s.setLicense(prepareSoftwareLicenses(doc, info)); + s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info)); + s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info)); + oafs.add(s); + break; + case "otherresearchproducts": + default: + final OtherResearchProduct o = new OtherResearchProduct(); + populateResultFields(o, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + o.setResulttype(MigrationConstants.OTHER_RESULTTYPE_QUALIFIER); + o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); + o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); + o.setTool(prepareOtherResearchProductTools(doc, info)); + oafs.add(o); + break; + } + + if (!oafs.isEmpty()) { + oafs.addAll(addProjectRels(doc, collectedFrom, info, lastUpdateTimestamp)); + oafs.addAll(addOtherResultRels(doc, collectedFrom, info, lastUpdateTimestamp)); + } + + return oafs; + } + + private List addProjectRels( + final Document doc, + final KeyValue collectedFrom, + final DataInfo info, + final long lastUpdateTimestamp) { + + final List res = new ArrayList<>(); + + final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); + + for (final Object o : doc.selectNodes("//oaf:projectid")) { + + final String originalId = ((Node) o).getText(); + + if (StringUtils.isNotBlank(originalId)) { + final String projectId = createOpenaireId(40, originalId, true); + + final Relation r1 = new Relation(); + r1.setRelType("resultProject"); + r1.setSubRelType("outcome"); + r1.setRelClass("isProducedBy"); + r1.setSource(docId); + r1.setTarget(projectId); + r1.setCollectedfrom(Arrays.asList(collectedFrom)); + r1.setDataInfo(info); + r1.setLastupdatetimestamp(lastUpdateTimestamp); + res.add(r1); + + final Relation r2 = new Relation(); + r2.setRelType("resultProject"); + r2.setSubRelType("outcome"); + r2.setRelClass("produces"); + r2.setSource(projectId); + r2.setTarget(docId); + r2.setCollectedfrom(Arrays.asList(collectedFrom)); + r2.setDataInfo(info); + r2.setLastupdatetimestamp(lastUpdateTimestamp); + res.add(r2); + } + } + + return res; + } + + protected abstract List addOtherResultRels( + final Document doc, + final KeyValue collectedFrom, + final DataInfo info, + final long lastUpdateTimestamp); + + private void populateResultFields( + final Result r, + final Document doc, + final KeyValue collectedFrom, + final KeyValue hostedBy, + final DataInfo info, + final long lastUpdateTimestamp) { + r.setDataInfo(info); + r.setLastupdatetimestamp(lastUpdateTimestamp); + r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false)); + r.setOriginalId(Arrays.asList(doc.valueOf("//dri:objIdentifier"))); + r.setCollectedfrom(Arrays.asList(collectedFrom)); + r + .setPid( + prepareListStructProps( + doc, "//oaf:identifier", "@identifierType", "dnet:pid_types", "dnet:pid_types", info)); + r.setDateofcollection(doc.valueOf("//dr:dateOfCollection")); + r.setDateoftransformation(doc.valueOf("//dr:dateOfTransformation")); + r.setExtraInfo(new ArrayList<>()); // NOT PRESENT IN MDSTORES + r.setOaiprovenance(prepareOAIprovenance(doc)); + r.setAuthor(prepareAuthors(doc, info)); + r.setLanguage(prepareLanguages(doc)); + r.setCountry(new ArrayList<>()); // NOT PRESENT IN MDSTORES + r.setSubject(prepareSubjects(doc, info)); + r.setTitle(prepareTitles(doc, info)); + r.setRelevantdate(prepareRelevantDates(doc, info)); + r.setDescription(prepareDescriptions(doc, info)); + r.setDateofacceptance(prepareField(doc, "//oaf:dateAccepted", info)); + r.setPublisher(preparePublisher(doc, info)); + r.setEmbargoenddate(prepareField(doc, "//oaf:embargoenddate", info)); + r.setSource(prepareSources(doc, info)); + r.setFulltext(new ArrayList<>()); // NOT PRESENT IN MDSTORES + r.setFormat(prepareFormats(doc, info)); + r.setContributor(prepareContributors(doc, info)); + r.setResourcetype(prepareResourceType(doc, info)); + r.setCoverage(prepareCoverages(doc, info)); + r.setContext(prepareContexts(doc, info)); + r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES + r.setInstance(prepareInstances(doc, info, collectedFrom, hostedBy)); + } + + private List prepareContexts(final Document doc, final DataInfo info) { + final List list = new ArrayList<>(); + for (final Object o : doc.selectNodes("//oaf:concept")) { + final String cid = ((Node) o).valueOf("@id"); + if (StringUtils.isNotBlank(cid)) { + final Context c = new Context(); + c.setId(cid); + c.setDataInfo(Arrays.asList(info)); + list.add(c); + } + } + return list; + } + + protected abstract Qualifier prepareResourceType(Document doc, DataInfo info); + + protected abstract List prepareInstances( + Document doc, DataInfo info, KeyValue collectedfrom, KeyValue hostedby); + + protected abstract List> prepareSources(Document doc, DataInfo info); + + protected abstract List prepareRelevantDates(Document doc, DataInfo info); + + protected abstract List> prepareCoverages(Document doc, DataInfo info); + + protected abstract List> prepareContributors(Document doc, DataInfo info); + + protected abstract List> prepareFormats(Document doc, DataInfo info); + + protected abstract Field preparePublisher(Document doc, DataInfo info); + + protected abstract List> prepareDescriptions(Document doc, DataInfo info); + + protected abstract List prepareTitles(Document doc, DataInfo info); + + protected abstract List prepareSubjects(Document doc, DataInfo info); + + protected abstract Qualifier prepareLanguages(Document doc); + + protected abstract List prepareAuthors(Document doc, DataInfo info); + + protected abstract List> prepareOtherResearchProductTools( + Document doc, DataInfo info); + + protected abstract List> prepareOtherResearchProductContactGroups( + Document doc, DataInfo info); + + protected abstract List> prepareOtherResearchProductContactPersons( + Document doc, DataInfo info); + + protected abstract Qualifier prepareSoftwareProgrammingLanguage(Document doc, DataInfo info); + + protected abstract Field prepareSoftwareCodeRepositoryUrl(Document doc, DataInfo info); + + protected abstract List prepareSoftwareLicenses(Document doc, DataInfo info); + + protected abstract List> prepareSoftwareDocumentationUrls( + Document doc, DataInfo info); + + protected abstract List prepareDatasetGeoLocations(Document doc, DataInfo info); + + protected abstract Field prepareDatasetMetadataVersionNumber(Document doc, DataInfo info); + + protected abstract Field prepareDatasetLastMetadataUpdate(Document doc, DataInfo info); + + protected abstract Field prepareDatasetVersion(Document doc, DataInfo info); + + protected abstract Field prepareDatasetSize(Document doc, DataInfo info); + + protected abstract Field prepareDatasetDevice(Document doc, DataInfo info); + + protected abstract Field prepareDatasetStorageDate(Document doc, DataInfo info); + + private Journal prepareJournal(final Document doc, final DataInfo info) { + final Node n = doc.selectSingleNode("//oaf:journal"); + if (n != null) { + final String name = n.getText(); + final String issnPrinted = n.valueOf("@issn"); + final String issnOnline = n.valueOf("@eissn"); + final String issnLinking = n.valueOf("@lissn"); + final String ep = n.valueOf("@ep"); + final String iss = n.valueOf("@iss"); + final String sp = n.valueOf("@sp"); + final String vol = n.valueOf("@vol"); + final String edition = n.valueOf("@edition"); + if (StringUtils.isNotBlank(name)) { + return journal( + name, + issnPrinted, + issnOnline, + issnLinking, + ep, + iss, + sp, + vol, + edition, + null, + null, + info); + } + } + return null; + } + + protected Qualifier prepareQualifier( + final Node node, final String xpath, final String schemeId, final String schemeName) { + final String classId = node.valueOf(xpath); + final String className = code2name.get(classId); + return qualifier(classId, className, schemeId, schemeName); + } + + protected List prepareListStructProps( + final Node node, + final String xpath, + final String xpathClassId, + final String schemeId, + final String schemeName, + final DataInfo info) { + final List res = new ArrayList<>(); + for (final Object o : node.selectNodes(xpath)) { + final Node n = (Node) o; + final String classId = n.valueOf(xpathClassId); + final String className = code2name.get(classId); + res.add(structuredProperty(n.getText(), classId, className, schemeId, schemeName, info)); + } + return res; + } + + protected List prepareListStructProps( + final Node node, final String xpath, final Qualifier qualifier, final DataInfo info) { + final List res = new ArrayList<>(); + for (final Object o : node.selectNodes(xpath)) { + final Node n = (Node) o; + res.add(structuredProperty(n.getText(), qualifier, info)); + } + return res; + } + + protected List prepareListStructProps( + final Node node, final String xpath, final DataInfo info) { + final List res = new ArrayList<>(); + for (final Object o : node.selectNodes(xpath)) { + final Node n = (Node) o; + res + .add( + structuredProperty( + n.getText(), + n.valueOf("@classid"), + n.valueOf("@classname"), + n.valueOf("@schemeid"), + n.valueOf("@schemename"), + info)); + } + return res; + } + + protected OAIProvenance prepareOAIprovenance(final Document doc) { + final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']"); + + if (n == null) { + return null; + } + + final String identifier = n.valueOf("./*[local-name()='identifier']"); + final String baseURL = n.valueOf("./*[local-name()='baseURL']"); + ; + final String metadataNamespace = n.valueOf("./*[local-name()='metadataNamespace']"); + ; + final boolean altered = n.valueOf("@altered").equalsIgnoreCase("true"); + final String datestamp = n.valueOf("./*[local-name()='datestamp']"); + ; + final String harvestDate = n.valueOf("@harvestDate"); + ; + + return oaiIProvenance(identifier, baseURL, metadataNamespace, altered, datestamp, harvestDate); + } + + protected DataInfo prepareDataInfo(final Document doc) { + final Node n = doc.selectSingleNode("//oaf:datainfo"); + + if (n == null) { + return dataInfo( + false, null, false, false, MigrationConstants.REPOSITORY_PROVENANCE_ACTIONS, "0.9"); + } + + final String paClassId = n.valueOf("./oaf:provenanceaction/@classid"); + final String paClassName = n.valueOf("./oaf:provenanceaction/@classname"); + final String paSchemeId = n.valueOf("./oaf:provenanceaction/@schemeid"); + final String paSchemeName = n.valueOf("./oaf:provenanceaction/@schemename"); + + final boolean deletedbyinference = Boolean.parseBoolean(n.valueOf("./oaf:deletedbyinference")); + final String inferenceprovenance = n.valueOf("./oaf:inferenceprovenance"); + final Boolean inferred = Boolean.parseBoolean(n.valueOf("./oaf:inferred")); + final String trust = n.valueOf("./oaf:trust"); + + return dataInfo( + deletedbyinference, + inferenceprovenance, + inferred, + false, + qualifier(paClassId, paClassName, paSchemeId, paSchemeName), + trust); + } + + protected Field prepareField(final Node node, final String xpath, final DataInfo info) { + return field(node.valueOf(xpath), info); + } + + protected List> prepareListFields( + final Node node, final String xpath, final DataInfo info) { + return listFields(info, prepareListString(node, xpath)); + } + + protected List prepareListString(final Node node, final String xpath) { + final List res = new ArrayList<>(); + for (final Object o : node.selectNodes(xpath)) { + final String s = ((Node) o).getText().trim(); + if (StringUtils.isNotBlank(s)) { + res.add(s); + } + } + return res; + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java index 8029f8422..1aab78afe 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/DispatchEntitiesApplication.java @@ -1,12 +1,10 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -18,66 +16,72 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; + public class DispatchEntitiesApplication { - private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesApplication.class); + private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesApplication.class); - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateMongoMdstoresApplication.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json"))); - parser.parseArgument(args); + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateMongoMdstoresApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/dispatch_entities_parameters.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String sourcePath = parser.get("sourcePath"); - final String targetPath = parser.get("graphRawPath"); + final String sourcePath = parser.get("sourcePath"); + final String targetPath = parser.get("graphRawPath"); - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, targetPath); - ModelSupport.oafTypes - .values() - .forEach(clazz -> processEntity(spark, clazz, sourcePath, targetPath)); - }); - } + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, targetPath); + ModelSupport.oafTypes + .values() + .forEach(clazz -> processEntity(spark, clazz, sourcePath, targetPath)); + }); + } - private static void processEntity( - final SparkSession spark, - final Class clazz, - final String sourcePath, - final String targetPath) { - final String type = clazz.getSimpleName().toLowerCase(); + private static void processEntity( + final SparkSession spark, + final Class clazz, + final String sourcePath, + final String targetPath) { + final String type = clazz.getSimpleName().toLowerCase(); - log.info("Processing entities ({}) in file: {}", type, sourcePath); + log.info("Processing entities ({}) in file: {}", type, sourcePath); - spark - .read() - .textFile(sourcePath) - .filter((FilterFunction) value -> isEntityType(value, type)) - .map( - (MapFunction) l -> StringUtils.substringAfter(l, "|"), - Encoders.STRING()) - .write() - .option("compression", "gzip") - .mode(SaveMode.Overwrite) - .text(targetPath + "/" + type); - } + spark + .read() + .textFile(sourcePath) + .filter((FilterFunction) value -> isEntityType(value, type)) + .map( + (MapFunction) l -> StringUtils.substringAfter(l, "|"), + Encoders.STRING()) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .text(targetPath + "/" + type); + } - private static boolean isEntityType(final String line, final String type) { - return StringUtils.substringBefore(line, "|").equalsIgnoreCase(type); - } + private static boolean isEntityType(final String line, final String type) { + return StringUtils.substringBefore(line, "|").equalsIgnoreCase(type); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index 21288ad98..ccc9f8a89 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -1,17 +1,13 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.IOException; import java.sql.SQLException; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileSystem; @@ -24,172 +20,182 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; public class GenerateEntitiesApplication { - private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class); + private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateMongoMdstoresApplication.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json"))); + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateMongoMdstoresApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/generate_entities_parameters.json"))); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String sourcePaths = parser.get("sourcePaths"); - final String targetPath = parser.get("targetPath"); + final String sourcePaths = parser.get("sourcePaths"); + final String targetPath = parser.get("targetPath"); - final String dbUrl = parser.get("postgresUrl"); - final String dbUser = parser.get("postgresUser"); - final String dbPassword = parser.get("postgresPassword"); + final String dbUrl = parser.get("postgresUrl"); + final String dbUser = parser.get("postgresUser"); + final String dbPassword = parser.get("postgresPassword"); - final Map code2name = loadClassNames(dbUrl, dbUser, dbPassword); + final Map code2name = loadClassNames(dbUrl, dbUser, dbPassword); - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, targetPath); - generateEntities(spark, code2name, sourcePaths, targetPath); - }); - } + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, targetPath); + generateEntities(spark, code2name, sourcePaths, targetPath); + }); + } - private static void generateEntities( - final SparkSession spark, - final Map code2name, - final String sourcePaths, - final String targetPath) { + private static void generateEntities( + final SparkSession spark, + final Map code2name, + final String sourcePaths, + final String targetPath) { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - final List existingSourcePaths = - Arrays.stream(sourcePaths.split(",")) - .filter(p -> exists(sc, p)) - .collect(Collectors.toList()); + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + final List existingSourcePaths = Arrays + .stream(sourcePaths.split(",")) + .filter(p -> exists(sc, p)) + .collect(Collectors.toList()); - log.info("Generate entities from files:"); - existingSourcePaths.forEach(log::info); + log.info("Generate entities from files:"); + existingSourcePaths.forEach(log::info); - JavaRDD inputRdd = sc.emptyRDD(); + JavaRDD inputRdd = sc.emptyRDD(); - for (final String sp : existingSourcePaths) { - inputRdd = - inputRdd.union( - sc.sequenceFile(sp, Text.class, Text.class) - .map(k -> new Tuple2<>(k._1().toString(), k._2().toString())) - .map(k -> convertToListOaf(k._1(), k._2(), code2name)) - .flatMap(list -> list.iterator())); - } + for (final String sp : existingSourcePaths) { + inputRdd = inputRdd + .union( + sc + .sequenceFile(sp, Text.class, Text.class) + .map(k -> new Tuple2<>(k._1().toString(), k._2().toString())) + .map(k -> convertToListOaf(k._1(), k._2(), code2name)) + .flatMap(list -> list.iterator())); + } - inputRdd - .mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf)) - .reduceByKey((o1, o2) -> merge(o1, o2)) - .map(Tuple2::_2) - .map( - oaf -> - oaf.getClass().getSimpleName().toLowerCase() - + "|" - + OBJECT_MAPPER.writeValueAsString(oaf)) - .saveAsTextFile(targetPath, GzipCodec.class); - } + inputRdd + .mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf)) + .reduceByKey((o1, o2) -> merge(o1, o2)) + .map(Tuple2::_2) + .map( + oaf -> oaf.getClass().getSimpleName().toLowerCase() + + "|" + + OBJECT_MAPPER.writeValueAsString(oaf)) + .saveAsTextFile(targetPath, GzipCodec.class); + } - private static Oaf merge(Oaf o1, Oaf o2) { - if (ModelSupport.isSubClass(o1, OafEntity.class)) { - ((OafEntity) o1).mergeFrom((OafEntity) o2); - } else if (ModelSupport.isSubClass(o1, Relation.class)) { - ((Relation) o1).mergeFrom((Relation) o2); - } else { - throw new RuntimeException("invalid Oaf type:" + o1.getClass().getCanonicalName()); - } - return o1; - } + private static Oaf merge(Oaf o1, Oaf o2) { + if (ModelSupport.isSubClass(o1, OafEntity.class)) { + ((OafEntity) o1).mergeFrom((OafEntity) o2); + } else if (ModelSupport.isSubClass(o1, Relation.class)) { + ((Relation) o1).mergeFrom((Relation) o2); + } else { + throw new RuntimeException("invalid Oaf type:" + o1.getClass().getCanonicalName()); + } + return o1; + } - private static List convertToListOaf( - final String id, final String s, final Map code2name) { - final String type = StringUtils.substringAfter(id, ":"); + private static List convertToListOaf( + final String id, final String s, final Map code2name) { + final String type = StringUtils.substringAfter(id, ":"); - switch (type.toLowerCase()) { - case "native_oaf": - return new OafToOafMapper(code2name).processMdRecord(s); - case "native_odf": - return new OdfToOafMapper(code2name).processMdRecord(s); - case "datasource": - return Arrays.asList(convertFromJson(s, Datasource.class)); - case "organization": - return Arrays.asList(convertFromJson(s, Organization.class)); - case "project": - return Arrays.asList(convertFromJson(s, Project.class)); - case "relation": - return Arrays.asList(convertFromJson(s, Relation.class)); - case "publication": - return Arrays.asList(convertFromJson(s, Publication.class)); - case "dataset": - return Arrays.asList(convertFromJson(s, Dataset.class)); - case "software": - return Arrays.asList(convertFromJson(s, Software.class)); - case "otherresearchproduct": - return Arrays.asList(convertFromJson(s, OtherResearchProduct.class)); - default: - throw new RuntimeException("type not managed: " + type.toLowerCase()); - } - } + switch (type.toLowerCase()) { + case "native_oaf": + return new OafToOafMapper(code2name).processMdRecord(s); + case "native_odf": + return new OdfToOafMapper(code2name).processMdRecord(s); + case "datasource": + return Arrays.asList(convertFromJson(s, Datasource.class)); + case "organization": + return Arrays.asList(convertFromJson(s, Organization.class)); + case "project": + return Arrays.asList(convertFromJson(s, Project.class)); + case "relation": + return Arrays.asList(convertFromJson(s, Relation.class)); + case "publication": + return Arrays.asList(convertFromJson(s, Publication.class)); + case "dataset": + return Arrays.asList(convertFromJson(s, Dataset.class)); + case "software": + return Arrays.asList(convertFromJson(s, Software.class)); + case "otherresearchproduct": + return Arrays.asList(convertFromJson(s, OtherResearchProduct.class)); + default: + throw new RuntimeException("type not managed: " + type.toLowerCase()); + } + } - private static Map loadClassNames( - final String dbUrl, final String dbUser, final String dbPassword) throws IOException { + private static Map loadClassNames( + final String dbUrl, final String dbUser, final String dbPassword) throws IOException { - log.info("Loading vocabulary terms from db..."); + log.info("Loading vocabulary terms from db..."); - final Map map = new HashMap<>(); + final Map map = new HashMap<>(); - try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) { - dbClient.processResults( - "select code, name from class", - rs -> { - try { - map.put(rs.getString("code"), rs.getString("name")); - } catch (final SQLException e) { - e.printStackTrace(); - } - }); - } + try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) { + dbClient + .processResults( + "select code, name from class", + rs -> { + try { + map.put(rs.getString("code"), rs.getString("name")); + } catch (final SQLException e) { + e.printStackTrace(); + } + }); + } - log.info("Found " + map.size() + " terms."); + log.info("Found " + map.size() + " terms."); - return map; - } + return map; + } - private static Oaf convertFromJson(final String s, final Class clazz) { - try { - return OBJECT_MAPPER.readValue(s, clazz); - } catch (final Exception e) { - log.error("Error parsing object of class: " + clazz); - log.error(s); - throw new RuntimeException(e); - } - } + private static Oaf convertFromJson(final String s, final Class clazz) { + try { + return OBJECT_MAPPER.readValue(s, clazz); + } catch (final Exception e) { + log.error("Error parsing object of class: " + clazz); + log.error(s); + throw new RuntimeException(e); + } + } - private static boolean exists(final JavaSparkContext context, final String pathToFile) { - try { - final FileSystem hdfs = FileSystem.get(context.hadoopConfiguration()); - final Path path = new Path(pathToFile); - return hdfs.exists(path); - } catch (final IOException e) { - throw new RuntimeException(e); - } - } + private static boolean exists(final JavaSparkContext context, final String pathToFile) { + try { + final FileSystem hdfs = FileSystem.get(context.hadoopConfiguration()); + final Path path = new Path(pathToFile); + return hdfs.exists(path); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java index 7667735cb..9b99097ce 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MergeClaimsApplication.java @@ -1,14 +1,11 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Objects; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; @@ -20,113 +17,118 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; public class MergeClaimsApplication { - private static final Logger log = LoggerFactory.getLogger(MergeClaimsApplication.class); + private static final Logger log = LoggerFactory.getLogger(MergeClaimsApplication.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateMongoMdstoresApplication.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json"))); - parser.parseArgument(args); + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateMongoMdstoresApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String rawGraphPath = parser.get("rawGraphPath"); - log.info("rawGraphPath: {}", rawGraphPath); + final String rawGraphPath = parser.get("rawGraphPath"); + log.info("rawGraphPath: {}", rawGraphPath); - final String claimsGraphPath = parser.get("claimsGraphPath"); - log.info("claimsGraphPath: {}", claimsGraphPath); + final String claimsGraphPath = parser.get("claimsGraphPath"); + log.info("claimsGraphPath: {}", claimsGraphPath); - final String outputRawGaphPath = parser.get("outputRawGaphPath"); - log.info("outputRawGaphPath: {}", outputRawGaphPath); + final String outputRawGaphPath = parser.get("outputRawGaphPath"); + log.info("outputRawGaphPath: {}", outputRawGaphPath); - String graphTableClassName = parser.get("graphTableClassName"); - log.info("graphTableClassName: {}", graphTableClassName); + String graphTableClassName = parser.get("graphTableClassName"); + log.info("graphTableClassName: {}", graphTableClassName); - Class clazz = (Class) Class.forName(graphTableClassName); + Class clazz = (Class) Class.forName(graphTableClassName); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - String type = clazz.getSimpleName().toLowerCase(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + String type = clazz.getSimpleName().toLowerCase(); - String rawPath = rawGraphPath + "/" + type; - String claimPath = claimsGraphPath + "/" + type; - String outPath = outputRawGaphPath + "/" + type; + String rawPath = rawGraphPath + "/" + type; + String claimPath = claimsGraphPath + "/" + type; + String outPath = outputRawGaphPath + "/" + type; - removeOutputDir(spark, outPath); - mergeByType(spark, rawPath, claimPath, outPath, clazz); - }); - } + removeOutputDir(spark, outPath); + mergeByType(spark, rawPath, claimPath, outPath, clazz); + }); + } - private static void mergeByType( - SparkSession spark, String rawPath, String claimPath, String outPath, Class clazz) { - Dataset> raw = - readFromPath(spark, rawPath, clazz) - .map( - (MapFunction>) - value -> new Tuple2<>(ModelSupport.idFn().apply(value), value), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + private static void mergeByType( + SparkSession spark, String rawPath, String claimPath, String outPath, Class clazz) { + Dataset> raw = readFromPath(spark, rawPath, clazz) + .map( + (MapFunction>) value -> new Tuple2<>(ModelSupport.idFn().apply(value), value), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); - final JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - Dataset> claim = - jsc.broadcast(readFromPath(spark, claimPath, clazz)) - .getValue() - .map( - (MapFunction>) - value -> new Tuple2<>(ModelSupport.idFn().apply(value), value), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + final JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + Dataset> claim = jsc + .broadcast(readFromPath(spark, claimPath, clazz)) + .getValue() + .map( + (MapFunction>) value -> new Tuple2<>(ModelSupport.idFn().apply(value), value), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); - raw.joinWith(claim, raw.col("_1").equalTo(claim.col("_1")), "full_outer") - .map( - (MapFunction, Tuple2>, T>) - value -> { - Optional> opRaw = Optional.ofNullable(value._1()); - Optional> opClaim = Optional.ofNullable(value._2()); + raw + .joinWith(claim, raw.col("_1").equalTo(claim.col("_1")), "full_outer") + .map( + (MapFunction, Tuple2>, T>) value -> { + Optional> opRaw = Optional.ofNullable(value._1()); + Optional> opClaim = Optional.ofNullable(value._2()); - return opRaw.isPresent() - ? opRaw.get()._2() - : opClaim.isPresent() ? opClaim.get()._2() : null; - }, - Encoders.bean(clazz)) - .filter(Objects::nonNull) - .map( - (MapFunction) value -> OBJECT_MAPPER.writeValueAsString(value), - Encoders.STRING()) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outPath); - } + return opRaw.isPresent() + ? opRaw.get()._2() + : opClaim.isPresent() ? opClaim.get()._2() : null; + }, + Encoders.bean(clazz)) + .filter(Objects::nonNull) + .map( + (MapFunction) value -> OBJECT_MAPPER.writeValueAsString(value), + Encoders.STRING()) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outPath); + } - private static Dataset readFromPath( - SparkSession spark, String path, Class clazz) { - return spark - .read() - .textFile(path) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), - Encoders.bean(clazz)) - .filter((FilterFunction) value -> Objects.nonNull(ModelSupport.idFn().apply(value))); - } + private static Dataset readFromPath( + SparkSession spark, String path, Class clazz) { + return spark + .read() + .textFile(path) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), + Encoders.bean(clazz)) + .filter((FilterFunction) value -> Objects.nonNull(ModelSupport.idFn().apply(value))); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index a5114abc3..aa63f9ebc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.asString; @@ -10,6 +11,23 @@ import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.listKeyValues; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.qualifier; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.structuredProperty; +import java.io.Closeable; +import java.io.IOException; +import java.sql.Array; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.Function; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; @@ -31,24 +49,8 @@ import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.Software; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.io.Closeable; -import java.io.IOException; -import java.sql.Array; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Date; -import java.util.List; -import java.util.function.Consumer; -import java.util.function.Function; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; public class MigrateDbEntitiesApplication extends AbstractMigrationApplication - implements Closeable { private static final Log log = LogFactory.getLog(MigrateDbEntitiesApplication.class); @@ -541,4 +543,5 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication dbClient.close(); } + } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java index 95f4477e8..00c1dc4bb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateMongoMdstoresApplication.java @@ -1,70 +1,73 @@ + package eu.dnetlib.dhp.oa.graph.raw; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; -import eu.dnetlib.dhp.oa.graph.raw.common.MdstoreClient; import java.io.Closeable; import java.io.IOException; import java.util.Map; import java.util.Map.Entry; + import org.apache.commons.io.IOUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; +import eu.dnetlib.dhp.oa.graph.raw.common.MdstoreClient; + public class MigrateMongoMdstoresApplication extends AbstractMigrationApplication - implements Closeable { + implements Closeable { - private static final Log log = LogFactory.getLog(MigrateMongoMdstoresApplication.class); + private static final Log log = LogFactory.getLog(MigrateMongoMdstoresApplication.class); - private final MdstoreClient mdstoreClient; + private final MdstoreClient mdstoreClient; - public static void main(final String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - MigrateMongoMdstoresApplication.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/migrate_mongo_mstores_parameters.json"))); - parser.parseArgument(args); + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + MigrateMongoMdstoresApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/migrate_mongo_mstores_parameters.json"))); + parser.parseArgument(args); - final String mongoBaseUrl = parser.get("mongoBaseUrl"); - final String mongoDb = parser.get("mongoDb"); + final String mongoBaseUrl = parser.get("mongoBaseUrl"); + final String mongoDb = parser.get("mongoDb"); - final String mdFormat = parser.get("mdFormat"); - final String mdLayout = parser.get("mdLayout"); - final String mdInterpretation = parser.get("mdInterpretation"); + final String mdFormat = parser.get("mdFormat"); + final String mdLayout = parser.get("mdLayout"); + final String mdInterpretation = parser.get("mdInterpretation"); - final String hdfsPath = parser.get("hdfsPath"); + final String hdfsPath = parser.get("hdfsPath"); - try (MigrateMongoMdstoresApplication app = - new MigrateMongoMdstoresApplication(hdfsPath, mongoBaseUrl, mongoDb)) { - app.execute(mdFormat, mdLayout, mdInterpretation); - } - } + try (MigrateMongoMdstoresApplication app = new MigrateMongoMdstoresApplication(hdfsPath, mongoBaseUrl, + mongoDb)) { + app.execute(mdFormat, mdLayout, mdInterpretation); + } + } - public MigrateMongoMdstoresApplication( - final String hdfsPath, final String mongoBaseUrl, final String mongoDb) throws Exception { - super(hdfsPath); - this.mdstoreClient = new MdstoreClient(mongoBaseUrl, mongoDb); - } + public MigrateMongoMdstoresApplication( + final String hdfsPath, final String mongoBaseUrl, final String mongoDb) throws Exception { + super(hdfsPath); + this.mdstoreClient = new MdstoreClient(mongoBaseUrl, mongoDb); + } - public void execute(final String format, final String layout, final String interpretation) { - final Map colls = - mdstoreClient.validCollections(format, layout, interpretation); - log.info("Found " + colls.size() + " mdstores"); + public void execute(final String format, final String layout, final String interpretation) { + final Map colls = mdstoreClient.validCollections(format, layout, interpretation); + log.info("Found " + colls.size() + " mdstores"); - for (final Entry entry : colls.entrySet()) { - log.info("Processing mdstore " + entry.getKey() + " (collection: " + entry.getValue() + ")"); - final String currentColl = entry.getValue(); + for (final Entry entry : colls.entrySet()) { + log.info("Processing mdstore " + entry.getKey() + " (collection: " + entry.getValue() + ")"); + final String currentColl = entry.getValue(); - for (final String xml : mdstoreClient.listRecords(currentColl)) { - emit(xml, "native_" + format); - } - } - } + for (final String xml : mdstoreClient.listRecords(currentColl)) { + emit(xml, "native_" + format); + } + } + } - @Override - public void close() throws IOException { - super.close(); - mdstoreClient.close(); - } + @Override + public void close() throws IOException { + super.close(); + mdstoreClient.close(); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 2a40e1802..286656149 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -1,8 +1,18 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.createOpenaireId; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.Node; + import eu.dnetlib.dhp.oa.graph.raw.common.PacePerson; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.DataInfo; @@ -14,254 +24,251 @@ import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.Node; public class OafToOafMapper extends AbstractMdRecordToOafMapper { - public OafToOafMapper(final Map code2name) { - super(code2name); - } + public OafToOafMapper(final Map code2name) { + super(code2name); + } - @Override - protected List prepareAuthors(final Document doc, final DataInfo info) { - final List res = new ArrayList<>(); - int pos = 1; - for (final Object o : doc.selectNodes("//dc:creator")) { - final Node n = (Node) o; - final Author author = new Author(); - author.setFullname(n.getText()); - author.setRank(pos++); - final PacePerson p = new PacePerson(n.getText(), false); - if (p.isAccurate()) { - author.setName(p.getNormalisedFirstName()); - author.setSurname(p.getNormalisedSurname()); - } - res.add(author); - } - return res; - } + @Override + protected List prepareAuthors(final Document doc, final DataInfo info) { + final List res = new ArrayList<>(); + int pos = 1; + for (final Object o : doc.selectNodes("//dc:creator")) { + final Node n = (Node) o; + final Author author = new Author(); + author.setFullname(n.getText()); + author.setRank(pos++); + final PacePerson p = new PacePerson(n.getText(), false); + if (p.isAccurate()) { + author.setName(p.getNormalisedFirstName()); + author.setSurname(p.getNormalisedSurname()); + } + res.add(author); + } + return res; + } - @Override - protected Qualifier prepareLanguages(final Document doc) { - return prepareQualifier(doc, "//dc:language", "dnet:languages", "dnet:languages"); - } + @Override + protected Qualifier prepareLanguages(final Document doc) { + return prepareQualifier(doc, "//dc:language", "dnet:languages", "dnet:languages"); + } - @Override - protected List prepareSubjects(final Document doc, final DataInfo info) { - return prepareListStructProps(doc, "//dc:subject", info); - } + @Override + protected List prepareSubjects(final Document doc, final DataInfo info) { + return prepareListStructProps(doc, "//dc:subject", info); + } - @Override - protected List prepareTitles(final Document doc, final DataInfo info) { - return prepareListStructProps(doc, "//dc:title", MAIN_TITLE_QUALIFIER, info); - } + @Override + protected List prepareTitles(final Document doc, final DataInfo info) { + return prepareListStructProps(doc, "//dc:title", MAIN_TITLE_QUALIFIER, info); + } - @Override - protected List> prepareDescriptions(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//dc:description", info); - } + @Override + protected List> prepareDescriptions(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//dc:description", info); + } - @Override - protected Field preparePublisher(final Document doc, final DataInfo info) { - return prepareField(doc, "//dc:publisher", info); - } + @Override + protected Field preparePublisher(final Document doc, final DataInfo info) { + return prepareField(doc, "//dc:publisher", info); + } - @Override - protected List> prepareFormats(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//dc:format", info); - } + @Override + protected List> prepareFormats(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//dc:format", info); + } - @Override - protected List> prepareContributors(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//dc:contributor", info); - } + @Override + protected List> prepareContributors(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//dc:contributor", info); + } - @Override - protected List> prepareCoverages(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//dc:coverage", info); - } + @Override + protected List> prepareCoverages(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//dc:coverage", info); + } - @Override - protected List prepareInstances( - final Document doc, - final DataInfo info, - final KeyValue collectedfrom, - final KeyValue hostedby) { - final List res = new ArrayList<>(); - for (final Object o : doc.selectNodes("//dc:identifier")) { - final String url = ((Node) o).getText().trim(); - if (url.startsWith("http")) { - final Instance instance = new Instance(); - instance.setUrl(Arrays.asList(url)); - instance.setInstancetype( - prepareQualifier( - doc, - "//dr:CobjCategory", - "dnet:publication_resource", - "dnet:publication_resource")); - instance.setCollectedfrom(collectedfrom); - instance.setHostedby(hostedby); - instance.setDateofacceptance(field(doc.valueOf("//oaf:dateAccepted"), info)); - instance.setDistributionlocation(doc.valueOf("//oaf:distributionlocation")); - instance.setAccessright( - prepareQualifier(doc, "//oaf:accessrights", "dnet:access_modes", "dnet:access_modes")); - instance.setLicense(field(doc.valueOf("//oaf:license"), info)); - instance.setRefereed(field(doc.valueOf("//oaf:refereed"), info)); - instance.setProcessingchargeamount( - field(doc.valueOf("//oaf:processingchargeamount"), info)); - instance.setProcessingchargecurrency( - field(doc.valueOf("//oaf:processingchargeamount/@currency"), info)); - res.add(instance); - } - } - return res; - } + @Override + protected List prepareInstances( + final Document doc, + final DataInfo info, + final KeyValue collectedfrom, + final KeyValue hostedby) { + final List res = new ArrayList<>(); + for (final Object o : doc.selectNodes("//dc:identifier")) { + final String url = ((Node) o).getText().trim(); + if (url.startsWith("http")) { + final Instance instance = new Instance(); + instance.setUrl(Arrays.asList(url)); + instance + .setInstancetype( + prepareQualifier( + doc, + "//dr:CobjCategory", + "dnet:publication_resource", + "dnet:publication_resource")); + instance.setCollectedfrom(collectedfrom); + instance.setHostedby(hostedby); + instance.setDateofacceptance(field(doc.valueOf("//oaf:dateAccepted"), info)); + instance.setDistributionlocation(doc.valueOf("//oaf:distributionlocation")); + instance + .setAccessright( + prepareQualifier(doc, "//oaf:accessrights", "dnet:access_modes", "dnet:access_modes")); + instance.setLicense(field(doc.valueOf("//oaf:license"), info)); + instance.setRefereed(field(doc.valueOf("//oaf:refereed"), info)); + instance + .setProcessingchargeamount( + field(doc.valueOf("//oaf:processingchargeamount"), info)); + instance + .setProcessingchargecurrency( + field(doc.valueOf("//oaf:processingchargeamount/@currency"), info)); + res.add(instance); + } + } + return res; + } - @Override - protected List> prepareSources(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//dc:source", info); - } + @Override + protected List> prepareSources(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//dc:source", info); + } - @Override - protected List prepareRelevantDates(final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List prepareRelevantDates(final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - // SOFTWARES + // SOFTWARES - @Override - protected Qualifier prepareSoftwareProgrammingLanguage(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Qualifier prepareSoftwareProgrammingLanguage(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareSoftwareCodeRepositoryUrl( - final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareSoftwareCodeRepositoryUrl( + final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected List prepareSoftwareLicenses( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List prepareSoftwareLicenses( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - @Override - protected List> prepareSoftwareDocumentationUrls( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List> prepareSoftwareDocumentationUrls( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - // DATASETS - @Override - protected List prepareDatasetGeoLocations(final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + // DATASETS + @Override + protected List prepareDatasetGeoLocations(final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetMetadataVersionNumber( - final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetMetadataVersionNumber( + final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetLastMetadataUpdate( - final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetLastMetadataUpdate( + final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetVersion(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetVersion(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetSize(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetSize(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetDevice(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetDevice(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - @Override - protected Field prepareDatasetStorageDate(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Field prepareDatasetStorageDate(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } - // OTHER PRODUCTS + // OTHER PRODUCTS - @Override - protected List> prepareOtherResearchProductTools( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List> prepareOtherResearchProductTools( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - @Override - protected List> prepareOtherResearchProductContactGroups( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List> prepareOtherResearchProductContactGroups( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - @Override - protected List> prepareOtherResearchProductContactPersons( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // NOT PRESENT IN OAF - } + @Override + protected List> prepareOtherResearchProductContactPersons( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // NOT PRESENT IN OAF + } - @Override - protected List addOtherResultRels( - final Document doc, - final KeyValue collectedFrom, - final DataInfo info, - final long lastUpdateTimestamp) { - final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); + @Override + protected List addOtherResultRels( + final Document doc, + final KeyValue collectedFrom, + final DataInfo info, + final long lastUpdateTimestamp) { + final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); - final List res = new ArrayList<>(); + final List res = new ArrayList<>(); - for (final Object o : doc.selectNodes("//*[local-name()='relatedDataset']")) { + for (final Object o : doc.selectNodes("//*[local-name()='relatedDataset']")) { - final String originalId = ((Node) o).getText(); + final String originalId = ((Node) o).getText(); - if (StringUtils.isNotBlank(originalId)) { + if (StringUtils.isNotBlank(originalId)) { - final String otherId = createOpenaireId(50, originalId, false); + final String otherId = createOpenaireId(50, originalId, false); - final Relation r1 = new Relation(); - r1.setRelType("resultResult"); - r1.setSubRelType("publicationDataset"); - r1.setRelClass("isRelatedTo"); - r1.setSource(docId); - r1.setTarget(otherId); - r1.setCollectedfrom(Arrays.asList(collectedFrom)); - r1.setDataInfo(info); - r1.setLastupdatetimestamp(lastUpdateTimestamp); - res.add(r1); + final Relation r1 = new Relation(); + r1.setRelType("resultResult"); + r1.setSubRelType("publicationDataset"); + r1.setRelClass("isRelatedTo"); + r1.setSource(docId); + r1.setTarget(otherId); + r1.setCollectedfrom(Arrays.asList(collectedFrom)); + r1.setDataInfo(info); + r1.setLastupdatetimestamp(lastUpdateTimestamp); + res.add(r1); - final Relation r2 = new Relation(); - r2.setRelType("resultResult"); - r2.setSubRelType("publicationDataset"); - r2.setRelClass("isRelatedTo"); - r2.setSource(otherId); - r2.setTarget(docId); - r2.setCollectedfrom(Arrays.asList(collectedFrom)); - r2.setDataInfo(info); - r2.setLastupdatetimestamp(lastUpdateTimestamp); - res.add(r2); - } - } - return res; - } + final Relation r2 = new Relation(); + r2.setRelType("resultResult"); + r2.setSubRelType("publicationDataset"); + r2.setRelClass("isRelatedTo"); + r2.setSource(otherId); + r2.setTarget(docId); + r2.setCollectedfrom(Arrays.asList(collectedFrom)); + r2.setDataInfo(info); + r2.setLastupdatetimestamp(lastUpdateTimestamp); + res.add(r2); + } + } + return res; + } - @Override - protected Qualifier prepareResourceType(final Document doc, final DataInfo info) { - return null; // NOT PRESENT IN OAF - } + @Override + protected Qualifier prepareResourceType(final Document doc, final DataInfo info) { + return null; // NOT PRESENT IN OAF + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index 64755a6eb..93b0eb29c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -1,9 +1,19 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.createOpenaireId; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.field; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.structuredProperty; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.Node; + import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Field; @@ -14,338 +24,337 @@ import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.Node; public class OdfToOafMapper extends AbstractMdRecordToOafMapper { - public OdfToOafMapper(final Map code2name) { - super(code2name); - } + public OdfToOafMapper(final Map code2name) { + super(code2name); + } - @Override - protected List prepareTitles(final Document doc, final DataInfo info) { - return prepareListStructProps(doc, "//datacite:title", MAIN_TITLE_QUALIFIER, info); - } + @Override + protected List prepareTitles(final Document doc, final DataInfo info) { + return prepareListStructProps(doc, "//datacite:title", MAIN_TITLE_QUALIFIER, info); + } - @Override - protected List prepareAuthors(final Document doc, final DataInfo info) { - final List res = new ArrayList<>(); - int pos = 1; - for (final Object o : doc.selectNodes("//datacite:creator")) { - final Node n = (Node) o; - final Author author = new Author(); - author.setFullname(n.valueOf("./datacite:creatorName")); - author.setName(n.valueOf("./datacite:givenName")); - author.setSurname(n.valueOf("./datacite:familyName")); - author.setAffiliation(prepareListFields(doc, "./datacite:affiliation", info)); - author.setPid(preparePids(doc, info)); - author.setRank(pos++); - res.add(author); - } - return res; - } + @Override + protected List prepareAuthors(final Document doc, final DataInfo info) { + final List res = new ArrayList<>(); + int pos = 1; + for (final Object o : doc.selectNodes("//datacite:creator")) { + final Node n = (Node) o; + final Author author = new Author(); + author.setFullname(n.valueOf("./datacite:creatorName")); + author.setName(n.valueOf("./datacite:givenName")); + author.setSurname(n.valueOf("./datacite:familyName")); + author.setAffiliation(prepareListFields(doc, "./datacite:affiliation", info)); + author.setPid(preparePids(doc, info)); + author.setRank(pos++); + res.add(author); + } + return res; + } - private List preparePids(final Document doc, final DataInfo info) { - final List res = new ArrayList<>(); - for (final Object o : doc.selectNodes("./datacite:nameIdentifier")) { - res.add( - structuredProperty( - ((Node) o).getText(), - prepareQualifier( - (Node) o, "./@nameIdentifierScheme", "dnet:pid_types", "dnet:pid_types"), - info)); - } - return res; - } + private List preparePids(final Document doc, final DataInfo info) { + final List res = new ArrayList<>(); + for (final Object o : doc.selectNodes("./datacite:nameIdentifier")) { + res + .add( + structuredProperty( + ((Node) o).getText(), + prepareQualifier( + (Node) o, "./@nameIdentifierScheme", "dnet:pid_types", "dnet:pid_types"), + info)); + } + return res; + } - @Override - protected List prepareInstances( - final Document doc, - final DataInfo info, - final KeyValue collectedfrom, - final KeyValue hostedby) { + @Override + protected List prepareInstances( + final Document doc, + final DataInfo info, + final KeyValue collectedfrom, + final KeyValue hostedby) { - final Instance instance = new Instance(); - instance.setUrl(new ArrayList<>()); - instance.setInstancetype( - prepareQualifier( - doc, "//dr:CobjCategory", "dnet:publication_resource", "dnet:publication_resource")); - instance.setCollectedfrom(collectedfrom); - instance.setHostedby(hostedby); - instance.setDateofacceptance(field(doc.valueOf("//oaf:dateAccepted"), info)); - instance.setDistributionlocation(doc.valueOf("//oaf:distributionlocation")); - instance.setAccessright( - prepareQualifier(doc, "//oaf:accessrights", "dnet:access_modes", "dnet:access_modes")); - instance.setLicense(field(doc.valueOf("//oaf:license"), info)); - instance.setRefereed(field(doc.valueOf("//oaf:refereed"), info)); - instance.setProcessingchargeamount(field(doc.valueOf("//oaf:processingchargeamount"), info)); - instance.setProcessingchargecurrency( - field(doc.valueOf("//oaf:processingchargeamount/@currency"), info)); + final Instance instance = new Instance(); + instance.setUrl(new ArrayList<>()); + instance + .setInstancetype( + prepareQualifier( + doc, "//dr:CobjCategory", "dnet:publication_resource", "dnet:publication_resource")); + instance.setCollectedfrom(collectedfrom); + instance.setHostedby(hostedby); + instance.setDateofacceptance(field(doc.valueOf("//oaf:dateAccepted"), info)); + instance.setDistributionlocation(doc.valueOf("//oaf:distributionlocation")); + instance + .setAccessright( + prepareQualifier(doc, "//oaf:accessrights", "dnet:access_modes", "dnet:access_modes")); + instance.setLicense(field(doc.valueOf("//oaf:license"), info)); + instance.setRefereed(field(doc.valueOf("//oaf:refereed"), info)); + instance.setProcessingchargeamount(field(doc.valueOf("//oaf:processingchargeamount"), info)); + instance + .setProcessingchargecurrency( + field(doc.valueOf("//oaf:processingchargeamount/@currency"), info)); - for (final Object o : - doc.selectNodes("//datacite:alternateIdentifier[@alternateIdentifierType='URL']")) { - instance.getUrl().add(((Node) o).getText().trim()); - } - for (final Object o : doc.selectNodes("//datacite:identifier[@identifierType='URL']")) { - instance.getUrl().add(((Node) o).getText().trim()); - } - for (final Object o : - doc.selectNodes("//datacite:alternateIdentifier[@alternateIdentifierType='DOI']")) { - instance.getUrl().add("http://dx.doi.org/" + ((Node) o).getText().trim()); - } - for (final Object o : doc.selectNodes("//datacite:identifier[@identifierType='DOI']")) { - instance.getUrl().add("http://dx.doi.org/" + ((Node) o).getText().trim()); - } - return Arrays.asList(instance); - } + for (final Object o : doc.selectNodes("//datacite:alternateIdentifier[@alternateIdentifierType='URL']")) { + instance.getUrl().add(((Node) o).getText().trim()); + } + for (final Object o : doc.selectNodes("//datacite:identifier[@identifierType='URL']")) { + instance.getUrl().add(((Node) o).getText().trim()); + } + for (final Object o : doc.selectNodes("//datacite:alternateIdentifier[@alternateIdentifierType='DOI']")) { + instance.getUrl().add("http://dx.doi.org/" + ((Node) o).getText().trim()); + } + for (final Object o : doc.selectNodes("//datacite:identifier[@identifierType='DOI']")) { + instance.getUrl().add("http://dx.doi.org/" + ((Node) o).getText().trim()); + } + return Arrays.asList(instance); + } - @Override - protected List> prepareSources(final Document doc, final DataInfo info) { - return new ArrayList<>(); // Not present in ODF ??? - } + @Override + protected List> prepareSources(final Document doc, final DataInfo info) { + return new ArrayList<>(); // Not present in ODF ??? + } - @Override - protected List prepareRelevantDates(final Document doc, final DataInfo info) { - final List res = new ArrayList<>(); - for (final Object o : doc.selectNodes("//datacite:date")) { - final String dateType = ((Node) o).valueOf("@dateType"); - if (StringUtils.isBlank(dateType) - && !dateType.equalsIgnoreCase("Accepted") - && !dateType.equalsIgnoreCase("Issued") - && !dateType.equalsIgnoreCase("Updated") - && !dateType.equalsIgnoreCase("Available")) { - res.add( - structuredProperty( - ((Node) o).getText(), - "UNKNOWN", - "UNKNOWN", - "dnet:dataCite_date", - "dnet:dataCite_date", - info)); - } - } - return res; - } + @Override + protected List prepareRelevantDates(final Document doc, final DataInfo info) { + final List res = new ArrayList<>(); + for (final Object o : doc.selectNodes("//datacite:date")) { + final String dateType = ((Node) o).valueOf("@dateType"); + if (StringUtils.isBlank(dateType) + && !dateType.equalsIgnoreCase("Accepted") + && !dateType.equalsIgnoreCase("Issued") + && !dateType.equalsIgnoreCase("Updated") + && !dateType.equalsIgnoreCase("Available")) { + res + .add( + structuredProperty( + ((Node) o).getText(), + "UNKNOWN", + "UNKNOWN", + "dnet:dataCite_date", + "dnet:dataCite_date", + info)); + } + } + return res; + } - @Override - protected List> prepareCoverages(final Document doc, final DataInfo info) { - return new ArrayList<>(); // Not present in ODF ??? - } + @Override + protected List> prepareCoverages(final Document doc, final DataInfo info) { + return new ArrayList<>(); // Not present in ODF ??? + } - @Override - protected List> prepareContributors(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//datacite:contributorName", info); - } + @Override + protected List> prepareContributors(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//datacite:contributorName", info); + } - @Override - protected List> prepareFormats(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//datacite:format", info); - } + @Override + protected List> prepareFormats(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//datacite:format", info); + } - @Override - protected Field preparePublisher(final Document doc, final DataInfo info) { - return prepareField(doc, "//datacite:publisher", info); - } + @Override + protected Field preparePublisher(final Document doc, final DataInfo info) { + return prepareField(doc, "//datacite:publisher", info); + } - @Override - protected List> prepareDescriptions(final Document doc, final DataInfo info) { - return prepareListFields(doc, "//datacite:description[@descriptionType='Abstract']", info); - } + @Override + protected List> prepareDescriptions(final Document doc, final DataInfo info) { + return prepareListFields(doc, "//datacite:description[@descriptionType='Abstract']", info); + } - @Override - protected List prepareSubjects(final Document doc, final DataInfo info) { - return prepareListStructProps(doc, "//datacite:subject", info); - } + @Override + protected List prepareSubjects(final Document doc, final DataInfo info) { + return prepareListStructProps(doc, "//datacite:subject", info); + } - @Override - protected Qualifier prepareLanguages(final Document doc) { - return prepareQualifier(doc, "//datacite:language", "dnet:languages", "dnet:languages"); - } + @Override + protected Qualifier prepareLanguages(final Document doc) { + return prepareQualifier(doc, "//datacite:language", "dnet:languages", "dnet:languages"); + } - @Override - protected List> prepareOtherResearchProductTools( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // Not present in ODF ??? - } + @Override + protected List> prepareOtherResearchProductTools( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // Not present in ODF ??? + } - @Override - protected List> prepareOtherResearchProductContactGroups( - final Document doc, final DataInfo info) { - return prepareListFields( - doc, - "//datacite:contributor[@contributorType='ContactGroup']/datacite:contributorName", - info); - } + @Override + protected List> prepareOtherResearchProductContactGroups( + final Document doc, final DataInfo info) { + return prepareListFields( + doc, + "//datacite:contributor[@contributorType='ContactGroup']/datacite:contributorName", + info); + } - @Override - protected List> prepareOtherResearchProductContactPersons( - final Document doc, final DataInfo info) { - return prepareListFields( - doc, - "//datacite:contributor[@contributorType='ContactPerson']/datacite:contributorName", - info); - } + @Override + protected List> prepareOtherResearchProductContactPersons( + final Document doc, final DataInfo info) { + return prepareListFields( + doc, + "//datacite:contributor[@contributorType='ContactPerson']/datacite:contributorName", + info); + } - @Override - protected Qualifier prepareSoftwareProgrammingLanguage(final Document doc, final DataInfo info) { - return prepareQualifier( - doc, "//datacite:format", "dnet:programming_languages", "dnet:programming_languages"); - } + @Override + protected Qualifier prepareSoftwareProgrammingLanguage(final Document doc, final DataInfo info) { + return prepareQualifier( + doc, "//datacite:format", "dnet:programming_languages", "dnet:programming_languages"); + } - @Override - protected Field prepareSoftwareCodeRepositoryUrl( - final Document doc, final DataInfo info) { - return null; // Not present in ODF ??? - } + @Override + protected Field prepareSoftwareCodeRepositoryUrl( + final Document doc, final DataInfo info) { + return null; // Not present in ODF ??? + } - @Override - protected List prepareSoftwareLicenses( - final Document doc, final DataInfo info) { - return new ArrayList<>(); // Not present in ODF ??? - } + @Override + protected List prepareSoftwareLicenses( + final Document doc, final DataInfo info) { + return new ArrayList<>(); // Not present in ODF ??? + } - @Override - protected List> prepareSoftwareDocumentationUrls( - final Document doc, final DataInfo info) { - return prepareListFields( - doc, - "//datacite:relatedIdentifier[@relatedIdentifierType='URL' and @relationType='IsDocumentedBy']", - info); - } + @Override + protected List> prepareSoftwareDocumentationUrls( + final Document doc, final DataInfo info) { + return prepareListFields( + doc, + "//datacite:relatedIdentifier[@relatedIdentifierType='URL' and @relationType='IsDocumentedBy']", + info); + } - // DATASETS + // DATASETS - @Override - protected List prepareDatasetGeoLocations(final Document doc, final DataInfo info) { - final List res = new ArrayList<>(); + @Override + protected List prepareDatasetGeoLocations(final Document doc, final DataInfo info) { + final List res = new ArrayList<>(); - for (final Object o : doc.selectNodes("//datacite:geoLocation")) { - final GeoLocation loc = new GeoLocation(); - loc.setBox(((Node) o).valueOf("./datacite:geoLocationBox")); - loc.setPlace(((Node) o).valueOf("./datacite:geoLocationPlace")); - loc.setPoint(((Node) o).valueOf("./datacite:geoLocationPoint")); - res.add(loc); - } - return res; - } + for (final Object o : doc.selectNodes("//datacite:geoLocation")) { + final GeoLocation loc = new GeoLocation(); + loc.setBox(((Node) o).valueOf("./datacite:geoLocationBox")); + loc.setPlace(((Node) o).valueOf("./datacite:geoLocationPlace")); + loc.setPoint(((Node) o).valueOf("./datacite:geoLocationPoint")); + res.add(loc); + } + return res; + } - @Override - protected Field prepareDatasetMetadataVersionNumber( - final Document doc, final DataInfo info) { - return null; // Not present in ODF ??? - } + @Override + protected Field prepareDatasetMetadataVersionNumber( + final Document doc, final DataInfo info) { + return null; // Not present in ODF ??? + } - @Override - protected Field prepareDatasetLastMetadataUpdate( - final Document doc, final DataInfo info) { - return prepareField(doc, "//datacite:date[@dateType='Updated']", info); - } + @Override + protected Field prepareDatasetLastMetadataUpdate( + final Document doc, final DataInfo info) { + return prepareField(doc, "//datacite:date[@dateType='Updated']", info); + } - @Override - protected Field prepareDatasetVersion(final Document doc, final DataInfo info) { - return prepareField(doc, "//datacite:version", info); - } + @Override + protected Field prepareDatasetVersion(final Document doc, final DataInfo info) { + return prepareField(doc, "//datacite:version", info); + } - @Override - protected Field prepareDatasetSize(final Document doc, final DataInfo info) { - return prepareField(doc, "//datacite:size", info); - } + @Override + protected Field prepareDatasetSize(final Document doc, final DataInfo info) { + return prepareField(doc, "//datacite:size", info); + } - @Override - protected Field prepareDatasetDevice(final Document doc, final DataInfo info) { - return null; // Not present in ODF ??? - } + @Override + protected Field prepareDatasetDevice(final Document doc, final DataInfo info) { + return null; // Not present in ODF ??? + } - @Override - protected Field prepareDatasetStorageDate(final Document doc, final DataInfo info) { - return prepareField(doc, "//datacite:date[@dateType='Issued']", info); - } + @Override + protected Field prepareDatasetStorageDate(final Document doc, final DataInfo info) { + return prepareField(doc, "//datacite:date[@dateType='Issued']", info); + } - @Override - protected List addOtherResultRels( - final Document doc, - final KeyValue collectedFrom, - final DataInfo info, - final long lastUpdateTimestamp) { + @Override + protected List addOtherResultRels( + final Document doc, + final KeyValue collectedFrom, + final DataInfo info, + final long lastUpdateTimestamp) { - final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); + final String docId = createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false); - final List res = new ArrayList<>(); + final List res = new ArrayList<>(); - for (final Object o : - doc.selectNodes("//datacite:relatedIdentifier[@relatedIdentifierType='OPENAIRE']")) { + for (final Object o : doc.selectNodes("//datacite:relatedIdentifier[@relatedIdentifierType='OPENAIRE']")) { - final String originalId = ((Node) o).getText(); + final String originalId = ((Node) o).getText(); - if (StringUtils.isNotBlank(originalId)) { - final String otherId = createOpenaireId(50, originalId, false); - final String type = ((Node) o).valueOf("@relationType"); + if (StringUtils.isNotBlank(originalId)) { + final String otherId = createOpenaireId(50, originalId, false); + final String type = ((Node) o).valueOf("@relationType"); - if (type.equals("IsSupplementTo")) { - res.add( - prepareOtherResultRel( - collectedFrom, - info, - lastUpdateTimestamp, - docId, - otherId, - "supplement", - "isSupplementTo")); - res.add( - prepareOtherResultRel( - collectedFrom, - info, - lastUpdateTimestamp, - otherId, - docId, - "supplement", - "isSupplementedBy")); - } else if (type.equals("IsPartOf")) { - res.add( - prepareOtherResultRel( - collectedFrom, info, lastUpdateTimestamp, docId, otherId, "part", "IsPartOf")); - res.add( - prepareOtherResultRel( - collectedFrom, info, lastUpdateTimestamp, otherId, docId, "part", "HasParts")); - } else { - } - } - } - return res; - } + if (type.equals("IsSupplementTo")) { + res + .add( + prepareOtherResultRel( + collectedFrom, + info, + lastUpdateTimestamp, + docId, + otherId, + "supplement", + "isSupplementTo")); + res + .add( + prepareOtherResultRel( + collectedFrom, + info, + lastUpdateTimestamp, + otherId, + docId, + "supplement", + "isSupplementedBy")); + } else if (type.equals("IsPartOf")) { + res + .add( + prepareOtherResultRel( + collectedFrom, info, lastUpdateTimestamp, docId, otherId, "part", "IsPartOf")); + res + .add( + prepareOtherResultRel( + collectedFrom, info, lastUpdateTimestamp, otherId, docId, "part", "HasParts")); + } else { + } + } + } + return res; + } - private Relation prepareOtherResultRel( - final KeyValue collectedFrom, - final DataInfo info, - final long lastUpdateTimestamp, - final String source, - final String target, - final String subRelType, - final String relClass) { - final Relation r = new Relation(); - r.setRelType("resultResult"); - r.setSubRelType(subRelType); - r.setRelClass(relClass); - r.setSource(source); - r.setTarget(target); - r.setCollectedfrom(Arrays.asList(collectedFrom)); - r.setDataInfo(info); - r.setLastupdatetimestamp(lastUpdateTimestamp); - return r; - } + private Relation prepareOtherResultRel( + final KeyValue collectedFrom, + final DataInfo info, + final long lastUpdateTimestamp, + final String source, + final String target, + final String subRelType, + final String relClass) { + final Relation r = new Relation(); + r.setRelType("resultResult"); + r.setSubRelType(subRelType); + r.setRelClass(relClass); + r.setSource(source); + r.setTarget(target); + r.setCollectedfrom(Arrays.asList(collectedFrom)); + r.setDataInfo(info); + r.setLastupdatetimestamp(lastUpdateTimestamp); + return r; + } - @Override - protected Qualifier prepareResourceType(final Document doc, final DataInfo info) { - return prepareQualifier( - doc, - "//*[local-name() = 'resource']//*[local-name() = 'resourceType']", - "dnet:dataCite_resource", - "dnet:dataCite_resource"); - } + @Override + protected Qualifier prepareResourceType(final Document doc, final DataInfo info) { + return prepareQualifier( + doc, + "//*[local-name() = 'resource']//*[local-name() = 'resourceType']", + "dnet:dataCite_resource", + "dnet:dataCite_resource"); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java index c7756be0d..f7579c0a0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/AbstractMigrationApplication.java @@ -1,9 +1,10 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; -import eu.dnetlib.dhp.schema.oaf.Oaf; import java.io.Closeable; import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -12,72 +13,74 @@ import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.codehaus.jackson.map.ObjectMapper; +import eu.dnetlib.dhp.schema.oaf.Oaf; + public class AbstractMigrationApplication implements Closeable { - private final AtomicInteger counter = new AtomicInteger(0); + private final AtomicInteger counter = new AtomicInteger(0); - private final Text key = new Text(); + private final Text key = new Text(); - private final Text value = new Text(); + private final Text value = new Text(); - private final SequenceFile.Writer writer; + private final SequenceFile.Writer writer; - private final ObjectMapper objectMapper = new ObjectMapper(); + private final ObjectMapper objectMapper = new ObjectMapper(); - private static final Log log = LogFactory.getLog(AbstractMigrationApplication.class); + private static final Log log = LogFactory.getLog(AbstractMigrationApplication.class); - protected AbstractMigrationApplication() { // ONLY FOR UNIT TEST - this.writer = null; - } + protected AbstractMigrationApplication() { // ONLY FOR UNIT TEST + this.writer = null; + } - public AbstractMigrationApplication(final String hdfsPath) throws Exception { + public AbstractMigrationApplication(final String hdfsPath) throws Exception { - log.info(String.format("Creating SequenceFile Writer, hdfsPath=%s", hdfsPath)); + log.info(String.format("Creating SequenceFile Writer, hdfsPath=%s", hdfsPath)); - this.writer = - SequenceFile.createWriter( - getConf(), - SequenceFile.Writer.file(new Path(hdfsPath)), - SequenceFile.Writer.keyClass(Text.class), - SequenceFile.Writer.valueClass(Text.class)); - } + this.writer = SequenceFile + .createWriter( + getConf(), + SequenceFile.Writer.file(new Path(hdfsPath)), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + } - private Configuration getConf() throws IOException { - final Configuration conf = new Configuration(); - /* - * conf.set("fs.defaultFS", hdfsNameNode); conf.set("fs.hdfs.impl", - * org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); conf.set("fs.file.impl", - * org.apache.hadoop.fs.LocalFileSystem.class.getName()); System.setProperty("HADOOP_USER_NAME", hdfsUser); - * System.setProperty("hadoop.home.dir", "/"); FileSystem.get(URI.create(hdfsNameNode), conf); - */ - return conf; - } + private Configuration getConf() throws IOException { + final Configuration conf = new Configuration(); + /* + * conf.set("fs.defaultFS", hdfsNameNode); conf.set("fs.hdfs.impl", + * org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); conf.set("fs.file.impl", + * org.apache.hadoop.fs.LocalFileSystem.class.getName()); System.setProperty("HADOOP_USER_NAME", hdfsUser); + * System.setProperty("hadoop.home.dir", "/"); FileSystem.get(URI.create(hdfsNameNode), conf); + */ + return conf; + } - protected void emit(final String s, final String type) { - try { - key.set(counter.getAndIncrement() + ":" + type); - value.set(s); - writer.append(key, value); - } catch (final Exception e) { - throw new RuntimeException(e); - } - } + protected void emit(final String s, final String type) { + try { + key.set(counter.getAndIncrement() + ":" + type); + value.set(s); + writer.append(key, value); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } - protected void emitOaf(final Oaf oaf) { - try { - emit(objectMapper.writeValueAsString(oaf), oaf.getClass().getSimpleName().toLowerCase()); - } catch (final Exception e) { - throw new RuntimeException(e); - } - } + protected void emitOaf(final Oaf oaf) { + try { + emit(objectMapper.writeValueAsString(oaf), oaf.getClass().getSimpleName().toLowerCase()); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } - public ObjectMapper getObjectMapper() { - return objectMapper; - } + public ObjectMapper getObjectMapper() { + return objectMapper; + } - @Override - public void close() throws IOException { - writer.hflush(); - writer.close(); - } + @Override + public void close() throws IOException { + writer.hflush(); + writer.close(); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java index ca7c9fffb..121df8131 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java @@ -1,61 +1,62 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; import java.io.Closeable; import java.io.IOException; import java.sql.*; import java.util.function.Consumer; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; public class DbClient implements Closeable { - private static final Log log = LogFactory.getLog(DbClient.class); + private static final Log log = LogFactory.getLog(DbClient.class); - private Connection connection; + private Connection connection; - public DbClient(final String address, final String login, final String password) { + public DbClient(final String address, final String login, final String password) { - try { - Class.forName("org.postgresql.Driver"); + try { + Class.forName("org.postgresql.Driver"); - this.connection = - StringUtils.isNoneBlank(login, password) - ? DriverManager.getConnection(address, login, password) - : DriverManager.getConnection(address); - this.connection.setAutoCommit(false); - } catch (final Exception e) { - log.error("Connection to postgresDB failed"); - throw new RuntimeException("Connection to postgresDB failed", e); - } - log.info("Opened database successfully"); - } + this.connection = StringUtils.isNoneBlank(login, password) + ? DriverManager.getConnection(address, login, password) + : DriverManager.getConnection(address); + this.connection.setAutoCommit(false); + } catch (final Exception e) { + log.error("Connection to postgresDB failed"); + throw new RuntimeException("Connection to postgresDB failed", e); + } + log.info("Opened database successfully"); + } - public void processResults(final String sql, final Consumer consumer) { + public void processResults(final String sql, final Consumer consumer) { - try (final Statement stmt = connection.createStatement()) { - stmt.setFetchSize(100); + try (final Statement stmt = connection.createStatement()) { + stmt.setFetchSize(100); - try (final ResultSet rs = stmt.executeQuery(sql)) { - while (rs.next()) { - consumer.accept(rs); - } - } catch (final SQLException e) { - log.error("Error executing sql query: " + sql, e); - throw new RuntimeException("Error executing sql query", e); - } - } catch (final SQLException e1) { - log.error("Error preparing sql statement", e1); - throw new RuntimeException("Error preparing sql statement", e1); - } - } + try (final ResultSet rs = stmt.executeQuery(sql)) { + while (rs.next()) { + consumer.accept(rs); + } + } catch (final SQLException e) { + log.error("Error executing sql query: " + sql, e); + throw new RuntimeException("Error executing sql query", e); + } + } catch (final SQLException e1) { + log.error("Error preparing sql statement", e1); + throw new RuntimeException("Error preparing sql statement", e1); + } + } - @Override - public void close() throws IOException { - try { - connection.close(); - } catch (final SQLException e) { - throw new RuntimeException(e); - } - } + @Override + public void close() throws IOException { + try { + connection.close(); + } catch (final SQLException e) { + throw new RuntimeException(e); + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java index 1602c9742..a2177935a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MdstoreClient.java @@ -1,100 +1,102 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; -import com.google.common.collect.Iterables; -import com.mongodb.MongoClient; -import com.mongodb.MongoClientURI; -import com.mongodb.client.MongoCollection; -import com.mongodb.client.MongoDatabase; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; import java.util.stream.StreamSupport; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.bson.Document; +import com.google.common.collect.Iterables; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientURI; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; + public class MdstoreClient implements Closeable { - private final MongoClient client; - private final MongoDatabase db; + private final MongoClient client; + private final MongoDatabase db; - private static final String COLL_METADATA = "metadata"; - private static final String COLL_METADATA_MANAGER = "metadataManager"; + private static final String COLL_METADATA = "metadata"; + private static final String COLL_METADATA_MANAGER = "metadataManager"; - private static final Log log = LogFactory.getLog(MdstoreClient.class); + private static final Log log = LogFactory.getLog(MdstoreClient.class); - public MdstoreClient(final String baseUrl, final String dbName) { - this.client = new MongoClient(new MongoClientURI(baseUrl)); - this.db = getDb(client, dbName); - } + public MdstoreClient(final String baseUrl, final String dbName) { + this.client = new MongoClient(new MongoClientURI(baseUrl)); + this.db = getDb(client, dbName); + } - public Map validCollections( - final String mdFormat, final String mdLayout, final String mdInterpretation) { + public Map validCollections( + final String mdFormat, final String mdLayout, final String mdInterpretation) { - final Map transactions = new HashMap<>(); - for (final Document entry : getColl(db, COLL_METADATA_MANAGER, true).find()) { - final String mdId = entry.getString("mdId"); - final String currentId = entry.getString("currentId"); - if (StringUtils.isNoneBlank(mdId, currentId)) { - transactions.put(mdId, currentId); - } - } + final Map transactions = new HashMap<>(); + for (final Document entry : getColl(db, COLL_METADATA_MANAGER, true).find()) { + final String mdId = entry.getString("mdId"); + final String currentId = entry.getString("currentId"); + if (StringUtils.isNoneBlank(mdId, currentId)) { + transactions.put(mdId, currentId); + } + } - final Map res = new HashMap<>(); - for (final Document entry : getColl(db, COLL_METADATA, true).find()) { - if (entry.getString("format").equals(mdFormat) - && entry.getString("layout").equals(mdLayout) - && entry.getString("interpretation").equals(mdInterpretation) - && transactions.containsKey(entry.getString("mdId"))) { - res.put(entry.getString("mdId"), transactions.get(entry.getString("mdId"))); - } - } + final Map res = new HashMap<>(); + for (final Document entry : getColl(db, COLL_METADATA, true).find()) { + if (entry.getString("format").equals(mdFormat) + && entry.getString("layout").equals(mdLayout) + && entry.getString("interpretation").equals(mdInterpretation) + && transactions.containsKey(entry.getString("mdId"))) { + res.put(entry.getString("mdId"), transactions.get(entry.getString("mdId"))); + } + } - return res; - } + return res; + } - private MongoDatabase getDb(final MongoClient client, final String dbName) { - if (!Iterables.contains(client.listDatabaseNames(), dbName)) { - final String err = - String.format("Database '%s' not found in %s", dbName, client.getAddress()); - log.warn(err); - throw new RuntimeException(err); - } - return client.getDatabase(dbName); - } + private MongoDatabase getDb(final MongoClient client, final String dbName) { + if (!Iterables.contains(client.listDatabaseNames(), dbName)) { + final String err = String.format("Database '%s' not found in %s", dbName, client.getAddress()); + log.warn(err); + throw new RuntimeException(err); + } + return client.getDatabase(dbName); + } - private MongoCollection getColl( - final MongoDatabase db, final String collName, final boolean abortIfMissing) { - if (!Iterables.contains(db.listCollectionNames(), collName)) { - final String err = - String.format( - String.format("Missing collection '%s' in database '%s'", collName, db.getName())); - log.warn(err); - if (abortIfMissing) { - throw new RuntimeException(err); - } else { - return null; - } - } - return db.getCollection(collName); - } + private MongoCollection getColl( + final MongoDatabase db, final String collName, final boolean abortIfMissing) { + if (!Iterables.contains(db.listCollectionNames(), collName)) { + final String err = String + .format( + String.format("Missing collection '%s' in database '%s'", collName, db.getName())); + log.warn(err); + if (abortIfMissing) { + throw new RuntimeException(err); + } else { + return null; + } + } + return db.getCollection(collName); + } - public Iterable listRecords(final String collName) { - final MongoCollection coll = getColl(db, collName, false); - return coll == null - ? new ArrayList<>() - : () -> - StreamSupport.stream(coll.find().spliterator(), false) - .filter(e -> e.containsKey("body")) - .map(e -> e.getString("body")) - .iterator(); - } + public Iterable listRecords(final String collName) { + final MongoCollection coll = getColl(db, collName, false); + return coll == null + ? new ArrayList<>() + : () -> StreamSupport + .stream(coll.find().spliterator(), false) + .filter(e -> e.containsKey("body")) + .map(e -> e.getString("body")) + .iterator(); + } - @Override - public void close() throws IOException { - client.close(); - } + @Override + public void close() throws IOException { + client.close(); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrationConstants.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrationConstants.java index 4e0b2dbd3..15bff9565 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrationConstants.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrationConstants.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.qualifier; @@ -6,26 +7,21 @@ import eu.dnetlib.dhp.schema.oaf.Qualifier; public class MigrationConstants { - public static final Qualifier PUBLICATION_RESULTTYPE_QUALIFIER = - qualifier("publication", "publication", "dnet:result_typologies", "dnet:result_typologies"); - public static final Qualifier DATASET_RESULTTYPE_QUALIFIER = - qualifier( - "dataset", "dataset", - "dnet:result_typologies", "dnet:result_typologies"); - public static final Qualifier SOFTWARE_RESULTTYPE_QUALIFIER = - qualifier( - "software", "software", - "dnet:result_typologies", "dnet:result_typologies"); - public static final Qualifier OTHER_RESULTTYPE_QUALIFIER = - qualifier( - "other", "other", - "dnet:result_typologies", "dnet:result_typologies"); - public static final Qualifier REPOSITORY_PROVENANCE_ACTIONS = - qualifier( - "sysimport:crosswalk:repository", "sysimport:crosswalk:repository", - "dnet:provenanceActions", "dnet:provenanceActions"); - public static final Qualifier ENTITYREGISTRY_PROVENANCE_ACTION = - qualifier( - "sysimport:crosswalk:entityregistry", "sysimport:crosswalk:entityregistry", - "dnet:provenanceActions", "dnet:provenanceActions"); + public static final Qualifier PUBLICATION_RESULTTYPE_QUALIFIER = qualifier( + "publication", "publication", "dnet:result_typologies", "dnet:result_typologies"); + public static final Qualifier DATASET_RESULTTYPE_QUALIFIER = qualifier( + "dataset", "dataset", + "dnet:result_typologies", "dnet:result_typologies"); + public static final Qualifier SOFTWARE_RESULTTYPE_QUALIFIER = qualifier( + "software", "software", + "dnet:result_typologies", "dnet:result_typologies"); + public static final Qualifier OTHER_RESULTTYPE_QUALIFIER = qualifier( + "other", "other", + "dnet:result_typologies", "dnet:result_typologies"); + public static final Qualifier REPOSITORY_PROVENANCE_ACTIONS = qualifier( + "sysimport:crosswalk:repository", "sysimport:crosswalk:repository", + "dnet:provenanceActions", "dnet:provenanceActions"); + public static final Qualifier ENTITYREGISTRY_PROVENANCE_ACTION = qualifier( + "sysimport:crosswalk:entityregistry", "sysimport:crosswalk:entityregistry", + "dnet:provenanceActions", "dnet:provenanceActions"); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java index b9788a05c..9beed2837 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java @@ -1,215 +1,220 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.utils.DHPUtils; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.DHPUtils; + public class OafMapperUtils { - public static KeyValue keyValue(final String k, final String v) { - final KeyValue kv = new KeyValue(); - kv.setKey(k); - kv.setValue(v); - return kv; - } + public static KeyValue keyValue(final String k, final String v) { + final KeyValue kv = new KeyValue(); + kv.setKey(k); + kv.setValue(v); + return kv; + } - public static List listKeyValues(final String... s) { - if (s.length % 2 > 0) { - throw new RuntimeException("Invalid number of parameters (k,v,k,v,....)"); - } + public static List listKeyValues(final String... s) { + if (s.length % 2 > 0) { + throw new RuntimeException("Invalid number of parameters (k,v,k,v,....)"); + } - final List list = new ArrayList<>(); - for (int i = 0; i < s.length; i += 2) { - list.add(keyValue(s[i], s[i + 1])); - } - return list; - } + final List list = new ArrayList<>(); + for (int i = 0; i < s.length; i += 2) { + list.add(keyValue(s[i], s[i + 1])); + } + return list; + } - public static Field field(final T value, final DataInfo info) { - if (value == null || StringUtils.isBlank(value.toString())) { - return null; - } + public static Field field(final T value, final DataInfo info) { + if (value == null || StringUtils.isBlank(value.toString())) { + return null; + } - final Field field = new Field<>(); - field.setValue(value); - field.setDataInfo(info); - return field; - } + final Field field = new Field<>(); + field.setValue(value); + field.setDataInfo(info); + return field; + } - public static List> listFields(final DataInfo info, final String... values) { - return Arrays.stream(values) - .map(v -> field(v, info)) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } + public static List> listFields(final DataInfo info, final String... values) { + return Arrays + .stream(values) + .map(v -> field(v, info)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } - public static List> listFields(final DataInfo info, final List values) { - return values.stream() - .map(v -> field(v, info)) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } + public static List> listFields(final DataInfo info, final List values) { + return values + .stream() + .map(v -> field(v, info)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } - public static Qualifier qualifier( - final String classid, - final String classname, - final String schemeid, - final String schemename) { - final Qualifier q = new Qualifier(); - q.setClassid(classid); - q.setClassname(classname); - q.setSchemeid(schemeid); - q.setSchemename(schemename); - return q; - } + public static Qualifier qualifier( + final String classid, + final String classname, + final String schemeid, + final String schemename) { + final Qualifier q = new Qualifier(); + q.setClassid(classid); + q.setClassname(classname); + q.setSchemeid(schemeid); + q.setSchemename(schemename); + return q; + } - public static StructuredProperty structuredProperty( - final String value, - final String classid, - final String classname, - final String schemeid, - final String schemename, - final DataInfo dataInfo) { + public static StructuredProperty structuredProperty( + final String value, + final String classid, + final String classname, + final String schemeid, + final String schemename, + final DataInfo dataInfo) { - return structuredProperty(value, qualifier(classid, classname, schemeid, schemename), dataInfo); - } + return structuredProperty(value, qualifier(classid, classname, schemeid, schemename), dataInfo); + } - public static StructuredProperty structuredProperty( - final String value, final Qualifier qualifier, final DataInfo dataInfo) { - if (value == null) { - return null; - } - final StructuredProperty sp = new StructuredProperty(); - sp.setValue(value); - sp.setQualifier(qualifier); - sp.setDataInfo(dataInfo); - return sp; - } + public static StructuredProperty structuredProperty( + final String value, final Qualifier qualifier, final DataInfo dataInfo) { + if (value == null) { + return null; + } + final StructuredProperty sp = new StructuredProperty(); + sp.setValue(value); + sp.setQualifier(qualifier); + sp.setDataInfo(dataInfo); + return sp; + } - public static ExtraInfo extraInfo( - final String name, - final String value, - final String typology, - final String provenance, - final String trust) { - final ExtraInfo info = new ExtraInfo(); - info.setName(name); - info.setValue(value); - info.setTypology(typology); - info.setProvenance(provenance); - info.setTrust(trust); - return info; - } + public static ExtraInfo extraInfo( + final String name, + final String value, + final String typology, + final String provenance, + final String trust) { + final ExtraInfo info = new ExtraInfo(); + info.setName(name); + info.setValue(value); + info.setTypology(typology); + info.setProvenance(provenance); + info.setTrust(trust); + return info; + } - public static OAIProvenance oaiIProvenance( - final String identifier, - final String baseURL, - final String metadataNamespace, - final Boolean altered, - final String datestamp, - final String harvestDate) { + public static OAIProvenance oaiIProvenance( + final String identifier, + final String baseURL, + final String metadataNamespace, + final Boolean altered, + final String datestamp, + final String harvestDate) { - final OriginDescription desc = new OriginDescription(); - desc.setIdentifier(identifier); - desc.setBaseURL(baseURL); - desc.setMetadataNamespace(metadataNamespace); - desc.setAltered(altered); - desc.setDatestamp(datestamp); - desc.setHarvestDate(harvestDate); + final OriginDescription desc = new OriginDescription(); + desc.setIdentifier(identifier); + desc.setBaseURL(baseURL); + desc.setMetadataNamespace(metadataNamespace); + desc.setAltered(altered); + desc.setDatestamp(datestamp); + desc.setHarvestDate(harvestDate); - final OAIProvenance p = new OAIProvenance(); - p.setOriginDescription(desc); + final OAIProvenance p = new OAIProvenance(); + p.setOriginDescription(desc); - return p; - } + return p; + } - public static Journal journal( - final String name, - final String issnPrinted, - final String issnOnline, - final String issnLinking, - final String ep, - final String iss, - final String sp, - final String vol, - final String edition, - final String conferenceplace, - final String conferencedate, - final DataInfo dataInfo) { + public static Journal journal( + final String name, + final String issnPrinted, + final String issnOnline, + final String issnLinking, + final String ep, + final String iss, + final String sp, + final String vol, + final String edition, + final String conferenceplace, + final String conferencedate, + final DataInfo dataInfo) { - if (StringUtils.isNotBlank(name) - || StringUtils.isNotBlank(issnPrinted) - || StringUtils.isNotBlank(issnOnline) - || StringUtils.isNotBlank(issnLinking)) { - final Journal j = new Journal(); - j.setName(name); - j.setIssnPrinted(issnPrinted); - j.setIssnOnline(issnOnline); - j.setIssnLinking(issnLinking); - j.setEp(ep); - j.setIss(iss); - j.setSp(sp); - j.setVol(vol); - j.setEdition(edition); - j.setConferenceplace(conferenceplace); - j.setConferencedate(conferencedate); - j.setDataInfo(dataInfo); - return j; - } else { - return null; - } - } + if (StringUtils.isNotBlank(name) + || StringUtils.isNotBlank(issnPrinted) + || StringUtils.isNotBlank(issnOnline) + || StringUtils.isNotBlank(issnLinking)) { + final Journal j = new Journal(); + j.setName(name); + j.setIssnPrinted(issnPrinted); + j.setIssnOnline(issnOnline); + j.setIssnLinking(issnLinking); + j.setEp(ep); + j.setIss(iss); + j.setSp(sp); + j.setVol(vol); + j.setEdition(edition); + j.setConferenceplace(conferenceplace); + j.setConferencedate(conferencedate); + j.setDataInfo(dataInfo); + return j; + } else { + return null; + } + } - public static DataInfo dataInfo( - final Boolean deletedbyinference, - final String inferenceprovenance, - final Boolean inferred, - final Boolean invisible, - final Qualifier provenanceaction, - final String trust) { - final DataInfo d = new DataInfo(); - d.setDeletedbyinference(deletedbyinference); - d.setInferenceprovenance(inferenceprovenance); - d.setInferred(inferred); - d.setInvisible(invisible); - d.setProvenanceaction(provenanceaction); - d.setTrust(trust); - return d; - } + public static DataInfo dataInfo( + final Boolean deletedbyinference, + final String inferenceprovenance, + final Boolean inferred, + final Boolean invisible, + final Qualifier provenanceaction, + final String trust) { + final DataInfo d = new DataInfo(); + d.setDeletedbyinference(deletedbyinference); + d.setInferenceprovenance(inferenceprovenance); + d.setInferred(inferred); + d.setInvisible(invisible); + d.setProvenanceaction(provenanceaction); + d.setTrust(trust); + return d; + } - public static String createOpenaireId( - final int prefix, final String originalId, final boolean to_md5) { - if (to_md5) { - final String nsPrefix = StringUtils.substringBefore(originalId, "::"); - final String rest = StringUtils.substringAfter(originalId, "::"); - return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest)); - } else { - return String.format("%s|%s", prefix, originalId); - } - } + public static String createOpenaireId( + final int prefix, final String originalId, final boolean to_md5) { + if (to_md5) { + final String nsPrefix = StringUtils.substringBefore(originalId, "::"); + final String rest = StringUtils.substringAfter(originalId, "::"); + return String.format("%s|%s::%s", prefix, nsPrefix, DHPUtils.md5(rest)); + } else { + return String.format("%s|%s", prefix, originalId); + } + } - public static String createOpenaireId( - final String type, final String originalId, final boolean to_md5) { - switch (type) { - case "datasource": - return createOpenaireId(10, originalId, to_md5); - case "organization": - return createOpenaireId(20, originalId, to_md5); - case "person": - return createOpenaireId(30, originalId, to_md5); - case "project": - return createOpenaireId(40, originalId, to_md5); - default: - return createOpenaireId(50, originalId, to_md5); - } - } + public static String createOpenaireId( + final String type, final String originalId, final boolean to_md5) { + switch (type) { + case "datasource": + return createOpenaireId(10, originalId, to_md5); + case "organization": + return createOpenaireId(20, originalId, to_md5); + case "person": + return createOpenaireId(30, originalId, to_md5); + case "project": + return createOpenaireId(40, originalId, to_md5); + default: + return createOpenaireId(50, originalId, to_md5); + } + } - public static String asString(final Object o) { - return o == null ? "" : o.toString(); - } + public static String asString(final Object o) { + return o == null ? "" : o.toString(); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java index 5317983b1..8adcd565b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/PacePerson.java @@ -1,178 +1,183 @@ + package eu.dnetlib.dhp.oa.graph.raw.common; +import java.nio.charset.Charset; +import java.text.Normalizer; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.text.WordUtils; + import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.hash.Hashing; -import java.nio.charset.Charset; -import java.text.Normalizer; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.text.WordUtils; public class PacePerson { - private static final String UTF8 = "UTF-8"; - private List name = Lists.newArrayList(); - private List surname = Lists.newArrayList(); - private List fullname = Lists.newArrayList(); - private final String original; + private static final String UTF8 = "UTF-8"; + private List name = Lists.newArrayList(); + private List surname = Lists.newArrayList(); + private List fullname = Lists.newArrayList(); + private final String original; - private static Set particles = null; + private static Set particles = null; - public static final String capitalize(final String s) { - return WordUtils.capitalize(s.toLowerCase(), ' ', '-'); - } + public static final String capitalize(final String s) { + return WordUtils.capitalize(s.toLowerCase(), ' ', '-'); + } - public static final String dotAbbreviations(final String s) { - return s.length() == 1 ? s + "." : s; - } + public static final String dotAbbreviations(final String s) { + return s.length() == 1 ? s + "." : s; + } - public static Set loadFromClasspath(final String classpath) { - final Set h = new HashSet<>(); - try { - for (final String s : IOUtils.readLines(PacePerson.class.getResourceAsStream(classpath))) { - h.add(s); - } - } catch (final Throwable e) { - return new HashSet<>(); - } - return h; - } + public static Set loadFromClasspath(final String classpath) { + final Set h = new HashSet<>(); + try { + for (final String s : IOUtils.readLines(PacePerson.class.getResourceAsStream(classpath))) { + h.add(s); + } + } catch (final Throwable e) { + return new HashSet<>(); + } + return h; + } - public PacePerson(String s, final boolean aggressive) { - original = s; - s = Normalizer.normalize(s, Normalizer.Form.NFD); - s = s.replaceAll("\\(.+\\)", ""); - s = s.replaceAll("\\[.+\\]", ""); - s = s.replaceAll("\\{.+\\}", ""); - s = s.replaceAll("\\s+-\\s+", "-"); - s = s.replaceAll("[\\p{Punct}&&[^,-]]", " "); - s = s.replaceAll("\\d", " "); - s = s.replaceAll("\\n", " "); - s = s.replaceAll("\\.", " "); - s = s.replaceAll("\\s+", " "); + public PacePerson(String s, final boolean aggressive) { + original = s; + s = Normalizer.normalize(s, Normalizer.Form.NFD); + s = s.replaceAll("\\(.+\\)", ""); + s = s.replaceAll("\\[.+\\]", ""); + s = s.replaceAll("\\{.+\\}", ""); + s = s.replaceAll("\\s+-\\s+", "-"); + s = s.replaceAll("[\\p{Punct}&&[^,-]]", " "); + s = s.replaceAll("\\d", " "); + s = s.replaceAll("\\n", " "); + s = s.replaceAll("\\.", " "); + s = s.replaceAll("\\s+", " "); - if (aggressive) { - s = s.replaceAll("[\\p{InCombiningDiacriticalMarks}&&[^,-]]", ""); - // s = s.replaceAll("[\\W&&[^,-]]", ""); - } + if (aggressive) { + s = s.replaceAll("[\\p{InCombiningDiacriticalMarks}&&[^,-]]", ""); + // s = s.replaceAll("[\\W&&[^,-]]", ""); + } - if (s.contains(",")) { - final String[] arr = s.split(","); - if (arr.length == 1) { - fullname = splitTerms(arr[0]); - } else if (arr.length > 1) { - surname = splitTerms(arr[0]); - name = splitTerms(arr[1]); - fullname.addAll(surname); - fullname.addAll(name); - } - } else { - fullname = splitTerms(s); + if (s.contains(",")) { + final String[] arr = s.split(","); + if (arr.length == 1) { + fullname = splitTerms(arr[0]); + } else if (arr.length > 1) { + surname = splitTerms(arr[0]); + name = splitTerms(arr[1]); + fullname.addAll(surname); + fullname.addAll(name); + } + } else { + fullname = splitTerms(s); - int lastInitialPosition = fullname.size(); - boolean hasSurnameInUpperCase = false; + int lastInitialPosition = fullname.size(); + boolean hasSurnameInUpperCase = false; - for (int i = 0; i < fullname.size(); i++) { - final String term = fullname.get(i); - if (term.length() == 1) { - lastInitialPosition = i; - } else if (term.equals(term.toUpperCase())) { - hasSurnameInUpperCase = true; - } - } + for (int i = 0; i < fullname.size(); i++) { + final String term = fullname.get(i); + if (term.length() == 1) { + lastInitialPosition = i; + } else if (term.equals(term.toUpperCase())) { + hasSurnameInUpperCase = true; + } + } - if (lastInitialPosition < fullname.size() - 1) { // Case: Michele G. Artini - name = fullname.subList(0, lastInitialPosition + 1); - surname = fullname.subList(lastInitialPosition + 1, fullname.size()); - } else if (hasSurnameInUpperCase) { // Case: Michele ARTINI - for (final String term : fullname) { - if (term.length() > 1 && term.equals(term.toUpperCase())) { - surname.add(term); - } else { - name.add(term); - } - } - } - } - } + if (lastInitialPosition < fullname.size() - 1) { // Case: Michele G. Artini + name = fullname.subList(0, lastInitialPosition + 1); + surname = fullname.subList(lastInitialPosition + 1, fullname.size()); + } else if (hasSurnameInUpperCase) { // Case: Michele ARTINI + for (final String term : fullname) { + if (term.length() > 1 && term.equals(term.toUpperCase())) { + surname.add(term); + } else { + name.add(term); + } + } + } + } + } - private List splitTerms(final String s) { - if (particles == null) { - particles = loadFromClasspath("/eu/dnetlib/dhp/oa/graph/pace/name_particles.txt"); - } + private List splitTerms(final String s) { + if (particles == null) { + particles = loadFromClasspath("/eu/dnetlib/dhp/oa/graph/pace/name_particles.txt"); + } - final List list = Lists.newArrayList(); - for (final String part : Splitter.on(" ").omitEmptyStrings().split(s)) { - if (!particles.contains(part.toLowerCase())) { - list.add(part); - } - } - return list; - } + final List list = Lists.newArrayList(); + for (final String part : Splitter.on(" ").omitEmptyStrings().split(s)) { + if (!particles.contains(part.toLowerCase())) { + list.add(part); + } + } + return list; + } - public List getName() { - return name; - } + public List getName() { + return name; + } - public String getNameString() { - return Joiner.on(" ").join(getName()); - } + public String getNameString() { + return Joiner.on(" ").join(getName()); + } - public List getSurname() { - return surname; - } + public List getSurname() { + return surname; + } - public List getFullname() { - return fullname; - } + public List getFullname() { + return fullname; + } - public String getOriginal() { - return original; - } + public String getOriginal() { + return original; + } - public String hash() { - return Hashing.murmur3_128() - .hashString(getNormalisedFullname(), Charset.forName(UTF8)) - .toString(); - } + public String hash() { + return Hashing + .murmur3_128() + .hashString(getNormalisedFullname(), Charset.forName(UTF8)) + .toString(); + } - public String getNormalisedFirstName() { - return Joiner.on(" ").join(getCapitalFirstnames()); - } + public String getNormalisedFirstName() { + return Joiner.on(" ").join(getCapitalFirstnames()); + } - public String getNormalisedSurname() { - return Joiner.on(" ").join(getCapitalSurname()); - } + public String getNormalisedSurname() { + return Joiner.on(" ").join(getCapitalSurname()); + } - public String getSurnameString() { - return Joiner.on(" ").join(getSurname()); - } + public String getSurnameString() { + return Joiner.on(" ").join(getSurname()); + } - public String getNormalisedFullname() { - return isAccurate() - ? getNormalisedSurname() + ", " + getNormalisedFirstName() - : Joiner.on(" ").join(fullname); - } + public String getNormalisedFullname() { + return isAccurate() + ? getNormalisedSurname() + ", " + getNormalisedFirstName() + : Joiner.on(" ").join(fullname); + } - public List getCapitalFirstnames() { - return Lists.newArrayList( - Iterables.transform(getNameWithAbbreviations(), PacePerson::capitalize)); - } + public List getCapitalFirstnames() { + return Lists + .newArrayList( + Iterables.transform(getNameWithAbbreviations(), PacePerson::capitalize)); + } - public List getCapitalSurname() { - return Lists.newArrayList(Iterables.transform(surname, PacePerson::capitalize)); - } + public List getCapitalSurname() { + return Lists.newArrayList(Iterables.transform(surname, PacePerson::capitalize)); + } - public List getNameWithAbbreviations() { - return Lists.newArrayList(Iterables.transform(name, PacePerson::dotAbbreviations)); - } + public List getNameWithAbbreviations() { + return Lists.newArrayList(Iterables.transform(name, PacePerson::dotAbbreviations)); + } - public boolean isAccurate() { - return name != null && surname != null && !name.isEmpty() && !surname.isEmpty(); - } + public boolean isAccurate() { + return name != null && surname != null && !name.isEmpty() && !surname.isEmpty(); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ImportDataFromMongo.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ImportDataFromMongo.java index 2787c61a9..bc40afbfd 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ImportDataFromMongo.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/ImportDataFromMongo.java @@ -1,12 +1,6 @@ + package eu.dnetlib.dhp.sx.graph; -import com.mongodb.DBObject; -import com.mongodb.MongoClient; -import com.mongodb.QueryBuilder; -import com.mongodb.client.FindIterable; -import com.mongodb.client.MongoCollection; -import com.mongodb.client.MongoDatabase; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -15,6 +9,7 @@ import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -25,127 +20,134 @@ import org.apache.hadoop.io.Text; import org.bson.Document; import org.bson.conversions.Bson; +import com.mongodb.DBObject; +import com.mongodb.MongoClient; +import com.mongodb.QueryBuilder; +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + /** - * This job is responsible to collect data from mongoDatabase and store in a sequence File on HDFS - * Mongo database contains information of each MDSTore in two collections: -metadata That contains - * info like: ID, format, layout, interpretation -metadataManager: that contains info : ID, - * mongoCollectionName from the metadata collection we filter the ids with Format, layout, and - * Interpretation from the metadataManager we get the current MONGO collection name which contains - * metadata XML see function getCurrentId - * - *

This Job will be called different times in base at the triple we want import, and generates - * for each triple a sequence file of XML + * This job is responsible to collect data from mongoDatabase and store in a sequence File on HDFS Mongo database + * contains information of each MDSTore in two collections: -metadata That contains info like: ID, format, layout, + * interpretation -metadataManager: that contains info : ID, mongoCollectionName from the metadata collection we filter + * the ids with Format, layout, and Interpretation from the metadataManager we get the current MONGO collection name + * which contains metadata XML see function getCurrentId + *

+ * This Job will be called different times in base at the triple we want import, and generates for each triple a + * sequence file of XML */ public class ImportDataFromMongo { - /** - * It requires in input some parameters described on a file - * eu/dnetlib/dhp/graph/sx/import_from_mongo_parameters.json - * - *

- the name node - the paht where store HDFS File - the mongo host - the mongo port - the - * metadata format to import - the metadata layout to import - the metadata interpretation to - * import - the mongo database Name - * - *

This params are encoded into args - * - * @param args - * @throws Exception - */ - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - ImportDataFromMongo.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/graph/argumentparser/import_from_mongo_parameters.json"))); - parser.parseArgument(args); - final int port = Integer.parseInt(parser.get("dbport")); - final String host = parser.get("dbhost"); + /** + * It requires in input some parameters described on a file + * eu/dnetlib/dhp/graph/sx/import_from_mongo_parameters.json + *

+ * - the name node - the paht where store HDFS File - the mongo host - the mongo port - the metadata format to + * import - the metadata layout to import - the metadata interpretation to import - the mongo database Name + *

+ * This params are encoded into args + * + * @param args + * @throws Exception + */ + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + ImportDataFromMongo.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/graph/argumentparser/import_from_mongo_parameters.json"))); + parser.parseArgument(args); + final int port = Integer.parseInt(parser.get("dbport")); + final String host = parser.get("dbhost"); - final String format = parser.get("format"); - final String layout = parser.get("layout"); - final String interpretation = parser.get("interpretation"); + final String format = parser.get("format"); + final String layout = parser.get("layout"); + final String interpretation = parser.get("interpretation"); - final String dbName = parser.get("dbName"); - final MongoClient client = new MongoClient(host, port); - MongoDatabase database = client.getDatabase(dbName); + final String dbName = parser.get("dbName"); + final MongoClient client = new MongoClient(host, port); + MongoDatabase database = client.getDatabase(dbName); - MongoCollection metadata = database.getCollection("metadata"); - MongoCollection metadataManager = database.getCollection("metadataManager"); - final DBObject query = - QueryBuilder.start("format") - .is(format) - .and("layout") - .is(layout) - .and("interpretation") - .is(interpretation) - .get(); - final List ids = new ArrayList<>(); - metadata - .find((Bson) query) - .forEach((Consumer) document -> ids.add(document.getString("mdId"))); - List databaseId = - ids.stream() - .map(it -> getCurrentId(it, metadataManager)) - .filter(Objects::nonNull) - .collect(Collectors.toList()); + MongoCollection metadata = database.getCollection("metadata"); + MongoCollection metadataManager = database.getCollection("metadataManager"); + final DBObject query = QueryBuilder + .start("format") + .is(format) + .and("layout") + .is(layout) + .and("interpretation") + .is(interpretation) + .get(); + final List ids = new ArrayList<>(); + metadata + .find((Bson) query) + .forEach((Consumer) document -> ids.add(document.getString("mdId"))); + List databaseId = ids + .stream() + .map(it -> getCurrentId(it, metadataManager)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); - final String hdfsuri = parser.get("namenode"); - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + final String hdfsuri = parser.get("namenode"); + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - FileSystem.get(URI.create(hdfsuri), conf); - Path hdfswritepath = new Path(parser.get("targetPath")); + FileSystem.get(URI.create(hdfsuri), conf); + Path hdfswritepath = new Path(parser.get("targetPath")); - final AtomicInteger counter = new AtomicInteger(0); - try (SequenceFile.Writer writer = - SequenceFile.createWriter( - conf, - SequenceFile.Writer.file(hdfswritepath), - SequenceFile.Writer.keyClass(IntWritable.class), - SequenceFile.Writer.valueClass(Text.class))) { - final IntWritable key = new IntWritable(counter.get()); - final Text value = new Text(); - databaseId.forEach( - id -> { - System.out.println("Reading :" + id); - MongoCollection collection = database.getCollection(id); - collection - .find() - .forEach( - (Consumer) - document -> { - key.set(counter.getAndIncrement()); - value.set(document.getString("body")); + final AtomicInteger counter = new AtomicInteger(0); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(hdfswritepath), + SequenceFile.Writer.keyClass(IntWritable.class), + SequenceFile.Writer.valueClass(Text.class))) { + final IntWritable key = new IntWritable(counter.get()); + final Text value = new Text(); + databaseId + .forEach( + id -> { + System.out.println("Reading :" + id); + MongoCollection collection = database.getCollection(id); + collection + .find() + .forEach( + (Consumer) document -> { + key.set(counter.getAndIncrement()); + value.set(document.getString("body")); - if (counter.get() % 10000 == 0) { - System.out.println("Added " + counter.get()); - } - try { - writer.append(key, value); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - }); - } - } + if (counter.get() % 10000 == 0) { + System.out.println("Added " + counter.get()); + } + try { + writer.append(key, value); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + }); + } + } - /** - * Return the name of mongo collection giving an MdStore ID - * - * @param mdId The id of the MDStore - * @param metadataManager The collection metadataManager on mongo which contains this information - * @return - */ - private static String getCurrentId( - final String mdId, final MongoCollection metadataManager) { - FindIterable result = - metadataManager.find((Bson) QueryBuilder.start("mdId").is(mdId).get()); - final Document item = result.first(); - return item == null ? null : item.getString("currentId"); - } + /** + * Return the name of mongo collection giving an MdStore ID + * + * @param mdId The id of the MDStore + * @param metadataManager The collection metadataManager on mongo which contains this information + * @return + */ + private static String getCurrentId( + final String mdId, final MongoCollection metadataManager) { + FindIterable result = metadataManager.find((Bson) QueryBuilder.start("mdId").is(mdId).get()); + final Document item = result.first(); + return item == null ? null : item.getString("currentId"); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkExtractEntitiesJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkExtractEntitiesJob.java index 457f987eb..4f015a9ad 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkExtractEntitiesJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkExtractEntitiesJob.java @@ -1,11 +1,10 @@ + package eu.dnetlib.dhp.sx.graph; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; -import net.minidev.json.JSONArray; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.compress.GzipCodec; @@ -13,107 +12,115 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; +import com.jayway.jsonpath.JsonPath; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import net.minidev.json.JSONArray; + /** - * This Job extracts a typology of entity and stores it in a new RDD This job is called different - * times, for each file generated by the Job {@link ImportDataFromMongo} and store the new RDD in a - * path that should be under a folder: extractedEntities/entity/version1 - * - *

at the end of this process we will have : extractedEntities/dataset/version1 - * extractedEntities/dataset/version2 extractedEntities/dataset/... - * extractedEntities/publication/version1 extractedEntities/publication/version2 - * extractedEntities/publication/... extractedEntities/unknown/version1 - * extractedEntities/unknown/version2 extractedEntities/unknown/... - * extractedEntities/relation/version1 extractedEntities/relation/version2 + * This Job extracts a typology of entity and stores it in a new RDD This job is called different times, for each file + * generated by the Job {@link ImportDataFromMongo} and store the new RDD in a path that should be under a folder: + * extractedEntities/entity/version1 + *

+ * at the end of this process we will have : extractedEntities/dataset/version1 extractedEntities/dataset/version2 + * extractedEntities/dataset/... extractedEntities/publication/version1 extractedEntities/publication/version2 + * extractedEntities/publication/... extractedEntities/unknown/version1 extractedEntities/unknown/version2 + * extractedEntities/unknown/... extractedEntities/relation/version1 extractedEntities/relation/version2 * extractedEntities/relation/... */ public class SparkExtractEntitiesJob { - static final String IDJSONPATH = "$.id"; - static final String SOURCEJSONPATH = "$.source"; - static final String TARGETJSONPATH = "$.target"; + static final String IDJSONPATH = "$.id"; + static final String SOURCEJSONPATH = "$.source"; + static final String TARGETJSONPATH = "$.target"; - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkExtractEntitiesJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/graph/argumentparser/input_extract_entities_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkExtractEntitiesJob.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String targetPath = parser.get("targetPath"); - final String tdir = parser.get("targetDir"); - final JavaRDD inputRDD = sc.textFile(inputPath); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkExtractEntitiesJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/graph/argumentparser/input_extract_entities_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkExtractEntitiesJob.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String targetPath = parser.get("targetPath"); + final String tdir = parser.get("targetDir"); + final JavaRDD inputRDD = sc.textFile(inputPath); - List entities = - Arrays.stream(parser.get("entities").split(",")) - .map(String::trim) - .collect(Collectors.toList()); - if (entities.stream().anyMatch("dataset"::equalsIgnoreCase)) { - // Extract Dataset - inputRDD - .filter(SparkExtractEntitiesJob::isDataset) - .saveAsTextFile(targetPath + "/dataset/" + tdir, GzipCodec.class); - } - if (entities.stream().anyMatch("unknown"::equalsIgnoreCase)) { - // Extract Unknown - inputRDD - .filter(SparkExtractEntitiesJob::isUnknown) - .saveAsTextFile(targetPath + "/unknown/" + tdir, GzipCodec.class); - } + List entities = Arrays + .stream(parser.get("entities").split(",")) + .map(String::trim) + .collect(Collectors.toList()); + if (entities.stream().anyMatch("dataset"::equalsIgnoreCase)) { + // Extract Dataset + inputRDD + .filter(SparkExtractEntitiesJob::isDataset) + .saveAsTextFile(targetPath + "/dataset/" + tdir, GzipCodec.class); + } + if (entities.stream().anyMatch("unknown"::equalsIgnoreCase)) { + // Extract Unknown + inputRDD + .filter(SparkExtractEntitiesJob::isUnknown) + .saveAsTextFile(targetPath + "/unknown/" + tdir, GzipCodec.class); + } - if (entities.stream().anyMatch("relation"::equalsIgnoreCase)) { - // Extract Relation - inputRDD - .filter(SparkExtractEntitiesJob::isRelation) - .saveAsTextFile(targetPath + "/relation/" + tdir, GzipCodec.class); - } - if (entities.stream().anyMatch("publication"::equalsIgnoreCase)) { - // Extract Relation - inputRDD - .filter(SparkExtractEntitiesJob::isPublication) - .saveAsTextFile(targetPath + "/publication/" + tdir, GzipCodec.class); - } - } + if (entities.stream().anyMatch("relation"::equalsIgnoreCase)) { + // Extract Relation + inputRDD + .filter(SparkExtractEntitiesJob::isRelation) + .saveAsTextFile(targetPath + "/relation/" + tdir, GzipCodec.class); + } + if (entities.stream().anyMatch("publication"::equalsIgnoreCase)) { + // Extract Relation + inputRDD + .filter(SparkExtractEntitiesJob::isPublication) + .saveAsTextFile(targetPath + "/publication/" + tdir, GzipCodec.class); + } + } - public static boolean isDataset(final String json) { - final String id = getJPathString(IDJSONPATH, json); - if (StringUtils.isBlank(id)) return false; - return id.startsWith("60|"); - } + public static boolean isDataset(final String json) { + final String id = getJPathString(IDJSONPATH, json); + if (StringUtils.isBlank(id)) + return false; + return id.startsWith("60|"); + } - public static boolean isPublication(final String json) { - final String id = getJPathString(IDJSONPATH, json); - if (StringUtils.isBlank(id)) return false; - return id.startsWith("50|"); - } + public static boolean isPublication(final String json) { + final String id = getJPathString(IDJSONPATH, json); + if (StringUtils.isBlank(id)) + return false; + return id.startsWith("50|"); + } - public static boolean isUnknown(final String json) { - final String id = getJPathString(IDJSONPATH, json); - if (StringUtils.isBlank(id)) return false; - return id.startsWith("70|"); - } + public static boolean isUnknown(final String json) { + final String id = getJPathString(IDJSONPATH, json); + if (StringUtils.isBlank(id)) + return false; + return id.startsWith("70|"); + } - public static boolean isRelation(final String json) { - final String source = getJPathString(SOURCEJSONPATH, json); - final String target = getJPathString(TARGETJSONPATH, json); - return StringUtils.isNotBlank(source) && StringUtils.isNotBlank(target); - } + public static boolean isRelation(final String json) { + final String source = getJPathString(SOURCEJSONPATH, json); + final String target = getJPathString(TARGETJSONPATH, json); + return StringUtils.isNotBlank(source) && StringUtils.isNotBlank(target); + } - public static String getJPathString(final String jsonPath, final String json) { - try { - Object o = JsonPath.read(json, jsonPath); - if (o instanceof String) return (String) o; - if (o instanceof JSONArray && ((JSONArray) o).size() > 0) - return (String) ((JSONArray) o).get(0); - return ""; - } catch (Exception e) { - return ""; - } - } + public static String getJPathString(final String jsonPath, final String json) { + try { + Object o = JsonPath.read(json, jsonPath); + if (o instanceof String) + return (String) o; + if (o instanceof JSONArray && ((JSONArray) o).size() > 0) + return (String) ((JSONArray) o).get(0); + return ""; + } catch (Exception e) { + return ""; + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java index cd8375df5..f3d7fd40f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkSXGeneratePidSimlarity.java @@ -1,7 +1,6 @@ + package eu.dnetlib.dhp.sx.graph; -import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; -import eu.dnetlib.dhp.utils.DHPUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -10,70 +9,67 @@ import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; + +import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; /** - * In some case the identifier generated for the Entity in @{@link SparkExtractEntitiesJob} is - * different from the identifier * associated by the aggregator, this means that some relation - * points to missing identifier To avoid this problem we store in the model the Id and the - * OriginalObJIdentifier This jobs extract this pair and creates a Similar relation that will be - * used in SparkMergeEntities + * In some case the identifier generated for the Entity in @{@link SparkExtractEntitiesJob} is different from the + * identifier * associated by the aggregator, this means that some relation points to missing identifier To avoid this + * problem we store in the model the Id and the OriginalObJIdentifier This jobs extract this pair and creates a Similar + * relation that will be used in SparkMergeEntities */ public class SparkSXGeneratePidSimlarity { - static final String IDJSONPATH = "$.id"; - static final String OBJIDPATH = "$.originalObjIdentifier"; + static final String IDJSONPATH = "$.id"; + static final String OBJIDPATH = "$.originalObjIdentifier"; - public static void generateDataFrame( - final SparkSession spark, - final JavaSparkContext sc, - final String inputPath, - final String targetPath) { + public static void generateDataFrame( + final SparkSession spark, + final JavaSparkContext sc, + final String inputPath, + final String targetPath) { - final JavaPairRDD datasetSimRel = - sc.textFile(inputPath + "/dataset/*") - .mapToPair( - (PairFunction) - k -> - new Tuple2<>( - DHPUtils.getJPathString(IDJSONPATH, k), - DHPUtils.getJPathString(OBJIDPATH, k))) - .filter( - t -> - !StringUtils.substringAfter(t._1(), "|") - .equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::"))) - .distinct(); + final JavaPairRDD datasetSimRel = sc + .textFile(inputPath + "/dataset/*") + .mapToPair( + (PairFunction) k -> new Tuple2<>( + DHPUtils.getJPathString(IDJSONPATH, k), + DHPUtils.getJPathString(OBJIDPATH, k))) + .filter( + t -> !StringUtils + .substringAfter(t._1(), "|") + .equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::"))) + .distinct(); - final JavaPairRDD publicationSimRel = - sc.textFile(inputPath + "/publication/*") - .mapToPair( - (PairFunction) - k -> - new Tuple2<>( - DHPUtils.getJPathString(IDJSONPATH, k), - DHPUtils.getJPathString(OBJIDPATH, k))) - .filter( - t -> - !StringUtils.substringAfter(t._1(), "|") - .equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::"))) - .distinct(); + final JavaPairRDD publicationSimRel = sc + .textFile(inputPath + "/publication/*") + .mapToPair( + (PairFunction) k -> new Tuple2<>( + DHPUtils.getJPathString(IDJSONPATH, k), + DHPUtils.getJPathString(OBJIDPATH, k))) + .filter( + t -> !StringUtils + .substringAfter(t._1(), "|") + .equalsIgnoreCase(StringUtils.substringAfter(t._2(), "::"))) + .distinct(); - JavaRDD simRel = - datasetSimRel - .union(publicationSimRel) - .map( - s -> { - final DLIRelation r = new DLIRelation(); - r.setSource(s._1()); - r.setTarget(s._2()); - r.setRelType("similar"); - return r; - }); - spark - .createDataset(simRel.rdd(), Encoders.bean(DLIRelation.class)) - .distinct() - .write() - .mode(SaveMode.Overwrite) - .save(targetPath + "/pid_simRel"); - } + JavaRDD simRel = datasetSimRel + .union(publicationSimRel) + .map( + s -> { + final DLIRelation r = new DLIRelation(); + r.setSource(s._1()); + r.setTarget(s._2()); + r.setRelType("similar"); + return r; + }); + spark + .createDataset(simRel.rdd(), Encoders.bean(DLIRelation.class)) + .distinct() + .write() + .mode(SaveMode.Overwrite) + .save(targetPath + "/pid_simRel"); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java index 3d1d9ec49..385ac4d1a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerCreateRawGraphJob.java @@ -1,20 +1,11 @@ + package eu.dnetlib.dhp.sx.graph; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; -import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; -import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; -import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; -import eu.dnetlib.dhp.utils.DHPUtils; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; -import net.minidev.json.JSONArray; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.FileStatus; @@ -31,228 +22,236 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.jayway.jsonpath.JsonPath; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; +import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; +import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; +import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; +import eu.dnetlib.dhp.utils.DHPUtils; +import net.minidev.json.JSONArray; import scala.Tuple2; /** - * This job is responsible of the creation of RAW Graph It is applied to the different entities - * generated from {@link SparkExtractEntitiesJob} In case of dataset, publication and Unknown - * Entities we group all the entities of the same type by their identifier, and then in the reduce - * phase we merge all the entities. Merge means: -merge all the metadata -merge the collected From - * values - * - *

In case of relation we need to make a different work: -Phase 1: Map reduce jobs Map: Get all - * Relation and emit a key constructed by (source, relType, Target) and the relation itself Reduce: - * Merge all relations Looking at the javadoc of {@link SparkSXGeneratePidSimlarity} we take the - * dataset of pid relation and joining by source and target we replace the wrong identifier in the - * relation with the correct ones. At the end we replace the new Dataset of Relation + * This job is responsible of the creation of RAW Graph It is applied to the different entities generated from + * {@link SparkExtractEntitiesJob} In case of dataset, publication and Unknown Entities we group all the entities of the + * same type by their identifier, and then in the reduce phase we merge all the entities. Merge means: -merge all the + * metadata -merge the collected From values + *

+ * In case of relation we need to make a different work: -Phase 1: Map reduce jobs Map: Get all Relation and emit a key + * constructed by (source, relType, Target) and the relation itself Reduce: Merge all relations Looking at the javadoc + * of {@link SparkSXGeneratePidSimlarity} we take the dataset of pid relation and joining by source and target we + * replace the wrong identifier in the relation with the correct ones. At the end we replace the new Dataset of Relation */ public class SparkScholexplorerCreateRawGraphJob { - static final String IDJSONPATH = "$.id"; - static final String SOURCEJSONPATH = "$.source"; - static final String TARGETJSONPATH = "$.target"; - static final String RELJSONPATH = "$.relType"; + static final String IDJSONPATH = "$.id"; + static final String SOURCEJSONPATH = "$.source"; + static final String TARGETJSONPATH = "$.target"; + static final String RELJSONPATH = "$.relType"; - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkScholexplorerCreateRawGraphJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/graph/argumentparser/merge_entities_scholix_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .config( - new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")) - .appName(SparkScholexplorerCreateRawGraphJob.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); - final String targetPath = parser.get("targetPath"); - final String entity = parser.get("entity"); - FileSystem fs = FileSystem.get(sc.sc().hadoopConfiguration()); - List subFolder = - Arrays.stream(fs.listStatus(new Path(inputPath))) - .filter(FileStatus::isDirectory) - .map(FileStatus::getPath) - .collect(Collectors.toList()); - List> inputRdd = new ArrayList<>(); - subFolder.forEach(p -> inputRdd.add(sc.textFile(p.toUri().getRawPath()))); - JavaRDD union = sc.emptyRDD(); - for (JavaRDD item : inputRdd) { - union = union.union(item); - } - switch (entity) { - case "dataset": - union - .mapToPair( - (PairFunction) - f -> { - final String id = getJPathString(IDJSONPATH, f); - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return new Tuple2<>(id, mapper.readValue(f, DLIDataset.class)); - }) - .reduceByKey( - (a, b) -> { - a.mergeFrom(b); - return a; - }) - .map( - item -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(item._2()); - }) - .saveAsTextFile(targetPath, GzipCodec.class); - break; - case "publication": - union - .mapToPair( - (PairFunction) - f -> { - final String id = getJPathString(IDJSONPATH, f); - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return new Tuple2<>(id, mapper.readValue(f, DLIPublication.class)); - }) - .reduceByKey( - (a, b) -> { - a.mergeFrom(b); - return a; - }) - .map( - item -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(item._2()); - }) - .saveAsTextFile(targetPath, GzipCodec.class); - break; - case "unknown": - union - .mapToPair( - (PairFunction) - f -> { - final String id = getJPathString(IDJSONPATH, f); - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return new Tuple2<>(id, mapper.readValue(f, DLIUnknown.class)); - }) - .reduceByKey( - (a, b) -> { - a.mergeFrom(b); - return a; - }) - .map( - item -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(item._2()); - }) - .saveAsTextFile(targetPath, GzipCodec.class); - break; - case "relation": - SparkSXGeneratePidSimlarity.generateDataFrame( - spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", "")); - RDD rdd = - union - .mapToPair( - (PairFunction) - f -> { - final String source = getJPathString(SOURCEJSONPATH, f); - final String target = getJPathString(TARGETJSONPATH, f); - final String reltype = getJPathString(RELJSONPATH, f); - ObjectMapper mapper = new ObjectMapper(); - mapper.configure( - DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - return new Tuple2<>( - DHPUtils.md5( - String.format( - "%s::%s::%s", - source.toLowerCase(), - reltype.toLowerCase(), - target.toLowerCase())), - mapper.readValue(f, DLIRelation.class)); - }) - .reduceByKey( - (a, b) -> { - a.mergeFrom(b); - return a; - }) - .map(Tuple2::_2) - .rdd(); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkScholexplorerCreateRawGraphJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/graph/argumentparser/merge_entities_scholix_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .config( + new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")) + .appName(SparkScholexplorerCreateRawGraphJob.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); + final String targetPath = parser.get("targetPath"); + final String entity = parser.get("entity"); + FileSystem fs = FileSystem.get(sc.sc().hadoopConfiguration()); + List subFolder = Arrays + .stream(fs.listStatus(new Path(inputPath))) + .filter(FileStatus::isDirectory) + .map(FileStatus::getPath) + .collect(Collectors.toList()); + List> inputRdd = new ArrayList<>(); + subFolder.forEach(p -> inputRdd.add(sc.textFile(p.toUri().getRawPath()))); + JavaRDD union = sc.emptyRDD(); + for (JavaRDD item : inputRdd) { + union = union.union(item); + } + switch (entity) { + case "dataset": + union + .mapToPair( + (PairFunction) f -> { + final String id = getJPathString(IDJSONPATH, f); + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return new Tuple2<>(id, mapper.readValue(f, DLIDataset.class)); + }) + .reduceByKey( + (a, b) -> { + a.mergeFrom(b); + return a; + }) + .map( + item -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(item._2()); + }) + .saveAsTextFile(targetPath, GzipCodec.class); + break; + case "publication": + union + .mapToPair( + (PairFunction) f -> { + final String id = getJPathString(IDJSONPATH, f); + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return new Tuple2<>(id, mapper.readValue(f, DLIPublication.class)); + }) + .reduceByKey( + (a, b) -> { + a.mergeFrom(b); + return a; + }) + .map( + item -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(item._2()); + }) + .saveAsTextFile(targetPath, GzipCodec.class); + break; + case "unknown": + union + .mapToPair( + (PairFunction) f -> { + final String id = getJPathString(IDJSONPATH, f); + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return new Tuple2<>(id, mapper.readValue(f, DLIUnknown.class)); + }) + .reduceByKey( + (a, b) -> { + a.mergeFrom(b); + return a; + }) + .map( + item -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(item._2()); + }) + .saveAsTextFile(targetPath, GzipCodec.class); + break; + case "relation": + SparkSXGeneratePidSimlarity + .generateDataFrame( + spark, sc, inputPath.replace("/relation", ""), targetPath.replace("/relation", "")); + RDD rdd = union + .mapToPair( + (PairFunction) f -> { + final String source = getJPathString(SOURCEJSONPATH, f); + final String target = getJPathString(TARGETJSONPATH, f); + final String reltype = getJPathString(RELJSONPATH, f); + ObjectMapper mapper = new ObjectMapper(); + mapper + .configure( + DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + return new Tuple2<>( + DHPUtils + .md5( + String + .format( + "%s::%s::%s", + source.toLowerCase(), + reltype.toLowerCase(), + target.toLowerCase())), + mapper.readValue(f, DLIRelation.class)); + }) + .reduceByKey( + (a, b) -> { + a.mergeFrom(b); + return a; + }) + .map(Tuple2::_2) + .rdd(); - spark - .createDataset(rdd, Encoders.bean(DLIRelation.class)) - .write() - .mode(SaveMode.Overwrite) - .save(targetPath); - Dataset rel_ds = spark.read().load(targetPath).as(Encoders.bean(Relation.class)); + spark + .createDataset(rdd, Encoders.bean(DLIRelation.class)) + .write() + .mode(SaveMode.Overwrite) + .save(targetPath); + Dataset rel_ds = spark.read().load(targetPath).as(Encoders.bean(Relation.class)); - System.out.println("LOADING PATH :" + targetPath.replace("/relation", "") + "/pid_simRel"); - Dataset sim_ds = - spark - .read() - .load(targetPath.replace("/relation", "") + "/pid_simRel") - .as(Encoders.bean(Relation.class)); + System.out.println("LOADING PATH :" + targetPath.replace("/relation", "") + "/pid_simRel"); + Dataset sim_ds = spark + .read() + .load(targetPath.replace("/relation", "") + "/pid_simRel") + .as(Encoders.bean(Relation.class)); - Dataset ids = - sim_ds.map( - (MapFunction) - relation -> { - final String type = StringUtils.substringBefore(relation.getSource(), "|"); - relation.setTarget( - String.format( - "%s|%s", - type, StringUtils.substringAfter(relation.getTarget(), "::"))); - return relation; - }, - Encoders.bean(Relation.class)); + Dataset ids = sim_ds + .map( + (MapFunction) relation -> { + final String type = StringUtils.substringBefore(relation.getSource(), "|"); + relation + .setTarget( + String + .format( + "%s|%s", + type, StringUtils.substringAfter(relation.getTarget(), "::"))); + return relation; + }, + Encoders.bean(Relation.class)); - final Dataset firstJoin = - rel_ds - .joinWith(ids, ids.col("target").equalTo(rel_ds.col("source")), "left_outer") - .map( - (MapFunction, Relation>) - s -> { - if (s._2() != null) { - s._1().setSource(s._2().getSource()); - } - return s._1(); - }, - Encoders.bean(Relation.class)); + final Dataset firstJoin = rel_ds + .joinWith(ids, ids.col("target").equalTo(rel_ds.col("source")), "left_outer") + .map( + (MapFunction, Relation>) s -> { + if (s._2() != null) { + s._1().setSource(s._2().getSource()); + } + return s._1(); + }, + Encoders.bean(Relation.class)); - Dataset secondJoin = - firstJoin - .joinWith(ids, ids.col("target").equalTo(firstJoin.col("target")), "left_outer") - .map( - (MapFunction, Relation>) - s -> { - if (s._2() != null) { - s._1().setTarget(s._2().getSource()); - } - return s._1(); - }, - Encoders.bean(Relation.class)); - secondJoin.write().mode(SaveMode.Overwrite).save(targetPath + "_fixed"); + Dataset secondJoin = firstJoin + .joinWith(ids, ids.col("target").equalTo(firstJoin.col("target")), "left_outer") + .map( + (MapFunction, Relation>) s -> { + if (s._2() != null) { + s._1().setTarget(s._2().getSource()); + } + return s._1(); + }, + Encoders.bean(Relation.class)); + secondJoin.write().mode(SaveMode.Overwrite).save(targetPath + "_fixed"); - FileSystem fileSystem = FileSystem.get(sc.hadoopConfiguration()); + FileSystem fileSystem = FileSystem.get(sc.hadoopConfiguration()); - fileSystem.delete(new Path(targetPath), true); - fileSystem.rename(new Path(targetPath + "_fixed"), new Path(targetPath)); - } - } + fileSystem.delete(new Path(targetPath), true); + fileSystem.rename(new Path(targetPath + "_fixed"), new Path(targetPath)); + } + } - public static String getJPathString(final String jsonPath, final String json) { - try { - Object o = JsonPath.read(json, jsonPath); - if (o instanceof String) return (String) o; - if (o instanceof JSONArray && ((JSONArray) o).size() > 0) - return (String) ((JSONArray) o).get(0); - return ""; - } catch (Exception e) { - return ""; - } - } + public static String getJPathString(final String jsonPath, final String json) { + try { + Object o = JsonPath.read(json, jsonPath); + if (o instanceof String) + return (String) o; + if (o instanceof JSONArray && ((JSONArray) o).size() > 0) + return (String) ((JSONArray) o).get(0); + return ""; + } catch (Exception e) { + return ""; + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporter.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporter.java index e0b0710c9..97f1251f0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporter.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporter.java @@ -1,11 +1,6 @@ + package eu.dnetlib.dhp.sx.graph; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.sx.graph.parser.DatasetScholexplorerParser; -import eu.dnetlib.dhp.sx.graph.parser.PublicationScholexplorerParser; -import eu.dnetlib.scholexplorer.relation.RelationMapper; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -13,56 +8,65 @@ import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.sql.SparkSession; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.sx.graph.parser.DatasetScholexplorerParser; +import eu.dnetlib.dhp.sx.graph.parser.PublicationScholexplorerParser; +import eu.dnetlib.scholexplorer.relation.RelationMapper; import scala.Tuple2; /** - * This Job read a sequential File containing XML stored in the aggregator and generates an RDD of - * heterogeneous entities like Dataset, Relation, Publication and Unknown + * This Job read a sequential File containing XML stored in the aggregator and generates an RDD of heterogeneous + * entities like Dataset, Relation, Publication and Unknown */ public class SparkScholexplorerGraphImporter { - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkScholexplorerGraphImporter.class.getResourceAsStream( - "/eu/dnetlib/dhp/sx/graph/argumentparser/input_graph_scholix_parameters.json"))); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkScholexplorerGraphImporter.class + .getResourceAsStream( + "/eu/dnetlib/dhp/sx/graph/argumentparser/input_graph_scholix_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkScholexplorerGraphImporter.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final String inputPath = parser.get("sourcePath"); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkScholexplorerGraphImporter.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final String inputPath = parser.get("sourcePath"); - RelationMapper relationMapper = RelationMapper.load(); + RelationMapper relationMapper = RelationMapper.load(); - sc.sequenceFile(inputPath, IntWritable.class, Text.class) - .map(Tuple2::_2) - .map(Text::toString) - .repartition(500) - .flatMap( - (FlatMapFunction) - record -> { - switch (parser.get("entity")) { - case "dataset": - final DatasetScholexplorerParser d = new DatasetScholexplorerParser(); - return d.parseObject(record, relationMapper).iterator(); - case "publication": - final PublicationScholexplorerParser p = new PublicationScholexplorerParser(); - return p.parseObject(record, relationMapper).iterator(); - default: - throw new IllegalArgumentException("wrong values of entities"); - } - }) - .map( - k -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(k); - }) - .saveAsTextFile(parser.get("targetPath"), GzipCodec.class); - } + sc + .sequenceFile(inputPath, IntWritable.class, Text.class) + .map(Tuple2::_2) + .map(Text::toString) + .repartition(500) + .flatMap( + (FlatMapFunction) record -> { + switch (parser.get("entity")) { + case "dataset": + final DatasetScholexplorerParser d = new DatasetScholexplorerParser(); + return d.parseObject(record, relationMapper).iterator(); + case "publication": + final PublicationScholexplorerParser p = new PublicationScholexplorerParser(); + return p.parseObject(record, relationMapper).iterator(); + default: + throw new IllegalArgumentException("wrong values of entities"); + } + }) + .map( + k -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(k); + }) + .saveAsTextFile(parser.get("targetPath"), GzipCodec.class); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java index 5e11c2a53..c97753fdc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/AbstractScholexplorerParser.java @@ -1,5 +1,17 @@ + package eu.dnetlib.dhp.sx.graph.parser; +import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import javax.xml.stream.XMLStreamReader; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.scholexplorer.DLIRelation; @@ -8,199 +20,195 @@ import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo; import eu.dnetlib.dhp.utils.DHPUtils; import eu.dnetlib.scholexplorer.relation.RelInfo; import eu.dnetlib.scholexplorer.relation.RelationMapper; -import java.util.*; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import javax.xml.stream.XMLStreamReader; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; public abstract class AbstractScholexplorerParser { - protected static final Log log = LogFactory.getLog(AbstractScholexplorerParser.class); - static final Pattern pattern = - Pattern.compile("10\\.\\d{4,9}/[-._;()/:A-Z0-9]+$", Pattern.CASE_INSENSITIVE); - private List datasetSubTypes = - Arrays.asList( - "dataset", - "software", - "film", - "sound", - "physicalobject", - "audiovisual", - "collection", - "other", - "study", - "metadata"); + protected static final Log log = LogFactory.getLog(AbstractScholexplorerParser.class); + static final Pattern pattern = Pattern.compile("10\\.\\d{4,9}/[-._;()/:A-Z0-9]+$", Pattern.CASE_INSENSITIVE); + private List datasetSubTypes = Arrays + .asList( + "dataset", + "software", + "film", + "sound", + "physicalobject", + "audiovisual", + "collection", + "other", + "study", + "metadata"); - public abstract List parseObject(final String record, final RelationMapper relMapper); + public abstract List parseObject(final String record, final RelationMapper relMapper); - protected Map getAttributes(final XMLStreamReader parser) { - final Map attributesMap = new HashMap<>(); - for (int i = 0; i < parser.getAttributeCount(); i++) { - attributesMap.put(parser.getAttributeLocalName(i), parser.getAttributeValue(i)); - } - return attributesMap; - } + protected Map getAttributes(final XMLStreamReader parser) { + final Map attributesMap = new HashMap<>(); + for (int i = 0; i < parser.getAttributeCount(); i++) { + attributesMap.put(parser.getAttributeLocalName(i), parser.getAttributeValue(i)); + } + return attributesMap; + } - protected List extractSubject(List subjects) { - final List subjectResult = new ArrayList<>(); - if (subjects != null && subjects.size() > 0) { - subjects.forEach( - subjectMap -> { - final StructuredProperty subject = new StructuredProperty(); - subject.setValue(subjectMap.getTextValue()); - final Qualifier schema = new Qualifier(); - schema.setClassid("dnet:subject"); - schema.setClassname("dnet:subject"); - schema.setSchemeid(subjectMap.getAttributes().get("subjectScheme")); - schema.setSchemename(subjectMap.getAttributes().get("subjectScheme")); - subject.setQualifier(schema); - subjectResult.add(subject); - }); - } - return subjectResult; - } + protected List extractSubject(List subjects) { + final List subjectResult = new ArrayList<>(); + if (subjects != null && subjects.size() > 0) { + subjects + .forEach( + subjectMap -> { + final StructuredProperty subject = new StructuredProperty(); + subject.setValue(subjectMap.getTextValue()); + final Qualifier schema = new Qualifier(); + schema.setClassid("dnet:subject"); + schema.setClassname("dnet:subject"); + schema.setSchemeid(subjectMap.getAttributes().get("subjectScheme")); + schema.setSchemename(subjectMap.getAttributes().get("subjectScheme")); + subject.setQualifier(schema); + subjectResult.add(subject); + }); + } + return subjectResult; + } - protected StructuredProperty extractIdentifier( - List identifierType, final String fieldName) { - final StructuredProperty pid = new StructuredProperty(); - if (identifierType != null && identifierType.size() > 0) { - final VtdUtilityParser.Node result = identifierType.get(0); - pid.setValue(result.getTextValue()); - final Qualifier pidType = new Qualifier(); - pidType.setClassname(result.getAttributes().get(fieldName)); - pidType.setClassid(result.getAttributes().get(fieldName)); - pidType.setSchemename("dnet:pid_types"); - pidType.setSchemeid("dnet:pid_types"); - pid.setQualifier(pidType); - return pid; - } - return null; - } + protected StructuredProperty extractIdentifier( + List identifierType, final String fieldName) { + final StructuredProperty pid = new StructuredProperty(); + if (identifierType != null && identifierType.size() > 0) { + final VtdUtilityParser.Node result = identifierType.get(0); + pid.setValue(result.getTextValue()); + final Qualifier pidType = new Qualifier(); + pidType.setClassname(result.getAttributes().get(fieldName)); + pidType.setClassid(result.getAttributes().get(fieldName)); + pidType.setSchemename("dnet:pid_types"); + pidType.setSchemeid("dnet:pid_types"); + pid.setQualifier(pidType); + return pid; + } + return null; + } - protected void inferPid(final StructuredProperty input) { - final Matcher matcher = pattern.matcher(input.getValue()); - if (matcher.find()) { - input.setValue(matcher.group()); - if (input.getQualifier() == null) { - input.setQualifier(new Qualifier()); - input.getQualifier().setSchemename("dnet:pid_types"); - input.getQualifier().setSchemeid("dnet:pid_types"); - } - input.getQualifier().setClassid("doi"); - input.getQualifier().setClassname("doi"); - } - } + protected void inferPid(final StructuredProperty input) { + final Matcher matcher = pattern.matcher(input.getValue()); + if (matcher.find()) { + input.setValue(matcher.group()); + if (input.getQualifier() == null) { + input.setQualifier(new Qualifier()); + input.getQualifier().setSchemename("dnet:pid_types"); + input.getQualifier().setSchemeid("dnet:pid_types"); + } + input.getQualifier().setClassid("doi"); + input.getQualifier().setClassname("doi"); + } + } - protected String generateId(final String pid, final String pidType, final String entityType) { - String type; - switch (entityType) { - case "publication": - type = "50|"; - break; - case "dataset": - type = "60|"; - break; - case "unknown": - type = "70|"; - break; - default: - throw new IllegalArgumentException("unexpected value " + entityType); - } - if ("dnet".equalsIgnoreCase(pidType)) return type + StringUtils.substringAfter(pid, "::"); + protected String generateId(final String pid, final String pidType, final String entityType) { + String type; + switch (entityType) { + case "publication": + type = "50|"; + break; + case "dataset": + type = "60|"; + break; + case "unknown": + type = "70|"; + break; + default: + throw new IllegalArgumentException("unexpected value " + entityType); + } + if ("dnet".equalsIgnoreCase(pidType)) + return type + StringUtils.substringAfter(pid, "::"); - return type - + DHPUtils.md5( - String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); - } + return type + + DHPUtils + .md5( + String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); + } - protected DLIUnknown createUnknownObject( - final String pid, - final String pidType, - final KeyValue cf, - final DataInfo di, - final String dateOfCollection) { - final DLIUnknown uk = new DLIUnknown(); - uk.setId(generateId(pid, pidType, "unknown")); - ProvenaceInfo pi = new ProvenaceInfo(); - pi.setId(cf.getKey()); - pi.setName(cf.getValue()); - pi.setCompletionStatus("incomplete"); - uk.setDataInfo(di); - uk.setDlicollectedfrom(Collections.singletonList(pi)); - final StructuredProperty sourcePid = new StructuredProperty(); - sourcePid.setValue(pid); - final Qualifier pt = new Qualifier(); - pt.setClassname(pidType); - pt.setClassid(pidType); - pt.setSchemename("dnet:pid_types"); - pt.setSchemeid("dnet:pid_types"); - sourcePid.setQualifier(pt); - uk.setPid(Collections.singletonList(sourcePid)); - uk.setDateofcollection(dateOfCollection); - return uk; - } + protected DLIUnknown createUnknownObject( + final String pid, + final String pidType, + final KeyValue cf, + final DataInfo di, + final String dateOfCollection) { + final DLIUnknown uk = new DLIUnknown(); + uk.setId(generateId(pid, pidType, "unknown")); + ProvenaceInfo pi = new ProvenaceInfo(); + pi.setId(cf.getKey()); + pi.setName(cf.getValue()); + pi.setCompletionStatus("incomplete"); + uk.setDataInfo(di); + uk.setDlicollectedfrom(Collections.singletonList(pi)); + final StructuredProperty sourcePid = new StructuredProperty(); + sourcePid.setValue(pid); + final Qualifier pt = new Qualifier(); + pt.setClassname(pidType); + pt.setClassid(pidType); + pt.setSchemename("dnet:pid_types"); + pt.setSchemeid("dnet:pid_types"); + sourcePid.setQualifier(pt); + uk.setPid(Collections.singletonList(sourcePid)); + uk.setDateofcollection(dateOfCollection); + return uk; + } - protected void generateRelations( - RelationMapper relationMapper, - Result parsedObject, - List result, - DataInfo di, - String dateOfCollection, - List relatedIdentifiers) { - if (relatedIdentifiers != null) { - result.addAll( - relatedIdentifiers.stream() - .flatMap( - n -> { - final List rels = new ArrayList<>(); - DLIRelation r = new DLIRelation(); - r.setSource(parsedObject.getId()); - final String relatedPid = n.getTextValue(); - final String relatedPidType = n.getAttributes().get("relatedIdentifierType"); - final String relatedType = - n.getAttributes().getOrDefault("entityType", "unknown"); - String relationSemantic = n.getAttributes().get("relationType"); - String inverseRelation; - final String targetId = generateId(relatedPid, relatedPidType, relatedType); - r.setDateOfCollection(dateOfCollection); - if (relationMapper.containsKey(relationSemantic.toLowerCase())) { - RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase()); - relationSemantic = relInfo.getOriginal(); - inverseRelation = relInfo.getInverse(); - } else { - relationSemantic = "Unknown"; - inverseRelation = "Unknown"; - } - r.setTarget(targetId); - r.setRelType(relationSemantic); - r.setRelClass("datacite"); - r.setCollectedfrom(parsedObject.getCollectedfrom()); - r.setDataInfo(di); - rels.add(r); - r = new DLIRelation(); - r.setDataInfo(di); - r.setSource(targetId); - r.setTarget(parsedObject.getId()); - r.setRelType(inverseRelation); - r.setRelClass("datacite"); - r.setCollectedfrom(parsedObject.getCollectedfrom()); - r.setDateOfCollection(dateOfCollection); - rels.add(r); - if ("unknown".equalsIgnoreCase(relatedType)) - result.add( - createUnknownObject( - relatedPid, - relatedPidType, - parsedObject.getCollectedfrom().get(0), - di, - dateOfCollection)); - return rels.stream(); - }) - .collect(Collectors.toList())); - } - } + protected void generateRelations( + RelationMapper relationMapper, + Result parsedObject, + List result, + DataInfo di, + String dateOfCollection, + List relatedIdentifiers) { + if (relatedIdentifiers != null) { + result + .addAll( + relatedIdentifiers + .stream() + .flatMap( + n -> { + final List rels = new ArrayList<>(); + DLIRelation r = new DLIRelation(); + r.setSource(parsedObject.getId()); + final String relatedPid = n.getTextValue(); + final String relatedPidType = n.getAttributes().get("relatedIdentifierType"); + final String relatedType = n.getAttributes().getOrDefault("entityType", "unknown"); + String relationSemantic = n.getAttributes().get("relationType"); + String inverseRelation; + final String targetId = generateId(relatedPid, relatedPidType, relatedType); + r.setDateOfCollection(dateOfCollection); + if (relationMapper.containsKey(relationSemantic.toLowerCase())) { + RelInfo relInfo = relationMapper.get(relationSemantic.toLowerCase()); + relationSemantic = relInfo.getOriginal(); + inverseRelation = relInfo.getInverse(); + } else { + relationSemantic = "Unknown"; + inverseRelation = "Unknown"; + } + r.setTarget(targetId); + r.setRelType(relationSemantic); + r.setRelClass("datacite"); + r.setCollectedfrom(parsedObject.getCollectedfrom()); + r.setDataInfo(di); + rels.add(r); + r = new DLIRelation(); + r.setDataInfo(di); + r.setSource(targetId); + r.setTarget(parsedObject.getId()); + r.setRelType(inverseRelation); + r.setRelClass("datacite"); + r.setCollectedfrom(parsedObject.getCollectedfrom()); + r.setDateOfCollection(dateOfCollection); + rels.add(r); + if ("unknown".equalsIgnoreCase(relatedType)) + result + .add( + createUnknownObject( + relatedPid, + relatedPidType, + parsedObject.getCollectedfrom().get(0), + di, + dateOfCollection)); + return rels.stream(); + }) + .collect(Collectors.toList())); + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java index 07b711106..f49163c87 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/DatasetScholexplorerParser.java @@ -1,270 +1,292 @@ + package eu.dnetlib.dhp.sx.graph.parser; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; + import com.ximpleware.AutoPilot; import com.ximpleware.VTDGen; import com.ximpleware.VTDNav; + import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; import eu.dnetlib.dhp.parser.utility.VtdUtilityParser.Node; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo; import eu.dnetlib.scholexplorer.relation.RelationMapper; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; public class DatasetScholexplorerParser extends AbstractScholexplorerParser { - @Override - public List parseObject(String record, final RelationMapper relationMapper) { - try { - final DLIDataset parsedObject = new DLIDataset(); - final VTDGen vg = new VTDGen(); - vg.setDoc(record.getBytes()); - final List result = new ArrayList<>(); - vg.parse(true); + @Override + public List parseObject(String record, final RelationMapper relationMapper) { + try { + final DLIDataset parsedObject = new DLIDataset(); + final VTDGen vg = new VTDGen(); + vg.setDoc(record.getBytes()); + final List result = new ArrayList<>(); + vg.parse(true); - final VTDNav vn = vg.getNav(); - final AutoPilot ap = new AutoPilot(vn); + final VTDNav vn = vg.getNav(); + final AutoPilot ap = new AutoPilot(vn); - DataInfo di = new DataInfo(); - di.setTrust("0.9"); - di.setDeletedbyinference(false); - di.setInvisible(false); - parsedObject.setDataInfo(di); + DataInfo di = new DataInfo(); + di.setTrust("0.9"); + di.setDeletedbyinference(false); + di.setInvisible(false); + parsedObject.setDataInfo(di); - parsedObject.setOriginalId( - Collections.singletonList( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); + parsedObject + .setOriginalId( + Collections + .singletonList( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); - parsedObject.setOriginalObjIdentifier( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']")); - String dateOfCollection = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); - parsedObject.setDateofcollection(dateOfCollection); + parsedObject + .setOriginalObjIdentifier( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']")); + String dateOfCollection = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); + parsedObject.setDateofcollection(dateOfCollection); - final String resolvedDate = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); + final String resolvedDate = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); - if (StringUtils.isNotBlank(resolvedDate)) { - StructuredProperty currentDate = new StructuredProperty(); - currentDate.setValue(resolvedDate); - final Qualifier dateQualifier = new Qualifier(); - dateQualifier.setClassname("resolvedDate"); - dateQualifier.setClassid("resolvedDate"); - dateQualifier.setSchemename("dnet::date"); - dateQualifier.setSchemeid("dnet::date"); - currentDate.setQualifier(dateQualifier); - parsedObject.setRelevantdate(Collections.singletonList(currentDate)); - } + if (StringUtils.isNotBlank(resolvedDate)) { + StructuredProperty currentDate = new StructuredProperty(); + currentDate.setValue(resolvedDate); + final Qualifier dateQualifier = new Qualifier(); + dateQualifier.setClassname("resolvedDate"); + dateQualifier.setClassid("resolvedDate"); + dateQualifier.setSchemename("dnet::date"); + dateQualifier.setSchemeid("dnet::date"); + currentDate.setQualifier(dateQualifier); + parsedObject.setRelevantdate(Collections.singletonList(currentDate)); + } - final String completionStatus = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='completionStatus']"); - final String provisionMode = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']"); + final String completionStatus = VtdUtilityParser + .getSingleValue(ap, vn, "//*[local-name()='completionStatus']"); + final String provisionMode = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']"); - final String publisher = - VtdUtilityParser.getSingleValue( - ap, vn, "//*[local-name()='resource']/*[local-name()='publisher']"); + final String publisher = VtdUtilityParser + .getSingleValue( + ap, vn, "//*[local-name()='resource']/*[local-name()='publisher']"); - List collectedFromNodes = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='collectedFrom']", - Arrays.asList("name", "id", "mode", "completionStatus")); + List collectedFromNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='collectedFrom']", + Arrays.asList("name", "id", "mode", "completionStatus")); - List resolvededFromNodes = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='resolvedFrom']", - Arrays.asList("name", "id", "mode", "completionStatus")); + List resolvededFromNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='resolvedFrom']", + Arrays.asList("name", "id", "mode", "completionStatus")); - Field pf = new Field<>(); - pf.setValue(publisher); + Field pf = new Field<>(); + pf.setValue(publisher); - parsedObject.setPublisher(pf); - final List provenances = new ArrayList<>(); - if (collectedFromNodes != null && collectedFromNodes.size() > 0) { - collectedFromNodes.forEach( - it -> { - final ProvenaceInfo provenance = new ProvenaceInfo(); - provenance.setId(it.getAttributes().get("id")); - provenance.setName(it.getAttributes().get("name")); - provenance.setCollectionMode(provisionMode); - provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); - provenances.add(provenance); - }); - } + parsedObject.setPublisher(pf); + final List provenances = new ArrayList<>(); + if (collectedFromNodes != null && collectedFromNodes.size() > 0) { + collectedFromNodes + .forEach( + it -> { + final ProvenaceInfo provenance = new ProvenaceInfo(); + provenance.setId(it.getAttributes().get("id")); + provenance.setName(it.getAttributes().get("name")); + provenance.setCollectionMode(provisionMode); + provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); + provenances.add(provenance); + }); + } - if (resolvededFromNodes != null && resolvededFromNodes.size() > 0) { - resolvededFromNodes.forEach( - it -> { - final ProvenaceInfo provenance = new ProvenaceInfo(); - provenance.setId(it.getAttributes().get("id")); - provenance.setName(it.getAttributes().get("name")); - provenance.setCollectionMode("resolved"); - provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); - provenances.add(provenance); - }); - } + if (resolvededFromNodes != null && resolvededFromNodes.size() > 0) { + resolvededFromNodes + .forEach( + it -> { + final ProvenaceInfo provenance = new ProvenaceInfo(); + provenance.setId(it.getAttributes().get("id")); + provenance.setName(it.getAttributes().get("name")); + provenance.setCollectionMode("resolved"); + provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); + provenances.add(provenance); + }); + } - parsedObject.setDlicollectedfrom(provenances); - parsedObject.setCollectedfrom( - parsedObject.getDlicollectedfrom().stream() - .map( - p -> { - final KeyValue cf = new KeyValue(); - cf.setKey(p.getId()); - cf.setValue(p.getName()); - return cf; - }) - .collect(Collectors.toList())); - parsedObject.setCompletionStatus( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='completionStatus']")); + parsedObject.setDlicollectedfrom(provenances); + parsedObject + .setCollectedfrom( + parsedObject + .getDlicollectedfrom() + .stream() + .map( + p -> { + final KeyValue cf = new KeyValue(); + cf.setKey(p.getId()); + cf.setValue(p.getName()); + return cf; + }) + .collect(Collectors.toList())); + parsedObject + .setCompletionStatus( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='completionStatus']")); - final List identifierType = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='resource']/*[local-name()='identifier']", - Collections.singletonList("identifierType")); + final List identifierType = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='resource']/*[local-name()='identifier']", + Collections.singletonList("identifierType")); - StructuredProperty currentPid = extractIdentifier(identifierType, "identifierType"); - if (currentPid == null) return null; - inferPid(currentPid); - parsedObject.setPid(Collections.singletonList(currentPid)); + StructuredProperty currentPid = extractIdentifier(identifierType, "identifierType"); + if (currentPid == null) + return null; + inferPid(currentPid); + parsedObject.setPid(Collections.singletonList(currentPid)); - final String sourceId = - generateId(currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset"); - parsedObject.setId(sourceId); + final String sourceId = generateId( + currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset"); + parsedObject.setId(sourceId); - List descs = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='description']"); - if (descs != null && descs.size() > 0) - parsedObject.setDescription( - descs.stream() - .map(it -> it.length() < 10000 ? it : it.substring(0, 10000)) - .map( - it -> { - final Field d = new Field<>(); - d.setValue(it); - return d; - }) - .collect(Collectors.toList())); + List descs = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='description']"); + if (descs != null && descs.size() > 0) + parsedObject + .setDescription( + descs + .stream() + .map(it -> it.length() < 10000 ? it : it.substring(0, 10000)) + .map( + it -> { + final Field d = new Field<>(); + d.setValue(it); + return d; + }) + .collect(Collectors.toList())); - final List relatedIdentifiers = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='relatedIdentifier']", - Arrays.asList( - "relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); + final List relatedIdentifiers = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='relatedIdentifier']", + Arrays + .asList( + "relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); - generateRelations( - relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); + generateRelations( + relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); - final List hostedBy = - VtdUtilityParser.getTextValuesWithAttributes( - ap, vn, "//*[local-name()='hostedBy']", Arrays.asList("id", "name")); + final List hostedBy = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//*[local-name()='hostedBy']", Arrays.asList("id", "name")); - if (hostedBy != null) { - parsedObject.setInstance( - hostedBy.stream() - .map( - it -> { - final Instance i = new Instance(); - i.setUrl(Collections.singletonList(currentPid.getValue())); - KeyValue h = new KeyValue(); - i.setHostedby(h); - h.setKey(it.getAttributes().get("id")); - h.setValue(it.getAttributes().get("name")); - return i; - }) - .collect(Collectors.toList())); - } + if (hostedBy != null) { + parsedObject + .setInstance( + hostedBy + .stream() + .map( + it -> { + final Instance i = new Instance(); + i.setUrl(Collections.singletonList(currentPid.getValue())); + KeyValue h = new KeyValue(); + i.setHostedby(h); + h.setKey(it.getAttributes().get("id")); + h.setValue(it.getAttributes().get("name")); + return i; + }) + .collect(Collectors.toList())); + } - List subjects = - extractSubject( - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='resource']//*[local-name()='subject']", - Collections.singletonList("subjectScheme"))); + List subjects = extractSubject( + VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='resource']//*[local-name()='subject']", + Collections.singletonList("subjectScheme"))); - parsedObject.setSubject(subjects); + parsedObject.setSubject(subjects); - Qualifier q = new Qualifier(); - q.setClassname("dataset"); - q.setClassid("dataset"); - q.setSchemename("dataset"); - q.setSchemeid("dataset"); - parsedObject.setResulttype(q); + Qualifier q = new Qualifier(); + q.setClassname("dataset"); + q.setClassid("dataset"); + q.setSchemename("dataset"); + q.setSchemeid("dataset"); + parsedObject.setResulttype(q); - parsedObject.setCompletionStatus(completionStatus); + parsedObject.setCompletionStatus(completionStatus); - final List creators = - VtdUtilityParser.getTextValue( - ap, - vn, - "//*[local-name()='resource']//*[local-name()='creator']/*[local-name()='creatorName']"); - if (creators != null && creators.size() > 0) { - parsedObject.setAuthor( - creators.stream() - .map( - a -> { - final Author author = new Author(); - author.setFullname(a); - return author; - }) - .collect(Collectors.toList())); - } - final List titles = - VtdUtilityParser.getTextValue( - ap, vn, "//*[local-name()='resource']//*[local-name()='title']"); - if (titles != null && titles.size() > 0) { - parsedObject.setTitle( - titles.stream() - .map( - t -> { - final StructuredProperty st = new StructuredProperty(); - st.setValue(t); - return st; - }) - .collect(Collectors.toList())); - } + final List creators = VtdUtilityParser + .getTextValue( + ap, + vn, + "//*[local-name()='resource']//*[local-name()='creator']/*[local-name()='creatorName']"); + if (creators != null && creators.size() > 0) { + parsedObject + .setAuthor( + creators + .stream() + .map( + a -> { + final Author author = new Author(); + author.setFullname(a); + return author; + }) + .collect(Collectors.toList())); + } + final List titles = VtdUtilityParser + .getTextValue( + ap, vn, "//*[local-name()='resource']//*[local-name()='title']"); + if (titles != null && titles.size() > 0) { + parsedObject + .setTitle( + titles + .stream() + .map( + t -> { + final StructuredProperty st = new StructuredProperty(); + st.setValue(t); + return st; + }) + .collect(Collectors.toList())); + } - final List dates = - VtdUtilityParser.getTextValue( - ap, - vn, - "//*[local-name()='resource']/*[local-name()='dates']/*[local-name()='date']"); + final List dates = VtdUtilityParser + .getTextValue( + ap, + vn, + "//*[local-name()='resource']/*[local-name()='dates']/*[local-name()='date']"); - if (dates != null && dates.size() > 0) { - parsedObject.setRelevantdate( - dates.stream() - .map( - cd -> { - StructuredProperty date = new StructuredProperty(); - date.setValue(cd); - final Qualifier dq = new Qualifier(); - dq.setClassname("date"); - dq.setClassid("date"); - dq.setSchemename("dnet::date"); - dq.setSchemeid("dnet::date"); - date.setQualifier(dq); - return date; - }) - .collect(Collectors.toList())); - } + if (dates != null && dates.size() > 0) { + parsedObject + .setRelevantdate( + dates + .stream() + .map( + cd -> { + StructuredProperty date = new StructuredProperty(); + date.setValue(cd); + final Qualifier dq = new Qualifier(); + dq.setClassname("date"); + dq.setClassid("date"); + dq.setSchemename("dnet::date"); + dq.setSchemeid("dnet::date"); + date.setQualifier(dq); + return date; + }) + .collect(Collectors.toList())); + } - result.add(parsedObject); - return result; - } catch (Throwable e) { - log.error("Error on parsing record " + record, e); - return null; - } - } + result.add(parsedObject); + return result; + } catch (Throwable e) { + log.error("Error on parsing record " + record, e); + return null; + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java index 2f7d48417..edbb444db 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/graph/parser/PublicationScholexplorerParser.java @@ -1,243 +1,259 @@ + package eu.dnetlib.dhp.sx.graph.parser; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; + import com.ximpleware.AutoPilot; import com.ximpleware.VTDGen; import com.ximpleware.VTDNav; + import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; import eu.dnetlib.dhp.parser.utility.VtdUtilityParser.Node; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; import eu.dnetlib.dhp.schema.scholexplorer.ProvenaceInfo; import eu.dnetlib.scholexplorer.relation.RelationMapper; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; public class PublicationScholexplorerParser extends AbstractScholexplorerParser { - @Override - public List parseObject(final String record, final RelationMapper relationMapper) { - try { - final List result = new ArrayList<>(); - final DLIPublication parsedObject = new DLIPublication(); - final VTDGen vg = new VTDGen(); - vg.setDoc(record.getBytes()); - vg.parse(true); + @Override + public List parseObject(final String record, final RelationMapper relationMapper) { + try { + final List result = new ArrayList<>(); + final DLIPublication parsedObject = new DLIPublication(); + final VTDGen vg = new VTDGen(); + vg.setDoc(record.getBytes()); + vg.parse(true); - final VTDNav vn = vg.getNav(); - final AutoPilot ap = new AutoPilot(vn); + final VTDNav vn = vg.getNav(); + final AutoPilot ap = new AutoPilot(vn); - final DataInfo di = new DataInfo(); - di.setTrust("0.9"); - di.setDeletedbyinference(false); - di.setInvisible(false); + final DataInfo di = new DataInfo(); + di.setTrust("0.9"); + di.setDeletedbyinference(false); + di.setInvisible(false); - String dateOfCollection = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); - parsedObject.setDateofcollection(dateOfCollection); + String dateOfCollection = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='dateOfCollection']"); + parsedObject.setDateofcollection(dateOfCollection); - final String resolvedDate = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); - parsedObject.setOriginalId( - Collections.singletonList( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); + final String resolvedDate = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='resolvedDate']"); + parsedObject + .setOriginalId( + Collections + .singletonList( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='recordIdentifier']"))); - if (StringUtils.isNotBlank(resolvedDate)) { - StructuredProperty currentDate = new StructuredProperty(); - currentDate.setValue(resolvedDate); - final Qualifier dateQualifier = new Qualifier(); - dateQualifier.setClassname("resolvedDate"); - dateQualifier.setClassid("resolvedDate"); - dateQualifier.setSchemename("dnet::date"); - dateQualifier.setSchemeid("dnet::date"); - currentDate.setQualifier(dateQualifier); - parsedObject.setRelevantdate(Collections.singletonList(currentDate)); - } + if (StringUtils.isNotBlank(resolvedDate)) { + StructuredProperty currentDate = new StructuredProperty(); + currentDate.setValue(resolvedDate); + final Qualifier dateQualifier = new Qualifier(); + dateQualifier.setClassname("resolvedDate"); + dateQualifier.setClassid("resolvedDate"); + dateQualifier.setSchemename("dnet::date"); + dateQualifier.setSchemeid("dnet::date"); + currentDate.setQualifier(dateQualifier); + parsedObject.setRelevantdate(Collections.singletonList(currentDate)); + } - final List pid = - VtdUtilityParser.getTextValuesWithAttributes( - ap, vn, "//*[local-name()='pid']", Arrays.asList("type")); + final List pid = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//*[local-name()='pid']", Arrays.asList("type")); - StructuredProperty currentPid = extractIdentifier(pid, "type"); - if (currentPid == null) return null; - inferPid(currentPid); - parsedObject.setPid(Collections.singletonList(currentPid)); - final String sourceId = - generateId(currentPid.getValue(), currentPid.getQualifier().getClassid(), "publication"); - parsedObject.setId(sourceId); + StructuredProperty currentPid = extractIdentifier(pid, "type"); + if (currentPid == null) + return null; + inferPid(currentPid); + parsedObject.setPid(Collections.singletonList(currentPid)); + final String sourceId = generateId( + currentPid.getValue(), currentPid.getQualifier().getClassid(), "publication"); + parsedObject.setId(sourceId); - parsedObject.setOriginalObjIdentifier( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']")); + parsedObject + .setOriginalObjIdentifier( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='objIdentifier']")); - String provisionMode = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']"); + String provisionMode = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='provisionMode']"); - List collectedFromNodes = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='collectedFrom']", - Arrays.asList("name", "id", "mode", "completionStatus")); + List collectedFromNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='collectedFrom']", + Arrays.asList("name", "id", "mode", "completionStatus")); - List resolvededFromNodes = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='resolvedFrom']", - Arrays.asList("name", "id", "mode", "completionStatus")); + List resolvededFromNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='resolvedFrom']", + Arrays.asList("name", "id", "mode", "completionStatus")); - final String publisher = - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='publisher']"); - Field pf = new Field<>(); - pf.setValue(publisher); + final String publisher = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='publisher']"); + Field pf = new Field<>(); + pf.setValue(publisher); - parsedObject.setPublisher(pf); - final List provenances = new ArrayList<>(); - if (collectedFromNodes != null && collectedFromNodes.size() > 0) { - collectedFromNodes.forEach( - it -> { - final ProvenaceInfo provenance = new ProvenaceInfo(); - provenance.setId(it.getAttributes().get("id")); - provenance.setName(it.getAttributes().get("name")); - provenance.setCollectionMode(provisionMode); - provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); - provenances.add(provenance); - }); - } + parsedObject.setPublisher(pf); + final List provenances = new ArrayList<>(); + if (collectedFromNodes != null && collectedFromNodes.size() > 0) { + collectedFromNodes + .forEach( + it -> { + final ProvenaceInfo provenance = new ProvenaceInfo(); + provenance.setId(it.getAttributes().get("id")); + provenance.setName(it.getAttributes().get("name")); + provenance.setCollectionMode(provisionMode); + provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); + provenances.add(provenance); + }); + } - if (resolvededFromNodes != null && resolvededFromNodes.size() > 0) { - resolvededFromNodes.forEach( - it -> { - final ProvenaceInfo provenance = new ProvenaceInfo(); - provenance.setId(it.getAttributes().get("id")); - provenance.setName(it.getAttributes().get("name")); - provenance.setCollectionMode("resolved"); - provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); - provenances.add(provenance); - }); - } + if (resolvededFromNodes != null && resolvededFromNodes.size() > 0) { + resolvededFromNodes + .forEach( + it -> { + final ProvenaceInfo provenance = new ProvenaceInfo(); + provenance.setId(it.getAttributes().get("id")); + provenance.setName(it.getAttributes().get("name")); + provenance.setCollectionMode("resolved"); + provenance.setCompletionStatus(it.getAttributes().get("completionStatus")); + provenances.add(provenance); + }); + } - parsedObject.setDlicollectedfrom(provenances); - parsedObject.setCompletionStatus( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='completionStatus']")); + parsedObject.setDlicollectedfrom(provenances); + parsedObject + .setCompletionStatus( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='completionStatus']")); - parsedObject.setCollectedfrom( - parsedObject.getDlicollectedfrom().stream() - .map( - p -> { - final KeyValue cf = new KeyValue(); - cf.setKey(p.getId()); - cf.setValue(p.getName()); - return cf; - }) - .collect(Collectors.toList())); + parsedObject + .setCollectedfrom( + parsedObject + .getDlicollectedfrom() + .stream() + .map( + p -> { + final KeyValue cf = new KeyValue(); + cf.setKey(p.getId()); + cf.setValue(p.getName()); + return cf; + }) + .collect(Collectors.toList())); - final List relatedIdentifiers = - VtdUtilityParser.getTextValuesWithAttributes( - ap, - vn, - "//*[local-name()='relatedIdentifier']", - Arrays.asList( - "relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); - generateRelations( - relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); + final List relatedIdentifiers = VtdUtilityParser + .getTextValuesWithAttributes( + ap, + vn, + "//*[local-name()='relatedIdentifier']", + Arrays + .asList( + "relatedIdentifierType", "relationType", "entityType", "inverseRelationType")); + generateRelations( + relationMapper, parsedObject, result, di, dateOfCollection, relatedIdentifiers); - final List hostedBy = - VtdUtilityParser.getTextValuesWithAttributes( - ap, vn, "//*[local-name()='hostedBy']", Arrays.asList("id", "name")); + final List hostedBy = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//*[local-name()='hostedBy']", Arrays.asList("id", "name")); - if (hostedBy != null) { - parsedObject.setInstance( - hostedBy.stream() - .map( - it -> { - final Instance i = new Instance(); - i.setUrl(Collections.singletonList(currentPid.getValue())); - KeyValue h = new KeyValue(); - i.setHostedby(h); - h.setKey(it.getAttributes().get("id")); - h.setValue(it.getAttributes().get("name")); - return i; - }) - .collect(Collectors.toList())); - } + if (hostedBy != null) { + parsedObject + .setInstance( + hostedBy + .stream() + .map( + it -> { + final Instance i = new Instance(); + i.setUrl(Collections.singletonList(currentPid.getValue())); + KeyValue h = new KeyValue(); + i.setHostedby(h); + h.setKey(it.getAttributes().get("id")); + h.setValue(it.getAttributes().get("name")); + return i; + }) + .collect(Collectors.toList())); + } - final List authorsNode = - VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='creator']"); - if (authorsNode != null) - parsedObject.setAuthor( - authorsNode.stream() - .map( - a -> { - final Author author = new Author(); - author.setFullname(a); - return author; - }) - .collect(Collectors.toList())); + final List authorsNode = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='creator']"); + if (authorsNode != null) + parsedObject + .setAuthor( + authorsNode + .stream() + .map( + a -> { + final Author author = new Author(); + author.setFullname(a); + return author; + }) + .collect(Collectors.toList())); - final List titles = - VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='title']"); - if (titles != null) { - parsedObject.setTitle( - titles.stream() - .map( - t -> { - final StructuredProperty st = new StructuredProperty(); - st.setValue(t); - return st; - }) - .collect(Collectors.toList())); - } + final List titles = VtdUtilityParser.getTextValue(ap, vn, "//*[local-name()='title']"); + if (titles != null) { + parsedObject + .setTitle( + titles + .stream() + .map( + t -> { + final StructuredProperty st = new StructuredProperty(); + st.setValue(t); + return st; + }) + .collect(Collectors.toList())); + } - Field description = new Field<>(); + Field description = new Field<>(); - description.setValue( - VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='description']")); + description + .setValue( + VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='description']")); - if (StringUtils.isNotBlank(description.getValue()) - && description.getValue().length() > 10000) { - description.setValue(description.getValue().substring(0, 10000)); - } + if (StringUtils.isNotBlank(description.getValue()) + && description.getValue().length() > 10000) { + description.setValue(description.getValue().substring(0, 10000)); + } - parsedObject.setDescription(Collections.singletonList(description)); + parsedObject.setDescription(Collections.singletonList(description)); - final String cd = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='date']"); + final String cd = VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='date']"); - StructuredProperty date = new StructuredProperty(); - date.setValue(cd); - final Qualifier dq = new Qualifier(); - dq.setClassname("date"); - dq.setClassid("date"); - dq.setSchemename("dnet::date"); - dq.setSchemeid("dnet::date"); - date.setQualifier(dq); - parsedObject.setRelevantdate(Collections.singletonList(date)); + StructuredProperty date = new StructuredProperty(); + date.setValue(cd); + final Qualifier dq = new Qualifier(); + dq.setClassname("date"); + dq.setClassid("date"); + dq.setSchemename("dnet::date"); + dq.setSchemeid("dnet::date"); + date.setQualifier(dq); + parsedObject.setRelevantdate(Collections.singletonList(date)); - List subjects = - extractSubject( - VtdUtilityParser.getTextValuesWithAttributes( - ap, vn, "//*[local-name()='subject']", Collections.singletonList("scheme"))); - parsedObject.setSubject(subjects); + List subjects = extractSubject( + VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//*[local-name()='subject']", Collections.singletonList("scheme"))); + parsedObject.setSubject(subjects); - parsedObject.setDataInfo(di); + parsedObject.setDataInfo(di); - parsedObject.setSubject(subjects); - Qualifier q = new Qualifier(); - q.setClassname("publication"); - q.setClassid("publication"); - q.setSchemename("publication"); - q.setSchemeid("publication"); - parsedObject.setResulttype(q); - result.add(parsedObject); - return result; + parsedObject.setSubject(subjects); + Qualifier q = new Qualifier(); + q.setClassname("publication"); + q.setClassid("publication"); + q.setSchemename("publication"); + q.setSchemeid("publication"); + parsedObject.setResulttype(q); + result.add(parsedObject); + return result; - } catch (Throwable e) { - log.error("Input record: " + record); - log.error("Error on parsing record ", e); - return null; - } - } + } catch (Throwable e) { + log.error("Input record: " + record); + log.error("Error on parsing record ", e); + return null; + } + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java index 06d9d1e8a..e95174670 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/GraphHiveImporterJobTest.java @@ -1,10 +1,10 @@ + package eu.dnetlib.dhp.oa.graph; -import eu.dnetlib.dhp.oa.graph.hive.GraphHiveImporterJob; -import eu.dnetlib.dhp.schema.common.ModelSupport; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.RandomStringUtils; import org.apache.spark.SparkConf; @@ -16,76 +16,82 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.oa.graph.hive.GraphHiveImporterJob; +import eu.dnetlib.dhp.schema.common.ModelSupport; + public class GraphHiveImporterJobTest { - private static final Logger log = LoggerFactory.getLogger(GraphHiveImporterJobTest.class); + private static final Logger log = LoggerFactory.getLogger(GraphHiveImporterJobTest.class); - public static final String JDBC_DERBY_TEMPLATE = - "jdbc:derby:;databaseName=%s/junit_metastore_db;create=true"; + public static final String JDBC_DERBY_TEMPLATE = "jdbc:derby:;databaseName=%s/junit_metastore_db;create=true"; - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - private static String dbName; + private static String dbName; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(GraphHiveImporterJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(GraphHiveImporterJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); - dbName = RandomStringUtils.randomAlphabetic(5); - log.info("using DB name {}", "test_" + dbName); + dbName = RandomStringUtils.randomAlphabetic(5); + log.info("using DB name {}", "test_" + dbName); - SparkConf conf = new SparkConf(); - conf.setAppName(GraphHiveImporterJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(GraphHiveImporterJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - conf.set( - "javax.jdo.option.ConnectionURL", - String.format(JDBC_DERBY_TEMPLATE, workingDir.resolve("warehouse").toString())); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf + .set( + "javax.jdo.option.ConnectionURL", + String.format(JDBC_DERBY_TEMPLATE, workingDir.resolve("warehouse").toString())); - spark = - SparkSession.builder() - .appName(GraphHiveImporterJobTest.class.getSimpleName()) - .config(conf) - .enableHiveSupport() - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(GraphHiveImporterJobTest.class.getSimpleName()) + .config(conf) + .enableHiveSupport() + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - @Test - public void testImportGraphAsHiveDB() throws Exception { + @Test + public void testImportGraphAsHiveDB() throws Exception { - GraphHiveImporterJob.main( - new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-inputPath", - getClass().getResource("/eu/dnetlib/dhp/oa/graph/sample").getPath(), - "-hiveMetastoreUris", - "", - "-hiveDbName", - dbName - }); + GraphHiveImporterJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-inputPath", + getClass().getResource("/eu/dnetlib/dhp/oa/graph/sample").getPath(), + "-hiveMetastoreUris", + "", + "-hiveDbName", + dbName + }); - ModelSupport.oafTypes.forEach( - (name, clazz) -> { - long count = spark.read().table(dbName + "." + name).count(); - int expected = name.equals("relation") ? 100 : 10; + ModelSupport.oafTypes + .forEach( + (name, clazz) -> { + long count = spark.read().table(dbName + "." + name).count(); + int expected = name.equals("relation") ? 100 : 10; - Assertions.assertEquals( - expected, count, String.format("%s should be %s", name, expected)); - }); - } + Assertions + .assertEquals( + expected, count, String.format("%s should be %s", name, expected)); + }); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 89740718b..951c97d9d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -6,14 +7,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.when; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; import java.io.IOException; import java.util.List; import java.util.Map; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.BeforeEach; @@ -22,124 +19,131 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Software; + @ExtendWith(MockitoExtension.class) public class MappersTest { - @Mock private Map code2name; + @Mock + private Map code2name; - @BeforeEach - public void setUp() throws Exception { - when(code2name.get(anyString())).thenAnswer(invocation -> invocation.getArgument(0)); - } + @BeforeEach + public void setUp() throws Exception { + when(code2name.get(anyString())).thenAnswer(invocation -> invocation.getArgument(0)); + } - @Test - void testPublication() throws IOException { - final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_record.xml")); + @Test + void testPublication() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_record.xml")); - final List list = new OafToOafMapper(code2name).processMdRecord(xml); + final List list = new OafToOafMapper(code2name).processMdRecord(xml); - assertEquals(3, list.size()); - assertTrue(list.get(0) instanceof Publication); - assertTrue(list.get(1) instanceof Relation); - assertTrue(list.get(2) instanceof Relation); + assertEquals(3, list.size()); + assertTrue(list.get(0) instanceof Publication); + assertTrue(list.get(1) instanceof Relation); + assertTrue(list.get(2) instanceof Relation); - final Publication p = (Publication) list.get(0); - final Relation r1 = (Relation) list.get(1); - final Relation r2 = (Relation) list.get(2); + final Publication p = (Publication) list.get(0); + final Relation r1 = (Relation) list.get(1); + final Relation r2 = (Relation) list.get(2); - assertValidId(p.getId()); - assertValidId(p.getCollectedfrom().get(0).getKey()); - assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); - assertTrue(p.getAuthor().size() > 0); - assertTrue(p.getSubject().size() > 0); - assertTrue(StringUtils.isNotBlank(p.getJournal().getIssnOnline())); - assertTrue(StringUtils.isNotBlank(p.getJournal().getName())); - assertTrue(p.getInstance().size() > 0); + assertValidId(p.getId()); + assertValidId(p.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); + assertTrue(p.getAuthor().size() > 0); + assertTrue(p.getSubject().size() > 0); + assertTrue(StringUtils.isNotBlank(p.getJournal().getIssnOnline())); + assertTrue(StringUtils.isNotBlank(p.getJournal().getName())); + assertTrue(p.getInstance().size() > 0); - assertValidId(r1.getSource()); - assertValidId(r1.getTarget()); - assertValidId(r2.getSource()); - assertValidId(r2.getTarget()); - assertValidId(r1.getCollectedfrom().get(0).getKey()); - assertValidId(r2.getCollectedfrom().get(0).getKey()); - assertNotNull(r1.getDataInfo()); - assertNotNull(r2.getDataInfo()); - assertNotNull(r1.getDataInfo().getTrust()); - assertNotNull(r2.getDataInfo().getTrust()); - assertEquals(r1.getSource(), r2.getTarget()); - assertEquals(r2.getSource(), r1.getTarget()); - assertTrue(StringUtils.isNotBlank(r1.getRelClass())); - assertTrue(StringUtils.isNotBlank(r2.getRelClass())); - assertTrue(StringUtils.isNotBlank(r1.getRelType())); - assertTrue(StringUtils.isNotBlank(r2.getRelType())); + assertValidId(r1.getSource()); + assertValidId(r1.getTarget()); + assertValidId(r2.getSource()); + assertValidId(r2.getTarget()); + assertValidId(r1.getCollectedfrom().get(0).getKey()); + assertValidId(r2.getCollectedfrom().get(0).getKey()); + assertNotNull(r1.getDataInfo()); + assertNotNull(r2.getDataInfo()); + assertNotNull(r1.getDataInfo().getTrust()); + assertNotNull(r2.getDataInfo().getTrust()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + assertTrue(StringUtils.isNotBlank(r1.getRelClass())); + assertTrue(StringUtils.isNotBlank(r2.getRelClass())); + assertTrue(StringUtils.isNotBlank(r1.getRelType())); + assertTrue(StringUtils.isNotBlank(r2.getRelType())); - // System.out.println(new ObjectMapper().writeValueAsString(r1)); - // System.out.println(new ObjectMapper().writeValueAsString(r2)); - } + // System.out.println(new ObjectMapper().writeValueAsString(r1)); + // System.out.println(new ObjectMapper().writeValueAsString(r2)); + } - @Test - void testDataset() throws IOException { - final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_dataset.xml")); + @Test + void testDataset() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_dataset.xml")); - final List list = new OdfToOafMapper(code2name).processMdRecord(xml); + final List list = new OdfToOafMapper(code2name).processMdRecord(xml); - assertEquals(3, list.size()); - assertTrue(list.get(0) instanceof Dataset); - assertTrue(list.get(1) instanceof Relation); - assertTrue(list.get(2) instanceof Relation); + assertEquals(3, list.size()); + assertTrue(list.get(0) instanceof Dataset); + assertTrue(list.get(1) instanceof Relation); + assertTrue(list.get(2) instanceof Relation); - final Dataset d = (Dataset) list.get(0); - final Relation r1 = (Relation) list.get(1); - final Relation r2 = (Relation) list.get(2); + final Dataset d = (Dataset) list.get(0); + final Relation r1 = (Relation) list.get(1); + final Relation r2 = (Relation) list.get(2); - assertValidId(d.getId()); - assertValidId(d.getCollectedfrom().get(0).getKey()); - assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); - assertTrue(d.getAuthor().size() > 0); - assertTrue(d.getSubject().size() > 0); - assertTrue(d.getInstance().size() > 0); - assertTrue(d.getContext().size() > 0); - assertTrue(d.getContext().get(0).getId().length() > 0); + assertValidId(d.getId()); + assertValidId(d.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); + assertTrue(d.getAuthor().size() > 0); + assertTrue(d.getSubject().size() > 0); + assertTrue(d.getInstance().size() > 0); + assertTrue(d.getContext().size() > 0); + assertTrue(d.getContext().get(0).getId().length() > 0); - assertValidId(r1.getSource()); - assertValidId(r1.getTarget()); - assertValidId(r2.getSource()); - assertValidId(r2.getTarget()); - assertNotNull(r1.getDataInfo()); - assertNotNull(r2.getDataInfo()); - assertNotNull(r1.getDataInfo().getTrust()); - assertNotNull(r2.getDataInfo().getTrust()); - assertEquals(r1.getSource(), r2.getTarget()); - assertEquals(r2.getSource(), r1.getTarget()); - assertTrue(StringUtils.isNotBlank(r1.getRelClass())); - assertTrue(StringUtils.isNotBlank(r2.getRelClass())); - assertTrue(StringUtils.isNotBlank(r1.getRelType())); - assertTrue(StringUtils.isNotBlank(r2.getRelType())); - } + assertValidId(r1.getSource()); + assertValidId(r1.getTarget()); + assertValidId(r2.getSource()); + assertValidId(r2.getTarget()); + assertNotNull(r1.getDataInfo()); + assertNotNull(r2.getDataInfo()); + assertNotNull(r1.getDataInfo().getTrust()); + assertNotNull(r2.getDataInfo().getTrust()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + assertTrue(StringUtils.isNotBlank(r1.getRelClass())); + assertTrue(StringUtils.isNotBlank(r2.getRelClass())); + assertTrue(StringUtils.isNotBlank(r1.getRelType())); + assertTrue(StringUtils.isNotBlank(r2.getRelType())); + } - @Test - void testSoftware() throws IOException { - final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_software.xml")); + @Test + void testSoftware() throws IOException { + final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_software.xml")); - final List list = new OdfToOafMapper(code2name).processMdRecord(xml); + final List list = new OdfToOafMapper(code2name).processMdRecord(xml); - assertEquals(1, list.size()); - assertTrue(list.get(0) instanceof Software); + assertEquals(1, list.size()); + assertTrue(list.get(0) instanceof Software); - final Software s = (Software) list.get(0); + final Software s = (Software) list.get(0); - assertValidId(s.getId()); - assertValidId(s.getCollectedfrom().get(0).getKey()); - assertTrue(StringUtils.isNotBlank(s.getTitle().get(0).getValue())); - assertTrue(s.getAuthor().size() > 0); - assertTrue(s.getSubject().size() > 0); - assertTrue(s.getInstance().size() > 0); - } + assertValidId(s.getId()); + assertValidId(s.getCollectedfrom().get(0).getKey()); + assertTrue(StringUtils.isNotBlank(s.getTitle().get(0).getValue())); + assertTrue(s.getAuthor().size() > 0); + assertTrue(s.getSubject().size() > 0); + assertTrue(s.getInstance().size() > 0); + } - private void assertValidId(final String id) { - assertEquals(49, id.length()); - assertEquals('|', id.charAt(2)); - assertEquals(':', id.charAt(15)); - assertEquals(':', id.charAt(16)); - } + private void assertValidId(final String id) { + assertEquals(49, id.length()); + assertEquals('|', id.charAt(2)); + assertEquals(':', id.charAt(15)); + assertEquals(':', id.charAt(16)); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java index 0d3a273ec..1bbe57ee8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java @@ -1,17 +1,10 @@ + package eu.dnetlib.dhp.oa.graph.raw; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.Organization; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import java.io.IOException; import java.sql.Array; import java.sql.Date; @@ -19,6 +12,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; import java.util.Objects; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.BeforeEach; @@ -28,316 +22,332 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Datasource; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.Organization; +import eu.dnetlib.dhp.schema.oaf.Project; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; + @ExtendWith(MockitoExtension.class) public class MigrateDbEntitiesApplicationTest { - private MigrateDbEntitiesApplication app; + private MigrateDbEntitiesApplication app; - @Mock private ResultSet rs; + @Mock + private ResultSet rs; - @BeforeEach - public void setUp() { - this.app = new MigrateDbEntitiesApplication(); - } + @BeforeEach + public void setUp() { + this.app = new MigrateDbEntitiesApplication(); + } - @Test - public void testProcessDatasource() throws Exception { - final List fields = prepareMocks("datasources_resultset_entry.json"); + @Test + public void testProcessDatasource() throws Exception { + final List fields = prepareMocks("datasources_resultset_entry.json"); - final List list = app.processDatasource(rs); - assertEquals(1, list.size()); - verifyMocks(fields); + final List list = app.processDatasource(rs); + assertEquals(1, list.size()); + verifyMocks(fields); - final Datasource ds = (Datasource) list.get(0); - assertValidId(ds.getId()); - assertValidId(ds.getCollectedfrom().get(0).getKey()); - assertEquals(ds.getOfficialname().getValue(), getValueAsString("officialname", fields)); - assertEquals(ds.getEnglishname().getValue(), getValueAsString("englishname", fields)); - assertEquals(ds.getContactemail().getValue(), getValueAsString("contactemail", fields)); - assertEquals(ds.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields)); - assertEquals(ds.getNamespaceprefix().getValue(), getValueAsString("namespaceprefix", fields)); - assertEquals( - ds.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); - } + final Datasource ds = (Datasource) list.get(0); + assertValidId(ds.getId()); + assertValidId(ds.getCollectedfrom().get(0).getKey()); + assertEquals(ds.getOfficialname().getValue(), getValueAsString("officialname", fields)); + assertEquals(ds.getEnglishname().getValue(), getValueAsString("englishname", fields)); + assertEquals(ds.getContactemail().getValue(), getValueAsString("contactemail", fields)); + assertEquals(ds.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields)); + assertEquals(ds.getNamespaceprefix().getValue(), getValueAsString("namespaceprefix", fields)); + assertEquals( + ds.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); + } - @Test - public void testProcessProject() throws Exception { - final List fields = prepareMocks("projects_resultset_entry.json"); + @Test + public void testProcessProject() throws Exception { + final List fields = prepareMocks("projects_resultset_entry.json"); - final List list = app.processProject(rs); - assertEquals(1, list.size()); - verifyMocks(fields); + final List list = app.processProject(rs); + assertEquals(1, list.size()); + verifyMocks(fields); - final Project p = (Project) list.get(0); - assertValidId(p.getId()); - assertValidId(p.getCollectedfrom().get(0).getKey()); - assertEquals(p.getAcronym().getValue(), getValueAsString("acronym", fields)); - assertEquals(p.getTitle().getValue(), getValueAsString("title", fields)); - assertEquals( - p.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); - } + final Project p = (Project) list.get(0); + assertValidId(p.getId()); + assertValidId(p.getCollectedfrom().get(0).getKey()); + assertEquals(p.getAcronym().getValue(), getValueAsString("acronym", fields)); + assertEquals(p.getTitle().getValue(), getValueAsString("title", fields)); + assertEquals( + p.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); + } - @Test - public void testProcessOrganization() throws Exception { - final List fields = prepareMocks("organizations_resultset_entry.json"); + @Test + public void testProcessOrganization() throws Exception { + final List fields = prepareMocks("organizations_resultset_entry.json"); - final List list = app.processOrganization(rs); + final List list = app.processOrganization(rs); - assertEquals(1, list.size()); + assertEquals(1, list.size()); - verifyMocks(fields); + verifyMocks(fields); - final Organization o = (Organization) list.get(0); - assertValidId(o.getId()); - assertValidId(o.getCollectedfrom().get(0).getKey()); - assertEquals(o.getLegalshortname().getValue(), getValueAsString("legalshortname", fields)); - assertEquals(o.getLegalname().getValue(), getValueAsString("legalname", fields)); - assertEquals(o.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields)); - assertEquals(o.getCountry().getClassid(), getValueAsString("country", fields).split("@@@")[0]); - assertEquals( - o.getCountry().getClassname(), getValueAsString("country", fields).split("@@@")[1]); - assertEquals(o.getCountry().getSchemeid(), getValueAsString("country", fields).split("@@@")[2]); - assertEquals( - o.getCountry().getSchemename(), getValueAsString("country", fields).split("@@@")[3]); - assertEquals( - o.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); - } + final Organization o = (Organization) list.get(0); + assertValidId(o.getId()); + assertValidId(o.getCollectedfrom().get(0).getKey()); + assertEquals(o.getLegalshortname().getValue(), getValueAsString("legalshortname", fields)); + assertEquals(o.getLegalname().getValue(), getValueAsString("legalname", fields)); + assertEquals(o.getWebsiteurl().getValue(), getValueAsString("websiteurl", fields)); + assertEquals(o.getCountry().getClassid(), getValueAsString("country", fields).split("@@@")[0]); + assertEquals( + o.getCountry().getClassname(), getValueAsString("country", fields).split("@@@")[1]); + assertEquals(o.getCountry().getSchemeid(), getValueAsString("country", fields).split("@@@")[2]); + assertEquals( + o.getCountry().getSchemename(), getValueAsString("country", fields).split("@@@")[3]); + assertEquals( + o.getCollectedfrom().get(0).getValue(), getValueAsString("collectedfromname", fields)); + } - @Test - public void testProcessDatasourceOrganization() throws Exception { - final List fields = prepareMocks("datasourceorganization_resultset_entry.json"); + @Test + public void testProcessDatasourceOrganization() throws Exception { + final List fields = prepareMocks("datasourceorganization_resultset_entry.json"); - final List list = app.processDatasourceOrganization(rs); + final List list = app.processDatasourceOrganization(rs); - assertEquals(2, list.size()); - verifyMocks(fields); + assertEquals(2, list.size()); + verifyMocks(fields); - final Relation r1 = (Relation) list.get(0); - final Relation r2 = (Relation) list.get(1); - assertValidId(r1.getSource()); - assertValidId(r2.getSource()); - assertEquals(r1.getSource(), r2.getTarget()); - assertEquals(r2.getSource(), r1.getTarget()); - } + final Relation r1 = (Relation) list.get(0); + final Relation r2 = (Relation) list.get(1); + assertValidId(r1.getSource()); + assertValidId(r2.getSource()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + } - @Test - public void testProcessProjectOrganization() throws Exception { - final List fields = prepareMocks("projectorganization_resultset_entry.json"); + @Test + public void testProcessProjectOrganization() throws Exception { + final List fields = prepareMocks("projectorganization_resultset_entry.json"); - final List list = app.processProjectOrganization(rs); + final List list = app.processProjectOrganization(rs); - assertEquals(2, list.size()); - verifyMocks(fields); + assertEquals(2, list.size()); + verifyMocks(fields); - final Relation r1 = (Relation) list.get(0); - final Relation r2 = (Relation) list.get(1); - assertValidId(r1.getSource()); - assertValidId(r2.getSource()); - assertEquals(r1.getSource(), r2.getTarget()); - assertEquals(r2.getSource(), r1.getTarget()); - assertValidId(r1.getCollectedfrom().get(0).getKey()); - assertValidId(r2.getCollectedfrom().get(0).getKey()); - } + final Relation r1 = (Relation) list.get(0); + final Relation r2 = (Relation) list.get(1); + assertValidId(r1.getSource()); + assertValidId(r2.getSource()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + assertValidId(r1.getCollectedfrom().get(0).getKey()); + assertValidId(r2.getCollectedfrom().get(0).getKey()); + } - @Test - public void testProcessClaims_context() throws Exception { - final List fields = prepareMocks("claimscontext_resultset_entry.json"); + @Test + public void testProcessClaims_context() throws Exception { + final List fields = prepareMocks("claimscontext_resultset_entry.json"); - final List list = app.processClaims(rs); + final List list = app.processClaims(rs); - assertEquals(1, list.size()); - assertTrue(list.get(0) instanceof Result); - final Result r = (Result) list.get(0); + assertEquals(1, list.size()); + assertTrue(list.get(0) instanceof Result); + final Result r = (Result) list.get(0); - verifyMocks(fields); + verifyMocks(fields); - assertValidId(r.getCollectedfrom().get(0).getKey()); - } + assertValidId(r.getCollectedfrom().get(0).getKey()); + } - @Test - public void testProcessClaims_rels() throws Exception { - final List fields = prepareMocks("claimsrel_resultset_entry.json"); + @Test + public void testProcessClaims_rels() throws Exception { + final List fields = prepareMocks("claimsrel_resultset_entry.json"); - final List list = app.processClaims(rs); + final List list = app.processClaims(rs); - assertEquals(2, list.size()); - verifyMocks(fields); + assertEquals(2, list.size()); + verifyMocks(fields); - assertTrue(list.get(0) instanceof Relation); - assertTrue(list.get(1) instanceof Relation); + assertTrue(list.get(0) instanceof Relation); + assertTrue(list.get(1) instanceof Relation); - final Relation r1 = (Relation) list.get(0); - final Relation r2 = (Relation) list.get(1); + final Relation r1 = (Relation) list.get(0); + final Relation r2 = (Relation) list.get(1); - assertValidId(r1.getSource()); - assertValidId(r1.getTarget()); - assertValidId(r2.getSource()); - assertValidId(r2.getTarget()); - assertNotNull(r1.getDataInfo()); - assertNotNull(r2.getDataInfo()); - assertNotNull(r1.getDataInfo().getTrust()); - assertNotNull(r2.getDataInfo().getTrust()); - assertEquals(r1.getSource(), r2.getTarget()); - assertEquals(r2.getSource(), r1.getTarget()); - assertTrue(StringUtils.isNotBlank(r1.getRelClass())); - assertTrue(StringUtils.isNotBlank(r2.getRelClass())); - assertTrue(StringUtils.isNotBlank(r1.getRelType())); - assertTrue(StringUtils.isNotBlank(r2.getRelType())); + assertValidId(r1.getSource()); + assertValidId(r1.getTarget()); + assertValidId(r2.getSource()); + assertValidId(r2.getTarget()); + assertNotNull(r1.getDataInfo()); + assertNotNull(r2.getDataInfo()); + assertNotNull(r1.getDataInfo().getTrust()); + assertNotNull(r2.getDataInfo().getTrust()); + assertEquals(r1.getSource(), r2.getTarget()); + assertEquals(r2.getSource(), r1.getTarget()); + assertTrue(StringUtils.isNotBlank(r1.getRelClass())); + assertTrue(StringUtils.isNotBlank(r2.getRelClass())); + assertTrue(StringUtils.isNotBlank(r1.getRelType())); + assertTrue(StringUtils.isNotBlank(r2.getRelType())); - assertValidId(r1.getCollectedfrom().get(0).getKey()); - assertValidId(r2.getCollectedfrom().get(0).getKey()); + assertValidId(r1.getCollectedfrom().get(0).getKey()); + assertValidId(r2.getCollectedfrom().get(0).getKey()); - // System.out.println(new ObjectMapper().writeValueAsString(r1)); - // System.out.println(new ObjectMapper().writeValueAsString(r2)); - } + // System.out.println(new ObjectMapper().writeValueAsString(r1)); + // System.out.println(new ObjectMapper().writeValueAsString(r2)); + } - private List prepareMocks(final String jsonFile) throws IOException, SQLException { - final String json = IOUtils.toString(getClass().getResourceAsStream(jsonFile)); - final ObjectMapper mapper = new ObjectMapper(); - final List list = mapper.readValue(json, new TypeReference>() {}); + private List prepareMocks(final String jsonFile) throws IOException, SQLException { + final String json = IOUtils.toString(getClass().getResourceAsStream(jsonFile)); + final ObjectMapper mapper = new ObjectMapper(); + final List list = mapper.readValue(json, new TypeReference>() { + }); - for (final TypedField tf : list) { - if (tf.getValue() == null) { - switch (tf.getType()) { - case "not_used": - break; - case "boolean": - Mockito.when(rs.getBoolean(tf.getField())).thenReturn(false); - break; - case "date": - Mockito.when(rs.getDate(tf.getField())).thenReturn(null); - break; - case "int": - Mockito.when(rs.getInt(tf.getField())).thenReturn(0); - break; - case "double": - Mockito.when(rs.getDouble(tf.getField())).thenReturn(0.0); - break; - case "array": - Mockito.when(rs.getArray(tf.getField())).thenReturn(null); - break; - case "string": - default: - Mockito.when(rs.getString(tf.getField())).thenReturn(null); - break; - } - } else { - switch (tf.getType()) { - case "not_used": - break; - case "boolean": - Mockito.when(rs.getBoolean(tf.getField())) - .thenReturn(Boolean.parseBoolean(tf.getValue().toString())); - break; - case "date": - Mockito.when(rs.getDate(tf.getField())) - .thenReturn(Date.valueOf(tf.getValue().toString())); - break; - case "int": - Mockito.when(rs.getInt(tf.getField())) - .thenReturn(new Integer(tf.getValue().toString())); - break; - case "double": - Mockito.when(rs.getDouble(tf.getField())) - .thenReturn(new Double(tf.getValue().toString())); - break; - case "array": - final Array arr = Mockito.mock(Array.class); - final String[] values = - ((List) tf.getValue()) - .stream() - .filter(Objects::nonNull) - .map(o -> o.toString()) - .toArray(String[]::new); + for (final TypedField tf : list) { + if (tf.getValue() == null) { + switch (tf.getType()) { + case "not_used": + break; + case "boolean": + Mockito.when(rs.getBoolean(tf.getField())).thenReturn(false); + break; + case "date": + Mockito.when(rs.getDate(tf.getField())).thenReturn(null); + break; + case "int": + Mockito.when(rs.getInt(tf.getField())).thenReturn(0); + break; + case "double": + Mockito.when(rs.getDouble(tf.getField())).thenReturn(0.0); + break; + case "array": + Mockito.when(rs.getArray(tf.getField())).thenReturn(null); + break; + case "string": + default: + Mockito.when(rs.getString(tf.getField())).thenReturn(null); + break; + } + } else { + switch (tf.getType()) { + case "not_used": + break; + case "boolean": + Mockito + .when(rs.getBoolean(tf.getField())) + .thenReturn(Boolean.parseBoolean(tf.getValue().toString())); + break; + case "date": + Mockito + .when(rs.getDate(tf.getField())) + .thenReturn(Date.valueOf(tf.getValue().toString())); + break; + case "int": + Mockito + .when(rs.getInt(tf.getField())) + .thenReturn(new Integer(tf.getValue().toString())); + break; + case "double": + Mockito + .when(rs.getDouble(tf.getField())) + .thenReturn(new Double(tf.getValue().toString())); + break; + case "array": + final Array arr = Mockito.mock(Array.class); + final String[] values = ((List) tf.getValue()) + .stream() + .filter(Objects::nonNull) + .map(o -> o.toString()) + .toArray(String[]::new); - Mockito.when(arr.getArray()).thenReturn(values); - Mockito.when(rs.getArray(tf.getField())).thenReturn(arr); - break; - case "string": - default: - Mockito.when(rs.getString(tf.getField())).thenReturn(tf.getValue().toString()); - break; - } - } - } + Mockito.when(arr.getArray()).thenReturn(values); + Mockito.when(rs.getArray(tf.getField())).thenReturn(arr); + break; + case "string": + default: + Mockito.when(rs.getString(tf.getField())).thenReturn(tf.getValue().toString()); + break; + } + } + } - return list; - } + return list; + } - private void verifyMocks(final List list) throws SQLException { - for (final TypedField tf : list) { + private void verifyMocks(final List list) throws SQLException { + for (final TypedField tf : list) { - switch (tf.getType()) { - case "not_used": - break; - case "boolean": - Mockito.verify(rs, Mockito.atLeastOnce()).getBoolean(tf.getField()); - break; - case "date": - Mockito.verify(rs, Mockito.atLeastOnce()).getDate(tf.getField()); - break; - case "int": - Mockito.verify(rs, Mockito.atLeastOnce()).getInt(tf.getField()); - break; - case "double": - Mockito.verify(rs, Mockito.atLeastOnce()).getDouble(tf.getField()); - break; - case "array": - Mockito.verify(rs, Mockito.atLeastOnce()).getArray(tf.getField()); - break; - case "string": - default: - Mockito.verify(rs, Mockito.atLeastOnce()).getString(tf.getField()); - break; - } - } - } + switch (tf.getType()) { + case "not_used": + break; + case "boolean": + Mockito.verify(rs, Mockito.atLeastOnce()).getBoolean(tf.getField()); + break; + case "date": + Mockito.verify(rs, Mockito.atLeastOnce()).getDate(tf.getField()); + break; + case "int": + Mockito.verify(rs, Mockito.atLeastOnce()).getInt(tf.getField()); + break; + case "double": + Mockito.verify(rs, Mockito.atLeastOnce()).getDouble(tf.getField()); + break; + case "array": + Mockito.verify(rs, Mockito.atLeastOnce()).getArray(tf.getField()); + break; + case "string": + default: + Mockito.verify(rs, Mockito.atLeastOnce()).getString(tf.getField()); + break; + } + } + } - private void assertValidId(final String id) { - assertEquals(49, id.length()); - assertEquals('|', id.charAt(2)); - assertEquals(':', id.charAt(15)); - assertEquals(':', id.charAt(16)); - } + private void assertValidId(final String id) { + assertEquals(49, id.length()); + assertEquals('|', id.charAt(2)); + assertEquals(':', id.charAt(15)); + assertEquals(':', id.charAt(16)); + } - private String getValueAsString(final String name, final List fields) { - return fields.stream() - .filter(f -> f.getField().equals(name)) - .map(TypedField::getValue) - .filter(Objects::nonNull) - .map(o -> o.toString()) - .findFirst() - .get(); - } + private String getValueAsString(final String name, final List fields) { + return fields + .stream() + .filter(f -> f.getField().equals(name)) + .map(TypedField::getValue) + .filter(Objects::nonNull) + .map(o -> o.toString()) + .findFirst() + .get(); + } } class TypedField { - private String field; - private String type; - private Object value; + private String field; + private String type; + private Object value; - public String getField() { - return field; - } + public String getField() { + return field; + } - public void setField(final String field) { - this.field = field; - } + public void setField(final String field) { + this.field = field; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(final String type) { - this.type = type; - } + public void setType(final String type) { + this.type = type; + } - public Object getValue() { - return value; - } + public Object getValue() { + return value; + } - public void setValue(final Object value) { - this.value = value; - } + public void setValue(final Object value) { + this.value = value; + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/ScholexplorerParserTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/ScholexplorerParserTest.java index f5ba4af55..d418da594 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/ScholexplorerParserTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/ScholexplorerParserTest.java @@ -1,35 +1,40 @@ + package eu.dnetlib.dhp.sx.graph; +import java.util.List; + +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.Test; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; + import eu.dnetlib.dhp.schema.oaf.Oaf; import eu.dnetlib.dhp.sx.graph.parser.DatasetScholexplorerParser; import eu.dnetlib.scholexplorer.relation.RelationMapper; -import java.util.List; -import org.apache.commons.io.IOUtils; -import org.junit.jupiter.api.Test; public class ScholexplorerParserTest { - @Test - public void testDataciteParser() throws Exception { - String xml = IOUtils.toString(this.getClass().getResourceAsStream("dmf.xml")); + @Test + public void testDataciteParser() throws Exception { + String xml = IOUtils.toString(this.getClass().getResourceAsStream("dmf.xml")); - DatasetScholexplorerParser p = new DatasetScholexplorerParser(); - List oaves = p.parseObject(xml, RelationMapper.load()); + DatasetScholexplorerParser p = new DatasetScholexplorerParser(); + List oaves = p.parseObject(xml, RelationMapper.load()); - ObjectMapper m = new ObjectMapper(); - m.enable(SerializationFeature.INDENT_OUTPUT); + ObjectMapper m = new ObjectMapper(); + m.enable(SerializationFeature.INDENT_OUTPUT); - oaves.forEach( - oaf -> { - try { - System.out.println(m.writeValueAsString(oaf)); - System.out.println("----------------------------"); - } catch (JsonProcessingException e) { + oaves + .forEach( + oaf -> { + try { + System.out.println(m.writeValueAsString(oaf)); + System.out.println("----------------------------"); + } catch (JsonProcessingException e) { - } - }); - } + } + }); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java index 7f32de318..ed3b6efdc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java @@ -1,3 +1,5 @@ + package eu.dnetlib.dhp.sx.graph; -public class SparkScholexplorerGraphImporterTest {} +public class SparkScholexplorerGraphImporterTest { +} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerMergeEntitiesJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerMergeEntitiesJobTest.java index af6385803..348a2b030 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerMergeEntitiesJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerMergeEntitiesJobTest.java @@ -1,3 +1,5 @@ + package eu.dnetlib.dhp.sx.graph; -public class SparkScholexplorerMergeEntitiesJobTest {} +public class SparkScholexplorerMergeEntitiesJobTest { +} diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java index f9756c88b..1b0cb4d05 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/ProvisionUtil.java @@ -1,46 +1,48 @@ + package eu.dnetlib.dhp.provision; +import org.apache.commons.lang3.StringUtils; + import eu.dnetlib.dhp.provision.scholix.summary.Typology; import eu.dnetlib.dhp.utils.DHPUtils; -import org.apache.commons.lang3.StringUtils; public class ProvisionUtil { - public static final String deletedByInferenceJPATH = "$.dataInfo.deletedbyinference"; - public static final String TARGETJSONPATH = "$.target"; - public static final String SOURCEJSONPATH = "$.source"; + public static final String deletedByInferenceJPATH = "$.dataInfo.deletedbyinference"; + public static final String TARGETJSONPATH = "$.target"; + public static final String SOURCEJSONPATH = "$.source"; - // public static RelatedItemInfo getItemType(final String item, final String idPath) { - // String targetId = DHPUtils.getJPathString(idPath, item); - // switch (StringUtils.substringBefore(targetId, "|")) { - // case "50": - // return new RelatedItemInfo(null,0,1,0); - // case "60": - // return new RelatedItemInfo(null,1,0,0); - // case "70": - // return new RelatedItemInfo(null,0,0,1); - // default: - // throw new RuntimeException("Unknonw target ID"); - // - // } - // - // } + // public static RelatedItemInfo getItemType(final String item, final String idPath) { + // String targetId = DHPUtils.getJPathString(idPath, item); + // switch (StringUtils.substringBefore(targetId, "|")) { + // case "50": + // return new RelatedItemInfo(null,0,1,0); + // case "60": + // return new RelatedItemInfo(null,1,0,0); + // case "70": + // return new RelatedItemInfo(null,0,0,1); + // default: + // throw new RuntimeException("Unknonw target ID"); + // + // } + // + // } - public static Boolean isNotDeleted(final String item) { - return !"true".equalsIgnoreCase(DHPUtils.getJPathString(deletedByInferenceJPATH, item)); - } + public static Boolean isNotDeleted(final String item) { + return !"true".equalsIgnoreCase(DHPUtils.getJPathString(deletedByInferenceJPATH, item)); + } - public static Typology getItemTypeFromId(String id) { + public static Typology getItemTypeFromId(String id) { - switch (StringUtils.substringBefore(id, "|")) { - case "50": - return Typology.publication; - case "60": - return Typology.dataset; - case "70": - return Typology.unknown; - default: - throw new RuntimeException("Unknonw ID type"); - } - } + switch (StringUtils.substringBefore(id, "|")) { + case "50": + return Typology.publication; + case "60": + return Typology.dataset; + case "70": + return Typology.unknown; + default: + throw new RuntimeException("Unknonw ID type"); + } + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java index 7e322ce06..28826612d 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/RelatedItemInfo.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.provision; import java.io.Serializable; @@ -5,53 +6,54 @@ import java.io.Serializable; /** This class models the information of related items */ public class RelatedItemInfo implements Serializable { - private String source; + private String source; - private long relatedDataset = 0; + private long relatedDataset = 0; - private long relatedPublication = 0; + private long relatedPublication = 0; - private long relatedUnknown = 0; + private long relatedUnknown = 0; - public RelatedItemInfo() {} + public RelatedItemInfo() { + } - public RelatedItemInfo( - String source, long relatedDataset, long relatedPublication, long relatedUnknown) { - this.source = source; - this.relatedDataset = relatedDataset; - this.relatedPublication = relatedPublication; - this.relatedUnknown = relatedUnknown; - } + public RelatedItemInfo( + String source, long relatedDataset, long relatedPublication, long relatedUnknown) { + this.source = source; + this.relatedDataset = relatedDataset; + this.relatedPublication = relatedPublication; + this.relatedUnknown = relatedUnknown; + } - public String getSource() { - return source; - } + public String getSource() { + return source; + } - public void setSource(String source) { - this.source = source; - } + public void setSource(String source) { + this.source = source; + } - public long getRelatedDataset() { - return relatedDataset; - } + public long getRelatedDataset() { + return relatedDataset; + } - public void setRelatedDataset(long relatedDataset) { - this.relatedDataset = relatedDataset; - } + public void setRelatedDataset(long relatedDataset) { + this.relatedDataset = relatedDataset; + } - public long getRelatedPublication() { - return relatedPublication; - } + public long getRelatedPublication() { + return relatedPublication; + } - public void setRelatedPublication(long relatedPublication) { - this.relatedPublication = relatedPublication; - } + public void setRelatedPublication(long relatedPublication) { + this.relatedPublication = relatedPublication; + } - public long getRelatedUnknown() { - return relatedUnknown; - } + public long getRelatedUnknown() { + return relatedUnknown; + } - public void setRelatedUnknown(int relatedUnknown) { - this.relatedUnknown = relatedUnknown; - } + public void setRelatedUnknown(int relatedUnknown) { + this.relatedUnknown = relatedUnknown; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.java index 14ffb32e5..df167f104 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkExtractRelationCount.java @@ -1,32 +1,34 @@ + package eu.dnetlib.dhp.provision; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; import org.apache.commons.io.IOUtils; import org.apache.spark.sql.*; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + /** - * SparkExtractRelationCount is a spark job that takes in input relation RDD and retrieve for each - * item in relation which are the number of - Related Dataset - Related Publication - Related - * Unknown + * SparkExtractRelationCount is a spark job that takes in input relation RDD and retrieve for each item in relation + * which are the number of - Related Dataset - Related Publication - Related Unknown */ public class SparkExtractRelationCount { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkExtractRelationCount.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/input_related_entities_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkExtractRelationCount.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkExtractRelationCount.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/input_related_entities_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkExtractRelationCount.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); - final String workingDirPath = parser.get("workingDirPath"); + final String workingDirPath = parser.get("workingDirPath"); - final String relationPath = parser.get("relationPath"); - DatasetJoiner.startJoin(spark, relationPath, workingDirPath + "/relatedItemCount"); - } + final String relationPath = parser.get("relationPath"); + DatasetJoiner.startJoin(spark, relationPath, workingDirPath + "/relatedItemCount"); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholix.java index 327bad94e..f9f3a58ce 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholix.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateScholix.java @@ -1,10 +1,6 @@ + package eu.dnetlib.dhp.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.provision.scholix.*; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; -import eu.dnetlib.dhp.schema.oaf.Relation; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -14,91 +10,100 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.provision.scholix.*; +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; +import eu.dnetlib.dhp.schema.oaf.Relation; import scala.Tuple2; public class SparkGenerateScholix { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkGenerateScholix.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json"))); - parser.parseArgument(args); - SparkConf conf = new SparkConf(); - conf.set("spark.sql.shuffle.partitions", "4000"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - final SparkSession spark = - SparkSession.builder() - .config(conf) - .appName(SparkExtractRelationCount.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkGenerateScholix.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json"))); + parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "4000"); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + final SparkSession spark = SparkSession + .builder() + .config(conf) + .appName(SparkExtractRelationCount.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); - conf.registerKryoClasses( - new Class[] { - Scholix.class, ScholixCollectedFrom.class, ScholixEntityId.class, - ScholixIdentifier.class, ScholixRelationship.class, ScholixResource.class - }); + conf + .registerKryoClasses( + new Class[] { + Scholix.class, ScholixCollectedFrom.class, ScholixEntityId.class, + ScholixIdentifier.class, ScholixRelationship.class, ScholixResource.class + }); - final String graphPath = parser.get("graphPath"); - final String workingDirPath = parser.get("workingDirPath"); + final String graphPath = parser.get("graphPath"); + final String workingDirPath = parser.get("workingDirPath"); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - final Dataset scholixSummary = - spark.read().load(workingDirPath + "/summary").as(Encoders.bean(ScholixSummary.class)); - final Dataset rels = - spark.read().load(graphPath + "/relation").as(Encoders.bean(Relation.class)); + final Dataset scholixSummary = spark + .read() + .load(workingDirPath + "/summary") + .as(Encoders.bean(ScholixSummary.class)); + final Dataset rels = spark.read().load(graphPath + "/relation").as(Encoders.bean(Relation.class)); - Dataset firstJoin = - scholixSummary - .joinWith(rels, scholixSummary.col("id").equalTo(rels.col("source"))) - .map( - (MapFunction, Scholix>) - f -> Scholix.generateScholixWithSource(f._1(), f._2()), - Encoders.bean(Scholix.class)); + Dataset firstJoin = scholixSummary + .joinWith(rels, scholixSummary.col("id").equalTo(rels.col("source"))) + .map( + (MapFunction, Scholix>) f -> Scholix + .generateScholixWithSource(f._1(), f._2()), + Encoders.bean(Scholix.class)); - firstJoin.write().mode(SaveMode.Overwrite).save(workingDirPath + "/scholix_1"); + firstJoin.write().mode(SaveMode.Overwrite).save(workingDirPath + "/scholix_1"); - Dataset scholix_final = - spark.read().load(workingDirPath + "/scholix_1").as(Encoders.bean(Scholix.class)); + Dataset scholix_final = spark + .read() + .load(workingDirPath + "/scholix_1") + .as(Encoders.bean(Scholix.class)); - scholixSummary - .map( - (MapFunction) ScholixResource::fromSummary, - Encoders.bean(ScholixResource.class)) - .repartition(1000) - .write() - .mode(SaveMode.Overwrite) - .save(workingDirPath + "/scholix_target"); + scholixSummary + .map( + (MapFunction) ScholixResource::fromSummary, + Encoders.bean(ScholixResource.class)) + .repartition(1000) + .write() + .mode(SaveMode.Overwrite) + .save(workingDirPath + "/scholix_target"); - Dataset target = - spark - .read() - .load(workingDirPath + "/scholix_target") - .as(Encoders.bean(ScholixResource.class)); + Dataset target = spark + .read() + .load(workingDirPath + "/scholix_target") + .as(Encoders.bean(ScholixResource.class)); - scholix_final - .joinWith( - target, scholix_final.col("identifier").equalTo(target.col("dnetIdentifier")), "inner") - .map( - (MapFunction, Scholix>) - f -> { - final Scholix scholix = f._1(); - final ScholixResource scholixTarget = f._2(); - scholix.setTarget(scholixTarget); - scholix.generateIdentifier(); - scholix.generatelinkPublisher(); - return scholix; - }, - Encoders.kryo(Scholix.class)) - .javaRDD() - .map( - s -> { - ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(s); - }) - .saveAsTextFile(workingDirPath + "/scholix_json", GzipCodec.class); - } + scholix_final + .joinWith( + target, scholix_final.col("identifier").equalTo(target.col("dnetIdentifier")), "inner") + .map( + (MapFunction, Scholix>) f -> { + final Scholix scholix = f._1(); + final ScholixResource scholixTarget = f._2(); + scholix.setTarget(scholixTarget); + scholix.generateIdentifier(); + scholix.generatelinkPublisher(); + return scholix; + }, + Encoders.kryo(Scholix.class)) + .javaRDD() + .map( + s -> { + ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(s); + }) + .saveAsTextFile(workingDirPath + "/scholix_json", GzipCodec.class); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummary.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummary.java index a4a19e833..04bde1099 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummary.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkGenerateSummary.java @@ -1,104 +1,106 @@ + package eu.dnetlib.dhp.provision; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; -import eu.dnetlib.dhp.utils.DHPUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; public class SparkGenerateSummary { - private static final String jsonIDPath = "$.id"; + private static final String jsonIDPath = "$.id"; - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkGenerateSummary.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json"))); - parser.parseArgument(args); - final SparkSession spark = - SparkSession.builder() - .appName(SparkExtractRelationCount.class.getSimpleName()) - .master(parser.get("master")) - .getOrCreate(); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkGenerateSummary.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/input_generate_summary_parameters.json"))); + parser.parseArgument(args); + final SparkSession spark = SparkSession + .builder() + .appName(SparkExtractRelationCount.class.getSimpleName()) + .master(parser.get("master")) + .getOrCreate(); - final String graphPath = parser.get("graphPath"); - final String workingDirPath = parser.get("workingDirPath"); + final String graphPath = parser.get("graphPath"); + final String workingDirPath = parser.get("workingDirPath"); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset rInfo = - spark - .read() - .load(workingDirPath + "/relatedItemCount") - .as(Encoders.bean(RelatedItemInfo.class)); + Dataset rInfo = spark + .read() + .load(workingDirPath + "/relatedItemCount") + .as(Encoders.bean(RelatedItemInfo.class)); - Dataset entity = - spark.createDataset( - sc.textFile( - graphPath + "/publication," + graphPath + "/dataset," + graphPath + "/unknown") - .map( - s -> - ScholixSummary.fromJsonOAF( - ProvisionUtil.getItemTypeFromId(DHPUtils.getJPathString(jsonIDPath, s)), - s)) - .rdd(), - Encoders.bean(ScholixSummary.class)); + Dataset entity = spark + .createDataset( + sc + .textFile( + graphPath + "/publication," + graphPath + "/dataset," + graphPath + "/unknown") + .map( + s -> ScholixSummary + .fromJsonOAF( + ProvisionUtil.getItemTypeFromId(DHPUtils.getJPathString(jsonIDPath, s)), + s)) + .rdd(), + Encoders.bean(ScholixSummary.class)); - Dataset summaryComplete = - rInfo - .joinWith(entity, rInfo.col("source").equalTo(entity.col("id"))) - .map( - (MapFunction, ScholixSummary>) - t -> { - ScholixSummary scholixSummary = t._2(); - RelatedItemInfo relatedItemInfo = t._1(); - scholixSummary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); - scholixSummary.setRelatedPublications( - relatedItemInfo.getRelatedPublication()); - scholixSummary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); - return scholixSummary; - }, - Encoders.bean(ScholixSummary.class)); + Dataset summaryComplete = rInfo + .joinWith(entity, rInfo.col("source").equalTo(entity.col("id"))) + .map( + (MapFunction, ScholixSummary>) t -> { + ScholixSummary scholixSummary = t._2(); + RelatedItemInfo relatedItemInfo = t._1(); + scholixSummary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); + scholixSummary + .setRelatedPublications( + relatedItemInfo.getRelatedPublication()); + scholixSummary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); + return scholixSummary; + }, + Encoders.bean(ScholixSummary.class)); - summaryComplete.write().save(workingDirPath + "/summary"); + summaryComplete.write().save(workingDirPath + "/summary"); - // JavaPairRDD relationCount = - // sc.textFile(workingDirPath+"/relatedItemCount").mapToPair((PairFunction) i -> new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)); - // - // JavaPairRDD entities = - // sc.textFile(graphPath + "/publication") - // .filter(ProvisionUtil::isNotDeleted) - // .mapToPair((PairFunction) i -> new - // Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) - // .union( - // sc.textFile(graphPath + "/dataset") - // .filter(ProvisionUtil::isNotDeleted) - // .mapToPair((PairFunction) - // i -> - // new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) - // ) - // .union( - // sc.textFile(graphPath + "/unknown") - // .filter(ProvisionUtil::isNotDeleted) - // .mapToPair((PairFunction) - // i -> - // new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) - // ); - // entities.join(relationCount).map((Function>, - // String>) k -> - // ScholixSummary.fromJsonOAF(ProvisionUtil.getItemTypeFromId(k._1()), - // k._2()._1(), k._2()._2())).saveAsTextFile(workingDirPath+"/summary", GzipCodec.class); - // - // - // ; + // JavaPairRDD relationCount = + // sc.textFile(workingDirPath+"/relatedItemCount").mapToPair((PairFunction) i -> new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)); + // + // JavaPairRDD entities = + // sc.textFile(graphPath + "/publication") + // .filter(ProvisionUtil::isNotDeleted) + // .mapToPair((PairFunction) i -> new + // Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) + // .union( + // sc.textFile(graphPath + "/dataset") + // .filter(ProvisionUtil::isNotDeleted) + // .mapToPair((PairFunction) + // i -> + // new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) + // ) + // .union( + // sc.textFile(graphPath + "/unknown") + // .filter(ProvisionUtil::isNotDeleted) + // .mapToPair((PairFunction) + // i -> + // new Tuple2<>(DHPUtils.getJPathString(jsonIDPath, i), i)) + // ); + // entities.join(relationCount).map((Function>, + // String>) k -> + // ScholixSummary.fromJsonOAF(ProvisionUtil.getItemTypeFromId(k._1()), + // k._2()._1(), k._2()._2())).saveAsTextFile(workingDirPath+"/summary", GzipCodec.class); + // + // + // ; - } + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java index bafdaa5d7..e79dad8d3 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/SparkIndexCollectionOnES.java @@ -1,10 +1,9 @@ + package eu.dnetlib.dhp.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; import java.util.HashMap; import java.util.Map; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -14,58 +13,62 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.elasticsearch.spark.rdd.api.java.JavaEsSpark; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; + public class SparkIndexCollectionOnES { - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkIndexCollectionOnES.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/index_on_es.json"))); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkIndexCollectionOnES.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/index_on_es.json"))); + parser.parseArgument(args); - SparkConf conf = - new SparkConf() - .setAppName(SparkIndexCollectionOnES.class.getSimpleName()) - .setMaster(parser.get("master")); + SparkConf conf = new SparkConf() + .setAppName(SparkIndexCollectionOnES.class.getSimpleName()) + .setMaster(parser.get("master")); - conf.set("spark.sql.shuffle.partitions", "4000"); + conf.set("spark.sql.shuffle.partitions", "4000"); - final String sourcePath = parser.get("sourcePath"); - final String index = parser.get("index"); - final String idPath = parser.get("idPath"); - final String type = parser.get("type"); + final String sourcePath = parser.get("sourcePath"); + final String index = parser.get("index"); + final String idPath = parser.get("idPath"); + final String type = parser.get("type"); - final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); + final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD inputRdd; + JavaRDD inputRdd; - if ("summary".equalsIgnoreCase(type)) - inputRdd = - spark - .read() - .load(sourcePath) - .as(Encoders.bean(ScholixSummary.class)) - .map( - (MapFunction) - f -> { - final ObjectMapper mapper = new ObjectMapper(); - return mapper.writeValueAsString(f); - }, - Encoders.STRING()) - .javaRDD(); - else inputRdd = sc.textFile(sourcePath); + if ("summary".equalsIgnoreCase(type)) + inputRdd = spark + .read() + .load(sourcePath) + .as(Encoders.bean(ScholixSummary.class)) + .map( + (MapFunction) f -> { + final ObjectMapper mapper = new ObjectMapper(); + return mapper.writeValueAsString(f); + }, + Encoders.STRING()) + .javaRDD(); + else + inputRdd = sc.textFile(sourcePath); - Map esCfg = new HashMap<>(); - esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54"); - esCfg.put("es.mapping.id", idPath); - esCfg.put("es.batch.write.retry.count", "8"); - esCfg.put("es.batch.write.retry.wait", "60s"); - esCfg.put("es.batch.size.entries", "200"); - esCfg.put("es.nodes.wan.only", "true"); - JavaEsSpark.saveJsonToEs(inputRdd, index, esCfg); - } + Map esCfg = new HashMap<>(); + esCfg.put("es.nodes", "10.19.65.51, 10.19.65.52, 10.19.65.53, 10.19.65.54"); + esCfg.put("es.mapping.id", idPath); + esCfg.put("es.batch.write.retry.count", "8"); + esCfg.put("es.batch.write.retry.wait", "60s"); + esCfg.put("es.batch.size.entries", "200"); + esCfg.put("es.nodes.wan.only", "true"); + JavaEsSpark.saveJsonToEs(inputRdd, index, esCfg); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java index 3130d8b98..d71415513 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/Scholix.java @@ -1,184 +1,200 @@ + package eu.dnetlib.dhp.provision.scholix; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.DHPUtils; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.DHPUtils; + public class Scholix implements Serializable { - private String publicationDate; + private String publicationDate; - private List publisher; + private List publisher; - private List linkprovider; + private List linkprovider; - private ScholixRelationship relationship; + private ScholixRelationship relationship; - private ScholixResource source; + private ScholixResource source; - private ScholixResource target; + private ScholixResource target; - private String identifier; + private String identifier; - public Scholix clone(final ScholixResource t) { - final Scholix clone = new Scholix(); - clone.setPublicationDate(publicationDate); - clone.setPublisher(publisher); - clone.setLinkprovider(linkprovider); - clone.setRelationship(relationship); - clone.setSource(source); - clone.setTarget(t); - clone.generatelinkPublisher(); - clone.generateIdentifier(); - return clone; - } + public Scholix clone(final ScholixResource t) { + final Scholix clone = new Scholix(); + clone.setPublicationDate(publicationDate); + clone.setPublisher(publisher); + clone.setLinkprovider(linkprovider); + clone.setRelationship(relationship); + clone.setSource(source); + clone.setTarget(t); + clone.generatelinkPublisher(); + clone.generateIdentifier(); + return clone; + } - public static Scholix generateScholixWithSource( - final String sourceSummaryJson, final String relation) { - final ObjectMapper mapper = new ObjectMapper(); + public static Scholix generateScholixWithSource( + final String sourceSummaryJson, final String relation) { + final ObjectMapper mapper = new ObjectMapper(); - try { - ScholixSummary scholixSummary = mapper.readValue(sourceSummaryJson, ScholixSummary.class); - Relation rel = mapper.readValue(relation, Relation.class); - final Scholix s = new Scholix(); - if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0) - s.setPublicationDate(scholixSummary.getDate().get(0)); - s.setLinkprovider( - rel.getCollectedfrom().stream() - .map( - cf -> - new ScholixEntityId( - cf.getValue(), - Collections.singletonList( - new ScholixIdentifier(cf.getKey(), "dnet_identifier")))) - .collect(Collectors.toList())); - s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null)); - s.setSource(ScholixResource.fromSummary(scholixSummary)); - return s; - } catch (Throwable e) { - throw new RuntimeException( - String.format("Summary: %s \n relation:%s", sourceSummaryJson, relation), e); - } - } + try { + ScholixSummary scholixSummary = mapper.readValue(sourceSummaryJson, ScholixSummary.class); + Relation rel = mapper.readValue(relation, Relation.class); + final Scholix s = new Scholix(); + if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0) + s.setPublicationDate(scholixSummary.getDate().get(0)); + s + .setLinkprovider( + rel + .getCollectedfrom() + .stream() + .map( + cf -> new ScholixEntityId( + cf.getValue(), + Collections + .singletonList( + new ScholixIdentifier(cf.getKey(), "dnet_identifier")))) + .collect(Collectors.toList())); + s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null)); + s.setSource(ScholixResource.fromSummary(scholixSummary)); + return s; + } catch (Throwable e) { + throw new RuntimeException( + String.format("Summary: %s \n relation:%s", sourceSummaryJson, relation), e); + } + } - public static Scholix generateScholixWithSource( - final ScholixSummary scholixSummary, final Relation rel) { - final Scholix s = new Scholix(); - if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0) - s.setPublicationDate(scholixSummary.getDate().get(0)); - s.setLinkprovider( - rel.getCollectedfrom().stream() - .map( - cf -> - new ScholixEntityId( - cf.getValue(), - Collections.singletonList( - new ScholixIdentifier(cf.getKey(), "dnet_identifier")))) - .collect(Collectors.toList())); - s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null)); - s.setSource(ScholixResource.fromSummary(scholixSummary)); + public static Scholix generateScholixWithSource( + final ScholixSummary scholixSummary, final Relation rel) { + final Scholix s = new Scholix(); + if (scholixSummary.getDate() != null && scholixSummary.getDate().size() > 0) + s.setPublicationDate(scholixSummary.getDate().get(0)); + s + .setLinkprovider( + rel + .getCollectedfrom() + .stream() + .map( + cf -> new ScholixEntityId( + cf.getValue(), + Collections + .singletonList( + new ScholixIdentifier(cf.getKey(), "dnet_identifier")))) + .collect(Collectors.toList())); + s.setRelationship(new ScholixRelationship(rel.getRelType(), rel.getRelClass(), null)); + s.setSource(ScholixResource.fromSummary(scholixSummary)); - s.setIdentifier(rel.getTarget()); - // ScholixResource mockTarget = new ScholixResource(); - // mockTarget.setDnetIdentifier(rel.getTarget()); - // s.setTarget(mockTarget); - // s.generateIdentifier(); - return s; - } + s.setIdentifier(rel.getTarget()); + // ScholixResource mockTarget = new ScholixResource(); + // mockTarget.setDnetIdentifier(rel.getTarget()); + // s.setTarget(mockTarget); + // s.generateIdentifier(); + return s; + } - public void generatelinkPublisher() { - Set publisher = new HashSet<>(); - if (source.getPublisher() != null) - publisher.addAll( - source.getPublisher().stream() - .map(ScholixEntityId::getName) - .collect(Collectors.toList())); - if (target.getPublisher() != null) - publisher.addAll( - target.getPublisher().stream() - .map(ScholixEntityId::getName) - .collect(Collectors.toList())); - this.publisher = - publisher.stream().map(k -> new ScholixEntityId(k, null)).collect(Collectors.toList()); - } + public void generatelinkPublisher() { + Set publisher = new HashSet<>(); + if (source.getPublisher() != null) + publisher + .addAll( + source + .getPublisher() + .stream() + .map(ScholixEntityId::getName) + .collect(Collectors.toList())); + if (target.getPublisher() != null) + publisher + .addAll( + target + .getPublisher() + .stream() + .map(ScholixEntityId::getName) + .collect(Collectors.toList())); + this.publisher = publisher.stream().map(k -> new ScholixEntityId(k, null)).collect(Collectors.toList()); + } - public void generateIdentifier() { - setIdentifier( - DHPUtils.md5( - String.format( - "%s::%s::%s", - source.getDnetIdentifier(), relationship.getName(), target.getDnetIdentifier()))); - } + public void generateIdentifier() { + setIdentifier( + DHPUtils + .md5( + String + .format( + "%s::%s::%s", + source.getDnetIdentifier(), relationship.getName(), target.getDnetIdentifier()))); + } - public Scholix addTarget(final String targetSummaryJson) { - final ObjectMapper mapper = new ObjectMapper(); + public Scholix addTarget(final String targetSummaryJson) { + final ObjectMapper mapper = new ObjectMapper(); - try { - ScholixSummary targetSummary = mapper.readValue(targetSummaryJson, ScholixSummary.class); - setTarget(ScholixResource.fromSummary(targetSummary)); - generateIdentifier(); - return this; - } catch (Throwable e) { - throw new RuntimeException(e); - } - } + try { + ScholixSummary targetSummary = mapper.readValue(targetSummaryJson, ScholixSummary.class); + setTarget(ScholixResource.fromSummary(targetSummary)); + generateIdentifier(); + return this; + } catch (Throwable e) { + throw new RuntimeException(e); + } + } - public String getPublicationDate() { - return publicationDate; - } + public String getPublicationDate() { + return publicationDate; + } - public void setPublicationDate(String publicationDate) { - this.publicationDate = publicationDate; - } + public void setPublicationDate(String publicationDate) { + this.publicationDate = publicationDate; + } - public List getPublisher() { - return publisher; - } + public List getPublisher() { + return publisher; + } - public void setPublisher(List publisher) { - this.publisher = publisher; - } + public void setPublisher(List publisher) { + this.publisher = publisher; + } - public List getLinkprovider() { - return linkprovider; - } + public List getLinkprovider() { + return linkprovider; + } - public void setLinkprovider(List linkprovider) { - this.linkprovider = linkprovider; - } + public void setLinkprovider(List linkprovider) { + this.linkprovider = linkprovider; + } - public ScholixRelationship getRelationship() { - return relationship; - } + public ScholixRelationship getRelationship() { + return relationship; + } - public void setRelationship(ScholixRelationship relationship) { - this.relationship = relationship; - } + public void setRelationship(ScholixRelationship relationship) { + this.relationship = relationship; + } - public ScholixResource getSource() { - return source; - } + public ScholixResource getSource() { + return source; + } - public void setSource(ScholixResource source) { - this.source = source; - } + public void setSource(ScholixResource source) { + this.source = source; + } - public ScholixResource getTarget() { - return target; - } + public ScholixResource getTarget() { + return target; + } - public void setTarget(ScholixResource target) { - this.target = target; - } + public void setTarget(ScholixResource target) { + this.target = target; + } - public String getIdentifier() { - return identifier; - } + public String getIdentifier() { + return identifier; + } - public void setIdentifier(String identifier) { - this.identifier = identifier; - } + public void setIdentifier(String identifier) { + this.identifier = identifier; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java index c55bbb111..9ce071fbc 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixCollectedFrom.java @@ -1,43 +1,45 @@ + package eu.dnetlib.dhp.provision.scholix; import java.io.Serializable; public class ScholixCollectedFrom implements Serializable { - private ScholixEntityId provider; - private String provisionMode; - private String completionStatus; + private ScholixEntityId provider; + private String provisionMode; + private String completionStatus; - public ScholixCollectedFrom() {} + public ScholixCollectedFrom() { + } - public ScholixCollectedFrom( - ScholixEntityId provider, String provisionMode, String completionStatus) { - this.provider = provider; - this.provisionMode = provisionMode; - this.completionStatus = completionStatus; - } + public ScholixCollectedFrom( + ScholixEntityId provider, String provisionMode, String completionStatus) { + this.provider = provider; + this.provisionMode = provisionMode; + this.completionStatus = completionStatus; + } - public ScholixEntityId getProvider() { - return provider; - } + public ScholixEntityId getProvider() { + return provider; + } - public void setProvider(ScholixEntityId provider) { - this.provider = provider; - } + public void setProvider(ScholixEntityId provider) { + this.provider = provider; + } - public String getProvisionMode() { - return provisionMode; - } + public String getProvisionMode() { + return provisionMode; + } - public void setProvisionMode(String provisionMode) { - this.provisionMode = provisionMode; - } + public void setProvisionMode(String provisionMode) { + this.provisionMode = provisionMode; + } - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java index 226c3d20a..e797017bc 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixEntityId.java @@ -1,32 +1,34 @@ + package eu.dnetlib.dhp.provision.scholix; import java.io.Serializable; import java.util.List; public class ScholixEntityId implements Serializable { - private String name; - private List identifiers; + private String name; + private List identifiers; - public ScholixEntityId() {} + public ScholixEntityId() { + } - public ScholixEntityId(String name, List identifiers) { - this.name = name; - this.identifiers = identifiers; - } + public ScholixEntityId(String name, List identifiers) { + this.name = name; + this.identifiers = identifiers; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public List getIdentifiers() { - return identifiers; - } + public List getIdentifiers() { + return identifiers; + } - public void setIdentifiers(List identifiers) { - this.identifiers = identifiers; - } + public void setIdentifiers(List identifiers) { + this.identifiers = identifiers; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java index 265ac1ef5..0dd15336a 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixIdentifier.java @@ -1,31 +1,33 @@ + package eu.dnetlib.dhp.provision.scholix; import java.io.Serializable; public class ScholixIdentifier implements Serializable { - private String identifier; - private String schema; + private String identifier; + private String schema; - public ScholixIdentifier() {} + public ScholixIdentifier() { + } - public ScholixIdentifier(String identifier, String schema) { - this.identifier = identifier; - this.schema = schema; - } + public ScholixIdentifier(String identifier, String schema) { + this.identifier = identifier; + this.schema = schema; + } - public String getIdentifier() { - return identifier; - } + public String getIdentifier() { + return identifier; + } - public void setIdentifier(String identifier) { - this.identifier = identifier; - } + public void setIdentifier(String identifier) { + this.identifier = identifier; + } - public String getSchema() { - return schema; - } + public String getSchema() { + return schema; + } - public void setSchema(String schema) { - this.schema = schema; - } + public void setSchema(String schema) { + this.schema = schema; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java index b363eff2c..0cbdf43e7 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixRelationship.java @@ -1,41 +1,43 @@ + package eu.dnetlib.dhp.provision.scholix; import java.io.Serializable; public class ScholixRelationship implements Serializable { - private String name; - private String schema; - private String inverse; + private String name; + private String schema; + private String inverse; - public ScholixRelationship() {} + public ScholixRelationship() { + } - public ScholixRelationship(String name, String schema, String inverse) { - this.name = name; - this.schema = schema; - this.inverse = inverse; - } + public ScholixRelationship(String name, String schema, String inverse) { + this.name = name; + this.schema = schema; + this.inverse = inverse; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getSchema() { - return schema; - } + public String getSchema() { + return schema; + } - public void setSchema(String schema) { - this.schema = schema; - } + public void setSchema(String schema) { + this.schema = schema; + } - public String getInverse() { - return inverse; - } + public String getInverse() { + return inverse; + } - public void setInverse(String inverse) { - this.inverse = inverse; - } + public void setInverse(String inverse) { + this.inverse = inverse; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java index 89342d281..6de30c748 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/ScholixResource.java @@ -1,137 +1,151 @@ + package eu.dnetlib.dhp.provision.scholix; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; import java.io.Serializable; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; + public class ScholixResource implements Serializable { - private List identifier; - private String dnetIdentifier; - private String objectType; - private String objectSubType; - private String title; - private List creator; - private String publicationDate; - private List publisher; - private List collectedFrom; + private List identifier; + private String dnetIdentifier; + private String objectType; + private String objectSubType; + private String title; + private List creator; + private String publicationDate; + private List publisher; + private List collectedFrom; - public static ScholixResource fromSummary(ScholixSummary summary) { + public static ScholixResource fromSummary(ScholixSummary summary) { - final ScholixResource resource = new ScholixResource(); + final ScholixResource resource = new ScholixResource(); - resource.setDnetIdentifier(summary.getId()); + resource.setDnetIdentifier(summary.getId()); - resource.setIdentifier( - summary.getLocalIdentifier().stream() - .map(i -> new ScholixIdentifier(i.getId(), i.getType())) - .collect(Collectors.toList())); + resource + .setIdentifier( + summary + .getLocalIdentifier() + .stream() + .map(i -> new ScholixIdentifier(i.getId(), i.getType())) + .collect(Collectors.toList())); - resource.setObjectType(summary.getTypology().toString()); + resource.setObjectType(summary.getTypology().toString()); - if (summary.getTitle() != null && summary.getTitle().size() > 0) - resource.setTitle(summary.getTitle().get(0)); + if (summary.getTitle() != null && summary.getTitle().size() > 0) + resource.setTitle(summary.getTitle().get(0)); - if (summary.getAuthor() != null) - resource.setCreator( - summary.getAuthor().stream() - .map(c -> new ScholixEntityId(c, null)) - .collect(Collectors.toList())); + if (summary.getAuthor() != null) + resource + .setCreator( + summary + .getAuthor() + .stream() + .map(c -> new ScholixEntityId(c, null)) + .collect(Collectors.toList())); - if (summary.getDate() != null && summary.getDate().size() > 0) - resource.setPublicationDate(summary.getDate().get(0)); - if (summary.getPublisher() != null) - resource.setPublisher( - summary.getPublisher().stream() - .map(p -> new ScholixEntityId(p, null)) - .collect(Collectors.toList())); - if (summary.getDatasources() != null) - resource.setCollectedFrom( - summary.getDatasources().stream() - .map( - d -> - new ScholixCollectedFrom( - new ScholixEntityId( - d.getDatasourceName(), - Collections.singletonList( - new ScholixIdentifier(d.getDatasourceId(), "dnet_identifier"))), - "collected", - d.getCompletionStatus())) - .collect(Collectors.toList())); - return resource; - } + if (summary.getDate() != null && summary.getDate().size() > 0) + resource.setPublicationDate(summary.getDate().get(0)); + if (summary.getPublisher() != null) + resource + .setPublisher( + summary + .getPublisher() + .stream() + .map(p -> new ScholixEntityId(p, null)) + .collect(Collectors.toList())); + if (summary.getDatasources() != null) + resource + .setCollectedFrom( + summary + .getDatasources() + .stream() + .map( + d -> new ScholixCollectedFrom( + new ScholixEntityId( + d.getDatasourceName(), + Collections + .singletonList( + new ScholixIdentifier(d.getDatasourceId(), "dnet_identifier"))), + "collected", + d.getCompletionStatus())) + .collect(Collectors.toList())); + return resource; + } - public List getIdentifier() { - return identifier; - } + public List getIdentifier() { + return identifier; + } - public void setIdentifier(List identifier) { - this.identifier = identifier; - } + public void setIdentifier(List identifier) { + this.identifier = identifier; + } - public String getDnetIdentifier() { - return dnetIdentifier; - } + public String getDnetIdentifier() { + return dnetIdentifier; + } - public void setDnetIdentifier(String dnetIdentifier) { - this.dnetIdentifier = dnetIdentifier; - } + public void setDnetIdentifier(String dnetIdentifier) { + this.dnetIdentifier = dnetIdentifier; + } - public String getObjectType() { - return objectType; - } + public String getObjectType() { + return objectType; + } - public void setObjectType(String objectType) { - this.objectType = objectType; - } + public void setObjectType(String objectType) { + this.objectType = objectType; + } - public String getObjectSubType() { - return objectSubType; - } + public String getObjectSubType() { + return objectSubType; + } - public void setObjectSubType(String objectSubType) { - this.objectSubType = objectSubType; - } + public void setObjectSubType(String objectSubType) { + this.objectSubType = objectSubType; + } - public String getTitle() { - return title; - } + public String getTitle() { + return title; + } - public void setTitle(String title) { - this.title = title; - } + public void setTitle(String title) { + this.title = title; + } - public List getCreator() { - return creator; - } + public List getCreator() { + return creator; + } - public void setCreator(List creator) { - this.creator = creator; - } + public void setCreator(List creator) { + this.creator = creator; + } - public String getPublicationDate() { - return publicationDate; - } + public String getPublicationDate() { + return publicationDate; + } - public void setPublicationDate(String publicationDate) { - this.publicationDate = publicationDate; - } + public void setPublicationDate(String publicationDate) { + this.publicationDate = publicationDate; + } - public List getPublisher() { - return publisher; - } + public List getPublisher() { + return publisher; + } - public void setPublisher(List publisher) { - this.publisher = publisher; - } + public void setPublisher(List publisher) { + this.publisher = publisher; + } - public List getCollectedFrom() { - return collectedFrom; - } + public List getCollectedFrom() { + return collectedFrom; + } - public void setCollectedFrom(List collectedFrom) { - this.collectedFrom = collectedFrom; - } + public void setCollectedFrom(List collectedFrom) { + this.collectedFrom = collectedFrom; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java index 0d58eacd6..6d6f46f54 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/CollectedFromType.java @@ -1,42 +1,44 @@ + package eu.dnetlib.dhp.provision.scholix.summary; import java.io.Serializable; public class CollectedFromType implements Serializable { - private String datasourceName; - private String datasourceId; - private String completionStatus; + private String datasourceName; + private String datasourceId; + private String completionStatus; - public CollectedFromType() {} + public CollectedFromType() { + } - public CollectedFromType(String datasourceName, String datasourceId, String completionStatus) { - this.datasourceName = datasourceName; - this.datasourceId = datasourceId; - this.completionStatus = completionStatus; - } + public CollectedFromType(String datasourceName, String datasourceId, String completionStatus) { + this.datasourceName = datasourceName; + this.datasourceId = datasourceId; + this.completionStatus = completionStatus; + } - public String getDatasourceName() { - return datasourceName; - } + public String getDatasourceName() { + return datasourceName; + } - public void setDatasourceName(String datasourceName) { - this.datasourceName = datasourceName; - } + public void setDatasourceName(String datasourceName) { + this.datasourceName = datasourceName; + } - public String getDatasourceId() { - return datasourceId; - } + public String getDatasourceId() { + return datasourceId; + } - public void setDatasourceId(String datasourceId) { - this.datasourceId = datasourceId; - } + public void setDatasourceId(String datasourceId) { + this.datasourceId = datasourceId; + } - public String getCompletionStatus() { - return completionStatus; - } + public String getCompletionStatus() { + return completionStatus; + } - public void setCompletionStatus(String completionStatus) { - this.completionStatus = completionStatus; - } + public void setCompletionStatus(String completionStatus) { + this.completionStatus = completionStatus; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java index 8338e3995..e9d94fccf 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/SchemeValue.java @@ -1,31 +1,33 @@ + package eu.dnetlib.dhp.provision.scholix.summary; import java.io.Serializable; public class SchemeValue implements Serializable { - private String scheme; - private String value; + private String scheme; + private String value; - public SchemeValue() {} + public SchemeValue() { + } - public SchemeValue(String scheme, String value) { - this.scheme = scheme; - this.value = value; - } + public SchemeValue(String scheme, String value) { + this.scheme = scheme; + this.value = value; + } - public String getScheme() { - return scheme; - } + public String getScheme() { + return scheme; + } - public void setScheme(String scheme) { - this.scheme = scheme; - } + public void setScheme(String scheme) { + this.scheme = scheme; + } - public String getValue() { - return value; - } + public String getValue() { + return value; + } - public void setValue(String value) { - this.value = value; - } + public void setValue(String value) { + this.value = value; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java index 50aa2a75c..e5ea8b9f5 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/ScholixSummary.java @@ -1,313 +1,353 @@ + package eu.dnetlib.dhp.provision.scholix.summary; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; + import eu.dnetlib.dhp.provision.RelatedItemInfo; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.schema.scholexplorer.DLIDataset; import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication; import eu.dnetlib.dhp.schema.scholexplorer.DLIUnknown; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; public class ScholixSummary implements Serializable { - private String id; - private List localIdentifier; - private Typology typology; - private List title; - private List author; - private List date; - private String description; - private List subject; - private List publisher; - private long relatedPublications; - private long relatedDatasets; - private long relatedUnknown; - private List datasources; + private String id; + private List localIdentifier; + private Typology typology; + private List title; + private List author; + private List date; + private String description; + private List subject; + private List publisher; + private long relatedPublications; + private long relatedDatasets; + private long relatedUnknown; + private List datasources; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public List getLocalIdentifier() { - return localIdentifier; - } + public List getLocalIdentifier() { + return localIdentifier; + } - public void setLocalIdentifier(List localIdentifier) { - this.localIdentifier = localIdentifier; - } + public void setLocalIdentifier(List localIdentifier) { + this.localIdentifier = localIdentifier; + } - public Typology getTypology() { - return typology; - } + public Typology getTypology() { + return typology; + } - public void setTypology(Typology typology) { - this.typology = typology; - } + public void setTypology(Typology typology) { + this.typology = typology; + } - public List getTitle() { - return title; - } + public List getTitle() { + return title; + } - public void setTitle(List title) { - this.title = title; - } + public void setTitle(List title) { + this.title = title; + } - public List getAuthor() { - return author; - } + public List getAuthor() { + return author; + } - public void setAuthor(List author) { - this.author = author; - } + public void setAuthor(List author) { + this.author = author; + } - public List getDate() { - return date; - } + public List getDate() { + return date; + } - public void setDate(List date) { - this.date = date; - } + public void setDate(List date) { + this.date = date; + } - @JsonProperty("abstract") - public String getDescription() { - return description; - } + @JsonProperty("abstract") + public String getDescription() { + return description; + } - @JsonProperty("abstract") - public void setDescription(String description) { - this.description = description; - } + @JsonProperty("abstract") + public void setDescription(String description) { + this.description = description; + } - public List getSubject() { - return subject; - } + public List getSubject() { + return subject; + } - public void setSubject(List subject) { - this.subject = subject; - } + public void setSubject(List subject) { + this.subject = subject; + } - public List getPublisher() { - return publisher; - } + public List getPublisher() { + return publisher; + } - public void setPublisher(List publisher) { - this.publisher = publisher; - } + public void setPublisher(List publisher) { + this.publisher = publisher; + } - public long getRelatedPublications() { - return relatedPublications; - } + public long getRelatedPublications() { + return relatedPublications; + } - public void setRelatedPublications(long relatedPublications) { - this.relatedPublications = relatedPublications; - } + public void setRelatedPublications(long relatedPublications) { + this.relatedPublications = relatedPublications; + } - public long getRelatedDatasets() { - return relatedDatasets; - } + public long getRelatedDatasets() { + return relatedDatasets; + } - public void setRelatedDatasets(long relatedDatasets) { - this.relatedDatasets = relatedDatasets; - } + public void setRelatedDatasets(long relatedDatasets) { + this.relatedDatasets = relatedDatasets; + } - public long getRelatedUnknown() { - return relatedUnknown; - } + public long getRelatedUnknown() { + return relatedUnknown; + } - public void setRelatedUnknown(long relatedUnknown) { - this.relatedUnknown = relatedUnknown; - } + public void setRelatedUnknown(long relatedUnknown) { + this.relatedUnknown = relatedUnknown; + } - public List getDatasources() { - return datasources; - } + public List getDatasources() { + return datasources; + } - public void setDatasources(List datasources) { - this.datasources = datasources; - } + public void setDatasources(List datasources) { + this.datasources = datasources; + } - public static ScholixSummary fromJsonOAF(final Typology oafType, final String oafJson) { - try { - final ObjectMapper mapper = new ObjectMapper(); - final RelatedItemInfo relatedItemInfo = new RelatedItemInfo(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - switch (oafType) { - case dataset: - return summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo); - case publication: - return summaryFromPublication( - mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo); - case unknown: - return summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo); - } - } catch (Throwable e) { - throw new RuntimeException(e); - } - return null; - } + public static ScholixSummary fromJsonOAF(final Typology oafType, final String oafJson) { + try { + final ObjectMapper mapper = new ObjectMapper(); + final RelatedItemInfo relatedItemInfo = new RelatedItemInfo(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + switch (oafType) { + case dataset: + return summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo); + case publication: + return summaryFromPublication( + mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo); + case unknown: + return summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo); + } + } catch (Throwable e) { + throw new RuntimeException(e); + } + return null; + } - public static String fromJsonOAF( - final Typology oafType, final String oafJson, final String relEntityJson) { - try { - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + public static String fromJsonOAF( + final Typology oafType, final String oafJson, final String relEntityJson) { + try { + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - RelatedItemInfo relatedItemInfo = mapper.readValue(relEntityJson, RelatedItemInfo.class); + RelatedItemInfo relatedItemInfo = mapper.readValue(relEntityJson, RelatedItemInfo.class); - switch (oafType) { - case dataset: - return mapper.writeValueAsString( - summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo)); - case publication: - return mapper.writeValueAsString( - summaryFromPublication( - mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo)); - case unknown: - return mapper.writeValueAsString( - summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo)); - } + switch (oafType) { + case dataset: + return mapper + .writeValueAsString( + summaryFromDataset(mapper.readValue(oafJson, DLIDataset.class), relatedItemInfo)); + case publication: + return mapper + .writeValueAsString( + summaryFromPublication( + mapper.readValue(oafJson, DLIPublication.class), relatedItemInfo)); + case unknown: + return mapper + .writeValueAsString( + summaryFromUnknown(mapper.readValue(oafJson, DLIUnknown.class), relatedItemInfo)); + } - } catch (Throwable e) { - throw new RuntimeException(e); - } + } catch (Throwable e) { + throw new RuntimeException(e); + } - return null; - } + return null; + } - private static ScholixSummary summaryFromDataset( - final DLIDataset item, final RelatedItemInfo relatedItemInfo) { - ScholixSummary summary = new ScholixSummary(); - summary.setId(item.getId()); + private static ScholixSummary summaryFromDataset( + final DLIDataset item, final RelatedItemInfo relatedItemInfo) { + ScholixSummary summary = new ScholixSummary(); + summary.setId(item.getId()); - if (item.getPid() != null) - summary.setLocalIdentifier( - item.getPid().stream() - .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) - .collect(Collectors.toList())); + if (item.getPid() != null) + summary + .setLocalIdentifier( + item + .getPid() + .stream() + .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) + .collect(Collectors.toList())); - summary.setTypology(Typology.dataset); - if (item.getTitle() != null) - summary.setTitle( - item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList())); + summary.setTypology(Typology.dataset); + if (item.getTitle() != null) + summary + .setTitle( + item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList())); - if (item.getAuthor() != null) { - summary.setAuthor( - item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList())); - } + if (item.getAuthor() != null) { + summary + .setAuthor( + item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList())); + } - if (item.getRelevantdate() != null) - summary.setDate( - item.getRelevantdate().stream() - .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname())) - .map(StructuredProperty::getValue) - .collect(Collectors.toList())); + if (item.getRelevantdate() != null) + summary + .setDate( + item + .getRelevantdate() + .stream() + .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname())) + .map(StructuredProperty::getValue) + .collect(Collectors.toList())); - if (item.getDescription() != null && item.getDescription().size() > 0) - summary.setDescription(item.getDescription().get(0).getValue()); + if (item.getDescription() != null && item.getDescription().size() > 0) + summary.setDescription(item.getDescription().get(0).getValue()); - if (item.getSubject() != null) { - summary.setSubject( - item.getSubject().stream() - .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue())) - .collect(Collectors.toList())); - } - if (item.getPublisher() != null) - summary.setPublisher(Collections.singletonList(item.getPublisher().getValue())); + if (item.getSubject() != null) { + summary + .setSubject( + item + .getSubject() + .stream() + .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue())) + .collect(Collectors.toList())); + } + if (item.getPublisher() != null) + summary.setPublisher(Collections.singletonList(item.getPublisher().getValue())); - summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); - summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); - summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); + summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); + summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); + summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); - if (item.getDlicollectedfrom() != null) - summary.setDatasources( - item.getDlicollectedfrom().stream() - .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) - .collect(Collectors.toList())); - return summary; - } + if (item.getDlicollectedfrom() != null) + summary + .setDatasources( + item + .getDlicollectedfrom() + .stream() + .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) + .collect(Collectors.toList())); + return summary; + } - private static ScholixSummary summaryFromPublication( - final DLIPublication item, final RelatedItemInfo relatedItemInfo) { - ScholixSummary summary = new ScholixSummary(); - summary.setId(item.getId()); + private static ScholixSummary summaryFromPublication( + final DLIPublication item, final RelatedItemInfo relatedItemInfo) { + ScholixSummary summary = new ScholixSummary(); + summary.setId(item.getId()); - if (item.getPid() != null) - summary.setLocalIdentifier( - item.getPid().stream() - .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) - .collect(Collectors.toList())); + if (item.getPid() != null) + summary + .setLocalIdentifier( + item + .getPid() + .stream() + .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) + .collect(Collectors.toList())); - summary.setTypology(Typology.publication); - if (item.getTitle() != null) - summary.setTitle( - item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList())); + summary.setTypology(Typology.publication); + if (item.getTitle() != null) + summary + .setTitle( + item.getTitle().stream().map(StructuredProperty::getValue).collect(Collectors.toList())); - if (item.getAuthor() != null) { - summary.setAuthor( - item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList())); - } + if (item.getAuthor() != null) { + summary + .setAuthor( + item.getAuthor().stream().map(Author::getFullname).collect(Collectors.toList())); + } - if (item.getRelevantdate() != null) - summary.setDate( - item.getRelevantdate().stream() - .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname())) - .map(StructuredProperty::getValue) - .collect(Collectors.toList())); + if (item.getRelevantdate() != null) + summary + .setDate( + item + .getRelevantdate() + .stream() + .filter(d -> "date".equalsIgnoreCase(d.getQualifier().getClassname())) + .map(StructuredProperty::getValue) + .collect(Collectors.toList())); - if (item.getDescription() != null && item.getDescription().size() > 0) - summary.setDescription(item.getDescription().get(0).getValue()); + if (item.getDescription() != null && item.getDescription().size() > 0) + summary.setDescription(item.getDescription().get(0).getValue()); - if (item.getSubject() != null) { - summary.setSubject( - item.getSubject().stream() - .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue())) - .collect(Collectors.toList())); - } + if (item.getSubject() != null) { + summary + .setSubject( + item + .getSubject() + .stream() + .map(s -> new SchemeValue(s.getQualifier().getClassid(), s.getValue())) + .collect(Collectors.toList())); + } - if (item.getPublisher() != null) - summary.setPublisher(Collections.singletonList(item.getPublisher().getValue())); + if (item.getPublisher() != null) + summary.setPublisher(Collections.singletonList(item.getPublisher().getValue())); - summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); - summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); - summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); + summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); + summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); + summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); - if (item.getDlicollectedfrom() != null) - summary.setDatasources( - item.getDlicollectedfrom().stream() - .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) - .collect(Collectors.toList())); + if (item.getDlicollectedfrom() != null) + summary + .setDatasources( + item + .getDlicollectedfrom() + .stream() + .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) + .collect(Collectors.toList())); - return summary; - } + return summary; + } - private static ScholixSummary summaryFromUnknown( - final DLIUnknown item, final RelatedItemInfo relatedItemInfo) { - ScholixSummary summary = new ScholixSummary(); - summary.setId(item.getId()); - if (item.getPid() != null) - summary.setLocalIdentifier( - item.getPid().stream() - .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) - .collect(Collectors.toList())); + private static ScholixSummary summaryFromUnknown( + final DLIUnknown item, final RelatedItemInfo relatedItemInfo) { + ScholixSummary summary = new ScholixSummary(); + summary.setId(item.getId()); + if (item.getPid() != null) + summary + .setLocalIdentifier( + item + .getPid() + .stream() + .map(p -> new TypedIdentifier(p.getValue(), p.getQualifier().getClassid())) + .collect(Collectors.toList())); - summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); - summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); - summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); - summary.setTypology(Typology.unknown); - if (item.getDlicollectedfrom() != null) - summary.setDatasources( - item.getDlicollectedfrom().stream() - .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) - .collect(Collectors.toList())); - return summary; - } + summary.setRelatedDatasets(relatedItemInfo.getRelatedDataset()); + summary.setRelatedPublications(relatedItemInfo.getRelatedPublication()); + summary.setRelatedUnknown(relatedItemInfo.getRelatedUnknown()); + summary.setTypology(Typology.unknown); + if (item.getDlicollectedfrom() != null) + summary + .setDatasources( + item + .getDlicollectedfrom() + .stream() + .map(c -> new CollectedFromType(c.getName(), c.getId(), c.getCompletionStatus())) + .collect(Collectors.toList())); + return summary; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java index 773695eff..c4148ad24 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/TypedIdentifier.java @@ -1,31 +1,33 @@ + package eu.dnetlib.dhp.provision.scholix.summary; import java.io.Serializable; public class TypedIdentifier implements Serializable { - private String id; - private String type; + private String id; + private String type; - public TypedIdentifier() {} + public TypedIdentifier() { + } - public TypedIdentifier(String id, String type) { - this.id = id; - this.type = type; - } + public TypedIdentifier(String id, String type) { + this.id = id; + this.type = type; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(String type) { - this.type = type; - } + public void setType(String type) { + this.type = type; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java index d90e224f9..effa32b6b 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/scholix/summary/Typology.java @@ -1,9 +1,8 @@ + package eu.dnetlib.dhp.provision.scholix.summary; import java.io.Serializable; public enum Typology implements Serializable { - dataset, - publication, - unknown + dataset, publication, unknown } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java index 7e8e7aef3..bc9562e08 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossRefParserJSON.java @@ -1,121 +1,131 @@ + package eu.dnetlib.dhp.provision.update; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + import com.google.gson.JsonArray; import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.JsonParser; + import eu.dnetlib.dhp.provision.scholix.ScholixCollectedFrom; import eu.dnetlib.dhp.provision.scholix.ScholixEntityId; import eu.dnetlib.dhp.provision.scholix.ScholixIdentifier; import eu.dnetlib.dhp.provision.scholix.ScholixResource; import eu.dnetlib.dhp.utils.DHPUtils; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; public class CrossRefParserJSON { - private static List collectedFrom = - generateCrossrefCollectedFrom("complete"); + private static List collectedFrom = generateCrossrefCollectedFrom("complete"); - public static ScholixResource parseRecord(final String record) { - if (record == null) return null; - JsonElement jElement = new JsonParser().parse(record); - JsonElement source = null; - if (jElement.getAsJsonObject().has("_source")) { - source = jElement.getAsJsonObject().get("_source"); - if (source == null || !source.isJsonObject()) return null; - } else if (jElement.getAsJsonObject().has("DOI")) { - source = jElement; - } else { - return null; - } + public static ScholixResource parseRecord(final String record) { + if (record == null) + return null; + JsonElement jElement = new JsonParser().parse(record); + JsonElement source = null; + if (jElement.getAsJsonObject().has("_source")) { + source = jElement.getAsJsonObject().get("_source"); + if (source == null || !source.isJsonObject()) + return null; + } else if (jElement.getAsJsonObject().has("DOI")) { + source = jElement; + } else { + return null; + } - final JsonObject message = source.getAsJsonObject(); - ScholixResource currentObject = new ScholixResource(); + final JsonObject message = source.getAsJsonObject(); + ScholixResource currentObject = new ScholixResource(); - if (message.get("DOI") != null) { - final String doi = message.get("DOI").getAsString(); - currentObject.setIdentifier(Collections.singletonList(new ScholixIdentifier(doi, "doi"))); - } + if (message.get("DOI") != null) { + final String doi = message.get("DOI").getAsString(); + currentObject.setIdentifier(Collections.singletonList(new ScholixIdentifier(doi, "doi"))); + } - if ((!message.get("created").isJsonNull()) - && (message.getAsJsonObject("created").get("date-time") != null)) { - currentObject.setPublicationDate( - message.getAsJsonObject("created").get("date-time").getAsString()); - } + if ((!message.get("created").isJsonNull()) + && (message.getAsJsonObject("created").get("date-time") != null)) { + currentObject + .setPublicationDate( + message.getAsJsonObject("created").get("date-time").getAsString()); + } - if (message.get("title") != null - && !message.get("title").isJsonNull() - && message.get("title").isJsonArray()) { + if (message.get("title") != null + && !message.get("title").isJsonNull() + && message.get("title").isJsonArray()) { - JsonArray array = message.get("title").getAsJsonArray(); - currentObject.setTitle(array.get(0).getAsString()); - } - if (message.get("author") != null && !message.get("author").isJsonNull()) { - JsonArray author = message.getAsJsonArray("author"); - List authorList = new ArrayList<>(); - for (JsonElement anAuthor : author) { - JsonObject currentAuth = anAuthor.getAsJsonObject(); + JsonArray array = message.get("title").getAsJsonArray(); + currentObject.setTitle(array.get(0).getAsString()); + } + if (message.get("author") != null && !message.get("author").isJsonNull()) { + JsonArray author = message.getAsJsonArray("author"); + List authorList = new ArrayList<>(); + for (JsonElement anAuthor : author) { + JsonObject currentAuth = anAuthor.getAsJsonObject(); - String family = ""; - String given = ""; - if (currentAuth != null - && currentAuth.get("family") != null - && !currentAuth.get("family").isJsonNull()) { - family = currentAuth.get("family").getAsString(); - } - if (currentAuth != null - && currentAuth.get("given") != null - && !currentAuth.get("given").isJsonNull()) { - given = currentAuth.get("given").getAsString(); - } - authorList.add(new ScholixEntityId(String.format("%s %s", family, given), null)); - } - currentObject.setCreator(authorList); - } - if (message.get("publisher") != null && !message.get("publisher").isJsonNull()) { - currentObject.setPublisher( - Collections.singletonList( - new ScholixEntityId(message.get("publisher").getAsString(), null))); - } - currentObject.setCollectedFrom(collectedFrom); - currentObject.setObjectType("publication"); - currentObject.setDnetIdentifier( - generateId(message.get("DOI").getAsString(), "doi", "publication")); + String family = ""; + String given = ""; + if (currentAuth != null + && currentAuth.get("family") != null + && !currentAuth.get("family").isJsonNull()) { + family = currentAuth.get("family").getAsString(); + } + if (currentAuth != null + && currentAuth.get("given") != null + && !currentAuth.get("given").isJsonNull()) { + given = currentAuth.get("given").getAsString(); + } + authorList.add(new ScholixEntityId(String.format("%s %s", family, given), null)); + } + currentObject.setCreator(authorList); + } + if (message.get("publisher") != null && !message.get("publisher").isJsonNull()) { + currentObject + .setPublisher( + Collections + .singletonList( + new ScholixEntityId(message.get("publisher").getAsString(), null))); + } + currentObject.setCollectedFrom(collectedFrom); + currentObject.setObjectType("publication"); + currentObject + .setDnetIdentifier( + generateId(message.get("DOI").getAsString(), "doi", "publication")); - return currentObject; - } + return currentObject; + } - private static List generateCrossrefCollectedFrom( - final String completionStatus) { - final ScholixEntityId scholixEntityId = - new ScholixEntityId( - "Crossref", - Collections.singletonList( - new ScholixIdentifier("dli_________::crossref", "dnet_identifier"))); - return Collections.singletonList( - new ScholixCollectedFrom(scholixEntityId, "resolved", completionStatus)); - } + private static List generateCrossrefCollectedFrom( + final String completionStatus) { + final ScholixEntityId scholixEntityId = new ScholixEntityId( + "Crossref", + Collections + .singletonList( + new ScholixIdentifier("dli_________::crossref", "dnet_identifier"))); + return Collections + .singletonList( + new ScholixCollectedFrom(scholixEntityId, "resolved", completionStatus)); + } - private static String generateId( - final String pid, final String pidType, final String entityType) { - String type; - switch (entityType) { - case "publication": - type = "50|"; - break; - case "dataset": - type = "60|"; - break; - case "unknown": - type = "70|"; - break; - default: - throw new IllegalArgumentException("unexpected value " + entityType); - } - return type - + DHPUtils.md5( - String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); - } + private static String generateId( + final String pid, final String pidType, final String entityType) { + String type; + switch (entityType) { + case "publication": + type = "50|"; + break; + case "dataset": + type = "60|"; + break; + case "unknown": + type = "70|"; + break; + default: + throw new IllegalArgumentException("unexpected value " + entityType); + } + return type + + DHPUtils + .md5( + String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java index e5aa38c1d..fac1da253 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/CrossrefClient.java @@ -1,10 +1,9 @@ + package eu.dnetlib.dhp.provision.update; -import com.google.gson.JsonElement; -import com.google.gson.JsonParser; -import eu.dnetlib.dhp.provision.scholix.ScholixResource; import java.io.ByteArrayOutputStream; import java.util.zip.Inflater; + import org.apache.commons.codec.binary.Base64; import org.apache.commons.io.IOUtils; import org.apache.http.client.methods.CloseableHttpResponse; @@ -12,77 +11,81 @@ import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; +import com.google.gson.JsonElement; +import com.google.gson.JsonParser; + +import eu.dnetlib.dhp.provision.scholix.ScholixResource; + public class CrossrefClient { - private String host; - private String index = "crossref"; - private String indexType = "item"; + private String host; + private String index = "crossref"; + private String indexType = "item"; - public CrossrefClient(String host) { - this.host = host; - } + public CrossrefClient(String host) { + this.host = host; + } - public String getHost() { - return host; - } + public String getHost() { + return host; + } - public void setHost(String host) { - this.host = host; - } + public void setHost(String host) { + this.host = host; + } - public String getIndex() { - return index; - } + public String getIndex() { + return index; + } - public void setIndex(String index) { - this.index = index; - } + public void setIndex(String index) { + this.index = index; + } - public String getIndexType() { - return indexType; - } + public String getIndexType() { + return indexType; + } - public void setIndexType(String indexType) { - this.indexType = indexType; - } + public void setIndexType(String indexType) { + this.indexType = indexType; + } - private static String decompressBlob(final String blob) { - try { - byte[] byteArray = Base64.decodeBase64(blob.getBytes()); - final Inflater decompresser = new Inflater(); - decompresser.setInput(byteArray); - final ByteArrayOutputStream bos = new ByteArrayOutputStream(byteArray.length); - byte[] buffer = new byte[8192]; - while (!decompresser.finished()) { - int size = decompresser.inflate(buffer); - bos.write(buffer, 0, size); - } - byte[] unzippeddata = bos.toByteArray(); - decompresser.end(); - return new String(unzippeddata); - } catch (Throwable e) { - throw new RuntimeException("Wrong record:" + blob, e); - } - } + private static String decompressBlob(final String blob) { + try { + byte[] byteArray = Base64.decodeBase64(blob.getBytes()); + final Inflater decompresser = new Inflater(); + decompresser.setInput(byteArray); + final ByteArrayOutputStream bos = new ByteArrayOutputStream(byteArray.length); + byte[] buffer = new byte[8192]; + while (!decompresser.finished()) { + int size = decompresser.inflate(buffer); + bos.write(buffer, 0, size); + } + byte[] unzippeddata = bos.toByteArray(); + decompresser.end(); + return new String(unzippeddata); + } catch (Throwable e) { + throw new RuntimeException("Wrong record:" + blob, e); + } + } - public ScholixResource getResourceByDOI(final String doi) { - try (CloseableHttpClient client = HttpClients.createDefault()) { - HttpGet httpGet = - new HttpGet( - String.format( - "http://%s:9200/%s/%s/%s", host, index, indexType, doi.replaceAll("/", "%2F"))); - CloseableHttpResponse response = client.execute(httpGet); - String json = IOUtils.toString(response.getEntity().getContent()); - if (json.contains("blob")) { - JsonParser p = new JsonParser(); - final JsonElement root = p.parse(json); - json = - decompressBlob( - root.getAsJsonObject().get("_source").getAsJsonObject().get("blob").getAsString()); - } - return CrossRefParserJSON.parseRecord(json); - } catch (Throwable e) { - return null; - } - } + public ScholixResource getResourceByDOI(final String doi) { + try (CloseableHttpClient client = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet( + String + .format( + "http://%s:9200/%s/%s/%s", host, index, indexType, doi.replaceAll("/", "%2F"))); + CloseableHttpResponse response = client.execute(httpGet); + String json = IOUtils.toString(response.getEntity().getContent()); + if (json.contains("blob")) { + JsonParser p = new JsonParser(); + final JsonElement root = p.parse(json); + json = decompressBlob( + root.getAsJsonObject().get("_source").getAsJsonObject().get("blob").getAsString()); + } + return CrossRefParserJSON.parseRecord(json); + } catch (Throwable e) { + return null; + } + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java index 3eed64d4d..10426b29c 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/Datacite2Scholix.java @@ -1,218 +1,229 @@ + package eu.dnetlib.dhp.provision.update; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.provision.scholix.*; -import eu.dnetlib.dhp.utils.DHPUtils; -import eu.dnetlib.scholexplorer.relation.RelInfo; -import eu.dnetlib.scholexplorer.relation.RelationMapper; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; +import com.jayway.jsonpath.JsonPath; + +import eu.dnetlib.dhp.provision.scholix.*; +import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.scholexplorer.relation.RelInfo; +import eu.dnetlib.scholexplorer.relation.RelationMapper; + public class Datacite2Scholix { - private String rootPath = "$.attributes"; - final RelationMapper relationMapper; + private String rootPath = "$.attributes"; + final RelationMapper relationMapper; - public Datacite2Scholix(RelationMapper relationMapper) { - this.relationMapper = relationMapper; - } + public Datacite2Scholix(RelationMapper relationMapper) { + this.relationMapper = relationMapper; + } - public List generateScholixFromJson(final String dJson) { - List> relIds = getRelatedIendtifiers(dJson); - relIds = - relIds != null - ? relIds.stream() - .filter( - m -> - m.containsKey("relatedIdentifierType") - && m.containsKey("relationType") - && m.containsKey("relatedIdentifier")) - .collect(Collectors.toList()) - : null; - if (relIds == null || relIds.size() == 0) return null; + public List generateScholixFromJson(final String dJson) { + List> relIds = getRelatedIendtifiers(dJson); + relIds = relIds != null + ? relIds + .stream() + .filter( + m -> m.containsKey("relatedIdentifierType") + && m.containsKey("relationType") + && m.containsKey("relatedIdentifier")) + .collect(Collectors.toList()) + : null; + if (relIds == null || relIds.size() == 0) + return null; - final String updated = JsonPath.read(dJson, rootPath + ".updated"); - ScholixResource resource = generateDataciteScholixResource(dJson); + final String updated = JsonPath.read(dJson, rootPath + ".updated"); + ScholixResource resource = generateDataciteScholixResource(dJson); - return relIds.stream() - .flatMap( - s -> { - try { - final List result = - generateScholix( - resource, - "" + s.get("relatedIdentifier"), - s.get("relatedIdentifierType"), - s.get("relationType"), - updated); - return result.stream(); - } catch (Throwable e) { - return new ArrayList().stream(); - } - }) - .collect(Collectors.toList()); - } + return relIds + .stream() + .flatMap( + s -> { + try { + final List result = generateScholix( + resource, + "" + s.get("relatedIdentifier"), + s.get("relatedIdentifierType"), + s.get("relationType"), + updated); + return result.stream(); + } catch (Throwable e) { + return new ArrayList().stream(); + } + }) + .collect(Collectors.toList()); + } - public String getRootPath() { - return rootPath; - } + public String getRootPath() { + return rootPath; + } - public void setRootPath(String rootPath) { - this.rootPath = rootPath; - } + public void setRootPath(String rootPath) { + this.rootPath = rootPath; + } - private List generateScholix( - ScholixResource source, - final String pid, - final String pidtype, - final String relType, - final String updated) { + private List generateScholix( + ScholixResource source, + final String pid, + final String pidtype, + final String relType, + final String updated) { - if ("doi".equalsIgnoreCase(pidtype)) { - ScholixResource target = new ScholixResource(); - target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype))); - final RelInfo relInfo = relationMapper.get(relType.toLowerCase()); - final ScholixRelationship rel = - new ScholixRelationship(relInfo.getOriginal(), "datacite", relInfo.getInverse()); - final ScholixEntityId provider = source.getCollectedFrom().get(0).getProvider(); - final Scholix s = new Scholix(); - s.setSource(source); - s.setTarget(target); - s.setLinkprovider(Collections.singletonList(provider)); - s.setPublisher(source.getPublisher()); - s.setRelationship(rel); - s.setPublicationDate(updated); - return Collections.singletonList(s); - } else { - final List result = new ArrayList<>(); - ScholixResource target = new ScholixResource(); - target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype))); - target.setDnetIdentifier(generateId(pid, pidtype, "unknown")); - target.setObjectType("unknown"); - target.setCollectedFrom(generateDataciteCollectedFrom("incomplete")); - final RelInfo relInfo = relationMapper.get(relType.toLowerCase()); - final ScholixRelationship rel = - new ScholixRelationship(relInfo.getOriginal(), "datacite", relInfo.getInverse()); - final ScholixEntityId provider = source.getCollectedFrom().get(0).getProvider(); - final Scholix s = new Scholix(); - s.setSource(source); - s.setTarget(target); - s.setLinkprovider(Collections.singletonList(provider)); - s.setPublisher(source.getPublisher()); - s.setRelationship(rel); - s.setPublicationDate(updated); - s.generateIdentifier(); - result.add(s); - final Scholix s2 = new Scholix(); - s2.setSource(target); - s2.setTarget(source); - s2.setLinkprovider(Collections.singletonList(provider)); - s2.setPublisher(source.getPublisher()); - s2.setRelationship( - new ScholixRelationship(relInfo.getInverse(), "datacite", relInfo.getOriginal())); - s2.setPublicationDate(updated); - s2.generateIdentifier(); - result.add(s2); - return result; - } - } + if ("doi".equalsIgnoreCase(pidtype)) { + ScholixResource target = new ScholixResource(); + target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype))); + final RelInfo relInfo = relationMapper.get(relType.toLowerCase()); + final ScholixRelationship rel = new ScholixRelationship(relInfo.getOriginal(), "datacite", + relInfo.getInverse()); + final ScholixEntityId provider = source.getCollectedFrom().get(0).getProvider(); + final Scholix s = new Scholix(); + s.setSource(source); + s.setTarget(target); + s.setLinkprovider(Collections.singletonList(provider)); + s.setPublisher(source.getPublisher()); + s.setRelationship(rel); + s.setPublicationDate(updated); + return Collections.singletonList(s); + } else { + final List result = new ArrayList<>(); + ScholixResource target = new ScholixResource(); + target.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, pidtype))); + target.setDnetIdentifier(generateId(pid, pidtype, "unknown")); + target.setObjectType("unknown"); + target.setCollectedFrom(generateDataciteCollectedFrom("incomplete")); + final RelInfo relInfo = relationMapper.get(relType.toLowerCase()); + final ScholixRelationship rel = new ScholixRelationship(relInfo.getOriginal(), "datacite", + relInfo.getInverse()); + final ScholixEntityId provider = source.getCollectedFrom().get(0).getProvider(); + final Scholix s = new Scholix(); + s.setSource(source); + s.setTarget(target); + s.setLinkprovider(Collections.singletonList(provider)); + s.setPublisher(source.getPublisher()); + s.setRelationship(rel); + s.setPublicationDate(updated); + s.generateIdentifier(); + result.add(s); + final Scholix s2 = new Scholix(); + s2.setSource(target); + s2.setTarget(source); + s2.setLinkprovider(Collections.singletonList(provider)); + s2.setPublisher(source.getPublisher()); + s2 + .setRelationship( + new ScholixRelationship(relInfo.getInverse(), "datacite", relInfo.getOriginal())); + s2.setPublicationDate(updated); + s2.generateIdentifier(); + result.add(s2); + return result; + } + } - public ScholixResource generateDataciteScholixResource(String dJson) { - ScholixResource resource = new ScholixResource(); - String DOI_PATH = rootPath + ".doi"; - final String doi = JsonPath.read(dJson, DOI_PATH); - resource.setIdentifier(Collections.singletonList(new ScholixIdentifier(doi, "doi"))); - resource.setObjectType(getType(dJson)); - resource.setDnetIdentifier(generateId(doi, "doi", resource.getObjectType())); - resource.setCollectedFrom(generateDataciteCollectedFrom("complete")); - final String publisher = JsonPath.read(dJson, rootPath + ".publisher"); - if (StringUtils.isNotBlank(publisher)) - resource.setPublisher(Collections.singletonList(new ScholixEntityId(publisher, null))); - final String date = getDate(dJson); - if (StringUtils.isNotBlank(date)) resource.setPublicationDate(date); - final String title = getTitle(dJson); - if (StringUtils.isNotBlank(title)) resource.setTitle(title); - resource.setCreator(getCreators(dJson)); - return resource; - } + public ScholixResource generateDataciteScholixResource(String dJson) { + ScholixResource resource = new ScholixResource(); + String DOI_PATH = rootPath + ".doi"; + final String doi = JsonPath.read(dJson, DOI_PATH); + resource.setIdentifier(Collections.singletonList(new ScholixIdentifier(doi, "doi"))); + resource.setObjectType(getType(dJson)); + resource.setDnetIdentifier(generateId(doi, "doi", resource.getObjectType())); + resource.setCollectedFrom(generateDataciteCollectedFrom("complete")); + final String publisher = JsonPath.read(dJson, rootPath + ".publisher"); + if (StringUtils.isNotBlank(publisher)) + resource.setPublisher(Collections.singletonList(new ScholixEntityId(publisher, null))); + final String date = getDate(dJson); + if (StringUtils.isNotBlank(date)) + resource.setPublicationDate(date); + final String title = getTitle(dJson); + if (StringUtils.isNotBlank(title)) + resource.setTitle(title); + resource.setCreator(getCreators(dJson)); + return resource; + } - private List getCreators(final String json) { - final List creatorName = JsonPath.read(json, rootPath + ".creators[*].name"); - if (creatorName != null && creatorName.size() > 0) { - return creatorName.stream() - .map(s -> new ScholixEntityId(s, null)) - .collect(Collectors.toList()); - } - return null; - } + private List getCreators(final String json) { + final List creatorName = JsonPath.read(json, rootPath + ".creators[*].name"); + if (creatorName != null && creatorName.size() > 0) { + return creatorName + .stream() + .map(s -> new ScholixEntityId(s, null)) + .collect(Collectors.toList()); + } + return null; + } - private String getTitle(final String json) { - final List titles = JsonPath.read(json, rootPath + ".titles[*].title"); - return titles != null && titles.size() > 0 ? titles.get(0) : null; - } + private String getTitle(final String json) { + final List titles = JsonPath.read(json, rootPath + ".titles[*].title"); + return titles != null && titles.size() > 0 ? titles.get(0) : null; + } - private String getDate(final String json) { - final List> dates = JsonPath.read(json, rootPath + ".dates"); - if (dates != null && dates.size() > 0) { + private String getDate(final String json) { + final List> dates = JsonPath.read(json, rootPath + ".dates"); + if (dates != null && dates.size() > 0) { - List> issued = - dates.stream() - .filter(s -> "issued".equalsIgnoreCase(s.get("dateType"))) - .collect(Collectors.toList()); - if (issued.size() > 0) return issued.get(0).get("date"); - } - return null; - } + List> issued = dates + .stream() + .filter(s -> "issued".equalsIgnoreCase(s.get("dateType"))) + .collect(Collectors.toList()); + if (issued.size() > 0) + return issued.get(0).get("date"); + } + return null; + } - private List generateDataciteCollectedFrom(final String completionStatus) { - final ScholixEntityId scholixEntityId = - new ScholixEntityId( - "Datasets in Datacite", - Collections.singletonList( - new ScholixIdentifier("dli_________::datacite", "dnet_identifier"))); - return Collections.singletonList( - new ScholixCollectedFrom(scholixEntityId, "collected", completionStatus)); - } + private List generateDataciteCollectedFrom(final String completionStatus) { + final ScholixEntityId scholixEntityId = new ScholixEntityId( + "Datasets in Datacite", + Collections + .singletonList( + new ScholixIdentifier("dli_________::datacite", "dnet_identifier"))); + return Collections + .singletonList( + new ScholixCollectedFrom(scholixEntityId, "collected", completionStatus)); + } - private String getType(final String json) { - try { - final String bibtext = JsonPath.read(json, rootPath + ".types.bibtex"); - if ("article".equalsIgnoreCase(bibtext)) { - return "publication"; - } - return "dataset"; - } catch (Throwable e) { - return "dataset"; - } - } + private String getType(final String json) { + try { + final String bibtext = JsonPath.read(json, rootPath + ".types.bibtex"); + if ("article".equalsIgnoreCase(bibtext)) { + return "publication"; + } + return "dataset"; + } catch (Throwable e) { + return "dataset"; + } + } - private List> getRelatedIendtifiers(final String json) { - String REL_IDENTIFIER_PATH = rootPath + ".relatedIdentifiers[*]"; - List> res = JsonPath.read(json, REL_IDENTIFIER_PATH); - return res; - } + private List> getRelatedIendtifiers(final String json) { + String REL_IDENTIFIER_PATH = rootPath + ".relatedIdentifiers[*]"; + List> res = JsonPath.read(json, REL_IDENTIFIER_PATH); + return res; + } - public static String generateId(final String pid, final String pidType, final String entityType) { - String type; - switch (entityType) { - case "publication": - type = "50|"; - break; - case "dataset": - type = "60|"; - break; - case "unknown": - type = "70|"; - break; - default: - throw new IllegalArgumentException("unexpected value " + entityType); - } - return type - + DHPUtils.md5( - String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); - } + public static String generateId(final String pid, final String pidType, final String entityType) { + String type; + switch (entityType) { + case "publication": + type = "50|"; + break; + case "dataset": + type = "60|"; + break; + case "unknown": + type = "70|"; + break; + default: + throw new IllegalArgumentException("unexpected value " + entityType); + } + return type + + DHPUtils + .md5( + String.format("%s::%s", pid.toLowerCase().trim(), pidType.toLowerCase().trim())); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClient.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClient.java index a4e77b37c..e84ec4376 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClient.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClient.java @@ -1,72 +1,75 @@ + package eu.dnetlib.dhp.provision.update; -import eu.dnetlib.dhp.provision.scholix.ScholixResource; import java.io.IOException; + import org.apache.commons.io.IOUtils; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; +import eu.dnetlib.dhp.provision.scholix.ScholixResource; + public class DataciteClient { - private String host; - private String index = "datacite"; - private String indexType = "dump"; - private Datacite2Scholix d2s; + private String host; + private String index = "datacite"; + private String indexType = "dump"; + private Datacite2Scholix d2s; - public DataciteClient(String host) { - this.host = host; + public DataciteClient(String host) { + this.host = host; - d2s = new Datacite2Scholix(null); - d2s.setRootPath("$._source.attributes"); - } + d2s = new Datacite2Scholix(null); + d2s.setRootPath("$._source.attributes"); + } - public Iterable getDatasetsFromTs(final Long timestamp) { - return () -> { - try { - return new DataciteClientIterator(host, index, timestamp); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - } + public Iterable getDatasetsFromTs(final Long timestamp) { + return () -> { + try { + return new DataciteClientIterator(host, index, timestamp); + } catch (IOException e) { + throw new RuntimeException(e); + } + }; + } - public String getHost() { - return host; - } + public String getHost() { + return host; + } - public void setHost(String host) { - this.host = host; - } + public void setHost(String host) { + this.host = host; + } - public String getIndex() { - return index; - } + public String getIndex() { + return index; + } - public void setIndex(String index) { - this.index = index; - } + public void setIndex(String index) { + this.index = index; + } - public String getIndexType() { - return indexType; - } + public String getIndexType() { + return indexType; + } - public void setIndexType(String indexType) { - this.indexType = indexType; - } + public void setIndexType(String indexType) { + this.indexType = indexType; + } - public ScholixResource getDatasetByDOI(final String doi) { - try (CloseableHttpClient client = HttpClients.createDefault()) { - HttpGet httpGet = - new HttpGet( - String.format( - "http://%s:9200/%s/%s/%s", host, index, indexType, doi.replaceAll("/", "%2F"))); - CloseableHttpResponse response = client.execute(httpGet); - final String json = IOUtils.toString(response.getEntity().getContent()); - return d2s.generateDataciteScholixResource(json); - } catch (Throwable e) { - return null; - } - } + public ScholixResource getDatasetByDOI(final String doi) { + try (CloseableHttpClient client = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet( + String + .format( + "http://%s:9200/%s/%s/%s", host, index, indexType, doi.replaceAll("/", "%2F"))); + CloseableHttpResponse response = client.execute(httpGet); + final String json = IOUtils.toString(response.getEntity().getContent()); + return d2s.generateDataciteScholixResource(json); + } catch (Throwable e) { + return null; + } + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java index fa9dc5646..2c70c8b09 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/DataciteClientIterator.java @@ -1,12 +1,11 @@ + package eu.dnetlib.dhp.provision.update; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.jayway.jsonpath.JsonPath; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; -import net.minidev.json.JSONArray; + import org.apache.commons.io.IOUtils; import org.apache.http.client.methods.CloseableHttpResponse; import org.apache.http.client.methods.HttpPost; @@ -14,103 +13,108 @@ import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.jayway.jsonpath.JsonPath; + +import net.minidev.json.JSONArray; + public class DataciteClientIterator implements Iterator { - static final String blobPath = "$.hits.hits[*]._source"; - static final String scrollIdPath = "$._scroll_id"; + static final String blobPath = "$.hits.hits[*]._source"; + static final String scrollIdPath = "$._scroll_id"; - String scrollId; + String scrollId; - List buffer; + List buffer; - final String esHost; - final String esIndex; - final ObjectMapper mapper = new ObjectMapper(); + final String esHost; + final String esIndex; + final ObjectMapper mapper = new ObjectMapper(); - public DataciteClientIterator(final String esHost, final String esIndex, long timestamp) - throws IOException { + public DataciteClientIterator(final String esHost, final String esIndex, long timestamp) + throws IOException { - this.esHost = esHost; - this.esIndex = esIndex; - // THIS FIX IS NECESSARY to avoid different timezone - timestamp -= (60 * 60 * 2); - final String body = - getResponse( - String.format("http://%s:9200/%s/_search?scroll=1m", esHost, esIndex), - String.format( - "{\"size\":1000, \"query\":{\"range\":{\"timestamp\":{\"gte\":%d}}}}", timestamp)); - scrollId = getJPathString(scrollIdPath, body); - buffer = getBlobs(body); - } + this.esHost = esHost; + this.esIndex = esIndex; + // THIS FIX IS NECESSARY to avoid different timezone + timestamp -= (60 * 60 * 2); + final String body = getResponse( + String.format("http://%s:9200/%s/_search?scroll=1m", esHost, esIndex), + String + .format( + "{\"size\":1000, \"query\":{\"range\":{\"timestamp\":{\"gte\":%d}}}}", timestamp)); + scrollId = getJPathString(scrollIdPath, body); + buffer = getBlobs(body); + } - public String getResponse(final String url, final String json) { - CloseableHttpClient client = HttpClients.createDefault(); - try { + public String getResponse(final String url, final String json) { + CloseableHttpClient client = HttpClients.createDefault(); + try { - HttpPost httpPost = new HttpPost(url); - if (json != null) { - StringEntity entity = new StringEntity(json); - httpPost.setEntity(entity); - httpPost.setHeader("Accept", "application/json"); - httpPost.setHeader("Content-type", "application/json"); - } - CloseableHttpResponse response = client.execute(httpPost); + HttpPost httpPost = new HttpPost(url); + if (json != null) { + StringEntity entity = new StringEntity(json); + httpPost.setEntity(entity); + httpPost.setHeader("Accept", "application/json"); + httpPost.setHeader("Content-type", "application/json"); + } + CloseableHttpResponse response = client.execute(httpPost); - return IOUtils.toString(response.getEntity().getContent()); - } catch (Throwable e) { - throw new RuntimeException("Error on executing request ", e); - } finally { - try { - client.close(); - } catch (IOException e) { - throw new RuntimeException("Unable to close client ", e); - } - } - } + return IOUtils.toString(response.getEntity().getContent()); + } catch (Throwable e) { + throw new RuntimeException("Error on executing request ", e); + } finally { + try { + client.close(); + } catch (IOException e) { + throw new RuntimeException("Unable to close client ", e); + } + } + } - private String getJPathString(final String jsonPath, final String json) { - try { - Object o = JsonPath.read(json, jsonPath); - if (o instanceof String) return (String) o; - return null; - } catch (Exception e) { - return ""; - } - } + private String getJPathString(final String jsonPath, final String json) { + try { + Object o = JsonPath.read(json, jsonPath); + if (o instanceof String) + return (String) o; + return null; + } catch (Exception e) { + return ""; + } + } - private List getBlobs(final String body) { - JSONArray array = JsonPath.read(body, blobPath); - return array.stream() - .map( - o -> { - try { - return mapper.writeValueAsString(o); - } catch (Throwable e) { - throw new RuntimeException(e); - } - }) - .collect(Collectors.toList()); - } + private List getBlobs(final String body) { + JSONArray array = JsonPath.read(body, blobPath); + return array + .stream() + .map( + o -> { + try { + return mapper.writeValueAsString(o); + } catch (Throwable e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + } - @Override - public boolean hasNext() { - return (buffer != null && !buffer.isEmpty()); - } + @Override + public boolean hasNext() { + return (buffer != null && !buffer.isEmpty()); + } - @Override - public String next() { - final String nextItem = buffer.remove(0); - if (buffer.isEmpty()) { - final String json_param = - String.format("{\"scroll_id\":\"%s\",\"scroll\" : \"1m\"}", scrollId); - final String body = - getResponse(String.format("http://%s:9200/_search/scroll", esHost), json_param); - try { - buffer = getBlobs(body); - } catch (Throwable e) { - System.out.println(body); - } - } - return nextItem; - } + @Override + public String next() { + final String nextItem = buffer.remove(0); + if (buffer.isEmpty()) { + final String json_param = String.format("{\"scroll_id\":\"%s\",\"scroll\" : \"1m\"}", scrollId); + final String body = getResponse(String.format("http://%s:9200/_search/scroll", esHost), json_param); + try { + buffer = getBlobs(body); + } catch (Throwable e) { + System.out.println(body); + } + } + return nextItem; + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/RetrieveUpdateFromDatacite.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/RetrieveUpdateFromDatacite.java index 15c396b10..e876d05a1 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/RetrieveUpdateFromDatacite.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/RetrieveUpdateFromDatacite.java @@ -1,11 +1,9 @@ + package eu.dnetlib.dhp.provision.update; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.provision.scholix.Scholix; -import eu.dnetlib.scholexplorer.relation.RelationMapper; import java.net.URI; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -14,54 +12,61 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.provision.scholix.Scholix; +import eu.dnetlib.scholexplorer.relation.RelationMapper; + public class RetrieveUpdateFromDatacite { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - RetrieveUpdateFromDatacite.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/input_retrieve_update_parameters.json"))); - parser.parseArgument(args); - final String hdfsuri = parser.get("namenode"); - Path hdfswritepath = new Path(parser.get("targetPath")); - final long timestamp = Long.parseLong(parser.get("timestamp")); - final String host = parser.get("indexHost"); - final String index = parser.get("indexName"); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + RetrieveUpdateFromDatacite.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/input_retrieve_update_parameters.json"))); + parser.parseArgument(args); + final String hdfsuri = parser.get("namenode"); + Path hdfswritepath = new Path(parser.get("targetPath")); + final long timestamp = Long.parseLong(parser.get("timestamp")); + final String host = parser.get("indexHost"); + final String index = parser.get("indexName"); - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - FileSystem.get(URI.create(hdfsuri), conf); - final Datacite2Scholix d2s = new Datacite2Scholix(RelationMapper.load()); - final ObjectMapper mapper = new ObjectMapper(); - try (SequenceFile.Writer writer = - SequenceFile.createWriter( - conf, - SequenceFile.Writer.file(hdfswritepath), - SequenceFile.Writer.keyClass(IntWritable.class), - SequenceFile.Writer.valueClass(Text.class))) { - final Text value = new Text(); - final IntWritable key = new IntWritable(); - int i = 0; - for (String dataset : new DataciteClient(host).getDatasetsFromTs(timestamp)) { - i++; - List scholix = d2s.generateScholixFromJson(dataset); - if (scholix != null) - for (Scholix s : scholix) { - key.set(i); - value.set(mapper.writeValueAsString(s)); - writer.append(key, value); - if (i % 10000 == 0) { - System.out.println("wrote " + i); - } - } - } - } - } + FileSystem.get(URI.create(hdfsuri), conf); + final Datacite2Scholix d2s = new Datacite2Scholix(RelationMapper.load()); + final ObjectMapper mapper = new ObjectMapper(); + try (SequenceFile.Writer writer = SequenceFile + .createWriter( + conf, + SequenceFile.Writer.file(hdfswritepath), + SequenceFile.Writer.keyClass(IntWritable.class), + SequenceFile.Writer.valueClass(Text.class))) { + final Text value = new Text(); + final IntWritable key = new IntWritable(); + int i = 0; + for (String dataset : new DataciteClient(host).getDatasetsFromTs(timestamp)) { + i++; + List scholix = d2s.generateScholixFromJson(dataset); + if (scholix != null) + for (Scholix s : scholix) { + key.set(i); + value.set(mapper.writeValueAsString(s)); + writer.append(key, value); + if (i % 10000 == 0) { + System.out.println("wrote " + i); + } + } + } + } + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java index 09a5c7c3d..981c471ae 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/provision/update/SparkResolveScholixTarget.java @@ -1,16 +1,11 @@ + package eu.dnetlib.dhp.provision.update; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.provision.scholix.Scholix; -import eu.dnetlib.dhp.provision.scholix.ScholixIdentifier; -import eu.dnetlib.dhp.provision.scholix.ScholixRelationship; -import eu.dnetlib.dhp.provision.scholix.ScholixResource; -import eu.dnetlib.dhp.utils.DHPUtils; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.IntWritable; @@ -20,150 +15,170 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.provision.scholix.Scholix; +import eu.dnetlib.dhp.provision.scholix.ScholixIdentifier; +import eu.dnetlib.dhp.provision.scholix.ScholixRelationship; +import eu.dnetlib.dhp.provision.scholix.ScholixResource; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; public class SparkResolveScholixTarget { - public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - SparkResolveScholixTarget.class.getResourceAsStream( - "/eu/dnetlib/dhp/provision/input_resolve_scholix_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkResolveScholixTarget.class + .getResourceAsStream( + "/eu/dnetlib/dhp/provision/input_resolve_scholix_parameters.json"))); + parser.parseArgument(args); - final SparkConf conf = new SparkConf(); + final SparkConf conf = new SparkConf(); - final String master = parser.get("master"); - final String sourcePath = parser.get("sourcePath"); - final String workingDirPath = parser.get("workingDirPath"); - final String indexHost = parser.get("indexHost"); - try (SparkSession spark = getSession(conf, master)) { + final String master = parser.get("master"); + final String sourcePath = parser.get("sourcePath"); + final String workingDirPath = parser.get("workingDirPath"); + final String indexHost = parser.get("indexHost"); + try (SparkSession spark = getSession(conf, master)) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - spark - .createDataset( - sc.sequenceFile(sourcePath, IntWritable.class, Text.class) - .map(Tuple2::_2) - .map(s -> new ObjectMapper().readValue(s.toString(), Scholix.class)) - .rdd(), - Encoders.bean(Scholix.class)) - .write() - .save(workingDirPath + "/stepA"); + spark + .createDataset( + sc + .sequenceFile(sourcePath, IntWritable.class, Text.class) + .map(Tuple2::_2) + .map(s -> new ObjectMapper().readValue(s.toString(), Scholix.class)) + .rdd(), + Encoders.bean(Scholix.class)) + .write() + .save(workingDirPath + "/stepA"); - Dataset s1 = - spark.read().load(workingDirPath + "/stepA").as(Encoders.bean(Scholix.class)); + Dataset s1 = spark.read().load(workingDirPath + "/stepA").as(Encoders.bean(Scholix.class)); - s1.where(s1.col("target.dnetIdentifier").isNull()) - .select(s1.col("target.identifier")) - .distinct() - .map( - (MapFunction) - f -> { - final String pid = ((Row) f.getList(0).get(0)).getString(0); - ScholixResource publication = - new CrossrefClient(indexHost).getResourceByDOI(pid); - if (publication != null) { - return publication; - } - ScholixResource dataset = new DataciteClient(indexHost).getDatasetByDOI(pid); - if (dataset != null) { - return dataset; - } - ScholixResource r = new ScholixResource(); - r.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, "doi"))); - r.setObjectType("unknown"); - r.setDnetIdentifier( - "70|" + DHPUtils.md5(String.format("%s::doi", pid.toLowerCase().trim()))); + s1 + .where(s1.col("target.dnetIdentifier").isNull()) + .select(s1.col("target.identifier")) + .distinct() + .map( + (MapFunction) f -> { + final String pid = ((Row) f.getList(0).get(0)).getString(0); + ScholixResource publication = new CrossrefClient(indexHost).getResourceByDOI(pid); + if (publication != null) { + return publication; + } + ScholixResource dataset = new DataciteClient(indexHost).getDatasetByDOI(pid); + if (dataset != null) { + return dataset; + } + ScholixResource r = new ScholixResource(); + r.setIdentifier(Collections.singletonList(new ScholixIdentifier(pid, "doi"))); + r.setObjectType("unknown"); + r + .setDnetIdentifier( + "70|" + DHPUtils.md5(String.format("%s::doi", pid.toLowerCase().trim()))); - return r; - }, - Encoders.bean(ScholixResource.class)) - .write() - .mode(SaveMode.Overwrite) - .save(workingDirPath + "/stepB"); + return r; + }, + Encoders.bean(ScholixResource.class)) + .write() + .mode(SaveMode.Overwrite) + .save(workingDirPath + "/stepB"); - Dataset s2 = - spark.read().load(workingDirPath + "/stepB").as(Encoders.bean(ScholixResource.class)); + Dataset s2 = spark + .read() + .load(workingDirPath + "/stepB") + .as(Encoders.bean(ScholixResource.class)); - s1.joinWith( - s2, - s1.col("target.identifier.identifier").equalTo(s2.col("identifier.identifier")), - "left") - .flatMap( - (FlatMapFunction, Scholix>) - f -> { - final List res = new ArrayList<>(); - final Scholix s = f._1(); - final ScholixResource target = f._2(); - if (StringUtils.isNotBlank(s.getIdentifier())) res.add(s); - else if (target == null) { - ScholixResource currentTarget = s.getTarget(); - currentTarget.setObjectType("unknown"); - currentTarget.setDnetIdentifier( - Datacite2Scholix.generateId( - currentTarget.getIdentifier().get(0).getIdentifier(), - currentTarget.getIdentifier().get(0).getSchema(), - currentTarget.getObjectType())); + s1 + .joinWith( + s2, + s1.col("target.identifier.identifier").equalTo(s2.col("identifier.identifier")), + "left") + .flatMap( + (FlatMapFunction, Scholix>) f -> { + final List res = new ArrayList<>(); + final Scholix s = f._1(); + final ScholixResource target = f._2(); + if (StringUtils.isNotBlank(s.getIdentifier())) + res.add(s); + else if (target == null) { + ScholixResource currentTarget = s.getTarget(); + currentTarget.setObjectType("unknown"); + currentTarget + .setDnetIdentifier( + Datacite2Scholix + .generateId( + currentTarget.getIdentifier().get(0).getIdentifier(), + currentTarget.getIdentifier().get(0).getSchema(), + currentTarget.getObjectType())); - s.generateIdentifier(); - res.add(s); - final Scholix inverse = new Scholix(); - inverse.setTarget(s.getSource()); - inverse.setSource(s.getTarget()); - inverse.setLinkprovider(s.getLinkprovider()); - inverse.setPublicationDate(s.getPublicationDate()); - inverse.setPublisher(s.getPublisher()); - inverse.setRelationship( - new ScholixRelationship( - s.getRelationship().getInverse(), - s.getRelationship().getSchema(), - s.getRelationship().getName())); - inverse.generateIdentifier(); - res.add(inverse); + s.generateIdentifier(); + res.add(s); + final Scholix inverse = new Scholix(); + inverse.setTarget(s.getSource()); + inverse.setSource(s.getTarget()); + inverse.setLinkprovider(s.getLinkprovider()); + inverse.setPublicationDate(s.getPublicationDate()); + inverse.setPublisher(s.getPublisher()); + inverse + .setRelationship( + new ScholixRelationship( + s.getRelationship().getInverse(), + s.getRelationship().getSchema(), + s.getRelationship().getName())); + inverse.generateIdentifier(); + res.add(inverse); - } else { - target.setIdentifier( - target.getIdentifier().stream() - .map( - d -> - new ScholixIdentifier( - d.getIdentifier().toLowerCase(), - d.getSchema().toLowerCase())) - .collect(Collectors.toList())); - s.setTarget(target); - s.generateIdentifier(); - res.add(s); - final Scholix inverse = new Scholix(); - inverse.setTarget(s.getSource()); - inverse.setSource(s.getTarget()); - inverse.setLinkprovider(s.getLinkprovider()); - inverse.setPublicationDate(s.getPublicationDate()); - inverse.setPublisher(s.getPublisher()); - inverse.setRelationship( - new ScholixRelationship( - s.getRelationship().getInverse(), - s.getRelationship().getSchema(), - s.getRelationship().getName())); - inverse.generateIdentifier(); - res.add(inverse); - } + } else { + target + .setIdentifier( + target + .getIdentifier() + .stream() + .map( + d -> new ScholixIdentifier( + d.getIdentifier().toLowerCase(), + d.getSchema().toLowerCase())) + .collect(Collectors.toList())); + s.setTarget(target); + s.generateIdentifier(); + res.add(s); + final Scholix inverse = new Scholix(); + inverse.setTarget(s.getSource()); + inverse.setSource(s.getTarget()); + inverse.setLinkprovider(s.getLinkprovider()); + inverse.setPublicationDate(s.getPublicationDate()); + inverse.setPublisher(s.getPublisher()); + inverse + .setRelationship( + new ScholixRelationship( + s.getRelationship().getInverse(), + s.getRelationship().getSchema(), + s.getRelationship().getName())); + inverse.generateIdentifier(); + res.add(inverse); + } - return res.iterator(); - }, - Encoders.bean(Scholix.class)) - .javaRDD() - .map(s -> new ObjectMapper().writeValueAsString(s)) - .saveAsTextFile(workingDirPath + "/resolved_json"); - } - } + return res.iterator(); + }, + Encoders.bean(Scholix.class)) + .javaRDD() + .map(s -> new ObjectMapper().writeValueAsString(s)) + .saveAsTextFile(workingDirPath + "/resolved_json"); + } + } - private static SparkSession getSession(SparkConf conf, String master) { - return SparkSession.builder() - .config(conf) - .appName(SparkResolveScholixTarget.class.getSimpleName()) - .master(master) - .getOrCreate(); - } + private static SparkSession getSession(SparkConf conf, String master) { + return SparkSession + .builder() + .config(conf) + .appName(SparkResolveScholixTarget.class.getSimpleName()) + .master(master) + .getOrCreate(); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java index 7dba9c95e..d9cbd22f3 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/DataciteClientTest.java @@ -1,46 +1,50 @@ + package eu.dnetlib.dhp.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.provision.scholix.Scholix; -import eu.dnetlib.dhp.provision.scholix.ScholixResource; -import eu.dnetlib.dhp.provision.update.*; -import eu.dnetlib.scholexplorer.relation.RelationMapper; import java.util.List; + import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.provision.scholix.Scholix; +import eu.dnetlib.dhp.provision.scholix.ScholixResource; +import eu.dnetlib.dhp.provision.update.*; +import eu.dnetlib.scholexplorer.relation.RelationMapper; + public class DataciteClientTest { - @Test - public void dataciteSCholixTest() throws Exception { - final String json = IOUtils.toString(getClass().getResourceAsStream("datacite.json")); - final RelationMapper mapper = RelationMapper.load(); + @Test + public void dataciteSCholixTest() throws Exception { + final String json = IOUtils.toString(getClass().getResourceAsStream("datacite.json")); + final RelationMapper mapper = RelationMapper.load(); - Datacite2Scholix ds = new Datacite2Scholix(mapper); - final List s = ds.generateScholixFromJson(json); - System.out.println(new ObjectMapper().writeValueAsString(s)); - } + Datacite2Scholix ds = new Datacite2Scholix(mapper); + final List s = ds.generateScholixFromJson(json); + System.out.println(new ObjectMapper().writeValueAsString(s)); + } - // public void testS() throws Exception { - // RetrieveUpdateFromDatacite.main(new String[]{ - // "-n", "file:///data/new_s2.txt", - // "-t", "/data/new_s2.txt", - // "-ts", "1586974078", - // "-ih", "ip-90-147-167-25.ct1.garrservices.it", - // "-in", "datacite", - // }); - // - // } + // public void testS() throws Exception { + // RetrieveUpdateFromDatacite.main(new String[]{ + // "-n", "file:///data/new_s2.txt", + // "-t", "/data/new_s2.txt", + // "-ts", "1586974078", + // "-ih", "ip-90-147-167-25.ct1.garrservices.it", + // "-in", "datacite", + // }); + // + // } - public void testResolveDataset() throws Exception { - DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); - ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5"); - Assertions.assertNotNull(datasetByDOI); - System.out.println(new ObjectMapper().writeValueAsString(datasetByDOI)); + public void testResolveDataset() throws Exception { + DataciteClient dc = new DataciteClient("ip-90-147-167-25.ct1.garrservices.it"); + ScholixResource datasetByDOI = dc.getDatasetByDOI("10.17182/hepdata.15392.v1/t5"); + Assertions.assertNotNull(datasetByDOI); + System.out.println(new ObjectMapper().writeValueAsString(datasetByDOI)); - CrossrefClient cr = new CrossrefClient("ip-90-147-167-25.ct1.garrservices.it"); - ScholixResource crossrefByDOI = cr.getResourceByDOI("10.26850/1678-4618eqj.v35.1.2010.p41-46"); - Assertions.assertNotNull(crossrefByDOI); - System.out.println(new ObjectMapper().writeValueAsString(crossrefByDOI)); - } + CrossrefClient cr = new CrossrefClient("ip-90-147-167-25.ct1.garrservices.it"); + ScholixResource crossrefByDOI = cr.getResourceByDOI("10.26850/1678-4618eqj.v35.1.2010.p41-46"); + Assertions.assertNotNull(crossrefByDOI); + System.out.println(new ObjectMapper().writeValueAsString(crossrefByDOI)); + } } diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/ExtractInfoTest.java b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/ExtractInfoTest.java index cc6e999ae..be97072b5 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/ExtractInfoTest.java +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/test/java/eu/dnetlib/dhp/provision/ExtractInfoTest.java @@ -1,27 +1,30 @@ + package eu.dnetlib.dhp.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.provision.scholix.Scholix; -import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.Test; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.provision.scholix.Scholix; +import eu.dnetlib.dhp.provision.scholix.summary.ScholixSummary; + public class ExtractInfoTest { - @Test - public void testSerialization() throws Exception { + @Test + public void testSerialization() throws Exception { - ScholixSummary summary = new ScholixSummary(); - summary.setDescription("descrizione"); - ObjectMapper mapper = new ObjectMapper(); - String json = mapper.writeValueAsString(summary); - System.out.println(json); - System.out.println(mapper.readValue(json, ScholixSummary.class).getDescription()); - } + ScholixSummary summary = new ScholixSummary(); + summary.setDescription("descrizione"); + ObjectMapper mapper = new ObjectMapper(); + String json = mapper.writeValueAsString(summary); + System.out.println(json); + System.out.println(mapper.readValue(json, ScholixSummary.class).getDescription()); + } - @Test - public void testScholix() throws Exception { - final String jsonSummary = IOUtils.toString(getClass().getResourceAsStream("summary.json")); - final String jsonRelation = IOUtils.toString(getClass().getResourceAsStream("relation.json")); - Scholix.generateScholixWithSource(jsonSummary, jsonRelation); - } + @Test + public void testScholix() throws Exception { + final String jsonSummary = IOUtils.toString(getClass().getResourceAsStream("summary.json")); + final String jsonRelation = IOUtils.toString(getClass().getResourceAsStream("relation.json")); + Scholix.generateScholixWithSource(jsonSummary, jsonRelation); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/AdjacencyListBuilderJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/AdjacencyListBuilderJob.java index 07b11010b..99247b756 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/AdjacencyListBuilderJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/AdjacencyListBuilderJob.java @@ -1,16 +1,12 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; -import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; -import eu.dnetlib.dhp.oa.provision.model.Tuple2; -import eu.dnetlib.dhp.schema.common.ModelSupport; import java.util.ArrayList; import java.util.List; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -21,103 +17,108 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; +import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; +import eu.dnetlib.dhp.oa.provision.model.Tuple2; +import eu.dnetlib.dhp.schema.common.ModelSupport; + /** - * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of - * linked objects. The operation considers all the entity types (publication, dataset, software, - * ORP, project, datasource, organization, and all the possible relationships (similarity links - * produced by the Dedup process are excluded). - * - *

The operation is implemented by sequentially joining one entity type at time (E) with the - * relationships (R), and again by E, finally grouped by E.id; - * - *

The workflow is organized in different parts aimed to to reduce the complexity of the - * operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted - * ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects - * - *

2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - - * T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting - * only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2): - * create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source - * join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) - * - *

3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - - * T ], mapping the result as JoinedEntity - * - *

4) XmlConverterJob: convert the JoinedEntities as XML records + * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The + * operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and + * all the possible relationships (similarity links produced by the Dedup process are excluded). + *

+ * The operation is implemented by sequentially joining one entity type at time (E) with the relationships (R), and + * again by E, finally grouped by E.id; + *

+ * The workflow is organized in different parts aimed to to reduce the complexity of the operation 1) + * PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference == + * false), each entity can be linked at most to 100 other objects + *

+ * 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type + * E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target = + * T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples + * (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) + *

+ * 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the + * result as JoinedEntity + *

+ * 4) XmlConverterJob: convert the JoinedEntities as XML records */ public class AdjacencyListBuilderJob { - private static final Logger log = LoggerFactory.getLogger(AdjacencyListBuilderJob.class); + private static final Logger log = LoggerFactory.getLogger(AdjacencyListBuilderJob.class); - public static final int MAX_LINKS = 100; + public static final int MAX_LINKS = 100; - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - AdjacencyListBuilderJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_build_adjacency_lists.json"))); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + AdjacencyListBuilderJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_build_adjacency_lists.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - createAdjacencyLists(spark, inputPath, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + createAdjacencyLists(spark, inputPath, outputPath); + }); + } - private static void createAdjacencyLists( - SparkSession spark, String inputPath, String outputPath) { + private static void createAdjacencyLists( + SparkSession spark, String inputPath, String outputPath) { - log.info("Reading joined entities from: {}", inputPath); - spark - .read() - .load(inputPath) - .as(Encoders.bean(EntityRelEntity.class)) - .groupByKey( - (MapFunction) value -> value.getEntity().getId(), - Encoders.STRING()) - .mapGroups( - (MapGroupsFunction) - (key, values) -> { - JoinedEntity j = new JoinedEntity(); - List links = new ArrayList<>(); - while (values.hasNext() && links.size() < MAX_LINKS) { - EntityRelEntity curr = values.next(); - if (j.getEntity() == null) { - j.setEntity(curr.getEntity()); - } - links.add(new Tuple2(curr.getRelation(), curr.getTarget())); - } - j.setLinks(links); - return j; - }, - Encoders.bean(JoinedEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + log.info("Reading joined entities from: {}", inputPath); + spark + .read() + .load(inputPath) + .as(Encoders.bean(EntityRelEntity.class)) + .groupByKey( + (MapFunction) value -> value.getEntity().getId(), + Encoders.STRING()) + .mapGroups( + (MapGroupsFunction) (key, values) -> { + JoinedEntity j = new JoinedEntity(); + List links = new ArrayList<>(); + while (values.hasNext() && links.size() < MAX_LINKS) { + EntityRelEntity curr = values.next(); + if (j.getEntity() == null) { + j.setEntity(curr.getEntity()); + } + links.add(new Tuple2(curr.getRelation(), curr.getTarget())); + } + j.setLinks(links); + return j; + }, + Encoders.bean(JoinedEntity.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java index a9c97155c..606fa4cc0 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase1.java @@ -1,21 +1,14 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.*; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; -import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; -import eu.dnetlib.dhp.oa.provision.model.SortableRelation; -import eu.dnetlib.dhp.schema.common.EntityType; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -25,224 +18,228 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; +import eu.dnetlib.dhp.oa.provision.model.RelatedEntity; +import eu.dnetlib.dhp.oa.provision.model.SortableRelation; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; /** - * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of - * linked objects. The operation considers all the entity types (publication, dataset, software, - * ORP, project, datasource, organization, and all the possible relationships (similarity links - * produced by the Dedup process are excluded). - * - *

The operation is implemented by sequentially joining one entity type at time (E) with the - * relationships (R), and again by E, finally grouped by E.id; - * - *

The workflow is organized in different parts aimed to to reduce the complexity of the - * operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted - * ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects - * - *

2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - - * T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting - * only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2): - * create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source - * join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) - * - *

3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - - * T ], mapping the result as JoinedEntity - * - *

4) XmlConverterJob: convert the JoinedEntities as XML records + * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The + * operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and + * all the possible relationships (similarity links produced by the Dedup process are excluded). + *

+ * The operation is implemented by sequentially joining one entity type at time (E) with the relationships (R), and + * again by E, finally grouped by E.id; + *

+ * The workflow is organized in different parts aimed to to reduce the complexity of the operation 1) + * PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference == + * false), each entity can be linked at most to 100 other objects + *

+ * 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type + * E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target = + * T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples + * (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) + *

+ * 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the + * result as JoinedEntity + *

+ * 4) XmlConverterJob: convert the JoinedEntities as XML records */ public class CreateRelatedEntitiesJob_phase1 { - private static final Logger log = LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase1.class); + private static final Logger log = LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase1.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareRelationsJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase1.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + String jsonConfiguration = IOUtils + .toString( + PrepareRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase1.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputRelationsPath = parser.get("inputRelationsPath"); - log.info("inputRelationsPath: {}", inputRelationsPath); + String inputRelationsPath = parser.get("inputRelationsPath"); + log.info("inputRelationsPath: {}", inputRelationsPath); - String inputEntityPath = parser.get("inputEntityPath"); - log.info("inputEntityPath: {}", inputEntityPath); + String inputEntityPath = parser.get("inputEntityPath"); + log.info("inputEntityPath: {}", inputEntityPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - String graphTableClassName = parser.get("graphTableClassName"); - log.info("graphTableClassName: {}", graphTableClassName); + String graphTableClassName = parser.get("graphTableClassName"); + log.info("graphTableClassName: {}", graphTableClassName); - Class entityClazz = - (Class) Class.forName(graphTableClassName); + Class entityClazz = (Class) Class.forName(graphTableClassName); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - joinRelationEntity(spark, inputRelationsPath, inputEntityPath, entityClazz, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + joinRelationEntity(spark, inputRelationsPath, inputEntityPath, entityClazz, outputPath); + }); + } - private static void joinRelationEntity( - SparkSession spark, - String inputRelationsPath, - String inputEntityPath, - Class clazz, - String outputPath) { + private static void joinRelationEntity( + SparkSession spark, + String inputRelationsPath, + String inputEntityPath, + Class clazz, + String outputPath) { - Dataset> relsByTarget = - readPathRelation(spark, inputRelationsPath) - .filter("dataInfo.deletedbyinference == false") - .map( - (MapFunction>) - r -> new Tuple2<>(r.getTarget(), r), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(SortableRelation.class))) - .cache(); + Dataset> relsByTarget = readPathRelation(spark, inputRelationsPath) + .filter("dataInfo.deletedbyinference == false") + .map( + (MapFunction>) r -> new Tuple2<>(r.getTarget(), r), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(SortableRelation.class))) + .cache(); - Dataset> entities = - readPathEntity(spark, inputEntityPath, clazz) - .filter("dataInfo.invisible == false") - .map( - (MapFunction) value -> asRelatedEntity(value, clazz), - Encoders.bean(RelatedEntity.class)) - .map( - (MapFunction>) - e -> new Tuple2<>(e.getId(), e), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class))) - .cache(); + Dataset> entities = readPathEntity(spark, inputEntityPath, clazz) + .filter("dataInfo.invisible == false") + .map( + (MapFunction) value -> asRelatedEntity(value, clazz), + Encoders.bean(RelatedEntity.class)) + .map( + (MapFunction>) e -> new Tuple2<>(e.getId(), e), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class))) + .cache(); - relsByTarget - .joinWith(entities, entities.col("_1").equalTo(relsByTarget.col("_1")), "inner") - .map( - (MapFunction< - Tuple2, Tuple2>, - EntityRelEntity>) - t -> new EntityRelEntity(t._1()._2(), t._2()._2()), - Encoders.bean(EntityRelEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath + "/" + EntityType.fromClass(clazz)); - } + relsByTarget + .joinWith(entities, entities.col("_1").equalTo(relsByTarget.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, EntityRelEntity>) t -> new EntityRelEntity( + t._1()._2(), t._2()._2()), + Encoders.bean(EntityRelEntity.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath + "/" + EntityType.fromClass(clazz)); + } - private static Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class entityClazz) { + private static Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class entityClazz) { - log.info("Reading Graph table from: {}", inputEntityPath); - return spark - .read() - .textFile(inputEntityPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, entityClazz), - Encoders.bean(entityClazz)); - } + log.info("Reading Graph table from: {}", inputEntityPath); + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, entityClazz), + Encoders.bean(entityClazz)); + } - public static RelatedEntity asRelatedEntity(E entity, Class clazz) { + public static RelatedEntity asRelatedEntity(E entity, Class clazz) { - final RelatedEntity re = new RelatedEntity(); - re.setId(entity.getId()); - re.setType(EntityType.fromClass(clazz).name()); + final RelatedEntity re = new RelatedEntity(); + re.setId(entity.getId()); + re.setType(EntityType.fromClass(clazz).name()); - re.setPid(entity.getPid()); - re.setCollectedfrom(entity.getCollectedfrom()); + re.setPid(entity.getPid()); + re.setCollectedfrom(entity.getCollectedfrom()); - switch (EntityType.fromClass(clazz)) { - case publication: - case dataset: - case otherresearchproduct: - case software: - Result result = (Result) entity; + switch (EntityType.fromClass(clazz)) { + case publication: + case dataset: + case otherresearchproduct: + case software: + Result result = (Result) entity; - if (result.getTitle() != null && !result.getTitle().isEmpty()) { - re.setTitle(result.getTitle().stream().findFirst().get()); - } + if (result.getTitle() != null && !result.getTitle().isEmpty()) { + re.setTitle(result.getTitle().stream().findFirst().get()); + } - re.setDateofacceptance(getValue(result.getDateofacceptance())); - re.setPublisher(getValue(result.getPublisher())); - re.setResulttype(result.getResulttype()); - re.setInstances(result.getInstance()); + re.setDateofacceptance(getValue(result.getDateofacceptance())); + re.setPublisher(getValue(result.getPublisher())); + re.setResulttype(result.getResulttype()); + re.setInstances(result.getInstance()); - // TODO still to be mapped - // re.setCodeRepositoryUrl(j.read("$.coderepositoryurl")); + // TODO still to be mapped + // re.setCodeRepositoryUrl(j.read("$.coderepositoryurl")); - break; - case datasource: - Datasource d = (Datasource) entity; + break; + case datasource: + Datasource d = (Datasource) entity; - re.setOfficialname(getValue(d.getOfficialname())); - re.setWebsiteurl(getValue(d.getWebsiteurl())); - re.setDatasourcetype(d.getDatasourcetype()); - re.setOpenairecompatibility(d.getOpenairecompatibility()); + re.setOfficialname(getValue(d.getOfficialname())); + re.setWebsiteurl(getValue(d.getWebsiteurl())); + re.setDatasourcetype(d.getDatasourcetype()); + re.setOpenairecompatibility(d.getOpenairecompatibility()); - break; - case organization: - Organization o = (Organization) entity; + break; + case organization: + Organization o = (Organization) entity; - re.setLegalname(getValue(o.getLegalname())); - re.setLegalshortname(getValue(o.getLegalshortname())); - re.setCountry(o.getCountry()); - re.setWebsiteurl(getValue(o.getWebsiteurl())); - break; - case project: - Project p = (Project) entity; + re.setLegalname(getValue(o.getLegalname())); + re.setLegalshortname(getValue(o.getLegalshortname())); + re.setCountry(o.getCountry()); + re.setWebsiteurl(getValue(o.getWebsiteurl())); + break; + case project: + Project p = (Project) entity; - re.setProjectTitle(getValue(p.getTitle())); - re.setCode(getValue(p.getCode())); - re.setAcronym(getValue(p.getAcronym())); - re.setContracttype(p.getContracttype()); + re.setProjectTitle(getValue(p.getTitle())); + re.setCode(getValue(p.getCode())); + re.setAcronym(getValue(p.getAcronym())); + re.setContracttype(p.getContracttype()); - List> f = p.getFundingtree(); - if (!f.isEmpty()) { - re.setFundingtree(f.stream().map(s -> s.getValue()).collect(Collectors.toList())); - } - break; - } - return re; - } + List> f = p.getFundingtree(); + if (!f.isEmpty()) { + re.setFundingtree(f.stream().map(s -> s.getValue()).collect(Collectors.toList())); + } + break; + } + return re; + } - private static String getValue(Field field) { - return getFieldValueWithDefault(field, ""); - } + private static String getValue(Field field) { + return getFieldValueWithDefault(field, ""); + } - private static T getFieldValueWithDefault(Field f, T defaultValue) { - return Optional.ofNullable(f) - .filter(Objects::nonNull) - .map(x -> x.getValue()) - .orElse(defaultValue); - } + private static T getFieldValueWithDefault(Field f, T defaultValue) { + return Optional + .ofNullable(f) + .filter(Objects::nonNull) + .map(x -> x.getValue()) + .orElse(defaultValue); + } - /** - * Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline - * delimited json text file, - * - * @param spark - * @param relationPath - * @return the Dataset containing all the relationships - */ - private static Dataset readPathRelation( - SparkSession spark, final String relationPath) { + /** + * Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param relationPath + * @return the Dataset containing all the relationships + */ + private static Dataset readPathRelation( + SparkSession spark, final String relationPath) { - log.info("Reading relations from: {}", relationPath); - return spark.read().load(relationPath).as(Encoders.bean(SortableRelation.class)); - } + log.info("Reading relations from: {}", relationPath); + return spark.read().load(relationPath).as(Encoders.bean(SortableRelation.class)); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java index 021ef86ba..403817019 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/CreateRelatedEntitiesJob_phase2.java @@ -1,17 +1,11 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; -import eu.dnetlib.dhp.oa.provision.model.TypedRow; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.List; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -23,203 +17,200 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.EntityRelEntity; +import eu.dnetlib.dhp.oa.provision.model.TypedRow; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.Seq; /** - * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of - * linked objects. The operation considers all the entity types (publication, dataset, software, - * ORP, project, datasource, organization, and all the possible relationships (similarity links - * produced by the Dedup process are excluded). - * - *

The operation is implemented by sequentially joining one entity type at time (E) with the - * relationships (R), and again by E, finally grouped by E.id; - * - *

The workflow is organized in different parts aimed to to reduce the complexity of the - * operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted - * ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects - * - *

2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - - * T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting - * only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2): - * create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source - * join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) - * - *

3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - - * T ], mapping the result as JoinedEntity - * - *

4) XmlConverterJob: convert the JoinedEntities as XML records + * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The + * operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and + * all the possible relationships (similarity links produced by the Dedup process are excluded). + *

+ * The operation is implemented by sequentially joining one entity type at time (E) with the relationships (R), and + * again by E, finally grouped by E.id; + *

+ * The workflow is organized in different parts aimed to to reduce the complexity of the operation 1) + * PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference == + * false), each entity can be linked at most to 100 other objects + *

+ * 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type + * E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target = + * T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples + * (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) + *

+ * 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the + * result as JoinedEntity + *

+ * 4) XmlConverterJob: convert the JoinedEntities as XML records */ public class CreateRelatedEntitiesJob_phase2 { - private static final Logger log = LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase2.class); + private static final Logger log = LoggerFactory.getLogger(CreateRelatedEntitiesJob_phase2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareRelationsJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase2.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + String jsonConfiguration = IOUtils + .toString( + PrepareRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_related_entities_pahase2.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputRelatedEntitiesPath = parser.get("inputRelatedEntitiesPath"); - log.info("inputRelatedEntitiesPath: {}", inputRelatedEntitiesPath); + String inputRelatedEntitiesPath = parser.get("inputRelatedEntitiesPath"); + log.info("inputRelatedEntitiesPath: {}", inputRelatedEntitiesPath); - String inputGraphRootPath = parser.get("inputGraphRootPath"); - log.info("inputGraphRootPath: {}", inputGraphRootPath); + String inputGraphRootPath = parser.get("inputGraphRootPath"); + log.info("inputGraphRootPath: {}", inputGraphRootPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - int numPartitions = Integer.parseInt(parser.get("numPartitions")); - log.info("numPartitions: {}", numPartitions); + int numPartitions = Integer.parseInt(parser.get("numPartitions")); + log.info("numPartitions: {}", numPartitions); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - joinAllEntities( - spark, inputRelatedEntitiesPath, inputGraphRootPath, outputPath, numPartitions); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + joinAllEntities( + spark, inputRelatedEntitiesPath, inputGraphRootPath, outputPath, numPartitions); + }); + } - private static void joinAllEntities( - SparkSession spark, - String inputRelatedEntitiesPath, - String inputGraphRootPath, - String outputPath, - int numPartitions) { + private static void joinAllEntities( + SparkSession spark, + String inputRelatedEntitiesPath, + String inputGraphRootPath, + String outputPath, + int numPartitions) { - Dataset> entities = - readAllEntities(spark, inputGraphRootPath, numPartitions); - Dataset> relsBySource = - readRelatedEntities(spark, inputRelatedEntitiesPath); + Dataset> entities = readAllEntities(spark, inputGraphRootPath, numPartitions); + Dataset> relsBySource = readRelatedEntities(spark, inputRelatedEntitiesPath); - entities - .joinWith(relsBySource, entities.col("_1").equalTo(relsBySource.col("_1")), "left_outer") - .map( - (MapFunction< - Tuple2, Tuple2>, - EntityRelEntity>) - value -> { - EntityRelEntity re = new EntityRelEntity(); - re.setEntity(value._1()._2()); - Optional related = - Optional.ofNullable(value._2()).map(Tuple2::_2); - if (related.isPresent()) { - re.setRelation(related.get().getRelation()); - re.setTarget(related.get().getTarget()); - } - return re; - }, - Encoders.bean(EntityRelEntity.class)) - .repartition(numPartitions) - .filter( - (FilterFunction) - value -> - value.getEntity() != null && StringUtils.isNotBlank(value.getEntity().getId())) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + entities + .joinWith(relsBySource, entities.col("_1").equalTo(relsBySource.col("_1")), "left_outer") + .map( + (MapFunction, Tuple2>, EntityRelEntity>) value -> { + EntityRelEntity re = new EntityRelEntity(); + re.setEntity(value._1()._2()); + Optional related = Optional.ofNullable(value._2()).map(Tuple2::_2); + if (related.isPresent()) { + re.setRelation(related.get().getRelation()); + re.setTarget(related.get().getTarget()); + } + return re; + }, + Encoders.bean(EntityRelEntity.class)) + .repartition(numPartitions) + .filter( + (FilterFunction) value -> value.getEntity() != null + && StringUtils.isNotBlank(value.getEntity().getId())) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - private static Dataset> readAllEntities( - SparkSession spark, String inputGraphPath, int numPartitions) { - Dataset publication = - readPathEntity(spark, inputGraphPath + "/publication", Publication.class); - Dataset dataset = - readPathEntity(spark, inputGraphPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class); - Dataset other = - readPathEntity(spark, inputGraphPath + "/otherresearchproduct", OtherResearchProduct.class); - Dataset software = - readPathEntity(spark, inputGraphPath + "/software", Software.class); - Dataset datasource = - readPathEntity(spark, inputGraphPath + "/datasource", Datasource.class); - Dataset organization = - readPathEntity(spark, inputGraphPath + "/organization", Organization.class); - Dataset project = readPathEntity(spark, inputGraphPath + "/project", Project.class); + private static Dataset> readAllEntities( + SparkSession spark, String inputGraphPath, int numPartitions) { + Dataset publication = readPathEntity(spark, inputGraphPath + "/publication", Publication.class); + Dataset dataset = readPathEntity( + spark, inputGraphPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class); + Dataset other = readPathEntity( + spark, inputGraphPath + "/otherresearchproduct", OtherResearchProduct.class); + Dataset software = readPathEntity(spark, inputGraphPath + "/software", Software.class); + Dataset datasource = readPathEntity(spark, inputGraphPath + "/datasource", Datasource.class); + Dataset organization = readPathEntity(spark, inputGraphPath + "/organization", Organization.class); + Dataset project = readPathEntity(spark, inputGraphPath + "/project", Project.class); - return publication - .union(dataset) - .union(other) - .union(software) - .union(datasource) - .union(organization) - .union(project) - .map( - (MapFunction>) - value -> new Tuple2<>(value.getId(), value), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(TypedRow.class))) - .repartition(numPartitions); - } + return publication + .union(dataset) + .union(other) + .union(software) + .union(datasource) + .union(organization) + .union(project) + .map( + (MapFunction>) value -> new Tuple2<>(value.getId(), value), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(TypedRow.class))) + .repartition(numPartitions); + } - private static Dataset> readRelatedEntities( - SparkSession spark, String inputRelatedEntitiesPath) { + private static Dataset> readRelatedEntities( + SparkSession spark, String inputRelatedEntitiesPath) { - log.info("Reading related entities from: {}", inputRelatedEntitiesPath); + log.info("Reading related entities from: {}", inputRelatedEntitiesPath); - final List paths = - HdfsSupport.listFiles(inputRelatedEntitiesPath, spark.sparkContext().hadoopConfiguration()); + final List paths = HdfsSupport + .listFiles(inputRelatedEntitiesPath, spark.sparkContext().hadoopConfiguration()); - log.info("Found paths: {}", String.join(",", paths)); + log.info("Found paths: {}", String.join(",", paths)); - return spark - .read() - .load(toSeq(paths)) - .as(Encoders.bean(EntityRelEntity.class)) - .map( - (MapFunction>) - value -> new Tuple2<>(value.getRelation().getSource(), value), - Encoders.tuple(Encoders.STRING(), Encoders.kryo(EntityRelEntity.class))); - } + return spark + .read() + .load(toSeq(paths)) + .as(Encoders.bean(EntityRelEntity.class)) + .map( + (MapFunction>) value -> new Tuple2<>( + value.getRelation().getSource(), value), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(EntityRelEntity.class))); + } - private static Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class entityClazz) { + private static Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class entityClazz) { - log.info("Reading Graph table from: {}", inputEntityPath); - return spark - .read() - .textFile(inputEntityPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, entityClazz), - Encoders.bean(entityClazz)) - .filter("dataInfo.invisible == false") - .map( - (MapFunction) - value -> getTypedRow(StringUtils.substringAfterLast(inputEntityPath, "/"), value), - Encoders.bean(TypedRow.class)); - } + log.info("Reading Graph table from: {}", inputEntityPath); + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, entityClazz), + Encoders.bean(entityClazz)) + .filter("dataInfo.invisible == false") + .map( + (MapFunction) value -> getTypedRow( + StringUtils.substringAfterLast(inputEntityPath, "/"), value), + Encoders.bean(TypedRow.class)); + } - private static TypedRow getTypedRow(String type, OafEntity entity) - throws JsonProcessingException { - TypedRow t = new TypedRow(); - t.setType(type); - t.setDeleted(entity.getDataInfo().getDeletedbyinference()); - t.setId(entity.getId()); - t.setOaf(OBJECT_MAPPER.writeValueAsString(entity)); - return t; - } + private static TypedRow getTypedRow(String type, OafEntity entity) + throws JsonProcessingException { + TypedRow t = new TypedRow(); + t.setType(type); + t.setDeleted(entity.getDataInfo().getDeletedbyinference()); + t.setId(entity.getId()); + t.setOaf(OBJECT_MAPPER.writeValueAsString(entity)); + return t; + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - private static Seq toSeq(List list) { - return JavaConverters.asScalaIteratorConverter(list.iterator()).asScala().toSeq(); - } + private static Seq toSeq(List list) { + return JavaConverters.asScalaIteratorConverter(list.iterator()).asScala().toSeq(); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java index 72eb15cbb..dbdc54fc0 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java @@ -1,15 +1,10 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.SortableRelation; -import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -22,139 +17,144 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.SortableRelation; +import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; import scala.Tuple2; /** - * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of - * linked objects. The operation considers all the entity types (publication, dataset, software, - * ORP, project, datasource, organization, and all the possible relationships (similarity links - * produced by the Dedup process are excluded). - * - *

The operation is implemented by sequentially joining one entity type at time (E) with the - * relationships (R), and again by E, finally grouped by E.id; - * - *

The workflow is organized in different parts aimed to to reduce the complexity of the - * operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted - * ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects - * - *

2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - - * T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting - * only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2): - * create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source - * join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) - * - *

3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - - * T ], mapping the result as JoinedEntity - * - *

4) XmlConverterJob: convert the JoinedEntities as XML records + * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The + * operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and + * all the possible relationships (similarity links produced by the Dedup process are excluded). + *

+ * The operation is implemented by sequentially joining one entity type at time (E) with the relationships (R), and + * again by E, finally grouped by E.id; + *

+ * The workflow is organized in different parts aimed to to reduce the complexity of the operation 1) + * PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference == + * false), each entity can be linked at most to 100 other objects + *

+ * 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type + * E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target = + * T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples + * (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) + *

+ * 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the + * result as JoinedEntity + *

+ * 4) XmlConverterJob: convert the JoinedEntities as XML records */ public class PrepareRelationsJob { - private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); + private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static final int MAX_RELS = 100; + public static final int MAX_RELS = 100; - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareRelationsJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareRelationsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json")); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputRelationsPath = parser.get("inputRelationsPath"); - log.info("inputRelationsPath: {}", inputRelationsPath); + String inputRelationsPath = parser.get("inputRelationsPath"); + log.info("inputRelationsPath: {}", inputRelationsPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareRelationsFromPaths(spark, inputRelationsPath, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareRelationsFromPaths(spark, inputRelationsPath, outputPath); + }); + } - private static void prepareRelationsFromPaths( - SparkSession spark, String inputRelationsPath, String outputPath) { - readPathRelation(spark, inputRelationsPath) - .filter("dataInfo.deletedbyinference == false") - .groupByKey( - (MapFunction) value -> value.getSource(), Encoders.STRING()) - .flatMapGroups( - (FlatMapGroupsFunction) - (key, values) -> Iterators.limit(values, MAX_RELS), - Encoders.bean(SortableRelation.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + private static void prepareRelationsFromPaths( + SparkSession spark, String inputRelationsPath, String outputPath) { + readPathRelation(spark, inputRelationsPath) + .filter("dataInfo.deletedbyinference == false") + .groupByKey( + (MapFunction) value -> value.getSource(), Encoders.STRING()) + .flatMapGroups( + (FlatMapGroupsFunction) (key, values) -> Iterators + .limit(values, MAX_RELS), + Encoders.bean(SortableRelation.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - /** - * Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline - * delimited json text file, - * - * @param spark - * @param inputPath - * @return the Dataset containing all the relationships - */ - private static Dataset readPathRelation( - SparkSession spark, final String inputPath) { - return spark - .read() - .textFile(inputPath) - .map( - (MapFunction) - value -> OBJECT_MAPPER.readValue(value, SortableRelation.class), - Encoders.bean(SortableRelation.class)); - } + /** + * Reads a Dataset of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param inputPath + * @return the Dataset containing all the relationships + */ + private static Dataset readPathRelation( + SparkSession spark, final String inputPath) { + return spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, SortableRelation.class), + Encoders.bean(SortableRelation.class)); + } - // TODO work in progress - private static void prepareRelationsRDDFromPaths( - SparkSession spark, String inputRelationsPath, String outputPath, int numPartitions) { - JavaRDD rels = - readPathRelationRDD(spark, inputRelationsPath).repartition(numPartitions); + // TODO work in progress + private static void prepareRelationsRDDFromPaths( + SparkSession spark, String inputRelationsPath, String outputPath, int numPartitions) { + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath).repartition(numPartitions); - RDD d = - rels.filter(rel -> !rel.getDataInfo().getDeletedbyinference()) // only - // consider - // those - // that are not virtually - // deleted - .mapToPair( - (PairFunction) - rel -> new Tuple2<>(rel, rel)) - .groupByKey(new RelationPartitioner(rels.getNumPartitions())) - .map(p -> Iterables.limit(p._2(), MAX_RELS)) - .flatMap(p -> p.iterator()) - .rdd(); + RDD d = rels + .filter(rel -> !rel.getDataInfo().getDeletedbyinference()) // only + // consider + // those + // that are not virtually + // deleted + .mapToPair( + (PairFunction) rel -> new Tuple2<>(rel, rel)) + .groupByKey(new RelationPartitioner(rels.getNumPartitions())) + .map(p -> Iterables.limit(p._2(), MAX_RELS)) + .flatMap(p -> p.iterator()) + .rdd(); - spark - .createDataset(d, Encoders.bean(SortableRelation.class)) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + spark + .createDataset(d, Encoders.bean(SortableRelation.class)) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - private static JavaRDD readPathRelationRDD( - SparkSession spark, final String inputPath) { - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, SortableRelation.class)); - } + private static JavaRDD readPathRelationRDD( + SparkSession spark, final String inputPath) { + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + return sc.textFile(inputPath).map(s -> OBJECT_MAPPER.readValue(s, SortableRelation.class)); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java index aabeae5ee..a88b28592 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlConverterJob.java @@ -1,19 +1,13 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Maps; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.provision.model.*; -import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; -import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.ArrayList; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.GzipCodec; @@ -27,178 +21,205 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.*; +import eu.dnetlib.dhp.oa.provision.utils.ContextMapper; +import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; /** - * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of - * linked objects. The operation considers all the entity types (publication, dataset, software, - * ORP, project, datasource, organization, and all the possible relationships (similarity links - * produced by the Dedup process are excluded). - * - *

The workflow is organized in different parts aimed to to reduce the complexity of the - * operation 1) PrepareRelationsJob: only consider relationships that are not virtually deleted - * ($.dataInfo.deletedbyinference == false), each entity can be linked at most to 100 other objects - * - *

2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - - * T): for each entity type E_i map E_i as RelatedEntity T_i to simplify the model and extracting - * only the necessary information join (R.target = T_i.id) save the tuples (R_i, T_i) (phase 2): - * create the union of all the entity types E, hash by id read the tuples (R, T), hash by R.source - * join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) - * - *

3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - - * T ], mapping the result as JoinedEntity - * - *

4) XmlConverterJob: convert the JoinedEntities as XML records + * Joins the graph nodes by resolving the links of distance = 1 to create an adjacency list of linked objects. The + * operation considers all the entity types (publication, dataset, software, ORP, project, datasource, organization, and + * all the possible relationships (similarity links produced by the Dedup process are excluded). + *

+ * The workflow is organized in different parts aimed to to reduce the complexity of the operation 1) + * PrepareRelationsJob: only consider relationships that are not virtually deleted ($.dataInfo.deletedbyinference == + * false), each entity can be linked at most to 100 other objects + *

+ * 2) JoinRelationEntityByTargetJob: (phase 1): prepare tuples [relation - target entity] (R - T): for each entity type + * E_i map E_i as RelatedEntity T_i to simplify the model and extracting only the necessary information join (R.target = + * T_i.id) save the tuples (R_i, T_i) (phase 2): create the union of all the entity types E, hash by id read the tuples + * (R, T), hash by R.source join E.id = (R, T).source, where E becomes the Source Entity S save the tuples (S, R, T) + *

+ * 3) AdjacencyListBuilderJob: given the tuple (S - R - T) we need to group by S.id -> List [ R - T ], mapping the + * result as JoinedEntity + *

+ * 4) XmlConverterJob: convert the JoinedEntities as XML records */ public class XmlConverterJob { - private static final Logger log = LoggerFactory.getLogger(XmlConverterJob.class); + private static final Logger log = LoggerFactory.getLogger(XmlConverterJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static final String schemaLocation = "https://www.openaire.eu/schema/1.0/oaf-1.0.xsd"; + public static final String schemaLocation = "https://www.openaire.eu/schema/1.0/oaf-1.0.xsd"; - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - XmlConverterJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_xml_converter.json"))); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + XmlConverterJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_xml_converter.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); + String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); - String otherDsTypeId = parser.get("otherDsTypeId"); - log.info("otherDsTypeId: {}", otherDsTypeId); + String otherDsTypeId = parser.get("otherDsTypeId"); + log.info("otherDsTypeId: {}", otherDsTypeId); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - convertToXml( - spark, inputPath, outputPath, ContextMapper.fromIS(isLookupUrl), otherDsTypeId); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + convertToXml( + spark, inputPath, outputPath, ContextMapper.fromIS(isLookupUrl), otherDsTypeId); + }); + } - private static void convertToXml( - SparkSession spark, - String inputPath, - String outputPath, - ContextMapper contextMapper, - String otherDsTypeId) { + private static void convertToXml( + SparkSession spark, + String inputPath, + String outputPath, + ContextMapper contextMapper, + String otherDsTypeId) { - final XmlRecordFactory recordFactory = - new XmlRecordFactory( - prepareAccumulators(spark.sparkContext()), - contextMapper, - false, - schemaLocation, - otherDsTypeId); + final XmlRecordFactory recordFactory = new XmlRecordFactory( + prepareAccumulators(spark.sparkContext()), + contextMapper, + false, + schemaLocation, + otherDsTypeId); - spark - .read() - .load(inputPath) - .as(Encoders.bean(JoinedEntity.class)) - .map( - (MapFunction) - j -> { - if (j.getLinks() != null) { - j.setLinks( - j.getLinks().stream() - .filter(t -> t.getRelation() != null & t.getRelatedEntity() != null) - .collect(Collectors.toCollection(ArrayList::new))); - } - return j; - }, - Encoders.bean(JoinedEntity.class)) - .map( - (MapFunction>) - je -> new Tuple2<>(je.getEntity().getId(), recordFactory.build(je)), - Encoders.tuple(Encoders.STRING(), Encoders.STRING())) - .javaRDD() - .mapToPair( - (PairFunction, Text, Text>) - t -> new Tuple2<>(new Text(t._1()), new Text(t._2()))) - .saveAsHadoopFile( - outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); - } + spark + .read() + .load(inputPath) + .as(Encoders.bean(JoinedEntity.class)) + .map( + (MapFunction) j -> { + if (j.getLinks() != null) { + j + .setLinks( + j + .getLinks() + .stream() + .filter(t -> t.getRelation() != null & t.getRelatedEntity() != null) + .collect(Collectors.toCollection(ArrayList::new))); + } + return j; + }, + Encoders.bean(JoinedEntity.class)) + .map( + (MapFunction>) je -> new Tuple2<>(je.getEntity().getId(), + recordFactory.build(je)), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .javaRDD() + .mapToPair( + (PairFunction, Text, Text>) t -> new Tuple2<>(new Text(t._1()), + new Text(t._2()))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - private static Map prepareAccumulators(SparkContext sc) { - Map accumulators = Maps.newHashMap(); - accumulators.put( - "resultResult_similarity_isAmongTopNSimilarDocuments", - sc.longAccumulator("resultResult_similarity_isAmongTopNSimilarDocuments")); - accumulators.put( - "resultResult_similarity_hasAmongTopNSimilarDocuments", - sc.longAccumulator("resultResult_similarity_hasAmongTopNSimilarDocuments")); - accumulators.put( - "resultResult_supplement_isSupplementTo", - sc.longAccumulator("resultResult_supplement_isSupplementTo")); - accumulators.put( - "resultResult_supplement_isSupplementedBy", - sc.longAccumulator("resultResult_supplement_isSupplementedBy")); - accumulators.put( - "resultResult_dedup_isMergedIn", sc.longAccumulator("resultResult_dedup_isMergedIn")); - accumulators.put("resultResult_dedup_merges", sc.longAccumulator("resultResult_dedup_merges")); + private static Map prepareAccumulators(SparkContext sc) { + Map accumulators = Maps.newHashMap(); + accumulators + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", + sc.longAccumulator("resultResult_similarity_isAmongTopNSimilarDocuments")); + accumulators + .put( + "resultResult_similarity_hasAmongTopNSimilarDocuments", + sc.longAccumulator("resultResult_similarity_hasAmongTopNSimilarDocuments")); + accumulators + .put( + "resultResult_supplement_isSupplementTo", + sc.longAccumulator("resultResult_supplement_isSupplementTo")); + accumulators + .put( + "resultResult_supplement_isSupplementedBy", + sc.longAccumulator("resultResult_supplement_isSupplementedBy")); + accumulators + .put( + "resultResult_dedup_isMergedIn", sc.longAccumulator("resultResult_dedup_isMergedIn")); + accumulators.put("resultResult_dedup_merges", sc.longAccumulator("resultResult_dedup_merges")); - accumulators.put( - "resultResult_publicationDataset_isRelatedTo", - sc.longAccumulator("resultResult_publicationDataset_isRelatedTo")); - accumulators.put( - "resultResult_relationship_isRelatedTo", - sc.longAccumulator("resultResult_relationship_isRelatedTo")); - accumulators.put( - "resultProject_outcome_isProducedBy", - sc.longAccumulator("resultProject_outcome_isProducedBy")); - accumulators.put( - "resultProject_outcome_produces", sc.longAccumulator("resultProject_outcome_produces")); - accumulators.put( - "resultOrganization_affiliation_isAuthorInstitutionOf", - sc.longAccumulator("resultOrganization_affiliation_isAuthorInstitutionOf")); + accumulators + .put( + "resultResult_publicationDataset_isRelatedTo", + sc.longAccumulator("resultResult_publicationDataset_isRelatedTo")); + accumulators + .put( + "resultResult_relationship_isRelatedTo", + sc.longAccumulator("resultResult_relationship_isRelatedTo")); + accumulators + .put( + "resultProject_outcome_isProducedBy", + sc.longAccumulator("resultProject_outcome_isProducedBy")); + accumulators + .put( + "resultProject_outcome_produces", sc.longAccumulator("resultProject_outcome_produces")); + accumulators + .put( + "resultOrganization_affiliation_isAuthorInstitutionOf", + sc.longAccumulator("resultOrganization_affiliation_isAuthorInstitutionOf")); - accumulators.put( - "resultOrganization_affiliation_hasAuthorInstitution", - sc.longAccumulator("resultOrganization_affiliation_hasAuthorInstitution")); - accumulators.put( - "projectOrganization_participation_hasParticipant", - sc.longAccumulator("projectOrganization_participation_hasParticipant")); - accumulators.put( - "projectOrganization_participation_isParticipant", - sc.longAccumulator("projectOrganization_participation_isParticipant")); - accumulators.put( - "organizationOrganization_dedup_isMergedIn", - sc.longAccumulator("organizationOrganization_dedup_isMergedIn")); - accumulators.put( - "organizationOrganization_dedup_merges", - sc.longAccumulator("resultProject_outcome_produces")); - accumulators.put( - "datasourceOrganization_provision_isProvidedBy", - sc.longAccumulator("datasourceOrganization_provision_isProvidedBy")); - accumulators.put( - "datasourceOrganization_provision_provides", - sc.longAccumulator("datasourceOrganization_provision_provides")); + accumulators + .put( + "resultOrganization_affiliation_hasAuthorInstitution", + sc.longAccumulator("resultOrganization_affiliation_hasAuthorInstitution")); + accumulators + .put( + "projectOrganization_participation_hasParticipant", + sc.longAccumulator("projectOrganization_participation_hasParticipant")); + accumulators + .put( + "projectOrganization_participation_isParticipant", + sc.longAccumulator("projectOrganization_participation_isParticipant")); + accumulators + .put( + "organizationOrganization_dedup_isMergedIn", + sc.longAccumulator("organizationOrganization_dedup_isMergedIn")); + accumulators + .put( + "organizationOrganization_dedup_merges", + sc.longAccumulator("resultProject_outcome_produces")); + accumulators + .put( + "datasourceOrganization_provision_isProvidedBy", + sc.longAccumulator("datasourceOrganization_provision_isProvidedBy")); + accumulators + .put( + "datasourceOrganization_provision_provides", + sc.longAccumulator("datasourceOrganization_provision_provides")); - return accumulators; - } + return accumulators; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java index ca81e0b3f..b9746f153 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/XmlIndexingJob.java @@ -1,25 +1,20 @@ + package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.lucidworks.spark.util.SolrSupport; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpDocumentNotFoundException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import java.io.IOException; import java.io.StringReader; import java.io.StringWriter; import java.text.SimpleDateFormat; import java.util.Date; import java.util.Optional; + import javax.xml.transform.Transformer; import javax.xml.transform.TransformerException; import javax.xml.transform.stream.StreamResult; import javax.xml.transform.stream.StreamSource; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.Text; @@ -30,197 +25,206 @@ import org.apache.spark.rdd.RDD; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.lucidworks.spark.util.SolrSupport; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpDocumentNotFoundException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + public class XmlIndexingJob { - private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class); + private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class); - private static final Integer DEFAULT_BATCH_SIZE = 1000; + private static final Integer DEFAULT_BATCH_SIZE = 1000; - private static final String LAYOUT = "index"; - private static final String INTERPRETATION = "openaire"; - private static final String SEPARATOR = "-"; - public static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'"; + private static final String LAYOUT = "index"; + private static final String INTERPRETATION = "openaire"; + private static final String SEPARATOR = "-"; + public static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'"; - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - final ArgumentApplicationParser parser = - new ArgumentApplicationParser( - IOUtils.toString( - XmlIndexingJob.class.getResourceAsStream( - "/eu/dnetlib/dhp/oa/provision/input_params_update_index.json"))); - parser.parseArgument(args); + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + XmlIndexingJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/provision/input_params_update_index.json"))); + parser.parseArgument(args); - Boolean isSparkSessionManaged = - Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); + final String inputPath = parser.get("inputPath"); + log.info("inputPath: {}", inputPath); - final String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); - final String format = parser.get("format"); - log.info("format: {}", format); + final String format = parser.get("format"); + log.info("format: {}", format); - final Integer batchSize = - parser.getObjectMap().containsKey("batchSize") - ? Integer.valueOf(parser.get("batchSize")) - : DEFAULT_BATCH_SIZE; - log.info("batchSize: {}", batchSize); + final Integer batchSize = parser.getObjectMap().containsKey("batchSize") + ? Integer.valueOf(parser.get("batchSize")) + : DEFAULT_BATCH_SIZE; + log.info("batchSize: {}", batchSize); - final ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); - final String fields = getLayoutSource(isLookup, format); - log.info("fields: {}", fields); + final ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl); + final String fields = getLayoutSource(isLookup, format); + log.info("fields: {}", fields); - final String xslt = getLayoutTransformer(isLookup); + final String xslt = getLayoutTransformer(isLookup); - final String dsId = getDsId(format, isLookup); - log.info("dsId: {}", dsId); + final String dsId = getDsId(format, isLookup); + log.info("dsId: {}", dsId); - final String zkHost = getZkHost(isLookup); - log.info("zkHost: {}", zkHost); + final String zkHost = getZkHost(isLookup); + log.info("zkHost: {}", zkHost); - final String version = getRecordDatestamp(); + final String version = getRecordDatestamp(); - final String indexRecordXslt = getLayoutTransformer(format, fields, xslt); - log.info("indexRecordTransformer {}", indexRecordXslt); + final String indexRecordXslt = getLayoutTransformer(format, fields, xslt); + log.info("indexRecordTransformer {}", indexRecordXslt); - final SparkConf conf = new SparkConf(); + final SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - RDD docs = - sc.sequenceFile(inputPath, Text.class, Text.class) - .map(t -> t._2().toString()) - .map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s)) - .map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s)) - .rdd(); + RDD docs = sc + .sequenceFile(inputPath, Text.class, Text.class) + .map(t -> t._2().toString()) + .map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s)) + .map(s -> new StreamingInputDocumentFactory(version, dsId).parseDocument(s)) + .rdd(); - final String collection = format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION; - SolrSupport.indexDocs(zkHost, collection, batchSize, docs); - }); - } + final String collection = format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION; + SolrSupport.indexDocs(zkHost, collection, batchSize, docs); + }); + } - private static String toIndexRecord(Transformer tr, final String record) { - final StreamResult res = new StreamResult(new StringWriter()); - try { - tr.transform(new StreamSource(new StringReader(record)), res); - return res.getWriter().toString(); - } catch (Throwable e) { - log.error("XPathException on record: \n {}", record, e); - throw new IllegalArgumentException(e); - } - } + private static String toIndexRecord(Transformer tr, final String record) { + final StreamResult res = new StreamResult(new StringWriter()); + try { + tr.transform(new StreamSource(new StringReader(record)), res); + return res.getWriter().toString(); + } catch (Throwable e) { + log.error("XPathException on record: \n {}", record, e); + throw new IllegalArgumentException(e); + } + } - /** - * Creates the XSLT responsible for building the index xml records. - * - * @param format Metadata format name (DMF|TMF) - * @param xslt xslt for building the index record transformer - * @param fields the list of fields - * @return the javax.xml.transform.Transformer - * @throws ISLookUpException could happen - * @throws IOException could happen - * @throws TransformerException could happen - */ - private static String getLayoutTransformer(String format, String fields, String xslt) - throws TransformerException { + /** + * Creates the XSLT responsible for building the index xml records. + * + * @param format Metadata format name (DMF|TMF) + * @param xslt xslt for building the index record transformer + * @param fields the list of fields + * @return the javax.xml.transform.Transformer + * @throws ISLookUpException could happen + * @throws IOException could happen + * @throws TransformerException could happen + */ + private static String getLayoutTransformer(String format, String fields, String xslt) + throws TransformerException { - final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt); - final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter()); + final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt); + final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter()); - layoutTransformer.setParameter("format", format); - layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt); + layoutTransformer.setParameter("format", format); + layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt); - return layoutToXsltXslt.getWriter().toString(); - } + return layoutToXsltXslt.getWriter().toString(); + } - /** - * method return a solr-compatible string representation of a date, used to mark all records as - * indexed today - * - * @return the parsed date - */ - public static String getRecordDatestamp() { - return new SimpleDateFormat(DATE_FORMAT).format(new Date()); - } + /** + * method return a solr-compatible string representation of a date, used to mark all records as indexed today + * + * @return the parsed date + */ + public static String getRecordDatestamp() { + return new SimpleDateFormat(DATE_FORMAT).format(new Date()); + } - /** - * Method retrieves from the information system the list of fields associated to the given - * MDFormat name - * - * @param isLookup the ISLookup service stub - * @param format the Metadata format name - * @return the string representation of the list of fields to be indexed - * @throws ISLookUpDocumentNotFoundException - * @throws ISLookUpException - */ - private static String getLayoutSource(final ISLookUpService isLookup, final String format) - throws ISLookUpDocumentNotFoundException, ISLookUpException { - return doLookup( - isLookup, - String.format( - "collection('')//RESOURCE_PROFILE[.//RESOURCE_TYPE/@value = 'MDFormatDSResourceType' and .//NAME='%s']//LAYOUT[@name='%s']", - format, LAYOUT)); - } + /** + * Method retrieves from the information system the list of fields associated to the given MDFormat name + * + * @param isLookup the ISLookup service stub + * @param format the Metadata format name + * @return the string representation of the list of fields to be indexed + * @throws ISLookUpDocumentNotFoundException + * @throws ISLookUpException + */ + private static String getLayoutSource(final ISLookUpService isLookup, final String format) + throws ISLookUpDocumentNotFoundException, ISLookUpException { + return doLookup( + isLookup, + String + .format( + "collection('')//RESOURCE_PROFILE[.//RESOURCE_TYPE/@value = 'MDFormatDSResourceType' and .//NAME='%s']//LAYOUT[@name='%s']", + format, LAYOUT)); + } - /** - * Method retrieves from the information system the openaireLayoutToRecordStylesheet - * - * @param isLookup the ISLookup service stub - * @return the string representation of the XSLT contained in the transformation rule profile - * @throws ISLookUpDocumentNotFoundException - * @throws ISLookUpException - */ - private static String getLayoutTransformer(ISLookUpService isLookup) throws ISLookUpException { - return doLookup( - isLookup, - "collection('/db/DRIVER/TransformationRuleDSResources/TransformationRuleDSResourceType')" - + "//RESOURCE_PROFILE[./BODY/CONFIGURATION/SCRIPT/TITLE/text() = 'openaireLayoutToRecordStylesheet']//CODE/node()"); - } + /** + * Method retrieves from the information system the openaireLayoutToRecordStylesheet + * + * @param isLookup the ISLookup service stub + * @return the string representation of the XSLT contained in the transformation rule profile + * @throws ISLookUpDocumentNotFoundException + * @throws ISLookUpException + */ + private static String getLayoutTransformer(ISLookUpService isLookup) throws ISLookUpException { + return doLookup( + isLookup, + "collection('/db/DRIVER/TransformationRuleDSResources/TransformationRuleDSResourceType')" + + "//RESOURCE_PROFILE[./BODY/CONFIGURATION/SCRIPT/TITLE/text() = 'openaireLayoutToRecordStylesheet']//CODE/node()"); + } - /** - * Method retrieves from the information system the IndexDS profile ID associated to the given - * MDFormat name - * - * @param format - * @param isLookup - * @return the IndexDS identifier - * @throws ISLookUpException - */ - private static String getDsId(String format, ISLookUpService isLookup) throws ISLookUpException { - return doLookup( - isLookup, - String.format( - "collection('/db/DRIVER/IndexDSResources/IndexDSResourceType')" - + "//RESOURCE_PROFILE[./BODY/CONFIGURATION/METADATA_FORMAT/text() = '%s']//RESOURCE_IDENTIFIER/@value/string()", - format)); - } + /** + * Method retrieves from the information system the IndexDS profile ID associated to the given MDFormat name + * + * @param format + * @param isLookup + * @return the IndexDS identifier + * @throws ISLookUpException + */ + private static String getDsId(String format, ISLookUpService isLookup) throws ISLookUpException { + return doLookup( + isLookup, + String + .format( + "collection('/db/DRIVER/IndexDSResources/IndexDSResourceType')" + + "//RESOURCE_PROFILE[./BODY/CONFIGURATION/METADATA_FORMAT/text() = '%s']//RESOURCE_IDENTIFIER/@value/string()", + format)); + } - /** - * Method retrieves from the information system the zookeeper quorum of the Solr server - * - * @param isLookup - * @return the zookeeper quorum of the Solr server - * @throws ISLookUpException - */ - private static String getZkHost(ISLookUpService isLookup) throws ISLookUpException { - return doLookup( - isLookup, - "for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='IndexServiceResourceType'] return $x//PROTOCOL[./@name='solr']/@address/string()"); - } + /** + * Method retrieves from the information system the zookeeper quorum of the Solr server + * + * @param isLookup + * @return the zookeeper quorum of the Solr server + * @throws ISLookUpException + */ + private static String getZkHost(ISLookUpService isLookup) throws ISLookUpException { + return doLookup( + isLookup, + "for $x in /RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='IndexServiceResourceType'] return $x//PROTOCOL[./@name='solr']/@address/string()"); + } - private static String doLookup(ISLookUpService isLookup, String xquery) throws ISLookUpException { - log.info(String.format("running xquery: %s", xquery)); - final String res = isLookup.getResourceProfileByQuery(xquery); - log.info(String.format("got response (100 chars): %s", StringUtils.left(res, 100) + " ...")); - return res; - } + private static String doLookup(ISLookUpService isLookup, String xquery) throws ISLookUpException { + log.info(String.format("running xquery: %s", xquery)); + final String res = isLookup.getResourceProfileByQuery(xquery); + log.info(String.format("got response (100 chars): %s", StringUtils.left(res, 100) + " ...")); + return res; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/EntityRelEntity.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/EntityRelEntity.java index e47356c13..a6b3c5591 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/EntityRelEntity.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/EntityRelEntity.java @@ -1,62 +1,67 @@ + package eu.dnetlib.dhp.oa.provision.model; -import com.google.common.base.Objects; import java.io.Serializable; +import com.google.common.base.Objects; + public class EntityRelEntity implements Serializable { - private TypedRow entity; - private SortableRelation relation; - private RelatedEntity target; + private TypedRow entity; + private SortableRelation relation; + private RelatedEntity target; - public EntityRelEntity() {} + public EntityRelEntity() { + } - public EntityRelEntity(SortableRelation relation, RelatedEntity target) { - this(null, relation, target); - } + public EntityRelEntity(SortableRelation relation, RelatedEntity target) { + this(null, relation, target); + } - public EntityRelEntity(TypedRow entity, SortableRelation relation, RelatedEntity target) { - this.entity = entity; - this.relation = relation; - this.target = target; - } + public EntityRelEntity(TypedRow entity, SortableRelation relation, RelatedEntity target) { + this.entity = entity; + this.relation = relation; + this.target = target; + } - public TypedRow getEntity() { - return entity; - } + public TypedRow getEntity() { + return entity; + } - public void setEntity(TypedRow entity) { - this.entity = entity; - } + public void setEntity(TypedRow entity) { + this.entity = entity; + } - public SortableRelation getRelation() { - return relation; - } + public SortableRelation getRelation() { + return relation; + } - public void setRelation(SortableRelation relation) { - this.relation = relation; - } + public void setRelation(SortableRelation relation) { + this.relation = relation; + } - public RelatedEntity getTarget() { - return target; - } + public RelatedEntity getTarget() { + return target; + } - public void setTarget(RelatedEntity target) { - this.target = target; - } + public void setTarget(RelatedEntity target) { + this.target = target; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - EntityRelEntity that = (EntityRelEntity) o; - return Objects.equal(entity, that.entity) - && Objects.equal(relation, that.relation) - && Objects.equal(target, that.target); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + EntityRelEntity that = (EntityRelEntity) o; + return Objects.equal(entity, that.entity) + && Objects.equal(relation, that.relation) + && Objects.equal(target, that.target); + } - @Override - public int hashCode() { - return Objects.hashCode(entity, relation, target); - } + @Override + public int hashCode() { + return Objects.hashCode(entity, relation, target); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java index daa069255..e29ec9d19 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/JoinedEntity.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.provision.model; import java.io.Serializable; @@ -5,25 +6,26 @@ import java.util.List; public class JoinedEntity implements Serializable { - private TypedRow entity; + private TypedRow entity; - private List links; + private List links; - public JoinedEntity() {} + public JoinedEntity() { + } - public TypedRow getEntity() { - return entity; - } + public TypedRow getEntity() { + return entity; + } - public void setEntity(TypedRow entity) { - this.entity = entity; - } + public void setEntity(TypedRow entity) { + this.entity = entity; + } - public List getLinks() { - return links; - } + public List getLinks() { + return links; + } - public void setLinks(List links) { - this.links = links; - } + public void setLinks(List links) { + this.links = links; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java index 9671d505c..e15ceff76 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntity.java @@ -1,289 +1,295 @@ + package eu.dnetlib.dhp.oa.provision.model; +import java.io.Serializable; +import java.util.List; + import com.google.common.base.Objects; + import eu.dnetlib.dhp.schema.oaf.Instance; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import java.io.Serializable; -import java.util.List; public class RelatedEntity implements Serializable { - private String id; - private String type; + private String id; + private String type; - // common fields - private StructuredProperty title; - private String websiteurl; // datasource, organizations, projects + // common fields + private StructuredProperty title; + private String websiteurl; // datasource, organizations, projects - // results - private String dateofacceptance; - private String publisher; - private List pid; - private String codeRepositoryUrl; - private Qualifier resulttype; - private List collectedfrom; - private List instances; + // results + private String dateofacceptance; + private String publisher; + private List pid; + private String codeRepositoryUrl; + private Qualifier resulttype; + private List collectedfrom; + private List instances; - // datasource - private String officialname; - private Qualifier datasourcetype; - private Qualifier datasourcetypeui; - private Qualifier openairecompatibility; - // private String aggregatortype; + // datasource + private String officialname; + private Qualifier datasourcetype; + private Qualifier datasourcetypeui; + private Qualifier openairecompatibility; + // private String aggregatortype; - // organization - private String legalname; - private String legalshortname; - private Qualifier country; + // organization + private String legalname; + private String legalshortname; + private Qualifier country; - // project - private String projectTitle; - private String code; - private String acronym; - private Qualifier contracttype; - private List fundingtree; + // project + private String projectTitle; + private String code; + private String acronym; + private Qualifier contracttype; + private List fundingtree; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(String type) { - this.type = type; - } + public void setType(String type) { + this.type = type; + } - public StructuredProperty getTitle() { - return title; - } + public StructuredProperty getTitle() { + return title; + } - public void setTitle(StructuredProperty title) { - this.title = title; - } + public void setTitle(StructuredProperty title) { + this.title = title; + } - public String getWebsiteurl() { - return websiteurl; - } + public String getWebsiteurl() { + return websiteurl; + } - public void setWebsiteurl(String websiteurl) { - this.websiteurl = websiteurl; - } + public void setWebsiteurl(String websiteurl) { + this.websiteurl = websiteurl; + } - public String getDateofacceptance() { - return dateofacceptance; - } + public String getDateofacceptance() { + return dateofacceptance; + } - public void setDateofacceptance(String dateofacceptance) { - this.dateofacceptance = dateofacceptance; - } + public void setDateofacceptance(String dateofacceptance) { + this.dateofacceptance = dateofacceptance; + } - public String getPublisher() { - return publisher; - } + public String getPublisher() { + return publisher; + } - public void setPublisher(String publisher) { - this.publisher = publisher; - } + public void setPublisher(String publisher) { + this.publisher = publisher; + } - public List getPid() { - return pid; - } + public List getPid() { + return pid; + } - public void setPid(List pid) { - this.pid = pid; - } + public void setPid(List pid) { + this.pid = pid; + } - public String getCodeRepositoryUrl() { - return codeRepositoryUrl; - } + public String getCodeRepositoryUrl() { + return codeRepositoryUrl; + } - public void setCodeRepositoryUrl(String codeRepositoryUrl) { - this.codeRepositoryUrl = codeRepositoryUrl; - } + public void setCodeRepositoryUrl(String codeRepositoryUrl) { + this.codeRepositoryUrl = codeRepositoryUrl; + } - public Qualifier getResulttype() { - return resulttype; - } + public Qualifier getResulttype() { + return resulttype; + } - public void setResulttype(Qualifier resulttype) { - this.resulttype = resulttype; - } + public void setResulttype(Qualifier resulttype) { + this.resulttype = resulttype; + } - public List getCollectedfrom() { - return collectedfrom; - } + public List getCollectedfrom() { + return collectedfrom; + } - public void setCollectedfrom(List collectedfrom) { - this.collectedfrom = collectedfrom; - } + public void setCollectedfrom(List collectedfrom) { + this.collectedfrom = collectedfrom; + } - public List getInstances() { - return instances; - } + public List getInstances() { + return instances; + } - public void setInstances(List instances) { - this.instances = instances; - } + public void setInstances(List instances) { + this.instances = instances; + } - public String getOfficialname() { - return officialname; - } + public String getOfficialname() { + return officialname; + } - public void setOfficialname(String officialname) { - this.officialname = officialname; - } + public void setOfficialname(String officialname) { + this.officialname = officialname; + } - public Qualifier getDatasourcetype() { - return datasourcetype; - } + public Qualifier getDatasourcetype() { + return datasourcetype; + } - public void setDatasourcetype(Qualifier datasourcetype) { - this.datasourcetype = datasourcetype; - } + public void setDatasourcetype(Qualifier datasourcetype) { + this.datasourcetype = datasourcetype; + } - public Qualifier getDatasourcetypeui() { - return datasourcetypeui; - } + public Qualifier getDatasourcetypeui() { + return datasourcetypeui; + } - public void setDatasourcetypeui(Qualifier datasourcetypeui) { - this.datasourcetypeui = datasourcetypeui; - } + public void setDatasourcetypeui(Qualifier datasourcetypeui) { + this.datasourcetypeui = datasourcetypeui; + } - public Qualifier getOpenairecompatibility() { - return openairecompatibility; - } + public Qualifier getOpenairecompatibility() { + return openairecompatibility; + } - public void setOpenairecompatibility(Qualifier openairecompatibility) { - this.openairecompatibility = openairecompatibility; - } + public void setOpenairecompatibility(Qualifier openairecompatibility) { + this.openairecompatibility = openairecompatibility; + } - public String getLegalname() { - return legalname; - } + public String getLegalname() { + return legalname; + } - public void setLegalname(String legalname) { - this.legalname = legalname; - } + public void setLegalname(String legalname) { + this.legalname = legalname; + } - public String getLegalshortname() { - return legalshortname; - } + public String getLegalshortname() { + return legalshortname; + } - public void setLegalshortname(String legalshortname) { - this.legalshortname = legalshortname; - } + public void setLegalshortname(String legalshortname) { + this.legalshortname = legalshortname; + } - public Qualifier getCountry() { - return country; - } + public Qualifier getCountry() { + return country; + } - public void setCountry(Qualifier country) { - this.country = country; - } + public void setCountry(Qualifier country) { + this.country = country; + } - public String getProjectTitle() { - return projectTitle; - } + public String getProjectTitle() { + return projectTitle; + } - public void setProjectTitle(String projectTitle) { - this.projectTitle = projectTitle; - } + public void setProjectTitle(String projectTitle) { + this.projectTitle = projectTitle; + } - public String getCode() { - return code; - } + public String getCode() { + return code; + } - public void setCode(String code) { - this.code = code; - } + public void setCode(String code) { + this.code = code; + } - public String getAcronym() { - return acronym; - } + public String getAcronym() { + return acronym; + } - public void setAcronym(String acronym) { - this.acronym = acronym; - } + public void setAcronym(String acronym) { + this.acronym = acronym; + } - public Qualifier getContracttype() { - return contracttype; - } + public Qualifier getContracttype() { + return contracttype; + } - public void setContracttype(Qualifier contracttype) { - this.contracttype = contracttype; - } + public void setContracttype(Qualifier contracttype) { + this.contracttype = contracttype; + } - public List getFundingtree() { - return fundingtree; - } + public List getFundingtree() { + return fundingtree; + } - public void setFundingtree(List fundingtree) { - this.fundingtree = fundingtree; - } + public void setFundingtree(List fundingtree) { + this.fundingtree = fundingtree; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - RelatedEntity that = (RelatedEntity) o; - return Objects.equal(id, that.id) - && Objects.equal(type, that.type) - && Objects.equal(title, that.title) - && Objects.equal(websiteurl, that.websiteurl) - && Objects.equal(dateofacceptance, that.dateofacceptance) - && Objects.equal(publisher, that.publisher) - && Objects.equal(pid, that.pid) - && Objects.equal(codeRepositoryUrl, that.codeRepositoryUrl) - && Objects.equal(resulttype, that.resulttype) - && Objects.equal(collectedfrom, that.collectedfrom) - && Objects.equal(instances, that.instances) - && Objects.equal(officialname, that.officialname) - && Objects.equal(datasourcetype, that.datasourcetype) - && Objects.equal(datasourcetypeui, that.datasourcetypeui) - && Objects.equal(openairecompatibility, that.openairecompatibility) - && Objects.equal(legalname, that.legalname) - && Objects.equal(legalshortname, that.legalshortname) - && Objects.equal(country, that.country) - && Objects.equal(projectTitle, that.projectTitle) - && Objects.equal(code, that.code) - && Objects.equal(acronym, that.acronym) - && Objects.equal(contracttype, that.contracttype) - && Objects.equal(fundingtree, that.fundingtree); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + RelatedEntity that = (RelatedEntity) o; + return Objects.equal(id, that.id) + && Objects.equal(type, that.type) + && Objects.equal(title, that.title) + && Objects.equal(websiteurl, that.websiteurl) + && Objects.equal(dateofacceptance, that.dateofacceptance) + && Objects.equal(publisher, that.publisher) + && Objects.equal(pid, that.pid) + && Objects.equal(codeRepositoryUrl, that.codeRepositoryUrl) + && Objects.equal(resulttype, that.resulttype) + && Objects.equal(collectedfrom, that.collectedfrom) + && Objects.equal(instances, that.instances) + && Objects.equal(officialname, that.officialname) + && Objects.equal(datasourcetype, that.datasourcetype) + && Objects.equal(datasourcetypeui, that.datasourcetypeui) + && Objects.equal(openairecompatibility, that.openairecompatibility) + && Objects.equal(legalname, that.legalname) + && Objects.equal(legalshortname, that.legalshortname) + && Objects.equal(country, that.country) + && Objects.equal(projectTitle, that.projectTitle) + && Objects.equal(code, that.code) + && Objects.equal(acronym, that.acronym) + && Objects.equal(contracttype, that.contracttype) + && Objects.equal(fundingtree, that.fundingtree); + } - @Override - public int hashCode() { - return Objects.hashCode( - id, - type, - title, - websiteurl, - dateofacceptance, - publisher, - pid, - codeRepositoryUrl, - resulttype, - collectedfrom, - instances, - officialname, - datasourcetype, - datasourcetypeui, - openairecompatibility, - legalname, - legalshortname, - country, - projectTitle, - code, - acronym, - contracttype, - fundingtree); - } + @Override + public int hashCode() { + return Objects + .hashCode( + id, + type, + title, + websiteurl, + dateofacceptance, + publisher, + pid, + codeRepositoryUrl, + resulttype, + collectedfrom, + instances, + officialname, + datasourcetype, + datasourcetypeui, + openairecompatibility, + legalname, + legalshortname, + country, + projectTitle, + code, + acronym, + contracttype, + fundingtree); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelation.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelation.java index 0a35a9752..7c866001b 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelation.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelation.java @@ -1,34 +1,38 @@ + package eu.dnetlib.dhp.oa.provision.model; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Maps; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.io.Serializable; import java.util.Map; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.schema.oaf.Relation; + public class SortableRelation extends Relation implements Comparable, Serializable { - private static final Map weights = Maps.newHashMap(); + private static final Map weights = Maps.newHashMap(); - static { - weights.put("outcome", 0); - weights.put("supplement", 1); - weights.put("publicationDataset", 2); - weights.put("relationship", 3); - weights.put("similarity", 4); - weights.put("affiliation", 5); + static { + weights.put("outcome", 0); + weights.put("supplement", 1); + weights.put("publicationDataset", 2); + weights.put("relationship", 3); + weights.put("similarity", 4); + weights.put("affiliation", 5); - weights.put("provision", 6); - weights.put("participation", 7); - weights.put("dedup", 8); - } + weights.put("provision", 6); + weights.put("participation", 7); + weights.put("dedup", 8); + } - @Override - public int compareTo(Relation o) { - return ComparisonChain.start() - .compare(weights.get(getSubRelType()), weights.get(o.getSubRelType())) - .compare(getSource(), o.getSource()) - .compare(getTarget(), o.getTarget()) - .result(); - } + @Override + public int compareTo(Relation o) { + return ComparisonChain + .start() + .compare(weights.get(getSubRelType()), weights.get(o.getSubRelType())) + .compare(getSource(), o.getSource()) + .compare(getTarget(), o.getTarget()) + .result(); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/Tuple2.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/Tuple2.java index e7e4aea3c..5ebe9c9eb 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/Tuple2.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/Tuple2.java @@ -1,48 +1,53 @@ + package eu.dnetlib.dhp.oa.provision.model; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.io.Serializable; import java.util.Objects; +import eu.dnetlib.dhp.schema.oaf.Relation; + public class Tuple2 implements Serializable { - private Relation relation; + private Relation relation; - private RelatedEntity relatedEntity; + private RelatedEntity relatedEntity; - public Tuple2() {} + public Tuple2() { + } - public Tuple2(Relation relation, RelatedEntity relatedEntity) { - this.relation = relation; - this.relatedEntity = relatedEntity; - } + public Tuple2(Relation relation, RelatedEntity relatedEntity) { + this.relation = relation; + this.relatedEntity = relatedEntity; + } - public Relation getRelation() { - return relation; - } + public Relation getRelation() { + return relation; + } - public void setRelation(Relation relation) { - this.relation = relation; - } + public void setRelation(Relation relation) { + this.relation = relation; + } - public RelatedEntity getRelatedEntity() { - return relatedEntity; - } + public RelatedEntity getRelatedEntity() { + return relatedEntity; + } - public void setRelatedEntity(RelatedEntity relatedEntity) { - this.relatedEntity = relatedEntity; - } + public void setRelatedEntity(RelatedEntity relatedEntity) { + this.relatedEntity = relatedEntity; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Tuple2 t2 = (Tuple2) o; - return getRelation().equals(t2.getRelation()); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + Tuple2 t2 = (Tuple2) o; + return getRelation().equals(t2.getRelation()); + } - @Override - public int hashCode() { - return Objects.hash(getRelation().hashCode()); - } + @Override + public int hashCode() { + return Objects.hash(getRelation().hashCode()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java index 01067707e..cbec372e4 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java @@ -1,60 +1,64 @@ + package eu.dnetlib.dhp.oa.provision.model; -import com.google.common.base.Objects; import java.io.Serializable; +import com.google.common.base.Objects; + public class TypedRow implements Serializable { - private String id; + private String id; - private Boolean deleted; + private Boolean deleted; - private String type; + private String type; - private String oaf; + private String oaf; - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(String id) { - this.id = id; - } + public void setId(String id) { + this.id = id; + } - public Boolean getDeleted() { - return deleted; - } + public Boolean getDeleted() { + return deleted; + } - public void setDeleted(Boolean deleted) { - this.deleted = deleted; - } + public void setDeleted(Boolean deleted) { + this.deleted = deleted; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(String type) { - this.type = type; - } + public void setType(String type) { + this.type = type; + } - public String getOaf() { - return oaf; - } + public String getOaf() { + return oaf; + } - public void setOaf(String oaf) { - this.oaf = oaf; - } + public void setOaf(String oaf) { + this.oaf = oaf; + } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - TypedRow typedRow2 = (TypedRow) o; - return Objects.equal(id, typedRow2.id); - } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + TypedRow typedRow2 = (TypedRow) o; + return Objects.equal(id, typedRow2.id); + } - @Override - public int hashCode() { - return Objects.hashCode(id); - } + @Override + public int hashCode() { + return Objects.hashCode(id); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextDef.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextDef.java index dc6170445..8afd6400c 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextDef.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextDef.java @@ -1,51 +1,52 @@ + package eu.dnetlib.dhp.oa.provision.utils; import java.io.Serializable; public class ContextDef implements Serializable { - private String id; - private String label; - private String name; - private String type; + private String id; + private String label; + private String name; + private String type; - public ContextDef(final String id, final String label, final String name, final String type) { - super(); - this.setId(id); - this.setLabel(label); - this.setName(name); - this.setType(type); - } + public ContextDef(final String id, final String label, final String name, final String type) { + super(); + this.setId(id); + this.setLabel(label); + this.setName(name); + this.setType(type); + } - public String getLabel() { - return label; - } + public String getLabel() { + return label; + } - public void setLabel(final String label) { - this.label = label; - } + public void setLabel(final String label) { + this.label = label; + } - public String getId() { - return id; - } + public String getId() { + return id; + } - public void setId(final String id) { - this.id = id; - } + public void setId(final String id) { + this.id = id; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(final String name) { - this.name = name; - } + public void setName(final String name) { + this.name = name; + } - public String getType() { - return type; - } + public String getType() { + return type; + } - public void setType(final String type) { - this.type = type; - } + public void setType(final String type) { + this.type = type; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextMapper.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextMapper.java index d1d6521db..ac418f2b9 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextMapper.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/ContextMapper.java @@ -1,46 +1,49 @@ + package eu.dnetlib.dhp.oa.provision.utils; -import com.google.common.base.Joiner; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import java.io.Serializable; import java.io.StringReader; import java.util.HashMap; + import org.dom4j.Document; import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; +import com.google.common.base.Joiner; + +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + public class ContextMapper extends HashMap implements Serializable { - private static final long serialVersionUID = 2159682308502487305L; + private static final long serialVersionUID = 2159682308502487305L; - private static final String XQUERY = - "for $x in //RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='ContextDSResourceType']//*[name()='context' or name()='category' or name()='concept'] return "; + private static final String XQUERY = "for $x in //RESOURCE_PROFILE[.//RESOURCE_TYPE/@value='ContextDSResourceType']//*[name()='context' or name()='category' or name()='concept'] return "; - public static ContextMapper fromIS(final String isLookupUrl) - throws DocumentException, ISLookUpException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - StringBuilder sb = new StringBuilder(""); - Joiner.on("").appendTo(sb, isLookUp.quickSearchProfile(XQUERY)); - sb.append(""); - return fromXml(sb.toString()); - } + public static ContextMapper fromIS(final String isLookupUrl) + throws DocumentException, ISLookUpException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + StringBuilder sb = new StringBuilder(""); + Joiner.on("").appendTo(sb, isLookUp.quickSearchProfile(XQUERY)); + sb.append(""); + return fromXml(sb.toString()); + } - public static ContextMapper fromXml(final String xml) throws DocumentException { - final ContextMapper contextMapper = new ContextMapper(); + public static ContextMapper fromXml(final String xml) throws DocumentException { + final ContextMapper contextMapper = new ContextMapper(); - final Document doc = new SAXReader().read(new StringReader(xml)); - for (Object o : doc.selectNodes("//entry")) { - Node node = (Node) o; - String id = node.valueOf("./@id"); - String label = node.valueOf("./@label"); - String name = node.valueOf("./@name"); - String type = node.valueOf("./@type") + ""; + final Document doc = new SAXReader().read(new StringReader(xml)); + for (Object o : doc.selectNodes("//entry")) { + Node node = (Node) o; + String id = node.valueOf("./@id"); + String label = node.valueOf("./@label"); + String name = node.valueOf("./@name"); + String type = node.valueOf("./@type") + ""; - contextMapper.put(id, new ContextDef(id, label, name, type)); - } - return contextMapper; - } + contextMapper.put(id, new ContextDef(id, label, name, type)); + } + return contextMapper; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/GraphMappingUtils.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/GraphMappingUtils.java index 96ffb4c90..0e742365a 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/GraphMappingUtils.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/GraphMappingUtils.java @@ -1,23 +1,27 @@ + package eu.dnetlib.dhp.oa.provision.utils; import static org.apache.commons.lang3.StringUtils.substringAfter; -import com.google.common.collect.Sets; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Set; +import com.google.common.collect.Sets; + +import eu.dnetlib.dhp.schema.oaf.*; + public class GraphMappingUtils { - public static final String SEPARATOR = "_"; + public static final String SEPARATOR = "_"; - public static Set authorPidTypes = Sets.newHashSet("orcid", "magidentifier"); + public static Set authorPidTypes = Sets.newHashSet("orcid", "magidentifier"); - public static String removePrefix(final String s) { - if (s.contains("|")) return substringAfter(s, "|"); - return s; - } + public static String removePrefix(final String s) { + if (s.contains("|")) + return substringAfter(s, "|"); + return s; + } - public static String getRelDescriptor(String relType, String subRelType, String relClass) { - return relType + SEPARATOR + subRelType + SEPARATOR + relClass; - } + public static String getRelDescriptor(String relType, String subRelType, String relClass) { + return relType + SEPARATOR + subRelType + SEPARATOR + relClass; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/LicenseComparator.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/LicenseComparator.java index 823997b6d..9dbac1936 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/LicenseComparator.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/LicenseComparator.java @@ -1,47 +1,69 @@ + package eu.dnetlib.dhp.oa.provision.utils; -import eu.dnetlib.dhp.schema.oaf.Qualifier; import java.util.Comparator; +import eu.dnetlib.dhp.schema.oaf.Qualifier; + public class LicenseComparator implements Comparator { - @Override - public int compare(Qualifier left, Qualifier right) { + @Override + public int compare(Qualifier left, Qualifier right) { - if (left == null && right == null) return 0; - if (left == null) return 1; - if (right == null) return -1; + if (left == null && right == null) + return 0; + if (left == null) + return 1; + if (right == null) + return -1; - String lClass = left.getClassid(); - String rClass = right.getClassid(); + String lClass = left.getClassid(); + String rClass = right.getClassid(); - if (lClass.equals(rClass)) return 0; + if (lClass.equals(rClass)) + return 0; - if (lClass.equals("OPEN SOURCE")) return -1; - if (rClass.equals("OPEN SOURCE")) return 1; + if (lClass.equals("OPEN SOURCE")) + return -1; + if (rClass.equals("OPEN SOURCE")) + return 1; - if (lClass.equals("OPEN")) return -1; - if (rClass.equals("OPEN")) return 1; + if (lClass.equals("OPEN")) + return -1; + if (rClass.equals("OPEN")) + return 1; - if (lClass.equals("6MONTHS")) return -1; - if (rClass.equals("6MONTHS")) return 1; + if (lClass.equals("6MONTHS")) + return -1; + if (rClass.equals("6MONTHS")) + return 1; - if (lClass.equals("12MONTHS")) return -1; - if (rClass.equals("12MONTHS")) return 1; + if (lClass.equals("12MONTHS")) + return -1; + if (rClass.equals("12MONTHS")) + return 1; - if (lClass.equals("EMBARGO")) return -1; - if (rClass.equals("EMBARGO")) return 1; + if (lClass.equals("EMBARGO")) + return -1; + if (rClass.equals("EMBARGO")) + return 1; - if (lClass.equals("RESTRICTED")) return -1; - if (rClass.equals("RESTRICTED")) return 1; + if (lClass.equals("RESTRICTED")) + return -1; + if (rClass.equals("RESTRICTED")) + return 1; - if (lClass.equals("CLOSED")) return -1; - if (rClass.equals("CLOSED")) return 1; + if (lClass.equals("CLOSED")) + return -1; + if (rClass.equals("CLOSED")) + return 1; - if (lClass.equals("UNKNOWN")) return -1; - if (rClass.equals("UNKNOWN")) return 1; + if (lClass.equals("UNKNOWN")) + return -1; + if (rClass.equals("UNKNOWN")) + return 1; - // Else (but unlikely), lexicographical ordering will do. - return lClass.compareTo(rClass); - } + // Else (but unlikely), lexicographical ordering will do. + return lClass.compareTo(rClass); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/RelationPartitioner.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/RelationPartitioner.java index 6db8b12de..bac2278e6 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/RelationPartitioner.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/RelationPartitioner.java @@ -1,29 +1,30 @@ + package eu.dnetlib.dhp.oa.provision.utils; -import eu.dnetlib.dhp.oa.provision.model.SortableRelation; import org.apache.spark.Partitioner; import org.apache.spark.util.Utils; +import eu.dnetlib.dhp.oa.provision.model.SortableRelation; + /** - * Used in combination with SortableRelationKey, allows to partition the records by source id, - * therefore allowing to sort relations sharing the same source id by the ordering defined in - * SortableRelationKey. + * Used in combination with SortableRelationKey, allows to partition the records by source id, therefore allowing to + * sort relations sharing the same source id by the ordering defined in SortableRelationKey. */ public class RelationPartitioner extends Partitioner { - private int numPartitions; + private int numPartitions; - public RelationPartitioner(int numPartitions) { - this.numPartitions = numPartitions; - } + public RelationPartitioner(int numPartitions) { + this.numPartitions = numPartitions; + } - @Override - public int numPartitions() { - return numPartitions; - } + @Override + public int numPartitions() { + return numPartitions; + } - @Override - public int getPartition(Object key) { - return Utils.nonNegativeMod(((SortableRelation) key).getSource().hashCode(), numPartitions()); - } + @Override + public int getPartition(Object key) { + return Utils.nonNegativeMod(((SortableRelation) key).getSource().hashCode(), numPartitions()); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/StreamingInputDocumentFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/StreamingInputDocumentFactory.java index c472e6e85..de221b2ee 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/StreamingInputDocumentFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/StreamingInputDocumentFactory.java @@ -1,262 +1,260 @@ + package eu.dnetlib.dhp.oa.provision.utils; -import com.google.common.collect.Lists; import java.io.StringReader; import java.io.StringWriter; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.List; + import javax.xml.stream.*; import javax.xml.stream.events.Namespace; import javax.xml.stream.events.StartElement; import javax.xml.stream.events.XMLEvent; + import org.apache.solr.common.SolrInputDocument; +import com.google.common.collect.Lists; + /** * Optimized version of the document parser, drop in replacement of InputDocumentFactory. - * - *

Faster because: - * + *

+ * Faster because: *

- * - *

This class is fully reentrant and can be invoked in parallel. + *

+ * This class is fully reentrant and can be invoked in parallel. * * @author claudio */ public class StreamingInputDocumentFactory { - private static final String INDEX_FIELD_PREFIX = "__"; + private static final String INDEX_FIELD_PREFIX = "__"; - private static final String DS_VERSION = INDEX_FIELD_PREFIX + "dsversion"; + private static final String DS_VERSION = INDEX_FIELD_PREFIX + "dsversion"; - private static final String DS_ID = INDEX_FIELD_PREFIX + "dsid"; + private static final String DS_ID = INDEX_FIELD_PREFIX + "dsid"; - private static final String RESULT = "result"; + private static final String RESULT = "result"; - private static final String INDEX_RESULT = INDEX_FIELD_PREFIX + RESULT; + private static final String INDEX_RESULT = INDEX_FIELD_PREFIX + RESULT; - private static final String INDEX_RECORD_ID = INDEX_FIELD_PREFIX + "indexrecordidentifier"; + private static final String INDEX_RECORD_ID = INDEX_FIELD_PREFIX + "indexrecordidentifier"; - private static final String outFormat = new String("yyyy-MM-dd'T'hh:mm:ss'Z'"); + private static final String outFormat = new String("yyyy-MM-dd'T'hh:mm:ss'Z'"); - private static final List dateFormats = - Arrays.asList("yyyy-MM-dd'T'hh:mm:ss", "yyyy-MM-dd", "dd-MM-yyyy", "dd/MM/yyyy", "yyyy"); + private static final List dateFormats = Arrays + .asList("yyyy-MM-dd'T'hh:mm:ss", "yyyy-MM-dd", "dd-MM-yyyy", "dd/MM/yyyy", "yyyy"); - private static final String DEFAULTDNETRESULT = "dnetResult"; + private static final String DEFAULTDNETRESULT = "dnetResult"; - private static final String TARGETFIELDS = "targetFields"; + private static final String TARGETFIELDS = "targetFields"; - private static final String INDEX_RECORD_ID_ELEMENT = "indexRecordIdentifier"; + private static final String INDEX_RECORD_ID_ELEMENT = "indexRecordIdentifier"; - private static final String ROOT_ELEMENT = "indexRecord"; + private static final String ROOT_ELEMENT = "indexRecord"; - private static final int MAX_FIELD_LENGTH = 25000; + private static final int MAX_FIELD_LENGTH = 25000; - private ThreadLocal inputFactory = - ThreadLocal.withInitial(() -> XMLInputFactory.newInstance()); + private ThreadLocal inputFactory = ThreadLocal.withInitial(() -> XMLInputFactory.newInstance()); - private ThreadLocal outputFactory = - ThreadLocal.withInitial(() -> XMLOutputFactory.newInstance()); + private ThreadLocal outputFactory = ThreadLocal.withInitial(() -> XMLOutputFactory.newInstance()); - private ThreadLocal eventFactory = - ThreadLocal.withInitial(() -> XMLEventFactory.newInstance()); + private ThreadLocal eventFactory = ThreadLocal.withInitial(() -> XMLEventFactory.newInstance()); - private String version; + private String version; - private String dsId; + private String dsId; - private String resultName = DEFAULTDNETRESULT; + private String resultName = DEFAULTDNETRESULT; - public StreamingInputDocumentFactory(final String version, final String dsId) { - this(version, dsId, DEFAULTDNETRESULT); - } + public StreamingInputDocumentFactory(final String version, final String dsId) { + this(version, dsId, DEFAULTDNETRESULT); + } - public StreamingInputDocumentFactory( - final String version, final String dsId, final String resultName) { - this.version = version; - this.dsId = dsId; - this.resultName = resultName; - } + public StreamingInputDocumentFactory( + final String version, final String dsId, final String resultName) { + this.version = version; + this.dsId = dsId; + this.resultName = resultName; + } - public SolrInputDocument parseDocument(final String inputDocument) { + public SolrInputDocument parseDocument(final String inputDocument) { - final StringWriter results = new StringWriter(); - final List nsList = Lists.newLinkedList(); - try { + final StringWriter results = new StringWriter(); + final List nsList = Lists.newLinkedList(); + try { - XMLEventReader parser = - inputFactory.get().createXMLEventReader(new StringReader(inputDocument)); + XMLEventReader parser = inputFactory.get().createXMLEventReader(new StringReader(inputDocument)); - final SolrInputDocument indexDocument = new SolrInputDocument(new HashMap<>()); + final SolrInputDocument indexDocument = new SolrInputDocument(new HashMap<>()); - while (parser.hasNext()) { - final XMLEvent event = parser.nextEvent(); - if ((event != null) && event.isStartElement()) { - final String localName = event.asStartElement().getName().getLocalPart(); + while (parser.hasNext()) { + final XMLEvent event = parser.nextEvent(); + if ((event != null) && event.isStartElement()) { + final String localName = event.asStartElement().getName().getLocalPart(); - if (ROOT_ELEMENT.equals(localName)) { - nsList.addAll(getNamespaces(event)); - } else if (INDEX_RECORD_ID_ELEMENT.equals(localName)) { - final XMLEvent text = parser.nextEvent(); - String recordId = getText(text); - indexDocument.addField(INDEX_RECORD_ID, recordId); - } else if (TARGETFIELDS.equals(localName)) { - parseTargetFields(indexDocument, parser); - } else if (resultName.equals(localName)) { - copyResult(indexDocument, results, parser, nsList, resultName); - } - } - } + if (ROOT_ELEMENT.equals(localName)) { + nsList.addAll(getNamespaces(event)); + } else if (INDEX_RECORD_ID_ELEMENT.equals(localName)) { + final XMLEvent text = parser.nextEvent(); + String recordId = getText(text); + indexDocument.addField(INDEX_RECORD_ID, recordId); + } else if (TARGETFIELDS.equals(localName)) { + parseTargetFields(indexDocument, parser); + } else if (resultName.equals(localName)) { + copyResult(indexDocument, results, parser, nsList, resultName); + } + } + } - if (version != null) { - indexDocument.addField(DS_VERSION, version); - } + if (version != null) { + indexDocument.addField(DS_VERSION, version); + } - if (dsId != null) { - indexDocument.addField(DS_ID, dsId); - } + if (dsId != null) { + indexDocument.addField(DS_ID, dsId); + } - if (!indexDocument.containsKey(INDEX_RECORD_ID)) { - indexDocument.clear(); - System.err.println("missing indexrecord id:\n" + inputDocument); - } + if (!indexDocument.containsKey(INDEX_RECORD_ID)) { + indexDocument.clear(); + System.err.println("missing indexrecord id:\n" + inputDocument); + } - return indexDocument; - } catch (XMLStreamException e) { - return new SolrInputDocument(); - } - } + return indexDocument; + } catch (XMLStreamException e) { + return new SolrInputDocument(); + } + } - private List getNamespaces(final XMLEvent event) { - final List res = Lists.newLinkedList(); - @SuppressWarnings("unchecked") - Iterator nsIter = event.asStartElement().getNamespaces(); - while (nsIter.hasNext()) { - Namespace ns = nsIter.next(); - res.add(ns); - } - return res; - } + private List getNamespaces(final XMLEvent event) { + final List res = Lists.newLinkedList(); + @SuppressWarnings("unchecked") + Iterator nsIter = event.asStartElement().getNamespaces(); + while (nsIter.hasNext()) { + Namespace ns = nsIter.next(); + res.add(ns); + } + return res; + } - /** - * Parse the targetFields block and add fields to the solr document. - * - * @param indexDocument - * @param parser - * @throws XMLStreamException - */ - protected void parseTargetFields( - final SolrInputDocument indexDocument, final XMLEventReader parser) - throws XMLStreamException { + /** + * Parse the targetFields block and add fields to the solr document. + * + * @param indexDocument + * @param parser + * @throws XMLStreamException + */ + protected void parseTargetFields( + final SolrInputDocument indexDocument, final XMLEventReader parser) + throws XMLStreamException { - boolean hasFields = false; + boolean hasFields = false; - while (parser.hasNext()) { - final XMLEvent targetEvent = parser.nextEvent(); - if (targetEvent.isEndElement() - && targetEvent.asEndElement().getName().getLocalPart().equals(TARGETFIELDS)) { - break; - } + while (parser.hasNext()) { + final XMLEvent targetEvent = parser.nextEvent(); + if (targetEvent.isEndElement() + && targetEvent.asEndElement().getName().getLocalPart().equals(TARGETFIELDS)) { + break; + } - if (targetEvent.isStartElement()) { - final String fieldName = targetEvent.asStartElement().getName().getLocalPart(); - final XMLEvent text = parser.nextEvent(); + if (targetEvent.isStartElement()) { + final String fieldName = targetEvent.asStartElement().getName().getLocalPart(); + final XMLEvent text = parser.nextEvent(); - String data = getText(text); + String data = getText(text); - addField(indexDocument, fieldName, data); - hasFields = true; - } - } + addField(indexDocument, fieldName, data); + hasFields = true; + } + } - if (!hasFields) { - indexDocument.clear(); - } - } + if (!hasFields) { + indexDocument.clear(); + } + } - /** - * Copy the /indexRecord/result element and children, preserving namespace declarations etc. - * - * @param indexDocument - * @param results - * @param parser - * @param nsList - * @throws XMLStreamException - */ - protected void copyResult( - final SolrInputDocument indexDocument, - final StringWriter results, - final XMLEventReader parser, - final List nsList, - final String dnetResult) - throws XMLStreamException { - final XMLEventWriter writer = outputFactory.get().createXMLEventWriter(results); + /** + * Copy the /indexRecord/result element and children, preserving namespace declarations etc. + * + * @param indexDocument + * @param results + * @param parser + * @param nsList + * @throws XMLStreamException + */ + protected void copyResult( + final SolrInputDocument indexDocument, + final StringWriter results, + final XMLEventReader parser, + final List nsList, + final String dnetResult) + throws XMLStreamException { + final XMLEventWriter writer = outputFactory.get().createXMLEventWriter(results); - for (Namespace ns : nsList) { - eventFactory.get().createNamespace(ns.getPrefix(), ns.getNamespaceURI()); - } + for (Namespace ns : nsList) { + eventFactory.get().createNamespace(ns.getPrefix(), ns.getNamespaceURI()); + } - StartElement newRecord = - eventFactory.get().createStartElement("", null, RESULT, null, nsList.iterator()); + StartElement newRecord = eventFactory.get().createStartElement("", null, RESULT, null, nsList.iterator()); - // new root record - writer.add(newRecord); + // new root record + writer.add(newRecord); - // copy the rest as it is - while (parser.hasNext()) { - final XMLEvent resultEvent = parser.nextEvent(); + // copy the rest as it is + while (parser.hasNext()) { + final XMLEvent resultEvent = parser.nextEvent(); - // TODO: replace with depth tracking instead of close tag tracking. - if (resultEvent.isEndElement() - && resultEvent.asEndElement().getName().getLocalPart().equals(dnetResult)) { - writer.add(eventFactory.get().createEndElement("", null, RESULT)); - break; - } + // TODO: replace with depth tracking instead of close tag tracking. + if (resultEvent.isEndElement() + && resultEvent.asEndElement().getName().getLocalPart().equals(dnetResult)) { + writer.add(eventFactory.get().createEndElement("", null, RESULT)); + break; + } - writer.add(resultEvent); - } - writer.close(); - indexDocument.addField(INDEX_RESULT, results.toString()); - } + writer.add(resultEvent); + } + writer.close(); + indexDocument.addField(INDEX_RESULT, results.toString()); + } - /** - * Helper used to add a field to a solr doc. It avoids to add empy fields - * - * @param indexDocument - * @param field - * @param value - */ - private final void addField( - final SolrInputDocument indexDocument, final String field, final String value) { - String cleaned = value.trim(); - if (!cleaned.isEmpty()) { - // log.info("\n\n adding field " + field.toLowerCase() + " value: " + cleaned + "\n"); - indexDocument.addField(field.toLowerCase(), cleaned); - } - } + /** + * Helper used to add a field to a solr doc. It avoids to add empy fields + * + * @param indexDocument + * @param field + * @param value + */ + private final void addField( + final SolrInputDocument indexDocument, final String field, final String value) { + String cleaned = value.trim(); + if (!cleaned.isEmpty()) { + // log.info("\n\n adding field " + field.toLowerCase() + " value: " + cleaned + "\n"); + indexDocument.addField(field.toLowerCase(), cleaned); + } + } - /** - * Helper used to get the string from a text element. - * - * @param text - * @return the - */ - protected final String getText(final XMLEvent text) { - if (text.isEndElement()) // log.warn("skipping because isEndOfElement " + - // text.asEndElement().getName().getLocalPart()); - return ""; + /** + * Helper used to get the string from a text element. + * + * @param text + * @return the + */ + protected final String getText(final XMLEvent text) { + if (text.isEndElement()) // log.warn("skipping because isEndOfElement " + + // text.asEndElement().getName().getLocalPart()); + return ""; - final String data = text.asCharacters().getData(); - if (data != null && data.length() > MAX_FIELD_LENGTH) { - return data.substring(0, MAX_FIELD_LENGTH); - } + final String data = text.asCharacters().getData(); + if (data != null && data.length() > MAX_FIELD_LENGTH) { + return data.substring(0, MAX_FIELD_LENGTH); + } - return data; - } + return data; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateFactory.java index 7c919d952..3d9cf1ae7 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateFactory.java @@ -1,113 +1,117 @@ + package eu.dnetlib.dhp.oa.provision.utils; import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.removePrefix; import static eu.dnetlib.dhp.oa.provision.utils.XmlSerializationUtils.escapeXml; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.OafEntity; import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; import org.stringtemplate.v4.ST; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.OafEntity; + public class TemplateFactory { - private TemplateResources resources; + private TemplateResources resources; - private static final char DELIMITER = '$'; + private static final char DELIMITER = '$'; - public TemplateFactory() { - try { - resources = new TemplateResources(); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } + public TemplateFactory() { + try { + resources = new TemplateResources(); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } - public String buildBody( - final String type, - final List metadata, - final List rels, - final List children, - final List extraInfo) { - ST body = getTemplate(resources.getEntity()); + public String buildBody( + final String type, + final List metadata, + final List rels, + final List children, + final List extraInfo) { + ST body = getTemplate(resources.getEntity()); - body.add("name", type); - body.add("metadata", metadata); - body.add("rels", rels); - body.add("children", children); - body.add("extrainfo", extraInfo); + body.add("name", type); + body.add("metadata", metadata); + body.add("rels", rels); + body.add("children", children); + body.add("extrainfo", extraInfo); - return body.render(); - } + return body.render(); + } - public String getChild(final String name, final String id, final List metadata) { - return getTemplate(resources.getChild()) - .add("name", name) - .add("hasId", !(id == null)) - .add("id", id != null ? escapeXml(removePrefix(id)) : "") - .add("metadata", metadata) - .render(); - } + public String getChild(final String name, final String id, final List metadata) { + return getTemplate(resources.getChild()) + .add("name", name) + .add("hasId", !(id == null)) + .add("id", id != null ? escapeXml(removePrefix(id)) : "") + .add("metadata", metadata) + .render(); + } - public String buildRecord( - final OafEntity entity, final String schemaLocation, final String body) { - return getTemplate(resources.getRecord()) - .add("id", escapeXml(removePrefix(entity.getId()))) - .add("dateofcollection", entity.getDateofcollection()) - .add("dateoftransformation", entity.getDateoftransformation()) - .add("schemaLocation", schemaLocation) - .add("it", body) - .render(); - } + public String buildRecord( + final OafEntity entity, final String schemaLocation, final String body) { + return getTemplate(resources.getRecord()) + .add("id", escapeXml(removePrefix(entity.getId()))) + .add("dateofcollection", entity.getDateofcollection()) + .add("dateoftransformation", entity.getDateoftransformation()) + .add("schemaLocation", schemaLocation) + .add("it", body) + .render(); + } - public String getRel( - final String type, - final String objIdentifier, - final Collection fields, - final String semanticclass, - final String semantischeme, - final DataInfo info) { - return getTemplate(resources.getRel()) - .add("type", type) - .add("objIdentifier", escapeXml(removePrefix(objIdentifier))) - .add("class", semanticclass) - .add("scheme", semantischeme) - .add("metadata", fields) - .add("inferred", info.getInferred()) - .add("trust", info.getTrust()) - .add("inferenceprovenance", info.getInferenceprovenance()) - .add( - "provenanceaction", - info.getProvenanceaction() != null ? info.getProvenanceaction().getClassid() : "") - .render(); - } + public String getRel( + final String type, + final String objIdentifier, + final Collection fields, + final String semanticclass, + final String semantischeme, + final DataInfo info) { + return getTemplate(resources.getRel()) + .add("type", type) + .add("objIdentifier", escapeXml(removePrefix(objIdentifier))) + .add("class", semanticclass) + .add("scheme", semantischeme) + .add("metadata", fields) + .add("inferred", info.getInferred()) + .add("trust", info.getTrust()) + .add("inferenceprovenance", info.getInferenceprovenance()) + .add( + "provenanceaction", + info.getProvenanceaction() != null ? info.getProvenanceaction().getClassid() : "") + .render(); + } - public String getInstance( - final String resultId, final List instancemetadata, final List webresources) { - return getTemplate(resources.getInstance()) - .add("instanceId", escapeXml(removePrefix(resultId))) - .add("metadata", instancemetadata) - .add( - "webresources", - webresources.stream() - .filter(StringUtils::isNotBlank) - .map(w -> getWebResource(w)) - .collect(Collectors.toList())) - .render(); - } + public String getInstance( + final String resultId, final List instancemetadata, final List webresources) { + return getTemplate(resources.getInstance()) + .add("instanceId", escapeXml(removePrefix(resultId))) + .add("metadata", instancemetadata) + .add( + "webresources", + webresources + .stream() + .filter(StringUtils::isNotBlank) + .map(w -> getWebResource(w)) + .collect(Collectors.toList())) + .render(); + } - private String getWebResource(final String identifier) { - return getTemplate(resources.getWebresource()) - .add("identifier", escapeXml(identifier)) - .render(); - } + private String getWebResource(final String identifier) { + return getTemplate(resources.getWebresource()) + .add("identifier", escapeXml(identifier)) + .render(); + } - // HELPERS + // HELPERS - private ST getTemplate(final String res) { - return new ST(res, DELIMITER, DELIMITER); - } + private ST getTemplate(final String res) { + return new ST(res, DELIMITER, DELIMITER); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateResources.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateResources.java index 3ffc33bd8..746f8ebe6 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateResources.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/TemplateResources.java @@ -1,50 +1,53 @@ + package eu.dnetlib.dhp.oa.provision.utils; -import com.google.common.io.Resources; import java.io.IOException; import java.nio.charset.StandardCharsets; +import com.google.common.io.Resources; + public class TemplateResources { - private String record = read("eu/dnetlib/dhp/oa/provision/template/record.st"); + private String record = read("eu/dnetlib/dhp/oa/provision/template/record.st"); - private String instance = read("eu/dnetlib/dhp/oa/provision/template/instance.st"); + private String instance = read("eu/dnetlib/dhp/oa/provision/template/instance.st"); - private String rel = read("eu/dnetlib/dhp/oa/provision/template/rel.st"); + private String rel = read("eu/dnetlib/dhp/oa/provision/template/rel.st"); - private String webresource = read("eu/dnetlib/dhp/oa/provision/template/webresource.st"); + private String webresource = read("eu/dnetlib/dhp/oa/provision/template/webresource.st"); - private String child = read("eu/dnetlib/dhp/oa/provision/template/child.st"); + private String child = read("eu/dnetlib/dhp/oa/provision/template/child.st"); - private String entity = read("eu/dnetlib/dhp/oa/provision/template/entity.st"); + private String entity = read("eu/dnetlib/dhp/oa/provision/template/entity.st"); - private static String read(final String classpathResource) throws IOException { - return Resources.toString(Resources.getResource(classpathResource), StandardCharsets.UTF_8); - } + private static String read(final String classpathResource) throws IOException { + return Resources.toString(Resources.getResource(classpathResource), StandardCharsets.UTF_8); + } - public TemplateResources() throws IOException {} + public TemplateResources() throws IOException { + } - public String getEntity() { - return entity; - } + public String getEntity() { + return entity; + } - public String getRecord() { - return record; - } + public String getRecord() { + return record; + } - public String getInstance() { - return instance; - } + public String getInstance() { + return instance; + } - public String getRel() { - return rel; - } + public String getRel() { + return rel; + } - public String getWebresource() { - return webresource; - } + public String getWebresource() { + return webresource; + } - public String getChild() { - return child; - } + public String getChild() { + return child; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 52a509409..f667d9f3c 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -1,23 +1,10 @@ + package eu.dnetlib.dhp.oa.provision.utils; import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.*; import static org.apache.commons.lang3.StringUtils.isNotBlank; import static org.apache.commons.lang3.StringUtils.substringBefore; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import com.mycila.xmltool.XMLDoc; -import com.mycila.xmltool.XMLTag; -import eu.dnetlib.dhp.oa.provision.model.*; -import eu.dnetlib.dhp.schema.common.EntityType; -import eu.dnetlib.dhp.schema.common.MainEntityType; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.Result; import java.io.IOException; import java.io.Serializable; import java.io.StringReader; @@ -27,9 +14,11 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; + import javax.xml.transform.*; import javax.xml.transform.dom.DOMSource; import javax.xml.transform.stream.StreamResult; + import org.apache.commons.lang3.StringUtils; import org.apache.spark.util.LongAccumulator; import org.dom4j.Document; @@ -40,1130 +29,1347 @@ import org.dom4j.io.OutputFormat; import org.dom4j.io.SAXReader; import org.dom4j.io.XMLWriter; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.mycila.xmltool.XMLDoc; +import com.mycila.xmltool.XMLTag; + +import eu.dnetlib.dhp.oa.provision.model.*; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.MainEntityType; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.Result; + public class XmlRecordFactory implements Serializable { - public static final String REL_SUBTYPE_DEDUP = "dedup"; - private Map accumulators; - - private Set specialDatasourceTypes; - - private ContextMapper contextMapper; - - private String schemaLocation; - - private boolean indent = false; - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public XmlRecordFactory( - final ContextMapper contextMapper, - final boolean indent, - final String schemaLocation, - final String otherDatasourceTypesUForUI) { - - this(Maps.newHashMap(), contextMapper, indent, schemaLocation, otherDatasourceTypesUForUI); - } - - public XmlRecordFactory( - final Map accumulators, - final ContextMapper contextMapper, - final boolean indent, - final String schemaLocation, - final String otherDatasourceTypesUForUI) { - - this.accumulators = accumulators; - this.contextMapper = contextMapper; - this.schemaLocation = schemaLocation; - this.specialDatasourceTypes = - Sets.newHashSet(Splitter.on(",").trimResults().split(otherDatasourceTypesUForUI)); - - this.indent = indent; - } - - public String build(final JoinedEntity je) { - - final Set contexts = Sets.newHashSet(); - - final OafEntity entity = toOafEntity(je.getEntity()); - TemplateFactory templateFactory = new TemplateFactory(); - try { - final EntityType type = EntityType.valueOf(je.getEntity().getType()); - final List metadata = metadata(type, entity, contexts); - - // rels has to be processed before the contexts because they enrich the contextMap with - // the - // funding info. - final List relations = - je.getLinks().stream() - .filter(t -> !REL_SUBTYPE_DEDUP.equalsIgnoreCase(t.getRelation().getSubRelType())) - .map(link -> mapRelation(link, templateFactory, contexts)) - .collect(Collectors.toCollection(ArrayList::new)); - - final String mainType = ModelSupport.getMainType(type); - metadata.addAll(buildContexts(mainType, contexts)); - metadata.add(XmlSerializationUtils.parseDataInfo(entity.getDataInfo())); - - final String body = - templateFactory.buildBody( - mainType, - metadata, - relations, - listChildren(entity, je, templateFactory), - listExtraInfo(entity)); - - return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); - } catch (final Throwable e) { - throw new RuntimeException(String.format("error building record '%s'", entity.getId()), e); - } - } - - private static OafEntity toOafEntity(TypedRow typedRow) { - return parseOaf(typedRow.getOaf(), typedRow.getType()); - } - - private static OafEntity parseOaf(final String json, final String type) { - try { - switch (EntityType.valueOf(type)) { - case publication: - return OBJECT_MAPPER.readValue(json, Publication.class); - case dataset: - return OBJECT_MAPPER.readValue(json, Dataset.class); - case otherresearchproduct: - return OBJECT_MAPPER.readValue(json, OtherResearchProduct.class); - case software: - return OBJECT_MAPPER.readValue(json, Software.class); - case datasource: - return OBJECT_MAPPER.readValue(json, Datasource.class); - case organization: - return OBJECT_MAPPER.readValue(json, Organization.class); - case project: - return OBJECT_MAPPER.readValue(json, Project.class); - default: - throw new IllegalArgumentException("invalid type: " + type); - } - } catch (IOException e) { - throw new IllegalArgumentException(e); - } - } - - private String printXML(String xml, boolean indent) { - try { - final Document doc = new SAXReader().read(new StringReader(xml)); - OutputFormat format = - indent ? OutputFormat.createPrettyPrint() : OutputFormat.createCompactFormat(); - format.setExpandEmptyElements(false); - format.setSuppressDeclaration(true); - StringWriter sw = new StringWriter(); - XMLWriter writer = new XMLWriter(sw, format); - writer.write(doc); - return sw.toString(); - } catch (IOException | DocumentException e) { - throw new IllegalArgumentException("Unable to indent XML. Invalid record:\n" + xml, e); - } - } - - private List metadata( - final EntityType type, final OafEntity entity, final Set contexts) { - - final List metadata = Lists.newArrayList(); - - if (entity.getCollectedfrom() != null) { - metadata.addAll( - entity.getCollectedfrom().stream() - .map(kv -> XmlSerializationUtils.mapKeyValue("collectedfrom", kv)) - .collect(Collectors.toList())); - } - if (entity.getOriginalId() != null) { - metadata.addAll( - entity.getOriginalId().stream() - .map(s -> XmlSerializationUtils.asXmlElement("originalId", s)) - .collect(Collectors.toList())); - } - if (entity.getPid() != null) { - metadata.addAll( - entity.getPid().stream() - .map(p -> XmlSerializationUtils.mapStructuredProperty("pid", p)) - .collect(Collectors.toList())); - } - - if (ModelSupport.isResult(type)) { - final Result r = (Result) entity; - - if (r.getContext() != null) { - contexts.addAll(r.getContext().stream().map(c -> c.getId()).collect(Collectors.toList())); - /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ - if (contexts.contains("dh-ch::subcommunity::2")) { - contexts.add("clarin"); - } - } - - if (r.getTitle() != null) { - metadata.addAll( - r.getTitle().stream() - .map(t -> XmlSerializationUtils.mapStructuredProperty("title", t)) - .collect(Collectors.toList())); - } - if (r.getBestaccessright() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("bestaccessright", r.getBestaccessright())); - } - if (r.getAuthor() != null) { - metadata.addAll( - r.getAuthor().stream() - .map( - a -> { - final StringBuilder sb = - new StringBuilder(" - isNotBlank(sp.getQualifier().getClassid()) - && isNotBlank(sp.getValue())) - .forEach( - sp -> { - String pidType = - XmlSerializationUtils.escapeXml( - sp.getQualifier().getClassid()) - .replaceAll("\\W", ""); - String pidValue = XmlSerializationUtils.escapeXml(sp.getValue()); - - // ugly hack: some records - // provide swapped pidtype and - // pidvalue - if (authorPidTypes.contains(pidValue.toLowerCase().trim())) { - sb.append(String.format(" %s=\"%s\"", pidValue, pidType)); - } else { - pidType = pidType.replaceAll("\\W", "").replaceAll("\\d", ""); - if (isNotBlank(pidType)) { - sb.append( - String.format( - " %s=\"%s\"", - pidType, - pidValue.toLowerCase().replaceAll("orcid", ""))); - } - } - }); - } - sb.append( - ">" + XmlSerializationUtils.escapeXml(a.getFullname()) + ""); - return sb.toString(); - }) - .collect(Collectors.toList())); - } - if (r.getContributor() != null) { - metadata.addAll( - r.getContributor().stream() - .map(c -> XmlSerializationUtils.asXmlElement("contributor", c.getValue())) - .collect(Collectors.toList())); - } - if (r.getCountry() != null) { - metadata.addAll( - r.getCountry().stream() - .map(c -> XmlSerializationUtils.mapQualifier("country", c)) - .collect(Collectors.toList())); - } - if (r.getCoverage() != null) { - metadata.addAll( - r.getCoverage().stream() - .map(c -> XmlSerializationUtils.asXmlElement("coverage", c.getValue())) - .collect(Collectors.toList())); - } - if (r.getDateofacceptance() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "dateofacceptance", r.getDateofacceptance().getValue())); - } - if (r.getDescription() != null) { - metadata.addAll( - r.getDescription().stream() - .map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue())) - .collect(Collectors.toList())); - } - if (r.getEmbargoenddate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("embargoenddate", r.getEmbargoenddate().getValue())); - } - if (r.getSubject() != null) { - metadata.addAll( - r.getSubject().stream() - .map(s -> XmlSerializationUtils.mapStructuredProperty("subject", s)) - .collect(Collectors.toList())); - } - if (r.getLanguage() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("language", r.getLanguage())); - } - if (r.getRelevantdate() != null) { - metadata.addAll( - r.getRelevantdate().stream() - .map(s -> XmlSerializationUtils.mapStructuredProperty("relevantdate", s)) - .collect(Collectors.toList())); - } - if (r.getPublisher() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("publisher", r.getPublisher().getValue())); - } - if (r.getSource() != null) { - metadata.addAll( - r.getSource().stream() - .map(c -> XmlSerializationUtils.asXmlElement("source", c.getValue())) - .collect(Collectors.toList())); - } - if (r.getFormat() != null) { - metadata.addAll( - r.getFormat().stream() - .map(c -> XmlSerializationUtils.asXmlElement("format", c.getValue())) - .collect(Collectors.toList())); - } - if (r.getResulttype() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("resulttype", r.getResulttype())); - } - if (r.getResourcetype() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("resourcetype", r.getResourcetype())); - } - } - - switch (type) { - case publication: - final Publication pub = (Publication) entity; - - if (pub.getJournal() != null) { - final Journal j = pub.getJournal(); - metadata.add(XmlSerializationUtils.mapJournal(j)); - } - - break; - case dataset: - final Dataset d = (Dataset) entity; - if (d.getDevice() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("device", d.getDevice().getValue())); - } - if (d.getLastmetadataupdate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "lastmetadataupdate", d.getLastmetadataupdate().getValue())); - } - if (d.getMetadataversionnumber() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "metadataversionnumber", d.getMetadataversionnumber().getValue())); - } - if (d.getSize() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("size", d.getSize().getValue())); - } - if (d.getStoragedate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("storagedate", d.getStoragedate().getValue())); - } - if (d.getVersion() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("version", d.getVersion().getValue())); - } - // TODO d.getGeolocation() - - break; - case otherresearchproduct: - final OtherResearchProduct orp = (OtherResearchProduct) entity; - - if (orp.getContactperson() != null) { - metadata.addAll( - orp.getContactperson().stream() - .map(c -> XmlSerializationUtils.asXmlElement("contactperson", c.getValue())) - .collect(Collectors.toList())); - } - - if (orp.getContactgroup() != null) { - metadata.addAll( - orp.getContactgroup().stream() - .map(c -> XmlSerializationUtils.asXmlElement("contactgroup", c.getValue())) - .collect(Collectors.toList())); - } - if (orp.getTool() != null) { - metadata.addAll( - orp.getTool().stream() - .map(c -> XmlSerializationUtils.asXmlElement("tool", c.getValue())) - .collect(Collectors.toList())); - } - break; - case software: - final Software s = (Software) entity; - - if (s.getDocumentationUrl() != null) { - metadata.addAll( - s.getDocumentationUrl().stream() - .map(c -> XmlSerializationUtils.asXmlElement("documentationUrl", c.getValue())) - .collect(Collectors.toList())); - } - if (s.getLicense() != null) { - metadata.addAll( - s.getLicense().stream() - .map(l -> XmlSerializationUtils.mapStructuredProperty("license", l)) - .collect(Collectors.toList())); - } - if (s.getCodeRepositoryUrl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "codeRepositoryUrl", s.getCodeRepositoryUrl().getValue())); - } - if (s.getProgrammingLanguage() != null) { - metadata.add( - XmlSerializationUtils.mapQualifier( - "programmingLanguage", s.getProgrammingLanguage())); - } - break; - case datasource: - final Datasource ds = (Datasource) entity; - - if (ds.getDatasourcetype() != null) { - mapDatasourceType(metadata, ds.getDatasourcetype()); - } - if (ds.getOpenairecompatibility() != null) { - metadata.add( - XmlSerializationUtils.mapQualifier( - "openairecompatibility", ds.getOpenairecompatibility())); - } - if (ds.getOfficialname() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("officialname", ds.getOfficialname().getValue())); - } - if (ds.getEnglishname() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("englishname", ds.getEnglishname().getValue())); - } - if (ds.getWebsiteurl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("websiteurl", ds.getWebsiteurl().getValue())); - } - if (ds.getLogourl() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("logourl", ds.getLogourl().getValue())); - } - if (ds.getContactemail() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("contactemail", ds.getContactemail().getValue())); - } - if (ds.getNamespaceprefix() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "namespaceprefix", ds.getNamespaceprefix().getValue())); - } - if (ds.getLatitude() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("latitude", ds.getLatitude().getValue())); - } - if (ds.getLongitude() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("longitude", ds.getLongitude().getValue())); - } - if (ds.getDateofvalidation() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "dateofvalidation", ds.getDateofvalidation().getValue())); - } - if (ds.getDescription() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("description", ds.getDescription().getValue())); - } - if (ds.getOdnumberofitems() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "odnumberofitems", ds.getOdnumberofitems().getValue())); - } - if (ds.getOdnumberofitemsdate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "odnumberofitemsdate", ds.getOdnumberofitemsdate().getValue())); - } - if (ds.getOdpolicies() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("odpolicies", ds.getOdpolicies().getValue())); - } - if (ds.getOdlanguages() != null) { - metadata.addAll( - ds.getOdlanguages().stream() - .map(c -> XmlSerializationUtils.asXmlElement("odlanguages", c.getValue())) - .collect(Collectors.toList())); - } - if (ds.getOdcontenttypes() != null) { - metadata.addAll( - ds.getOdcontenttypes().stream() - .map(c -> XmlSerializationUtils.asXmlElement("odcontenttypes", c.getValue())) - .collect(Collectors.toList())); - } - if (ds.getAccessinfopackage() != null) { - metadata.addAll( - ds.getAccessinfopackage().stream() - .map(c -> XmlSerializationUtils.asXmlElement("accessinfopackage", c.getValue())) - .collect(Collectors.toList())); - } - if (ds.getReleaseenddate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "releasestartdate", ds.getReleaseenddate().getValue())); - } - if (ds.getReleaseenddate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "releaseenddate", ds.getReleaseenddate().getValue())); - } - if (ds.getMissionstatementurl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "missionstatementurl", ds.getMissionstatementurl().getValue())); - } - if (ds.getDataprovider() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "dataprovider", ds.getDataprovider().getValue().toString())); - } - if (ds.getServiceprovider() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "serviceprovider", ds.getServiceprovider().getValue().toString())); - } - if (ds.getDatabaseaccesstype() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "databaseaccesstype", ds.getDatabaseaccesstype().getValue())); - } - if (ds.getDatauploadtype() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "datauploadtype", ds.getDatauploadtype().getValue())); - } - if (ds.getDatabaseaccessrestriction() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "databaseaccessrestriction", ds.getDatabaseaccessrestriction().getValue())); - } - if (ds.getDatauploadrestriction() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "datauploadrestriction", ds.getDatauploadrestriction().getValue())); - } - if (ds.getVersioning() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "versioning", ds.getVersioning().getValue().toString())); - } - if (ds.getCitationguidelineurl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "citationguidelineurl", ds.getCitationguidelineurl().getValue())); - } - if (ds.getQualitymanagementkind() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "qualitymanagementkind", ds.getQualitymanagementkind().getValue())); - } - if (ds.getPidsystems() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("pidsystems", ds.getPidsystems().getValue())); - } - if (ds.getCertificates() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("certificates", ds.getCertificates().getValue())); - } - if (ds.getPolicies() != null) { - metadata.addAll( - ds.getPolicies().stream() - .map(kv -> XmlSerializationUtils.mapKeyValue("policies", kv)) - .collect(Collectors.toList())); - } - if (ds.getJournal() != null) { - metadata.add(XmlSerializationUtils.mapJournal(ds.getJournal())); - } - if (ds.getSubjects() != null) { - metadata.addAll( - ds.getSubjects().stream() - .map(sp -> XmlSerializationUtils.mapStructuredProperty("subjects", sp)) - .collect(Collectors.toList())); - } - - break; - case organization: - final Organization o = (Organization) entity; - - if (o.getLegalshortname() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "legalshortname", o.getLegalshortname().getValue())); - } - if (o.getLegalname() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("legalname", o.getLegalname().getValue())); - } - if (o.getAlternativeNames() != null) { - metadata.addAll( - o.getAlternativeNames().stream() - .map(c -> XmlSerializationUtils.asXmlElement("alternativeNames", c.getValue())) - .collect(Collectors.toList())); - } - if (o.getWebsiteurl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("websiteurl", o.getWebsiteurl().getValue())); - } - if (o.getLogourl() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("websiteurl", o.getLogourl().getValue())); - } - - if (o.getEclegalbody() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("eclegalbody", o.getEclegalbody().getValue())); - } - if (o.getEclegalperson() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("eclegalperson", o.getEclegalperson().getValue())); - } - if (o.getEcnonprofit() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("ecnonprofit", o.getEcnonprofit().getValue())); - } - if (o.getEcresearchorganization() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "ecresearchorganization", o.getEcresearchorganization().getValue())); - } - if (o.getEchighereducation() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "echighereducation", o.getEchighereducation().getValue())); - } - if (o.getEcinternationalorganization() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "ecinternationalorganizationeurinterests", - o.getEcinternationalorganization().getValue())); - } - if (o.getEcinternationalorganization() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "ecinternationalorganization", o.getEcinternationalorganization().getValue())); - } - if (o.getEcenterprise() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("ecenterprise", o.getEcenterprise().getValue())); - } - if (o.getEcsmevalidated() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "ecsmevalidated", o.getEcsmevalidated().getValue())); - } - if (o.getEcnutscode() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("ecnutscode", o.getEcnutscode().getValue())); - } - if (o.getCountry() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("country", o.getCountry())); - } - - break; - case project: - final Project p = (Project) entity; - - if (p.getWebsiteurl() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("websiteurl", p.getWebsiteurl().getValue())); - } - if (p.getCode() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("code", p.getCode().getValue())); - } - if (p.getAcronym() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("acronym", p.getAcronym().getValue())); - } - if (p.getTitle() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("title", p.getTitle().getValue())); - } - if (p.getStartdate() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("startdate", p.getStartdate().getValue())); - } - if (p.getEnddate() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("enddate", p.getEnddate().getValue())); - } - if (p.getCallidentifier() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "callidentifier", p.getCallidentifier().getValue())); - } - if (p.getKeywords() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("keywords", p.getKeywords().getValue())); - } - if (p.getDuration() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("duration", p.getDuration().getValue())); - } - if (p.getEcarticle29_3() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("ecarticle29_3", p.getEcarticle29_3().getValue())); - } - if (p.getSubjects() != null) { - metadata.addAll( - p.getSubjects().stream() - .map(sp -> XmlSerializationUtils.mapStructuredProperty("subject", sp)) - .collect(Collectors.toList())); - } - if (p.getContracttype() != null) { - metadata.add(XmlSerializationUtils.mapQualifier("contracttype", p.getContracttype())); - } - if (p.getEcsc39() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("ecsc39", p.getEcsc39().getValue())); - } - if (p.getContactfullname() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement( - "contactfullname", p.getContactfullname().getValue())); - } - if (p.getContactfax() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("contactfax", p.getContactfax().getValue())); - } - if (p.getContactphone() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("contactphone", p.getContactphone().getValue())); - } - if (p.getContactemail() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("contactemail", p.getContactemail().getValue())); - } - if (p.getSummary() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("summary", p.getSummary().getValue())); - } - if (p.getCurrency() != null) { - metadata.add(XmlSerializationUtils.asXmlElement("currency", p.getCurrency().getValue())); - } - if (p.getTotalcost() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("totalcost", p.getTotalcost().toString())); - } - if (p.getFundedamount() != null) { - metadata.add( - XmlSerializationUtils.asXmlElement("fundedamount", p.getFundedamount().toString())); - } - if (p.getFundingtree() != null) { - metadata.addAll( - p.getFundingtree().stream().map(ft -> ft.getValue()).collect(Collectors.toList())); - } - - break; - default: - throw new IllegalArgumentException("invalid entity type: " + type); - } - - return metadata; - } - - private void mapDatasourceType(List metadata, final Qualifier dsType) { - metadata.add(XmlSerializationUtils.mapQualifier("datasourcetype", dsType)); - - if (specialDatasourceTypes.contains(dsType.getClassid())) { - dsType.setClassid("other"); - dsType.setClassname("other"); - } - metadata.add(XmlSerializationUtils.mapQualifier("datasourcetypeui", dsType)); - } - - private String mapRelation(Tuple2 link, TemplateFactory templateFactory, Set contexts) { - final Relation rel = link.getRelation(); - final RelatedEntity re = link.getRelatedEntity(); - final String targetType = link.getRelatedEntity().getType(); - - final List metadata = Lists.newArrayList(); - switch (EntityType.valueOf(targetType)) { - case publication: - case dataset: - case otherresearchproduct: - case software: - if (re.getTitle() != null && isNotBlank(re.getTitle().getValue())) { - metadata.add(XmlSerializationUtils.mapStructuredProperty("title", re.getTitle())); - } - if (isNotBlank(re.getDateofacceptance())) { - metadata.add( - XmlSerializationUtils.asXmlElement("dateofacceptance", re.getDateofacceptance())); - } - if (isNotBlank(re.getPublisher())) { - metadata.add(XmlSerializationUtils.asXmlElement("publisher", re.getPublisher())); - } - if (isNotBlank(re.getCodeRepositoryUrl())) { - metadata.add( - XmlSerializationUtils.asXmlElement("coderepositoryurl", re.getCodeRepositoryUrl())); - } - if (re.getResulttype() != null & re.getResulttype().isBlank()) { - metadata.add(XmlSerializationUtils.mapQualifier("resulttype", re.getResulttype())); - } - if (re.getCollectedfrom() != null) { - metadata.addAll( - re.getCollectedfrom().stream() - .map(kv -> XmlSerializationUtils.mapKeyValue("collectedfrom", kv)) - .collect(Collectors.toList())); - } - if (re.getPid() != null) { - metadata.addAll( - re.getPid().stream() - .map(p -> XmlSerializationUtils.mapStructuredProperty("pid", p)) - .collect(Collectors.toList())); - } - break; - case datasource: - if (isNotBlank(re.getOfficialname())) { - metadata.add(XmlSerializationUtils.asXmlElement("officialname", re.getOfficialname())); - } - if (re.getDatasourcetype() != null & !re.getDatasourcetype().isBlank()) { - mapDatasourceType(metadata, re.getDatasourcetype()); - } - if (re.getOpenairecompatibility() != null & !re.getOpenairecompatibility().isBlank()) { - metadata.add( - XmlSerializationUtils.mapQualifier( - "openairecompatibility", re.getOpenairecompatibility())); - } - break; - case organization: - if (isNotBlank(re.getLegalname())) { - metadata.add(XmlSerializationUtils.asXmlElement("legalname", re.getLegalname())); - } - if (isNotBlank(re.getLegalshortname())) { - metadata.add( - XmlSerializationUtils.asXmlElement("legalshortname", re.getLegalshortname())); - } - if (re.getCountry() != null & !re.getCountry().isBlank()) { - metadata.add(XmlSerializationUtils.mapQualifier("country", re.getCountry())); - } - break; - case project: - if (isNotBlank(re.getProjectTitle())) { - metadata.add(XmlSerializationUtils.asXmlElement("title", re.getProjectTitle())); - } - if (isNotBlank(re.getCode())) { - metadata.add(XmlSerializationUtils.asXmlElement("code", re.getCode())); - } - if (isNotBlank(re.getAcronym())) { - metadata.add(XmlSerializationUtils.asXmlElement("acronym", re.getAcronym())); - } - if (re.getContracttype() != null & !re.getContracttype().isBlank()) { - metadata.add(XmlSerializationUtils.mapQualifier("contracttype", re.getContracttype())); - } - if (re.getFundingtree() != null & contexts != null) { - metadata.addAll( - re.getFundingtree().stream() - .peek(ft -> fillContextMap(ft, contexts)) - .map(ft -> getRelFundingTree(ft)) - .collect(Collectors.toList())); - } - break; - default: - throw new IllegalArgumentException("invalid target type: " + targetType); - } - final DataInfo info = rel.getDataInfo(); - final String scheme = ModelSupport.getScheme(re.getType(), targetType); - - if (StringUtils.isBlank(scheme)) { - throw new IllegalArgumentException( - String.format("missing scheme for: <%s - %s>", re.getType(), targetType)); - } - - final String accumulatorName = - getRelDescriptor(rel.getRelType(), rel.getSubRelType(), rel.getRelClass()); - if (accumulators.containsKey(accumulatorName)) { - accumulators.get(accumulatorName).add(1); - } - - return templateFactory.getRel( - targetType, rel.getTarget(), Sets.newHashSet(metadata), rel.getRelClass(), scheme, info); - } - - private List listChildren( - final OafEntity entity, JoinedEntity je, TemplateFactory templateFactory) { - - final List children = Lists.newArrayList(); - EntityType entityType = EntityType.valueOf(je.getEntity().getType()); - - children.addAll( - je.getLinks().stream() - .filter(link -> REL_SUBTYPE_DEDUP.equalsIgnoreCase(link.getRelation().getSubRelType())) - .map(link -> mapRelation(link, templateFactory, null)) - .collect(Collectors.toCollection(ArrayList::new))); - - if (MainEntityType.result.toString().equals(ModelSupport.getMainType(entityType))) { - final List instances = ((Result) entity).getInstance(); - if (instances != null) { - for (final Instance instance : ((Result) entity).getInstance()) { - - final List fields = Lists.newArrayList(); - - if (instance.getAccessright() != null && !instance.getAccessright().isBlank()) { - fields.add( - XmlSerializationUtils.mapQualifier("accessright", instance.getAccessright())); - } - if (instance.getCollectedfrom() != null) { - fields.add( - XmlSerializationUtils.mapKeyValue("collectedfrom", instance.getCollectedfrom())); - } - if (instance.getHostedby() != null) { - fields.add(XmlSerializationUtils.mapKeyValue("hostedby", instance.getHostedby())); - } - if (instance.getDateofacceptance() != null - && isNotBlank(instance.getDateofacceptance().getValue())) { - fields.add( - XmlSerializationUtils.asXmlElement( - "dateofacceptance", instance.getDateofacceptance().getValue())); - } - if (instance.getInstancetype() != null && !instance.getInstancetype().isBlank()) { - fields.add( - XmlSerializationUtils.mapQualifier("instancetype", instance.getInstancetype())); - } - if (isNotBlank(instance.getDistributionlocation())) { - fields.add( - XmlSerializationUtils.asXmlElement( - "distributionlocation", instance.getDistributionlocation())); - } - if (instance.getRefereed() != null && isNotBlank(instance.getRefereed().getValue())) { - fields.add( - XmlSerializationUtils.asXmlElement("refereed", instance.getRefereed().getValue())); - } - if (instance.getProcessingchargeamount() != null - && isNotBlank(instance.getProcessingchargeamount().getValue())) { - fields.add( - XmlSerializationUtils.asXmlElement( - "processingchargeamount", instance.getProcessingchargeamount().getValue())); - } - if (instance.getProcessingchargecurrency() != null - && isNotBlank(instance.getProcessingchargecurrency().getValue())) { - fields.add( - XmlSerializationUtils.asXmlElement( - "processingchargecurrency", instance.getProcessingchargecurrency().getValue())); - } - - children.add( - templateFactory.getInstance( - instance.getHostedby().getKey(), fields, instance.getUrl())); - } - } - final List ext = ((Result) entity).getExternalReference(); - if (ext != null) { - for (final ExternalReference er : ((Result) entity).getExternalReference()) { - - final List fields = Lists.newArrayList(); - - if (isNotBlank(er.getSitename())) { - fields.add(XmlSerializationUtils.asXmlElement("sitename", er.getSitename())); - } - if (isNotBlank(er.getLabel())) { - fields.add(XmlSerializationUtils.asXmlElement("label", er.getLabel())); - } - if (isNotBlank(er.getUrl())) { - fields.add(XmlSerializationUtils.asXmlElement("url", er.getUrl())); - } - if (isNotBlank(er.getDescription())) { - fields.add(XmlSerializationUtils.asXmlElement("description", er.getDescription())); - } - if (isNotBlank(er.getUrl())) { - fields.add(XmlSerializationUtils.mapQualifier("qualifier", er.getQualifier())); - } - if (isNotBlank(er.getRefidentifier())) { - fields.add(XmlSerializationUtils.asXmlElement("refidentifier", er.getRefidentifier())); - } - if (isNotBlank(er.getQuery())) { - fields.add(XmlSerializationUtils.asXmlElement("query", er.getQuery())); - } - - children.add(templateFactory.getChild("externalreference", null, fields)); - } - } - } - - return children; - } - - private List listExtraInfo(OafEntity entity) { - final List extraInfo = entity.getExtraInfo(); - return extraInfo != null - ? extraInfo.stream() - .map(e -> XmlSerializationUtils.mapExtraInfo(e)) - .collect(Collectors.toList()) - : Lists.newArrayList(); - } - - private List buildContexts(final String type, final Set contexts) { - final List res = Lists.newArrayList(); - - if ((contextMapper != null) - && !contextMapper.isEmpty() - && MainEntityType.result.toString().equals(type)) { - - XMLTag document = XMLDoc.newDocument(true).addRoot("contextRoot"); - - for (final String context : contexts) { - - String id = ""; - for (final String token : Splitter.on("::").split(context)) { - id += token; - - final ContextDef def = contextMapper.get(id); - - if (def == null) { - continue; - // throw new IllegalStateException(String.format("cannot find context for id - // '%s'", - // id)); - } - - if (def.getName().equals("context")) { - final String xpath = "//context/@id='" + def.getId() + "'"; - if (!document.gotoRoot().rawXpathBoolean(xpath, new Object())) { - document = addContextDef(document.gotoRoot(), def); - } - } - - if (def.getName().equals("category")) { - final String rootId = substringBefore(def.getId(), "::"); - document = - addContextDef( - document.gotoRoot().gotoTag("//context[./@id='" + rootId + "']", new Object()), - def); - } - - if (def.getName().equals("concept")) { - document = addContextDef(document, def).gotoParent(); - } - id += "::"; - } - } - final Transformer transformer = getTransformer(); - for (final org.w3c.dom.Element x : document.gotoRoot().getChildElement()) { - try { - res.add(asStringElement(x, transformer)); - } catch (final TransformerException e) { - throw new RuntimeException(e); - } - } - } - - return res; - } - - private Transformer getTransformer() { - try { - Transformer transformer = TransformerFactory.newInstance().newTransformer(); - transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "yes"); - return transformer; - } catch (TransformerConfigurationException e) { - throw new IllegalStateException("unable to create javax.xml.transform.Transformer", e); - } - } - - private XMLTag addContextDef(final XMLTag tag, final ContextDef def) { - tag.addTag(def.getName()).addAttribute("id", def.getId()).addAttribute("label", def.getLabel()); - if ((def.getType() != null) && !def.getType().isEmpty()) { - tag.addAttribute("type", def.getType()); - } - return tag; - } - - private String asStringElement(final org.w3c.dom.Element element, final Transformer transformer) - throws TransformerException { - final StringWriter buffer = new StringWriter(); - transformer.transform(new DOMSource(element), new StreamResult(buffer)); - return buffer.toString(); - } - - private void fillContextMap(final String xmlTree, final Set contexts) { - - Document fundingPath; - try { - fundingPath = new SAXReader().read(new StringReader(xmlTree)); - } catch (final DocumentException e) { - throw new RuntimeException(e); - } - try { - final Node funder = fundingPath.selectSingleNode("//funder"); - - if (funder != null) { - - final String funderShortName = funder.valueOf("./shortname"); - contexts.add(funderShortName); - - contextMapper.put( - funderShortName, - new ContextDef(funderShortName, funder.valueOf("./name"), "context", "funding")); - final Node level0 = fundingPath.selectSingleNode("//funding_level_0"); - if (level0 != null) { - final String level0Id = Joiner.on("::").join(funderShortName, level0.valueOf("./name")); - contextMapper.put( - level0Id, new ContextDef(level0Id, level0.valueOf("./description"), "category", "")); - final Node level1 = fundingPath.selectSingleNode("//funding_level_1"); - if (level1 == null) { - contexts.add(level0Id); - } else { - final String level1Id = Joiner.on("::").join(level0Id, level1.valueOf("./name")); - contextMapper.put( - level1Id, new ContextDef(level1Id, level1.valueOf("./description"), "concept", "")); - final Node level2 = fundingPath.selectSingleNode("//funding_level_2"); - if (level2 == null) { - contexts.add(level1Id); - } else { - final String level2Id = Joiner.on("::").join(level1Id, level2.valueOf("./name")); - contextMapper.put( - level2Id, - new ContextDef(level2Id, level2.valueOf("./description"), "concept", "")); - contexts.add(level2Id); - } - } - } - } - } catch (final NullPointerException e) { - throw new IllegalArgumentException("malformed funding path: " + xmlTree, e); - } - } - - @SuppressWarnings("unchecked") - protected static String getRelFundingTree(final String xmlTree) { - String funding = ""; - try { - final Document ftree = new SAXReader().read(new StringReader(xmlTree)); - funding = ""; - - funding += getFunderElement(ftree); - - for (final Object o : - Lists.reverse( - ftree.selectNodes("//fundingtree//*[starts-with(local-name(),'funding_level_')]"))) { - final Element e = (Element) o; - final String _id = e.valueOf("./id"); - funding += - "<" - + e.getName() - + " name=\"" - + XmlSerializationUtils.escapeXml(e.valueOf("./name")) - + "\">" - + XmlSerializationUtils.escapeXml(_id) - + ""; - } - } catch (final DocumentException e) { - throw new IllegalArgumentException( - "unable to parse funding tree: " + xmlTree + "\n" + e.getMessage()); - } finally { - funding += ""; - } - return funding; - } - - private static String getFunderElement(final Document ftree) { - final String funderId = ftree.valueOf("//fundingtree/funder/id"); - final String funderShortName = ftree.valueOf("//fundingtree/funder/shortname"); - final String funderName = ftree.valueOf("//fundingtree/funder/name"); - final String funderJurisdiction = ftree.valueOf("//fundingtree/funder/jurisdiction"); - - return ""; - } + public static final String REL_SUBTYPE_DEDUP = "dedup"; + private Map accumulators; + + private Set specialDatasourceTypes; + + private ContextMapper contextMapper; + + private String schemaLocation; + + private boolean indent = false; + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public XmlRecordFactory( + final ContextMapper contextMapper, + final boolean indent, + final String schemaLocation, + final String otherDatasourceTypesUForUI) { + + this(Maps.newHashMap(), contextMapper, indent, schemaLocation, otherDatasourceTypesUForUI); + } + + public XmlRecordFactory( + final Map accumulators, + final ContextMapper contextMapper, + final boolean indent, + final String schemaLocation, + final String otherDatasourceTypesUForUI) { + + this.accumulators = accumulators; + this.contextMapper = contextMapper; + this.schemaLocation = schemaLocation; + this.specialDatasourceTypes = Sets.newHashSet(Splitter.on(",").trimResults().split(otherDatasourceTypesUForUI)); + + this.indent = indent; + } + + public String build(final JoinedEntity je) { + + final Set contexts = Sets.newHashSet(); + + final OafEntity entity = toOafEntity(je.getEntity()); + TemplateFactory templateFactory = new TemplateFactory(); + try { + final EntityType type = EntityType.valueOf(je.getEntity().getType()); + final List metadata = metadata(type, entity, contexts); + + // rels has to be processed before the contexts because they enrich the contextMap with + // the + // funding info. + final List relations = je + .getLinks() + .stream() + .filter(t -> !REL_SUBTYPE_DEDUP.equalsIgnoreCase(t.getRelation().getSubRelType())) + .map(link -> mapRelation(link, templateFactory, contexts)) + .collect(Collectors.toCollection(ArrayList::new)); + + final String mainType = ModelSupport.getMainType(type); + metadata.addAll(buildContexts(mainType, contexts)); + metadata.add(XmlSerializationUtils.parseDataInfo(entity.getDataInfo())); + + final String body = templateFactory + .buildBody( + mainType, + metadata, + relations, + listChildren(entity, je, templateFactory), + listExtraInfo(entity)); + + return printXML(templateFactory.buildRecord(entity, schemaLocation, body), indent); + } catch (final Throwable e) { + throw new RuntimeException(String.format("error building record '%s'", entity.getId()), e); + } + } + + private static OafEntity toOafEntity(TypedRow typedRow) { + return parseOaf(typedRow.getOaf(), typedRow.getType()); + } + + private static OafEntity parseOaf(final String json, final String type) { + try { + switch (EntityType.valueOf(type)) { + case publication: + return OBJECT_MAPPER.readValue(json, Publication.class); + case dataset: + return OBJECT_MAPPER.readValue(json, Dataset.class); + case otherresearchproduct: + return OBJECT_MAPPER.readValue(json, OtherResearchProduct.class); + case software: + return OBJECT_MAPPER.readValue(json, Software.class); + case datasource: + return OBJECT_MAPPER.readValue(json, Datasource.class); + case organization: + return OBJECT_MAPPER.readValue(json, Organization.class); + case project: + return OBJECT_MAPPER.readValue(json, Project.class); + default: + throw new IllegalArgumentException("invalid type: " + type); + } + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + private String printXML(String xml, boolean indent) { + try { + final Document doc = new SAXReader().read(new StringReader(xml)); + OutputFormat format = indent ? OutputFormat.createPrettyPrint() : OutputFormat.createCompactFormat(); + format.setExpandEmptyElements(false); + format.setSuppressDeclaration(true); + StringWriter sw = new StringWriter(); + XMLWriter writer = new XMLWriter(sw, format); + writer.write(doc); + return sw.toString(); + } catch (IOException | DocumentException e) { + throw new IllegalArgumentException("Unable to indent XML. Invalid record:\n" + xml, e); + } + } + + private List metadata( + final EntityType type, final OafEntity entity, final Set contexts) { + + final List metadata = Lists.newArrayList(); + + if (entity.getCollectedfrom() != null) { + metadata + .addAll( + entity + .getCollectedfrom() + .stream() + .map(kv -> XmlSerializationUtils.mapKeyValue("collectedfrom", kv)) + .collect(Collectors.toList())); + } + if (entity.getOriginalId() != null) { + metadata + .addAll( + entity + .getOriginalId() + .stream() + .map(s -> XmlSerializationUtils.asXmlElement("originalId", s)) + .collect(Collectors.toList())); + } + if (entity.getPid() != null) { + metadata + .addAll( + entity + .getPid() + .stream() + .map(p -> XmlSerializationUtils.mapStructuredProperty("pid", p)) + .collect(Collectors.toList())); + } + + if (ModelSupport.isResult(type)) { + final Result r = (Result) entity; + + if (r.getContext() != null) { + contexts.addAll(r.getContext().stream().map(c -> c.getId()).collect(Collectors.toList())); + /* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */ + if (contexts.contains("dh-ch::subcommunity::2")) { + contexts.add("clarin"); + } + } + + if (r.getTitle() != null) { + metadata + .addAll( + r + .getTitle() + .stream() + .map(t -> XmlSerializationUtils.mapStructuredProperty("title", t)) + .collect(Collectors.toList())); + } + if (r.getBestaccessright() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("bestaccessright", r.getBestaccessright())); + } + if (r.getAuthor() != null) { + metadata + .addAll( + r + .getAuthor() + .stream() + .map( + a -> { + final StringBuilder sb = new StringBuilder(" isNotBlank(sp.getQualifier().getClassid()) + && isNotBlank(sp.getValue())) + .forEach( + sp -> { + String pidType = XmlSerializationUtils + .escapeXml( + sp.getQualifier().getClassid()) + .replaceAll("\\W", ""); + String pidValue = XmlSerializationUtils.escapeXml(sp.getValue()); + + // ugly hack: some records + // provide swapped pidtype and + // pidvalue + if (authorPidTypes.contains(pidValue.toLowerCase().trim())) { + sb.append(String.format(" %s=\"%s\"", pidValue, pidType)); + } else { + pidType = pidType.replaceAll("\\W", "").replaceAll("\\d", ""); + if (isNotBlank(pidType)) { + sb + .append( + String + .format( + " %s=\"%s\"", + pidType, + pidValue + .toLowerCase() + .replaceAll("orcid", ""))); + } + } + }); + } + sb + .append( + ">" + XmlSerializationUtils.escapeXml(a.getFullname()) + ""); + return sb.toString(); + }) + .collect(Collectors.toList())); + } + if (r.getContributor() != null) { + metadata + .addAll( + r + .getContributor() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("contributor", c.getValue())) + .collect(Collectors.toList())); + } + if (r.getCountry() != null) { + metadata + .addAll( + r + .getCountry() + .stream() + .map(c -> XmlSerializationUtils.mapQualifier("country", c)) + .collect(Collectors.toList())); + } + if (r.getCoverage() != null) { + metadata + .addAll( + r + .getCoverage() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("coverage", c.getValue())) + .collect(Collectors.toList())); + } + if (r.getDateofacceptance() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "dateofacceptance", r.getDateofacceptance().getValue())); + } + if (r.getDescription() != null) { + metadata + .addAll( + r + .getDescription() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue())) + .collect(Collectors.toList())); + } + if (r.getEmbargoenddate() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("embargoenddate", r.getEmbargoenddate().getValue())); + } + if (r.getSubject() != null) { + metadata + .addAll( + r + .getSubject() + .stream() + .map(s -> XmlSerializationUtils.mapStructuredProperty("subject", s)) + .collect(Collectors.toList())); + } + if (r.getLanguage() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("language", r.getLanguage())); + } + if (r.getRelevantdate() != null) { + metadata + .addAll( + r + .getRelevantdate() + .stream() + .map(s -> XmlSerializationUtils.mapStructuredProperty("relevantdate", s)) + .collect(Collectors.toList())); + } + if (r.getPublisher() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("publisher", r.getPublisher().getValue())); + } + if (r.getSource() != null) { + metadata + .addAll( + r + .getSource() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("source", c.getValue())) + .collect(Collectors.toList())); + } + if (r.getFormat() != null) { + metadata + .addAll( + r + .getFormat() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("format", c.getValue())) + .collect(Collectors.toList())); + } + if (r.getResulttype() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("resulttype", r.getResulttype())); + } + if (r.getResourcetype() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("resourcetype", r.getResourcetype())); + } + } + + switch (type) { + case publication: + final Publication pub = (Publication) entity; + + if (pub.getJournal() != null) { + final Journal j = pub.getJournal(); + metadata.add(XmlSerializationUtils.mapJournal(j)); + } + + break; + case dataset: + final Dataset d = (Dataset) entity; + if (d.getDevice() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("device", d.getDevice().getValue())); + } + if (d.getLastmetadataupdate() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "lastmetadataupdate", d.getLastmetadataupdate().getValue())); + } + if (d.getMetadataversionnumber() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "metadataversionnumber", d.getMetadataversionnumber().getValue())); + } + if (d.getSize() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("size", d.getSize().getValue())); + } + if (d.getStoragedate() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("storagedate", d.getStoragedate().getValue())); + } + if (d.getVersion() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("version", d.getVersion().getValue())); + } + // TODO d.getGeolocation() + + break; + case otherresearchproduct: + final OtherResearchProduct orp = (OtherResearchProduct) entity; + + if (orp.getContactperson() != null) { + metadata + .addAll( + orp + .getContactperson() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("contactperson", c.getValue())) + .collect(Collectors.toList())); + } + + if (orp.getContactgroup() != null) { + metadata + .addAll( + orp + .getContactgroup() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("contactgroup", c.getValue())) + .collect(Collectors.toList())); + } + if (orp.getTool() != null) { + metadata + .addAll( + orp + .getTool() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("tool", c.getValue())) + .collect(Collectors.toList())); + } + break; + case software: + final Software s = (Software) entity; + + if (s.getDocumentationUrl() != null) { + metadata + .addAll( + s + .getDocumentationUrl() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("documentationUrl", c.getValue())) + .collect(Collectors.toList())); + } + if (s.getLicense() != null) { + metadata + .addAll( + s + .getLicense() + .stream() + .map(l -> XmlSerializationUtils.mapStructuredProperty("license", l)) + .collect(Collectors.toList())); + } + if (s.getCodeRepositoryUrl() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "codeRepositoryUrl", s.getCodeRepositoryUrl().getValue())); + } + if (s.getProgrammingLanguage() != null) { + metadata + .add( + XmlSerializationUtils + .mapQualifier( + "programmingLanguage", s.getProgrammingLanguage())); + } + break; + case datasource: + final Datasource ds = (Datasource) entity; + + if (ds.getDatasourcetype() != null) { + mapDatasourceType(metadata, ds.getDatasourcetype()); + } + if (ds.getOpenairecompatibility() != null) { + metadata + .add( + XmlSerializationUtils + .mapQualifier( + "openairecompatibility", ds.getOpenairecompatibility())); + } + if (ds.getOfficialname() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("officialname", ds.getOfficialname().getValue())); + } + if (ds.getEnglishname() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("englishname", ds.getEnglishname().getValue())); + } + if (ds.getWebsiteurl() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("websiteurl", ds.getWebsiteurl().getValue())); + } + if (ds.getLogourl() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("logourl", ds.getLogourl().getValue())); + } + if (ds.getContactemail() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("contactemail", ds.getContactemail().getValue())); + } + if (ds.getNamespaceprefix() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "namespaceprefix", ds.getNamespaceprefix().getValue())); + } + if (ds.getLatitude() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("latitude", ds.getLatitude().getValue())); + } + if (ds.getLongitude() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("longitude", ds.getLongitude().getValue())); + } + if (ds.getDateofvalidation() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "dateofvalidation", ds.getDateofvalidation().getValue())); + } + if (ds.getDescription() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("description", ds.getDescription().getValue())); + } + if (ds.getOdnumberofitems() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "odnumberofitems", ds.getOdnumberofitems().getValue())); + } + if (ds.getOdnumberofitemsdate() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "odnumberofitemsdate", ds.getOdnumberofitemsdate().getValue())); + } + if (ds.getOdpolicies() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("odpolicies", ds.getOdpolicies().getValue())); + } + if (ds.getOdlanguages() != null) { + metadata + .addAll( + ds + .getOdlanguages() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("odlanguages", c.getValue())) + .collect(Collectors.toList())); + } + if (ds.getOdcontenttypes() != null) { + metadata + .addAll( + ds + .getOdcontenttypes() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("odcontenttypes", c.getValue())) + .collect(Collectors.toList())); + } + if (ds.getAccessinfopackage() != null) { + metadata + .addAll( + ds + .getAccessinfopackage() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("accessinfopackage", c.getValue())) + .collect(Collectors.toList())); + } + if (ds.getReleaseenddate() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "releasestartdate", ds.getReleaseenddate().getValue())); + } + if (ds.getReleaseenddate() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "releaseenddate", ds.getReleaseenddate().getValue())); + } + if (ds.getMissionstatementurl() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "missionstatementurl", ds.getMissionstatementurl().getValue())); + } + if (ds.getDataprovider() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "dataprovider", ds.getDataprovider().getValue().toString())); + } + if (ds.getServiceprovider() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "serviceprovider", ds.getServiceprovider().getValue().toString())); + } + if (ds.getDatabaseaccesstype() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "databaseaccesstype", ds.getDatabaseaccesstype().getValue())); + } + if (ds.getDatauploadtype() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "datauploadtype", ds.getDatauploadtype().getValue())); + } + if (ds.getDatabaseaccessrestriction() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "databaseaccessrestriction", ds.getDatabaseaccessrestriction().getValue())); + } + if (ds.getDatauploadrestriction() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "datauploadrestriction", ds.getDatauploadrestriction().getValue())); + } + if (ds.getVersioning() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "versioning", ds.getVersioning().getValue().toString())); + } + if (ds.getCitationguidelineurl() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "citationguidelineurl", ds.getCitationguidelineurl().getValue())); + } + if (ds.getQualitymanagementkind() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "qualitymanagementkind", ds.getQualitymanagementkind().getValue())); + } + if (ds.getPidsystems() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("pidsystems", ds.getPidsystems().getValue())); + } + if (ds.getCertificates() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("certificates", ds.getCertificates().getValue())); + } + if (ds.getPolicies() != null) { + metadata + .addAll( + ds + .getPolicies() + .stream() + .map(kv -> XmlSerializationUtils.mapKeyValue("policies", kv)) + .collect(Collectors.toList())); + } + if (ds.getJournal() != null) { + metadata.add(XmlSerializationUtils.mapJournal(ds.getJournal())); + } + if (ds.getSubjects() != null) { + metadata + .addAll( + ds + .getSubjects() + .stream() + .map(sp -> XmlSerializationUtils.mapStructuredProperty("subjects", sp)) + .collect(Collectors.toList())); + } + + break; + case organization: + final Organization o = (Organization) entity; + + if (o.getLegalshortname() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "legalshortname", o.getLegalshortname().getValue())); + } + if (o.getLegalname() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("legalname", o.getLegalname().getValue())); + } + if (o.getAlternativeNames() != null) { + metadata + .addAll( + o + .getAlternativeNames() + .stream() + .map(c -> XmlSerializationUtils.asXmlElement("alternativeNames", c.getValue())) + .collect(Collectors.toList())); + } + if (o.getWebsiteurl() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("websiteurl", o.getWebsiteurl().getValue())); + } + if (o.getLogourl() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("websiteurl", o.getLogourl().getValue())); + } + + if (o.getEclegalbody() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("eclegalbody", o.getEclegalbody().getValue())); + } + if (o.getEclegalperson() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("eclegalperson", o.getEclegalperson().getValue())); + } + if (o.getEcnonprofit() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("ecnonprofit", o.getEcnonprofit().getValue())); + } + if (o.getEcresearchorganization() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "ecresearchorganization", o.getEcresearchorganization().getValue())); + } + if (o.getEchighereducation() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "echighereducation", o.getEchighereducation().getValue())); + } + if (o.getEcinternationalorganization() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "ecinternationalorganizationeurinterests", + o.getEcinternationalorganization().getValue())); + } + if (o.getEcinternationalorganization() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "ecinternationalorganization", o.getEcinternationalorganization().getValue())); + } + if (o.getEcenterprise() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("ecenterprise", o.getEcenterprise().getValue())); + } + if (o.getEcsmevalidated() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "ecsmevalidated", o.getEcsmevalidated().getValue())); + } + if (o.getEcnutscode() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("ecnutscode", o.getEcnutscode().getValue())); + } + if (o.getCountry() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("country", o.getCountry())); + } + + break; + case project: + final Project p = (Project) entity; + + if (p.getWebsiteurl() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("websiteurl", p.getWebsiteurl().getValue())); + } + if (p.getCode() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("code", p.getCode().getValue())); + } + if (p.getAcronym() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("acronym", p.getAcronym().getValue())); + } + if (p.getTitle() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("title", p.getTitle().getValue())); + } + if (p.getStartdate() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("startdate", p.getStartdate().getValue())); + } + if (p.getEnddate() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("enddate", p.getEnddate().getValue())); + } + if (p.getCallidentifier() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "callidentifier", p.getCallidentifier().getValue())); + } + if (p.getKeywords() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("keywords", p.getKeywords().getValue())); + } + if (p.getDuration() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("duration", p.getDuration().getValue())); + } + if (p.getEcarticle29_3() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("ecarticle29_3", p.getEcarticle29_3().getValue())); + } + if (p.getSubjects() != null) { + metadata + .addAll( + p + .getSubjects() + .stream() + .map(sp -> XmlSerializationUtils.mapStructuredProperty("subject", sp)) + .collect(Collectors.toList())); + } + if (p.getContracttype() != null) { + metadata.add(XmlSerializationUtils.mapQualifier("contracttype", p.getContracttype())); + } + if (p.getEcsc39() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("ecsc39", p.getEcsc39().getValue())); + } + if (p.getContactfullname() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement( + "contactfullname", p.getContactfullname().getValue())); + } + if (p.getContactfax() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("contactfax", p.getContactfax().getValue())); + } + if (p.getContactphone() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("contactphone", p.getContactphone().getValue())); + } + if (p.getContactemail() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("contactemail", p.getContactemail().getValue())); + } + if (p.getSummary() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("summary", p.getSummary().getValue())); + } + if (p.getCurrency() != null) { + metadata.add(XmlSerializationUtils.asXmlElement("currency", p.getCurrency().getValue())); + } + if (p.getTotalcost() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("totalcost", p.getTotalcost().toString())); + } + if (p.getFundedamount() != null) { + metadata + .add( + XmlSerializationUtils.asXmlElement("fundedamount", p.getFundedamount().toString())); + } + if (p.getFundingtree() != null) { + metadata + .addAll( + p.getFundingtree().stream().map(ft -> ft.getValue()).collect(Collectors.toList())); + } + + break; + default: + throw new IllegalArgumentException("invalid entity type: " + type); + } + + return metadata; + } + + private void mapDatasourceType(List metadata, final Qualifier dsType) { + metadata.add(XmlSerializationUtils.mapQualifier("datasourcetype", dsType)); + + if (specialDatasourceTypes.contains(dsType.getClassid())) { + dsType.setClassid("other"); + dsType.setClassname("other"); + } + metadata.add(XmlSerializationUtils.mapQualifier("datasourcetypeui", dsType)); + } + + private String mapRelation(Tuple2 link, TemplateFactory templateFactory, Set contexts) { + final Relation rel = link.getRelation(); + final RelatedEntity re = link.getRelatedEntity(); + final String targetType = link.getRelatedEntity().getType(); + + final List metadata = Lists.newArrayList(); + switch (EntityType.valueOf(targetType)) { + case publication: + case dataset: + case otherresearchproduct: + case software: + if (re.getTitle() != null && isNotBlank(re.getTitle().getValue())) { + metadata.add(XmlSerializationUtils.mapStructuredProperty("title", re.getTitle())); + } + if (isNotBlank(re.getDateofacceptance())) { + metadata + .add( + XmlSerializationUtils.asXmlElement("dateofacceptance", re.getDateofacceptance())); + } + if (isNotBlank(re.getPublisher())) { + metadata.add(XmlSerializationUtils.asXmlElement("publisher", re.getPublisher())); + } + if (isNotBlank(re.getCodeRepositoryUrl())) { + metadata + .add( + XmlSerializationUtils.asXmlElement("coderepositoryurl", re.getCodeRepositoryUrl())); + } + if (re.getResulttype() != null & re.getResulttype().isBlank()) { + metadata.add(XmlSerializationUtils.mapQualifier("resulttype", re.getResulttype())); + } + if (re.getCollectedfrom() != null) { + metadata + .addAll( + re + .getCollectedfrom() + .stream() + .map(kv -> XmlSerializationUtils.mapKeyValue("collectedfrom", kv)) + .collect(Collectors.toList())); + } + if (re.getPid() != null) { + metadata + .addAll( + re + .getPid() + .stream() + .map(p -> XmlSerializationUtils.mapStructuredProperty("pid", p)) + .collect(Collectors.toList())); + } + break; + case datasource: + if (isNotBlank(re.getOfficialname())) { + metadata.add(XmlSerializationUtils.asXmlElement("officialname", re.getOfficialname())); + } + if (re.getDatasourcetype() != null & !re.getDatasourcetype().isBlank()) { + mapDatasourceType(metadata, re.getDatasourcetype()); + } + if (re.getOpenairecompatibility() != null & !re.getOpenairecompatibility().isBlank()) { + metadata + .add( + XmlSerializationUtils + .mapQualifier( + "openairecompatibility", re.getOpenairecompatibility())); + } + break; + case organization: + if (isNotBlank(re.getLegalname())) { + metadata.add(XmlSerializationUtils.asXmlElement("legalname", re.getLegalname())); + } + if (isNotBlank(re.getLegalshortname())) { + metadata + .add( + XmlSerializationUtils.asXmlElement("legalshortname", re.getLegalshortname())); + } + if (re.getCountry() != null & !re.getCountry().isBlank()) { + metadata.add(XmlSerializationUtils.mapQualifier("country", re.getCountry())); + } + break; + case project: + if (isNotBlank(re.getProjectTitle())) { + metadata.add(XmlSerializationUtils.asXmlElement("title", re.getProjectTitle())); + } + if (isNotBlank(re.getCode())) { + metadata.add(XmlSerializationUtils.asXmlElement("code", re.getCode())); + } + if (isNotBlank(re.getAcronym())) { + metadata.add(XmlSerializationUtils.asXmlElement("acronym", re.getAcronym())); + } + if (re.getContracttype() != null & !re.getContracttype().isBlank()) { + metadata.add(XmlSerializationUtils.mapQualifier("contracttype", re.getContracttype())); + } + if (re.getFundingtree() != null & contexts != null) { + metadata + .addAll( + re + .getFundingtree() + .stream() + .peek(ft -> fillContextMap(ft, contexts)) + .map(ft -> getRelFundingTree(ft)) + .collect(Collectors.toList())); + } + break; + default: + throw new IllegalArgumentException("invalid target type: " + targetType); + } + final DataInfo info = rel.getDataInfo(); + final String scheme = ModelSupport.getScheme(re.getType(), targetType); + + if (StringUtils.isBlank(scheme)) { + throw new IllegalArgumentException( + String.format("missing scheme for: <%s - %s>", re.getType(), targetType)); + } + + final String accumulatorName = getRelDescriptor(rel.getRelType(), rel.getSubRelType(), rel.getRelClass()); + if (accumulators.containsKey(accumulatorName)) { + accumulators.get(accumulatorName).add(1); + } + + return templateFactory + .getRel( + targetType, rel.getTarget(), Sets.newHashSet(metadata), rel.getRelClass(), scheme, info); + } + + private List listChildren( + final OafEntity entity, JoinedEntity je, TemplateFactory templateFactory) { + + final List children = Lists.newArrayList(); + EntityType entityType = EntityType.valueOf(je.getEntity().getType()); + + children + .addAll( + je + .getLinks() + .stream() + .filter(link -> REL_SUBTYPE_DEDUP.equalsIgnoreCase(link.getRelation().getSubRelType())) + .map(link -> mapRelation(link, templateFactory, null)) + .collect(Collectors.toCollection(ArrayList::new))); + + if (MainEntityType.result.toString().equals(ModelSupport.getMainType(entityType))) { + final List instances = ((Result) entity).getInstance(); + if (instances != null) { + for (final Instance instance : ((Result) entity).getInstance()) { + + final List fields = Lists.newArrayList(); + + if (instance.getAccessright() != null && !instance.getAccessright().isBlank()) { + fields + .add( + XmlSerializationUtils.mapQualifier("accessright", instance.getAccessright())); + } + if (instance.getCollectedfrom() != null) { + fields + .add( + XmlSerializationUtils.mapKeyValue("collectedfrom", instance.getCollectedfrom())); + } + if (instance.getHostedby() != null) { + fields.add(XmlSerializationUtils.mapKeyValue("hostedby", instance.getHostedby())); + } + if (instance.getDateofacceptance() != null + && isNotBlank(instance.getDateofacceptance().getValue())) { + fields + .add( + XmlSerializationUtils + .asXmlElement( + "dateofacceptance", instance.getDateofacceptance().getValue())); + } + if (instance.getInstancetype() != null && !instance.getInstancetype().isBlank()) { + fields + .add( + XmlSerializationUtils.mapQualifier("instancetype", instance.getInstancetype())); + } + if (isNotBlank(instance.getDistributionlocation())) { + fields + .add( + XmlSerializationUtils + .asXmlElement( + "distributionlocation", instance.getDistributionlocation())); + } + if (instance.getRefereed() != null && isNotBlank(instance.getRefereed().getValue())) { + fields + .add( + XmlSerializationUtils.asXmlElement("refereed", instance.getRefereed().getValue())); + } + if (instance.getProcessingchargeamount() != null + && isNotBlank(instance.getProcessingchargeamount().getValue())) { + fields + .add( + XmlSerializationUtils + .asXmlElement( + "processingchargeamount", instance.getProcessingchargeamount().getValue())); + } + if (instance.getProcessingchargecurrency() != null + && isNotBlank(instance.getProcessingchargecurrency().getValue())) { + fields + .add( + XmlSerializationUtils + .asXmlElement( + "processingchargecurrency", instance.getProcessingchargecurrency().getValue())); + } + + children + .add( + templateFactory + .getInstance( + instance.getHostedby().getKey(), fields, instance.getUrl())); + } + } + final List ext = ((Result) entity).getExternalReference(); + if (ext != null) { + for (final ExternalReference er : ((Result) entity).getExternalReference()) { + + final List fields = Lists.newArrayList(); + + if (isNotBlank(er.getSitename())) { + fields.add(XmlSerializationUtils.asXmlElement("sitename", er.getSitename())); + } + if (isNotBlank(er.getLabel())) { + fields.add(XmlSerializationUtils.asXmlElement("label", er.getLabel())); + } + if (isNotBlank(er.getUrl())) { + fields.add(XmlSerializationUtils.asXmlElement("url", er.getUrl())); + } + if (isNotBlank(er.getDescription())) { + fields.add(XmlSerializationUtils.asXmlElement("description", er.getDescription())); + } + if (isNotBlank(er.getUrl())) { + fields.add(XmlSerializationUtils.mapQualifier("qualifier", er.getQualifier())); + } + if (isNotBlank(er.getRefidentifier())) { + fields.add(XmlSerializationUtils.asXmlElement("refidentifier", er.getRefidentifier())); + } + if (isNotBlank(er.getQuery())) { + fields.add(XmlSerializationUtils.asXmlElement("query", er.getQuery())); + } + + children.add(templateFactory.getChild("externalreference", null, fields)); + } + } + } + + return children; + } + + private List listExtraInfo(OafEntity entity) { + final List extraInfo = entity.getExtraInfo(); + return extraInfo != null + ? extraInfo + .stream() + .map(e -> XmlSerializationUtils.mapExtraInfo(e)) + .collect(Collectors.toList()) + : Lists.newArrayList(); + } + + private List buildContexts(final String type, final Set contexts) { + final List res = Lists.newArrayList(); + + if ((contextMapper != null) + && !contextMapper.isEmpty() + && MainEntityType.result.toString().equals(type)) { + + XMLTag document = XMLDoc.newDocument(true).addRoot("contextRoot"); + + for (final String context : contexts) { + + String id = ""; + for (final String token : Splitter.on("::").split(context)) { + id += token; + + final ContextDef def = contextMapper.get(id); + + if (def == null) { + continue; + // throw new IllegalStateException(String.format("cannot find context for id + // '%s'", + // id)); + } + + if (def.getName().equals("context")) { + final String xpath = "//context/@id='" + def.getId() + "'"; + if (!document.gotoRoot().rawXpathBoolean(xpath, new Object())) { + document = addContextDef(document.gotoRoot(), def); + } + } + + if (def.getName().equals("category")) { + final String rootId = substringBefore(def.getId(), "::"); + document = addContextDef( + document.gotoRoot().gotoTag("//context[./@id='" + rootId + "']", new Object()), + def); + } + + if (def.getName().equals("concept")) { + document = addContextDef(document, def).gotoParent(); + } + id += "::"; + } + } + final Transformer transformer = getTransformer(); + for (final org.w3c.dom.Element x : document.gotoRoot().getChildElement()) { + try { + res.add(asStringElement(x, transformer)); + } catch (final TransformerException e) { + throw new RuntimeException(e); + } + } + } + + return res; + } + + private Transformer getTransformer() { + try { + Transformer transformer = TransformerFactory.newInstance().newTransformer(); + transformer.setOutputProperty(OutputKeys.OMIT_XML_DECLARATION, "yes"); + return transformer; + } catch (TransformerConfigurationException e) { + throw new IllegalStateException("unable to create javax.xml.transform.Transformer", e); + } + } + + private XMLTag addContextDef(final XMLTag tag, final ContextDef def) { + tag.addTag(def.getName()).addAttribute("id", def.getId()).addAttribute("label", def.getLabel()); + if ((def.getType() != null) && !def.getType().isEmpty()) { + tag.addAttribute("type", def.getType()); + } + return tag; + } + + private String asStringElement(final org.w3c.dom.Element element, final Transformer transformer) + throws TransformerException { + final StringWriter buffer = new StringWriter(); + transformer.transform(new DOMSource(element), new StreamResult(buffer)); + return buffer.toString(); + } + + private void fillContextMap(final String xmlTree, final Set contexts) { + + Document fundingPath; + try { + fundingPath = new SAXReader().read(new StringReader(xmlTree)); + } catch (final DocumentException e) { + throw new RuntimeException(e); + } + try { + final Node funder = fundingPath.selectSingleNode("//funder"); + + if (funder != null) { + + final String funderShortName = funder.valueOf("./shortname"); + contexts.add(funderShortName); + + contextMapper + .put( + funderShortName, + new ContextDef(funderShortName, funder.valueOf("./name"), "context", "funding")); + final Node level0 = fundingPath.selectSingleNode("//funding_level_0"); + if (level0 != null) { + final String level0Id = Joiner.on("::").join(funderShortName, level0.valueOf("./name")); + contextMapper + .put( + level0Id, new ContextDef(level0Id, level0.valueOf("./description"), "category", "")); + final Node level1 = fundingPath.selectSingleNode("//funding_level_1"); + if (level1 == null) { + contexts.add(level0Id); + } else { + final String level1Id = Joiner.on("::").join(level0Id, level1.valueOf("./name")); + contextMapper + .put( + level1Id, new ContextDef(level1Id, level1.valueOf("./description"), "concept", "")); + final Node level2 = fundingPath.selectSingleNode("//funding_level_2"); + if (level2 == null) { + contexts.add(level1Id); + } else { + final String level2Id = Joiner.on("::").join(level1Id, level2.valueOf("./name")); + contextMapper + .put( + level2Id, + new ContextDef(level2Id, level2.valueOf("./description"), "concept", "")); + contexts.add(level2Id); + } + } + } + } + } catch (final NullPointerException e) { + throw new IllegalArgumentException("malformed funding path: " + xmlTree, e); + } + } + + @SuppressWarnings("unchecked") + protected static String getRelFundingTree(final String xmlTree) { + String funding = ""; + try { + final Document ftree = new SAXReader().read(new StringReader(xmlTree)); + funding = ""; + + funding += getFunderElement(ftree); + + for (final Object o : Lists + .reverse( + ftree.selectNodes("//fundingtree//*[starts-with(local-name(),'funding_level_')]"))) { + final Element e = (Element) o; + final String _id = e.valueOf("./id"); + funding += "<" + + e.getName() + + " name=\"" + + XmlSerializationUtils.escapeXml(e.valueOf("./name")) + + "\">" + + XmlSerializationUtils.escapeXml(_id) + + ""; + } + } catch (final DocumentException e) { + throw new IllegalArgumentException( + "unable to parse funding tree: " + xmlTree + "\n" + e.getMessage()); + } finally { + funding += ""; + } + return funding; + } + + private static String getFunderElement(final Document ftree) { + final String funderId = ftree.valueOf("//fundingtree/funder/id"); + final String funderShortName = ftree.valueOf("//fundingtree/funder/shortname"); + final String funderName = ftree.valueOf("//fundingtree/funder/name"); + final String funderJurisdiction = ftree.valueOf("//fundingtree/funder/jurisdiction"); + + return ""; + } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java index 0b3109bde..bc3b3107d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlSerializationUtils.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.oa.provision.utils; import static eu.dnetlib.dhp.oa.provision.utils.GraphMappingUtils.removePrefix; @@ -8,149 +9,151 @@ import eu.dnetlib.dhp.schema.oaf.*; public class XmlSerializationUtils { - // XML 1.0 - // #x9 | #xA | #xD | [#x20-#xD7FF] | [#xE000-#xFFFD] | [#x10000-#x10FFFF] - private static final String xml10pattern = - "[^" + "\u0009\r\n" + "\u0020-\uD7FF" + "\uE000-\uFFFD" + "\ud800\udc00-\udbff\udfff" + "]"; + // XML 1.0 + // #x9 | #xA | #xD | [#x20-#xD7FF] | [#xE000-#xFFFD] | [#x10000-#x10FFFF] + private static final String xml10pattern = "[^" + "\u0009\r\n" + "\u0020-\uD7FF" + "\uE000-\uFFFD" + + "\ud800\udc00-\udbff\udfff" + "]"; - public static String mapJournal(Journal j) { - final String attrs = - new StringBuilder() - .append(attr("issn", j.getIssnPrinted())) - .append(attr("eissn", j.getIssnOnline())) - .append(attr("lissn", j.getIssnLinking())) - .append(attr("ep", j.getEp())) - .append(attr("iss", j.getIss())) - .append(attr("sp", j.getSp())) - .append(attr("vol", j.getVol())) - .toString() - .trim(); + public static String mapJournal(Journal j) { + final String attrs = new StringBuilder() + .append(attr("issn", j.getIssnPrinted())) + .append(attr("eissn", j.getIssnOnline())) + .append(attr("lissn", j.getIssnLinking())) + .append(attr("ep", j.getEp())) + .append(attr("iss", j.getIss())) + .append(attr("sp", j.getSp())) + .append(attr("vol", j.getVol())) + .toString() + .trim(); - return new StringBuilder() - .append("") - .append(escapeXml(j.getName())) - .append("") - .toString(); - } + return new StringBuilder() + .append("") + .append(escapeXml(j.getName())) + .append("") + .toString(); + } - private static String attr(final String name, final String value) { - return isNotBlank(value) ? name + "=\"" + escapeXml(value) + "\" " : ""; - } + private static String attr(final String name, final String value) { + return isNotBlank(value) ? name + "=\"" + escapeXml(value) + "\" " : ""; + } - public static String mapStructuredProperty(String name, StructuredProperty t) { - return asXmlElement( - name, t.getValue(), t.getQualifier(), t.getDataInfo() != null ? t.getDataInfo() : null); - } + public static String mapStructuredProperty(String name, StructuredProperty t) { + return asXmlElement( + name, t.getValue(), t.getQualifier(), t.getDataInfo() != null ? t.getDataInfo() : null); + } - public static String mapQualifier(String name, Qualifier q) { - return asXmlElement(name, "", q, null); - } + public static String mapQualifier(String name, Qualifier q) { + return asXmlElement(name, "", q, null); + } - public static String escapeXml(final String value) { - return value - .replaceAll("&", "&") - .replaceAll("<", "<") - .replaceAll(">", ">") - .replaceAll("\"", """) - .replaceAll("'", "'") - .replaceAll(xml10pattern, ""); - } + public static String escapeXml(final String value) { + return value + .replaceAll("&", "&") + .replaceAll("<", "<") + .replaceAll(">", ">") + .replaceAll("\"", """) + .replaceAll("'", "'") + .replaceAll(xml10pattern, ""); + } - public static String parseDataInfo(final DataInfo dataInfo) { - return new StringBuilder() - .append("") - .append(asXmlElement("inferred", dataInfo.getInferred() + "")) - .append(asXmlElement("deletedbyinference", dataInfo.getDeletedbyinference() + "")) - .append(asXmlElement("trust", dataInfo.getTrust() + "")) - .append(asXmlElement("inferenceprovenance", dataInfo.getInferenceprovenance() + "")) - .append(asXmlElement("provenanceaction", null, dataInfo.getProvenanceaction(), null)) - .append("") - .toString(); - } + public static String parseDataInfo(final DataInfo dataInfo) { + return new StringBuilder() + .append("") + .append(asXmlElement("inferred", dataInfo.getInferred() + "")) + .append(asXmlElement("deletedbyinference", dataInfo.getDeletedbyinference() + "")) + .append(asXmlElement("trust", dataInfo.getTrust() + "")) + .append(asXmlElement("inferenceprovenance", dataInfo.getInferenceprovenance() + "")) + .append(asXmlElement("provenanceaction", null, dataInfo.getProvenanceaction(), null)) + .append("") + .toString(); + } - private static StringBuilder dataInfoAsAttributes(final StringBuilder sb, final DataInfo info) { - return sb.append( - attr("inferred", info.getInferred() != null ? info.getInferred().toString() : "")) - .append(attr("inferenceprovenance", info.getInferenceprovenance())) - .append( - attr( - "provenanceaction", - info.getProvenanceaction() != null ? info.getProvenanceaction().getClassid() : "")) - .append(attr("trust", info.getTrust())); - } + private static StringBuilder dataInfoAsAttributes(final StringBuilder sb, final DataInfo info) { + return sb + .append( + attr("inferred", info.getInferred() != null ? info.getInferred().toString() : "")) + .append(attr("inferenceprovenance", info.getInferenceprovenance())) + .append( + attr( + "provenanceaction", + info.getProvenanceaction() != null ? info.getProvenanceaction().getClassid() : "")) + .append(attr("trust", info.getTrust())); + } - public static String mapKeyValue(final String name, final KeyValue kv) { - return new StringBuilder() - .append("<") - .append(name) - .append(" name=\"") - .append(escapeXml(kv.getValue())) - .append("\" id=\"") - .append(escapeXml(removePrefix(kv.getKey()))) - .append("\"/>") - .toString(); - } + public static String mapKeyValue(final String name, final KeyValue kv) { + return new StringBuilder() + .append("<") + .append(name) + .append(" name=\"") + .append(escapeXml(kv.getValue())) + .append("\" id=\"") + .append(escapeXml(removePrefix(kv.getKey()))) + .append("\"/>") + .toString(); + } - public static String mapExtraInfo(final ExtraInfo e) { - return new StringBuilder("") - .append(e.getValue()) - .append("") - .toString(); - } + public static String mapExtraInfo(final ExtraInfo e) { + return new StringBuilder("") + .append(e.getValue()) + .append("") + .toString(); + } - public static String asXmlElement(final String name, final String value) { - return asXmlElement(name, value, null, null); - } + public static String asXmlElement(final String name, final String value) { + return asXmlElement(name, value, null, null); + } - public static String asXmlElement( - final String name, final String value, final Qualifier q, final DataInfo info) { - StringBuilder sb = new StringBuilder(); - sb.append("<"); - sb.append(name); - if (q != null) { - sb.append(getAttributes(q)); - } - if (info != null) { - sb.append(" ") - .append(attr("inferred", info.getInferred() != null ? info.getInferred().toString() : "")) - .append(attr("inferenceprovenance", info.getInferenceprovenance())) - .append( - attr( - "provenanceaction", - info.getProvenanceaction() != null - ? info.getProvenanceaction().getClassid() - : "")) - .append(attr("trust", info.getTrust())); - } - if (isBlank(value)) { - sb.append("/>"); - return sb.toString(); - } + public static String asXmlElement( + final String name, final String value, final Qualifier q, final DataInfo info) { + StringBuilder sb = new StringBuilder(); + sb.append("<"); + sb.append(name); + if (q != null) { + sb.append(getAttributes(q)); + } + if (info != null) { + sb + .append(" ") + .append(attr("inferred", info.getInferred() != null ? info.getInferred().toString() : "")) + .append(attr("inferenceprovenance", info.getInferenceprovenance())) + .append( + attr( + "provenanceaction", + info.getProvenanceaction() != null + ? info.getProvenanceaction().getClassid() + : "")) + .append(attr("trust", info.getTrust())); + } + if (isBlank(value)) { + sb.append("/>"); + return sb.toString(); + } - sb.append(">"); - sb.append(escapeXml(value)); - sb.append(""); + sb.append(">"); + sb.append(escapeXml(value)); + sb.append(""); - return sb.toString(); - } + return sb.toString(); + } - public static String getAttributes(final Qualifier q) { - if (q == null || q.isBlank()) return ""; + public static String getAttributes(final Qualifier q) { + if (q == null || q.isBlank()) + return ""; - return new StringBuilder(" ") - .append(attr("classid", q.getClassid())) - .append(attr("classname", q.getClassname())) - .append(attr("schemeid", q.getSchemeid())) - .append(attr("schemename", q.getSchemename())) - .toString(); - } + return new StringBuilder(" ") + .append(attr("classid", q.getClassid())) + .append(attr("classname", q.getClassname())) + .append(attr("schemeid", q.getSchemeid())) + .append(attr("schemename", q.getSchemename())) + .toString(); + } } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/GraphJoinerTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/GraphJoinerTest.java index 21feb1637..8afe03d6d 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/GraphJoinerTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/GraphJoinerTest.java @@ -1,39 +1,42 @@ + package eu.dnetlib.dhp.oa.provision; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.junit.jupiter.api.BeforeEach; public class GraphJoinerTest { - private ClassLoader cl = getClass().getClassLoader(); - private Path workingDir; - private Path inputDir; - private Path outputDir; + private ClassLoader cl = getClass().getClassLoader(); + private Path workingDir; + private Path inputDir; + private Path outputDir; - @BeforeEach - public void before() throws IOException { - workingDir = Files.createTempDirectory("promote_action_set"); - inputDir = workingDir.resolve("input"); - outputDir = workingDir.resolve("output"); - } + @BeforeEach + public void before() throws IOException { + workingDir = Files.createTempDirectory("promote_action_set"); + inputDir = workingDir.resolve("input"); + outputDir = workingDir.resolve("output"); + } - private static void copyFiles(Path source, Path target) throws IOException { - Files.list(source) - .forEach( - f -> { - try { - if (Files.isDirectory(f)) { - Path subTarget = Files.createDirectories(target.resolve(f.getFileName())); - copyFiles(f, subTarget); - } else { - Files.copy(f, target.resolve(f.getFileName())); - } - } catch (IOException e) { - e.printStackTrace(); - throw new RuntimeException(e); - } - }); - } + private static void copyFiles(Path source, Path target) throws IOException { + Files + .list(source) + .forEach( + f -> { + try { + if (Files.isDirectory(f)) { + Path subTarget = Files.createDirectories(target.resolve(f.getFileName())); + copyFiles(f, subTarget); + } else { + Files.copy(f, target.resolve(f.getFileName())); + } + } catch (IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 1a0d98182..a2fb0c0ef 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -1,12 +1,9 @@ + package eu.dnetlib.dhp; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; -import eu.dnetlib.dhp.schema.oaf.*; import java.io.IOException; import java.util.*; + import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.function.MapFunction; @@ -14,190 +11,189 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.oaf.*; + public class PropagationConstant { - public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional"; + public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional"; - public static final String PROPAGATION_DATA_INFO_TYPE = "propagation"; + public static final String PROPAGATION_DATA_INFO_TYPE = "propagation"; - public static final String TRUE = "true"; + public static final String TRUE = "true"; - public static final String DNET_COUNTRY_SCHEMA = "dnet:countries"; - public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; - public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; + public static final String DNET_COUNTRY_SCHEMA = "dnet:countries"; + public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; + public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; - public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; - public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = - "Propagation of country to result collected from datasources of type institutional repositories"; + public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; + public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; - public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = - "result:organization:instrepo"; - public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = - "Propagation of affiliation to result collected from datasources of type institutional repository"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID = "result:organization:instrepo"; + public static final String PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME = "Propagation of affiliation to result collected from datasources of type institutional repository"; - public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = - "result:project:semrel"; - public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = - "Propagation of result to project through semantic relation"; + public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID = "result:project:semrel"; + public static final String PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME = "Propagation of result to project through semantic relation"; - public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = - "result:community:semrel"; - public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = - " Propagation of result belonging to community through semantic relation"; + public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID = "result:community:semrel"; + public static final String PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME = " Propagation of result belonging to community through semantic relation"; - public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = - "result:community:organization"; - public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = - " Propagation of result belonging to community through organization"; + public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = "result:community:organization"; + public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = " Propagation of result belonging to community through organization"; - public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = - "authorpid:result"; - public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = - "Propagation of authors pid to result through semantic relations"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "authorpid:result"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of authors pid to result through semantic relations"; - public static final String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; + public static final String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; - public static final String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; - public static final String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; - public static final String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; - public static final String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; + public static final String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; + public static final String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; + public static final String RELATION_ORGANIZATION_RESULT_REL_CLASS = "isAuthorInstitutionOf"; + public static final String RELATION_RESULT_ORGANIZATION_REL_CLASS = "hasAuthorInstitution"; - public static final String RELATION_RESULTRESULT_REL_TYPE = "resultResult"; + public static final String RELATION_RESULTRESULT_REL_TYPE = "resultResult"; - public static final String RELATION_RESULTPROJECT_REL_TYPE = "resultProject"; - public static final String RELATION_RESULTPROJECT_SUBREL_TYPE = "outcome"; - public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; - public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; + public static final String RELATION_RESULTPROJECT_REL_TYPE = "resultProject"; + public static final String RELATION_RESULTPROJECT_SUBREL_TYPE = "outcome"; + public static final String RELATION_RESULT_PROJECT_REL_CLASS = "isProducedBy"; + public static final String RELATION_PROJECT_RESULT_REL_CLASS = "produces"; - public static final String RELATION_REPRESENTATIVERESULT_RESULT_CLASS = "merges"; + public static final String RELATION_REPRESENTATIVERESULT_RESULT_CLASS = "merges"; - public static final String PROPAGATION_AUTHOR_PID = "ORCID"; + public static final String PROPAGATION_AUTHOR_PID = "ORCID"; - public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static Country getCountry(String classid, String classname) { - Country nc = new Country(); - nc.setClassid(classid); - nc.setClassname(classname); - nc.setSchemename(DNET_COUNTRY_SCHEMA); - nc.setSchemeid(DNET_COUNTRY_SCHEMA); - nc.setDataInfo( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, - PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME)); - return nc; - } + public static Country getCountry(String classid, String classname) { + Country nc = new Country(); + nc.setClassid(classid); + nc.setClassname(classname); + nc.setSchemename(DNET_COUNTRY_SCHEMA); + nc.setSchemeid(DNET_COUNTRY_SCHEMA); + nc + .setDataInfo( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, + PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME)); + return nc; + } - public static DataInfo getDataInfo( - String inference_provenance, String inference_class_id, String inference_class_name) { - DataInfo di = new DataInfo(); - di.setInferred(true); - di.setDeletedbyinference(false); - di.setTrust("0.85"); - di.setInferenceprovenance(inference_provenance); - di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); - return di; - } + public static DataInfo getDataInfo( + String inference_provenance, String inference_class_id, String inference_class_name) { + DataInfo di = new DataInfo(); + di.setInferred(true); + di.setDeletedbyinference(false); + di.setTrust("0.85"); + di.setInferenceprovenance(inference_provenance); + di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name)); + return di; + } - public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { - Qualifier pa = new Qualifier(); - pa.setClassid(inference_class_id); - pa.setClassname(inference_class_name); - pa.setSchemeid(DNET_SCHEMA_ID); - pa.setSchemename(DNET_SCHEMA_NAME); - return pa; - } + public static Qualifier getQualifier(String inference_class_id, String inference_class_name) { + Qualifier pa = new Qualifier(); + pa.setClassid(inference_class_id); + pa.setClassname(inference_class_name); + pa.setSchemeid(DNET_SCHEMA_ID); + pa.setSchemename(DNET_SCHEMA_NAME); + return pa; + } - public static Relation getRelation( - String source, - String target, - String rel_class, - String rel_type, - String subrel_type, - String inference_provenance, - String inference_class_id, - String inference_class_name) { - Relation r = new Relation(); - r.setSource(source); - r.setTarget(target); - r.setRelClass(rel_class); - r.setRelType(rel_type); - r.setSubRelType(subrel_type); - r.setDataInfo(getDataInfo(inference_provenance, inference_class_id, inference_class_name)); - return r; - } + public static Relation getRelation( + String source, + String target, + String rel_class, + String rel_type, + String subrel_type, + String inference_provenance, + String inference_class_id, + String inference_class_name) { + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(rel_class); + r.setRelType(rel_type); + r.setSubRelType(subrel_type); + r.setDataInfo(getDataInfo(inference_provenance, inference_class_id, inference_class_name)); + return r; + } - public static String getConstraintList(String text, List constraints) { - String ret = " and (" + text + constraints.get(0) + "'"; - for (int i = 1; i < constraints.size(); i++) { - ret += " OR " + text + constraints.get(i) + "'"; - } - ret += ")"; - return ret; - } + public static String getConstraintList(String text, List constraints) { + String ret = " and (" + text + constraints.get(0) + "'"; + for (int i = 1; i < constraints.size(); i++) { + ret += " OR " + text + constraints.get(i) + "'"; + } + ret += ")"; + return ret; + } - public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { - if (fs.exists(new Path(outputPath))) { - fs.delete(new Path(outputPath), true); - } - fs.mkdirs(new Path(outputPath)); - } + public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { + if (fs.exists(new Path(outputPath))) { + fs.delete(new Path(outputPath), true); + } + fs.mkdirs(new Path(outputPath)); + } - public static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + public static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } - public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) { - return Optional.ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - } + public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) { + return Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + } - public static Boolean isTest(ArgumentApplicationParser parser) { - return Optional.ofNullable(parser.get("isTest")) - .map(Boolean::valueOf) - .orElse(Boolean.FALSE); - } + public static Boolean isTest(ArgumentApplicationParser parser) { + return Optional + .ofNullable(parser.get("isTest")) + .map(Boolean::valueOf) + .orElse(Boolean.FALSE); + } - public static void createCfHbforresult(SparkSession spark) { - String query; - query = - "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " - + "FROM ( SELECT id, instance " - + "FROM result " - + " WHERE datainfo.deletedbyinference = false) ds " - + "LATERAL VIEW EXPLODE(instance) i AS inst"; - org.apache.spark.sql.Dataset cfhb = spark.sql(query); - cfhb.createOrReplaceTempView("cfhb"); - } + public static void createCfHbforresult(SparkSession spark) { + String query; + query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " + + "FROM ( SELECT id, instance " + + "FROM result " + + " WHERE datainfo.deletedbyinference = false) ds " + + "LATERAL VIEW EXPLODE(instance) i AS inst"; + org.apache.spark.sql.Dataset cfhb = spark.sql(query); + cfhb.createOrReplaceTempView("cfhb"); + } - public static org.apache.spark.sql.Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class resultClazz) { + public static org.apache.spark.sql.Dataset readPathEntity( + SparkSession spark, String inputEntityPath, Class resultClazz) { - return spark.read() - .textFile(inputEntityPath) - .map( - (MapFunction) - value -> OBJECT_MAPPER.readValue(value, resultClazz), - Encoders.bean(resultClazz)); - } + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), + Encoders.bean(resultClazz)); + } - public static org.apache.spark.sql.Dataset readRelations( - SparkSession spark, String inputPath) { - return spark.read() - .textFile(inputPath) - .map( - (MapFunction) - value -> OBJECT_MAPPER.readValue(value, Relation.class), - Encoders.bean(Relation.class)); - } + public static org.apache.spark.sql.Dataset readRelations( + SparkSession spark, String inputPath) { + return spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), + Encoders.bean(Relation.class)); + } - public static org.apache.spark.sql.Dataset readResultCommunityList( - SparkSession spark, String possibleUpdatesPath) { - return spark.read() - .textFile(possibleUpdatesPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), - Encoders.bean(ResultCommunityList.class)); - } + public static org.apache.spark.sql.Dataset readResultCommunityList( + SparkSession spark, String possibleUpdatesPath) { + return spark + .read() + .textFile(possibleUpdatesPath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), + Encoders.bean(ResultCommunityList.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java index a33919d19..c29043a2d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java @@ -1,19 +1,20 @@ + package eu.dnetlib.dhp; +import java.util.List; + import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import java.util.List; public class QueryInformationSystem { - private static final String XQUERY = - "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" - + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" - + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" - + " return $x//CONFIGURATION/context/@id/string()"; + private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" + + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" + + " return $x//CONFIGURATION/context/@id/string()"; - public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - return isLookUp.quickSearchProfile(XQUERY); - } + public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + return isLookUp.quickSearchProfile(XQUERY); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java index 32c893261..271cc6bb3 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java @@ -1,24 +1,25 @@ + package eu.dnetlib.dhp.countrypropagation; import java.io.Serializable; public class CountrySbs implements Serializable { - private String classid; - private String classname; + private String classid; + private String classname; - public String getClassid() { - return classid; - } + public String getClassid() { + return classid; + } - public void setClassid(String classid) { - this.classid = classid; - } + public void setClassid(String classid) { + this.classid = classid; + } - public String getClassname() { - return classname; - } + public String getClassname() { + return classname; + } - public void setClassname(String classname) { - this.classname = classname; - } + public void setClassname(String classname) { + this.classname = classname; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java index 0ac8b108e..642192f73 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java @@ -1,24 +1,25 @@ + package eu.dnetlib.dhp.countrypropagation; import java.io.Serializable; public class DatasourceCountry implements Serializable { - private String dataSourceId; - private CountrySbs country; + private String dataSourceId; + private CountrySbs country; - public String getDataSourceId() { - return dataSourceId; - } + public String getDataSourceId() { + return dataSourceId; + } - public void setDataSourceId(String dataSourceId) { - this.dataSourceId = dataSourceId; - } + public void setDataSourceId(String dataSourceId) { + this.dataSourceId = dataSourceId; + } - public CountrySbs getCountry() { - return country; - } + public CountrySbs getCountry() { + return country; + } - public void setCountry(CountrySbs country) { - this.country = country; - } + public void setCountry(CountrySbs country) { + this.country = country; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index 0604bb019..56185eb72 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -1,13 +1,12 @@ + package eu.dnetlib.dhp.countrypropagation; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Arrays; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -18,118 +17,125 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; + /** - * For the association of the country to the datasource The association is computed only for - * datasource of specific type or having whitelisted ids The country is registered in the - * Organization associated to the Datasource, so the relation provides between Datasource and - * Organization is exploited to get the country for the datasource + * For the association of the country to the datasource The association is computed only for datasource of specific type + * or having whitelisted ids The country is registered in the Organization associated to the Datasource, so the relation + * provides between Datasource and Organization is exploited to get the country for the datasource */ public class PrepareDatasourceCountryAssociation { - private static final Logger log = - LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareDatasourceCountryAssociation.class.getResourceAsStream( - "/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + PrepareDatasourceCountryAssociation.class + .getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareDatasourceCountryAssociation( - spark, - Arrays.asList(parser.get("whitelist").split(";")), - Arrays.asList(parser.get("allowedtypes").split(";")), - inputPath, - outputPath); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareDatasourceCountryAssociation( + spark, + Arrays.asList(parser.get("whitelist").split(";")), + Arrays.asList(parser.get("allowedtypes").split(";")), + inputPath, + outputPath); + }); + } - private static void prepareDatasourceCountryAssociation( - SparkSession spark, - List whitelist, - List allowedtypes, - String inputPath, - String outputPath) { - String whitelisted = ""; - for (String i : whitelist) { - whitelisted += " OR id = '" + i + "'"; - } - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + private static void prepareDatasourceCountryAssociation( + SparkSession spark, + List whitelist, + List allowedtypes, + String inputPath, + String outputPath) { + String whitelisted = ""; + for (String i : whitelist) { + whitelisted += " OR id = '" + i + "'"; + } + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset datasource = - spark.createDataset( - sc.textFile(inputPath + "/datasource") - .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)) - .rdd(), - Encoders.bean(Datasource.class)); + Dataset datasource = spark + .createDataset( + sc + .textFile(inputPath + "/datasource") + .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)) + .rdd(), + Encoders.bean(Datasource.class)); - Dataset relation = - spark.createDataset( - sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + Dataset relation = spark + .createDataset( + sc + .textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); - Dataset organization = - spark.createDataset( - sc.textFile(inputPath + "/organization") - .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)) - .rdd(), - Encoders.bean(Organization.class)); + Dataset organization = spark + .createDataset( + sc + .textFile(inputPath + "/organization") + .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)) + .rdd(), + Encoders.bean(Organization.class)); - datasource.createOrReplaceTempView("datasource"); - relation.createOrReplaceTempView("relation"); - organization.createOrReplaceTempView("organization"); + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); - String query = - "SELECT source dataSourceId, named_struct('classid', country.classid, 'classname', country.classname) country " - + "FROM ( SELECT id " - + " FROM datasource " - + " WHERE (datainfo.deletedbyinference = false " - + whitelisted - + ") " - + getConstraintList("datasourcetype.classid = '", allowedtypes) - + ") d " - + "JOIN ( SELECT source, target " - + " FROM relation " - + " WHERE relclass = '" - + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS - + "' " - + " AND datainfo.deletedbyinference = false ) rel " - + "ON d.id = rel.source " - + "JOIN (SELECT id, country " - + " FROM organization " - + " WHERE datainfo.deletedbyinference = false " - + " AND length(country.classid)>0) o " - + "ON o.id = rel.target"; + String query = "SELECT source dataSourceId, named_struct('classid', country.classid, 'classname', country.classname) country " + + "FROM ( SELECT id " + + " FROM datasource " + + " WHERE (datainfo.deletedbyinference = false " + + whitelisted + + ") " + + getConstraintList("datasourcetype.classid = '", allowedtypes) + + ") d " + + "JOIN ( SELECT source, target " + + " FROM relation " + + " WHERE relclass = '" + + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + + "' " + + " AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source " + + "JOIN (SELECT id, country " + + " FROM organization " + + " WHERE datainfo.deletedbyinference = false " + + " AND length(country.classid)>0) o " + + "ON o.id = rel.target"; - spark.sql(query) - .as(Encoders.bean(DatasourceCountry.class)) - .toJavaRDD() - .map(c -> OBJECT_MAPPER.writeValueAsString(c)) - .saveAsTextFile(outputPath, GzipCodec.class); - } + spark + .sql(query) + .as(Encoders.bean(DatasourceCountry.class)) + .toJavaRDD() + .map(c -> OBJECT_MAPPER.writeValueAsString(c)) + .saveAsTextFile(outputPath, GzipCodec.class); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java index 69f7a59e9..8c29424f2 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java @@ -1,25 +1,26 @@ + package eu.dnetlib.dhp.countrypropagation; import java.io.Serializable; import java.util.ArrayList; public class ResultCountrySet implements Serializable { - private String resultId; - private ArrayList countrySet; + private String resultId; + private ArrayList countrySet; - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public ArrayList getCountrySet() { - return countrySet; - } + public ArrayList getCountrySet() { + return countrySet; + } - public void setCountrySet(ArrayList countrySet) { - this.countrySet = countrySet; - } + public void setCountrySet(ArrayList countrySet) { + this.countrySet = countrySet; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index cd37e79eb..15f9e9b60 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -1,12 +1,11 @@ + package eu.dnetlib.dhp.countrypropagation; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; @@ -16,194 +15,193 @@ import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; public class SparkCountryPropagationJob2 { - private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob2.class); + private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkCountryPropagationJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + SparkCountryPropagationJob2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String datasourcecountrypath = parser.get("preparedInfoPath"); - log.info("preparedInfoPath: {}", datasourcecountrypath); + final String datasourcecountrypath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", datasourcecountrypath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - execPropagation( - spark, - datasourcecountrypath, - inputPath, - outputPath, - resultClazz, - saveGraph); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + execPropagation( + spark, + datasourcecountrypath, + inputPath, + outputPath, + resultClazz, + saveGraph); + }); + } - private static void execPropagation( - SparkSession spark, - String datasourcecountrypath, - String inputPath, - String outputPath, - Class resultClazz, - boolean saveGraph) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + private static void execPropagation( + SparkSession spark, + String datasourcecountrypath, + String inputPath, + String outputPath, + Class resultClazz, + boolean saveGraph) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - // Load file with preprocessed association datasource - country - Dataset datasourcecountryassoc = - readAssocDatasourceCountry(spark, datasourcecountrypath); - // broadcasting the result of the preparation step - Broadcast> broadcast_datasourcecountryassoc = - sc.broadcast(datasourcecountryassoc); + // Load file with preprocessed association datasource - country + Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); + // broadcasting the result of the preparation step + Broadcast> broadcast_datasourcecountryassoc = sc.broadcast(datasourcecountryassoc); - Dataset potentialUpdates = - getPotentialResultToUpdate( - spark, inputPath, resultClazz, broadcast_datasourcecountryassoc) - .as(Encoders.bean(ResultCountrySet.class)); + Dataset potentialUpdates = getPotentialResultToUpdate( + spark, inputPath, resultClazz, broadcast_datasourcecountryassoc) + .as(Encoders.bean(ResultCountrySet.class)); - if (saveGraph) { - updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); - } - } + if (saveGraph) { + updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); + } + } - private static void updateResultTable( - SparkSession spark, - Dataset potentialUpdates, - String inputPath, - Class resultClazz, - String outputPath) { + private static void updateResultTable( + SparkSession spark, + Dataset potentialUpdates, + String inputPath, + Class resultClazz, + String outputPath) { - log.info("Reading Graph table from: {}", inputPath); - Dataset result = readPathEntity(spark, inputPath, resultClazz); + log.info("Reading Graph table from: {}", inputPath); + Dataset result = readPathEntity(spark, inputPath, resultClazz); - Dataset> result_pair = - result.map( - r -> new Tuple2<>(r.getId(), r), - Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); + Dataset> result_pair = result + .map( + r -> new Tuple2<>(r.getId(), r), + Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); - Dataset new_table = - result_pair - .joinWith( - potentialUpdates, - result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), - "left_outer") - .map( - (MapFunction, ResultCountrySet>, R>) - value -> { - R r = value._1()._2(); - Optional potentialNewCountries = - Optional.ofNullable(value._2()); - if (potentialNewCountries.isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : r.getCountry()) { - countries.add(country.getClassid()); - } - Result res = new Result(); - res.setId(r.getId()); - List countryList = new ArrayList<>(); - for (CountrySbs country : - potentialNewCountries - .get() - .getCountrySet()) { - if (!countries.contains(country.getClassid())) { - countryList.add( - getCountry( - country.getClassid(), - country.getClassname())); - } - } - res.setCountry(countryList); - r.mergeFrom(res); - } - return r; - }, - Encoders.bean(resultClazz)); + Dataset new_table = result_pair + .joinWith( + potentialUpdates, + result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), + "left_outer") + .map( + (MapFunction, ResultCountrySet>, R>) value -> { + R r = value._1()._2(); + Optional potentialNewCountries = Optional.ofNullable(value._2()); + if (potentialNewCountries.isPresent()) { + HashSet countries = new HashSet<>(); + for (Qualifier country : r.getCountry()) { + countries.add(country.getClassid()); + } + Result res = new Result(); + res.setId(r.getId()); + List countryList = new ArrayList<>(); + for (CountrySbs country : potentialNewCountries + .get() + .getCountrySet()) { + if (!countries.contains(country.getClassid())) { + countryList + .add( + getCountry( + country.getClassid(), + country.getClassname())); + } + } + res.setCountry(countryList); + r.mergeFrom(res); + } + return r; + }, + Encoders.bean(resultClazz)); - log.info("Saving graph table to path: {}", outputPath); - // log.info("number of saved recordsa: {}", new_table.count()); - new_table.toJSON().write().option("compression", "gzip").text(outputPath); - } + log.info("Saving graph table to path: {}", outputPath); + // log.info("number of saved recordsa: {}", new_table.count()); + new_table.toJSON().write().option("compression", "gzip").text(outputPath); + } - private static Dataset getPotentialResultToUpdate( - SparkSession spark, - String inputPath, - Class resultClazz, - Broadcast> broadcast_datasourcecountryassoc) { + private static Dataset getPotentialResultToUpdate( + SparkSession spark, + String inputPath, + Class resultClazz, + Broadcast> broadcast_datasourcecountryassoc) { - Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.createOrReplaceTempView("result"); - // log.info("number of results: {}", result.count()); - createCfHbforresult(spark); - return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); - } + Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + // log.info("number of results: {}", result.count()); + createCfHbforresult(spark); + return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); + } - private static Dataset countryPropagationAssoc( - SparkSession spark, - Broadcast> broadcast_datasourcecountryassoc) { + private static Dataset countryPropagationAssoc( + SparkSession spark, + Broadcast> broadcast_datasourcecountryassoc) { - Dataset datasource_country = broadcast_datasourcecountryassoc.value(); - datasource_country.createOrReplaceTempView("datasource_country"); - log.info("datasource_country number : {}", datasource_country.count()); + Dataset datasource_country = broadcast_datasourcecountryassoc.value(); + datasource_country.createOrReplaceTempView("datasource_country"); + log.info("datasource_country number : {}", datasource_country.count()); - String query = - "SELECT id resultId, collect_set(country) countrySet " - + "FROM ( SELECT id, country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON cf = dataSourceId " - + "UNION ALL " - + "SELECT id , country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON hb = dataSourceId ) tmp " - + "GROUP BY id"; - Dataset potentialUpdates = spark.sql(query); - // log.info("potential update number : {}", potentialUpdates.count()); - return potentialUpdates; - } + String query = "SELECT id resultId, collect_set(country) countrySet " + + "FROM ( SELECT id, country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON cf = dataSourceId " + + "UNION ALL " + + "SELECT id , country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON hb = dataSourceId ) tmp " + + "GROUP BY id"; + Dataset potentialUpdates = spark.sql(query); + // log.info("potential update number : {}", potentialUpdates.count()); + return potentialUpdates; + } - private static Dataset readAssocDatasourceCountry( - SparkSession spark, String relationPath) { - return spark.read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), - Encoders.bean(DatasourceCountry.class)); - } + private static Dataset readAssocDatasourceCountry( + SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), + Encoders.bean(DatasourceCountry.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java index 048e8ae46..c1644a589 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java @@ -1,40 +1,41 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; public class AutoritativeAuthor { - String name; - String surname; - String fullname; - String orcid; + String name; + String surname; + String fullname; + String orcid; - public String getName() { - return name; - } + public String getName() { + return name; + } - public void setName(String name) { - this.name = name; - } + public void setName(String name) { + this.name = name; + } - public String getSurname() { - return surname; - } + public String getSurname() { + return surname; + } - public void setSurname(String surname) { - this.surname = surname; - } + public void setSurname(String surname) { + this.surname = surname; + } - public String getFullname() { - return fullname; - } + public String getFullname() { + return fullname; + } - public void setFullname(String fullname) { - this.fullname = fullname; - } + public void setFullname(String fullname) { + this.fullname = fullname; + } - public String getOrcid() { - return orcid; - } + public String getOrcid() { + return orcid; + } - public void setOrcid(String orcid) { - this.orcid = orcid; - } + public void setOrcid(String orcid) { + this.orcid = orcid; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index 9bc34eb73..1baec07c5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -1,15 +1,12 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import java.util.Arrays; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -20,115 +17,121 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; + public class PrepareResultOrcidAssociationStep1 { - private static final Logger log = - LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkOrcidToResultFromSemRelJob3.class.getResourceAsStream( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkOrcidToResultFromSemRelJob3.class + .getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - final String resultType = - resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - prepareInfo( - spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo( + spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); + }); + } - private static void prepareInfo( - SparkSession spark, - String inputPath, - String outputPath, - Class resultClazz, - String resultType, - List allowedsemrel) { + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + Class resultClazz, + String resultType, + List allowedsemrel) { - // read the relation table and the table related to the result it is using - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = - spark.createDataset( - sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); - relation.createOrReplaceTempView("relation"); + // read the relation table and the table related to the result it is using + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + org.apache.spark.sql.Dataset relation = spark + .createDataset( + sc + .textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); + relation.createOrReplaceTempView("relation"); - log.info("Reading Graph table from: {}", inputPath + "/" + resultType); - Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + log.info("Reading Graph table from: {}", inputPath + "/" + resultType); + Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); - result.createOrReplaceTempView("result"); + result.createOrReplaceTempView("result"); - getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath + "/" + resultType); - } + getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath + "/" + resultType); + } - private static void getPossibleResultOrcidAssociation( - SparkSession spark, List allowedsemrel, String outputPath) { - String query = - " select target resultId, author authorList" - + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " - + " from ( " - + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " - + " from result " - + " lateral view explode (author) a as MyT " - + " lateral view explode (MyT.pid) p as MyP " - + " where MyP.qualifier.classid = 'ORCID') tmp " - + " group by id) r_t " - + " join (" - + " select source, target " - + " from relation " - + " where datainfo.deletedbyinference = false " - + getConstraintList(" relclass = '", allowedsemrel) - + ") rel_rel " - + " on source = id"; + private static void getPossibleResultOrcidAssociation( + SparkSession spark, List allowedsemrel, String outputPath) { + String query = " select target resultId, author authorList" + + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + + " from ( " + + " select id, MyT.fullname, MyT.name, MyT.surname, MyP.value orcid " + + " from result " + + " lateral view explode (author) a as MyT " + + " lateral view explode (MyT.pid) p as MyP " + + " where MyP.qualifier.classid = 'ORCID') tmp " + + " group by id) r_t " + + " join (" + + " select source, target " + + " from relation " + + " where datainfo.deletedbyinference = false " + + getConstraintList(" relclass = '", allowedsemrel) + + ") rel_rel " + + " on source = id"; - spark.sql(query) - .as(Encoders.bean(ResultOrcidList.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); - // .toJSON() - // .write() - // .mode(SaveMode.Append) - // .option("compression","gzip") - // .text(outputPath) - // ; - } + spark + .sql(query) + .as(Encoders.bean(ResultOrcidList.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + // .toJSON() + // .write() + // .mode(SaveMode.Append) + // .option("compression","gzip") + // .text(outputPath) + // ; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index 658c97f6c..a8380e8b9 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -1,101 +1,107 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; import java.util.HashSet; import java.util.Set; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; import scala.Tuple2; public class PrepareResultOrcidAssociationStep2 { - private static final Logger log = - LoggerFactory.getLogger(PrepareResultOrcidAssociationStep2.class); + private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareResultOrcidAssociationStep2.class.getResourceAsStream( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareResultOrcidAssociationStep2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - mergeInfo(spark, inputPath, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + mergeInfo(spark, inputPath, outputPath); + }); + } - private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { + private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssoc = - readAssocResultOrcidList(spark, inputPath + "/publication") - .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) - .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) - .union(readAssocResultOrcidList(spark, inputPath + "/software")); + Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath + "/publication") + .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) + .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) + .union(readAssocResultOrcidList(spark, inputPath + "/software")); - resultOrcidAssoc - .toJavaRDD() - .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) - .reduceByKey( - (a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } - Set orcid_set = new HashSet<>(); - a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); + resultOrcidAssoc + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) + .reduceByKey( + (a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set orcid_set = new HashSet<>(); + a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); - b.getAuthorList().stream() - .forEach( - aa -> { - if (!orcid_set.contains(aa.getOrcid())) { - a.getAuthorList().add(aa); - orcid_set.add(aa.getOrcid()); - } - }); - return a; - }) - .map(c -> c._2()) - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); - } + b + .getAuthorList() + .stream() + .forEach( + aa -> { + if (!orcid_set.contains(aa.getOrcid())) { + a.getAuthorList().add(aa); + orcid_set.add(aa.getOrcid()); + } + }); + return a; + }) + .map(c -> c._2()) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } - private static Dataset readAssocResultOrcidList( - SparkSession spark, String relationPath) { - return spark.read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), - Encoders.bean(ResultOrcidList.class)); - } + private static Dataset readAssocResultOrcidList( + SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), + Encoders.bean(ResultOrcidList.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java index 9e2bc6e31..54b415d1c 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; import java.io.Serializable; @@ -5,22 +6,22 @@ import java.util.ArrayList; import java.util.List; public class ResultOrcidList implements Serializable { - String resultId; - List authorList = new ArrayList<>(); + String resultId; + List authorList = new ArrayList<>(); - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public List getAuthorList() { - return authorList; - } + public List getAuthorList() { + return authorList; + } - public void setAuthorList(List authorList) { - this.authorList = authorList; - } + public void setAuthorList(List authorList) { + this.authorList = authorList; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index 75527552a..997b58bf2 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -1,15 +1,12 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import java.util.List; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -20,174 +17,181 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Author; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; + public class SparkOrcidToResultFromSemRelJob3 { - private static final Logger log = - LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); + private static final Logger log = LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkOrcidToResultFromSemRelJob3.class.getResourceAsStream( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkOrcidToResultFromSemRelJob3.class + .getResourceAsStream( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String possibleUpdates = parser.get("possibleUpdatesPath"); - log.info("possibleUpdatesPath: {}", possibleUpdates); + final String possibleUpdates = parser.get("possibleUpdatesPath"); + log.info("possibleUpdatesPath: {}", possibleUpdates); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - if (saveGraph) - execPropagation(spark, possibleUpdates, inputPath, outputPath, resultClazz); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + if (saveGraph) + execPropagation(spark, possibleUpdates, inputPath, outputPath, resultClazz); + }); + } - private static void execPropagation( - SparkSession spark, - String possibleUpdatesPath, - String inputPath, - String outputPath, - Class resultClazz) { + private static void execPropagation( + SparkSession spark, + String possibleUpdatesPath, + String inputPath, + String outputPath, + Class resultClazz) { - // read possible updates (resultId and list of possible orcid to add - Dataset possible_updates = - readAssocResultOrcidList(spark, possibleUpdatesPath); - // read the result we have been considering - Dataset result = readPathEntity(spark, inputPath, resultClazz); - // make join result left_outer with possible updates + // read possible updates (resultId and list of possible orcid to add + Dataset possible_updates = readAssocResultOrcidList(spark, possibleUpdatesPath); + // read the result we have been considering + Dataset result = readPathEntity(spark, inputPath, resultClazz); + // make join result left_outer with possible updates - result.joinWith( - possible_updates, - result.col("id").equalTo(possible_updates.col("resultId")), - "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rol = Optional.ofNullable(value._2()); - if (rol.isPresent()) { - List toenrich_author = ret.getAuthor(); - List autoritativeAuthors = - rol.get().getAuthorList(); - for (Author author : toenrich_author) { - if (!containsAllowedPid(author)) { - enrichAuthor(author, autoritativeAuthors); - } - } - } + result + .joinWith( + possible_updates, + result.col("id").equalTo(possible_updates.col("resultId")), + "left_outer") + .map( + value -> { + R ret = value._1(); + Optional rol = Optional.ofNullable(value._2()); + if (rol.isPresent()) { + List toenrich_author = ret.getAuthor(); + List autoritativeAuthors = rol.get().getAuthorList(); + for (Author author : toenrich_author) { + if (!containsAllowedPid(author)) { + enrichAuthor(author, autoritativeAuthors); + } + } + } - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + return ret; + }, + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } - private static Dataset readAssocResultOrcidList( - SparkSession spark, String relationPath) { - return spark.read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), - Encoders.bean(ResultOrcidList.class)); - } + private static Dataset readAssocResultOrcidList( + SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), + Encoders.bean(ResultOrcidList.class)); + } - private static void enrichAuthor(Author a, List au) { - for (AutoritativeAuthor aa : au) { - if (enrichAuthor(aa, a)) { - return; - } - } - } + private static void enrichAuthor(Author a, List au) { + for (AutoritativeAuthor aa : au) { + if (enrichAuthor(aa, a)) { + return; + } + } + } - private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, Author author) { - boolean toaddpid = false; + private static boolean enrichAuthor(AutoritativeAuthor autoritative_author, Author author) { + boolean toaddpid = false; - if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { - if (StringUtils.isNoneEmpty(author.getSurname())) { - if (autoritative_author - .getSurname() - .trim() - .equalsIgnoreCase(author.getSurname().trim())) { + if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) { + if (StringUtils.isNoneEmpty(author.getSurname())) { + if (autoritative_author + .getSurname() + .trim() + .equalsIgnoreCase(author.getSurname().trim())) { - // have the same surname. Check the name - if (StringUtils.isNoneEmpty(autoritative_author.getName())) { - if (StringUtils.isNoneEmpty(author.getName())) { - if (autoritative_author - .getName() - .trim() - .equalsIgnoreCase(author.getName().trim())) { - toaddpid = true; - } - // they could be differently written (i.e. only the initials of the name - // in one of the two - if (autoritative_author - .getName() - .trim() - .substring(0, 0) - .equalsIgnoreCase(author.getName().trim().substring(0, 0))) { - toaddpid = true; - } - } - } - } - } - } - if (toaddpid) { - StructuredProperty p = new StructuredProperty(); - p.setValue(autoritative_author.getOrcid()); - p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID)); - p.setDataInfo( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, - PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - author.addPid(p); - } - return toaddpid; - } + // have the same surname. Check the name + if (StringUtils.isNoneEmpty(autoritative_author.getName())) { + if (StringUtils.isNoneEmpty(author.getName())) { + if (autoritative_author + .getName() + .trim() + .equalsIgnoreCase(author.getName().trim())) { + toaddpid = true; + } + // they could be differently written (i.e. only the initials of the name + // in one of the two + if (autoritative_author + .getName() + .trim() + .substring(0, 0) + .equalsIgnoreCase(author.getName().trim().substring(0, 0))) { + toaddpid = true; + } + } + } + } + } + } + if (toaddpid) { + StructuredProperty p = new StructuredProperty(); + p.setValue(autoritative_author.getOrcid()); + p.setQualifier(getQualifier(PROPAGATION_AUTHOR_PID, PROPAGATION_AUTHOR_PID)); + p + .setDataInfo( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, + PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); + author.addPid(p); + } + return toaddpid; + } - private static boolean containsAllowedPid(Author a) { - for (StructuredProperty pid : a.getPid()) { - if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { - return true; - } - } - return false; - } + private static boolean containsAllowedPid(Author a) { + for (StructuredProperty pid : a.getPid()) { + if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { + return true; + } + } + return false; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index cf970048d..b8579156b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -1,16 +1,13 @@ + package eu.dnetlib.dhp.projecttoresult; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.PropagationConstant.getConstraintList; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.util.Arrays; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -19,134 +16,141 @@ import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import eu.dnetlib.dhp.schema.oaf.Relation; + public class PrepareProjectResultsAssociation { - private static final Logger log = - LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareProjectResultsAssociation.class.getResourceAsStream( - "/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + PrepareProjectResultsAssociation.class + .getResourceAsStream( + "/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String potentialUpdatePath = parser.get("potentialUpdatePath"); - log.info("potentialUpdatePath {}: ", potentialUpdatePath); + final String potentialUpdatePath = parser.get("potentialUpdatePath"); + log.info("potentialUpdatePath {}: ", potentialUpdatePath); - String alreadyLinkedPath = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath: {} ", alreadyLinkedPath); + String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath: {} ", alreadyLinkedPath); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - // removeOutputDir(spark, potentialUpdatePath); - // removeOutputDir(spark, alreadyLinkedPath); - prepareResultProjProjectResults( - spark, - inputPath, - potentialUpdatePath, - alreadyLinkedPath, - allowedsemrel); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + // removeOutputDir(spark, potentialUpdatePath); + // removeOutputDir(spark, alreadyLinkedPath); + prepareResultProjProjectResults( + spark, + inputPath, + potentialUpdatePath, + alreadyLinkedPath, + allowedsemrel); + }); + } - private static void prepareResultProjProjectResults( - SparkSession spark, - String inputPath, - String potentialUpdatePath, - String alreadyLinkedPath, - List allowedsemrel) { - JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset relation = - spark.createDataset( - sc.textFile(inputPath) - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + private static void prepareResultProjProjectResults( + SparkSession spark, + String inputPath, + String potentialUpdatePath, + String alreadyLinkedPath, + List allowedsemrel) { + JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + Dataset relation = spark + .createDataset( + sc + .textFile(inputPath) + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); - relation.createOrReplaceTempView("relation"); + relation.createOrReplaceTempView("relation"); - String query = - "SELECT source, target " - + " FROM relation " - + " WHERE datainfo.deletedbyinference = false " - + " AND relClass = '" - + RELATION_RESULT_PROJECT_REL_CLASS - + "'"; + String query = "SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_RESULT_PROJECT_REL_CLASS + + "'"; - Dataset resproj_relation = spark.sql(query); - resproj_relation.createOrReplaceTempView("resproj_relation"); + Dataset resproj_relation = spark.sql(query); + resproj_relation.createOrReplaceTempView("resproj_relation"); - query = - "SELECT resultId, collect_set(projectId) projectSet " - + "FROM ( " - + "SELECT r1.target resultId, r2.target projectId " - + " FROM (SELECT source, target " - + " FROM relation " - + " WHERE datainfo.deletedbyinference = false " - + getConstraintList(" relClass = '", allowedsemrel) - + " ) r1" - + " JOIN resproj_relation r2 " - + " ON r1.source = r2.source " - + " ) tmp " - + "GROUP BY resultId "; - // query = - // "SELECT projectId, collect_set(resId) resultSet " - // + "FROM (" - // + " SELECT r1.target resId, r2.target projectId " - // + " FROM (SELECT source, target " - // + " FROM relation " - // + " WHERE datainfo.deletedbyinference = false " - // + getConstraintList(" relClass = '", allowedsemrel) - // + ") r1" - // + " JOIN resproj_relation r2 " - // + " ON r1.source = r2.source " - // + " ) tmp " - // + "GROUP BY projectId "; + query = "SELECT resultId, collect_set(projectId) projectSet " + + "FROM ( " + + "SELECT r1.target resultId, r2.target projectId " + + " FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + getConstraintList(" relClass = '", allowedsemrel) + + " ) r1" + + " JOIN resproj_relation r2 " + + " ON r1.source = r2.source " + + " ) tmp " + + "GROUP BY resultId "; + // query = + // "SELECT projectId, collect_set(resId) resultSet " + // + "FROM (" + // + " SELECT r1.target resId, r2.target projectId " + // + " FROM (SELECT source, target " + // + " FROM relation " + // + " WHERE datainfo.deletedbyinference = false " + // + getConstraintList(" relClass = '", allowedsemrel) + // + ") r1" + // + " JOIN resproj_relation r2 " + // + " ON r1.source = r2.source " + // + " ) tmp " + // + "GROUP BY projectId "; - spark.sql(query) - .as(Encoders.bean(ResultProjectSet.class)) - // .toJSON() - // .write() - // .mode(SaveMode.Overwrite) - // .option("compression", "gzip") - // .text(potentialUpdatePath); - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + spark + .sql(query) + .as(Encoders.bean(ResultProjectSet.class)) + // .toJSON() + // .write() + // .mode(SaveMode.Overwrite) + // .option("compression", "gzip") + // .text(potentialUpdatePath); + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(potentialUpdatePath, GzipCodec.class); - query = - "SELECT source resultId, collect_set(target) projectSet " - + "FROM resproj_relation " - + "GROUP BY source"; + query = "SELECT source resultId, collect_set(target) projectSet " + + "FROM resproj_relation " + + "GROUP BY source"; - spark.sql(query) - .as(Encoders.bean(ResultProjectSet.class)) - // .toJSON() - // .write() - // .mode(SaveMode.Overwrite) - // .option("compression", "gzip") - // .text(alreadyLinkedPath); - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); - } + spark + .sql(query) + .as(Encoders.bean(ResultProjectSet.class)) + // .toJSON() + // .write() + // .mode(SaveMode.Overwrite) + // .option("compression", "gzip") + // .text(alreadyLinkedPath); + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java index 183ae1489..1d5280874 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java @@ -1,25 +1,26 @@ + package eu.dnetlib.dhp.projecttoresult; import java.io.Serializable; import java.util.ArrayList; public class ResultProjectSet implements Serializable { - private String resultId; - private ArrayList projectSet; + private String resultId; + private ArrayList projectSet; - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public ArrayList getProjectSet() { - return projectSet; - } + public ArrayList getProjectSet() { + return projectSet; + } - public void setProjectSet(ArrayList project) { - this.projectSet = project; - } + public void setProjectSet(ArrayList project) { + this.projectSet = project; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java index e32242a90..4be072901 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java @@ -1,149 +1,159 @@ + package eu.dnetlib.dhp.projecttoresult; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.util.ArrayList; import java.util.List; import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import eu.dnetlib.dhp.schema.oaf.Relation; + public class SparkResultToProjectThroughSemRelJob3 { - private static final Logger log = - LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkResultToProjectThroughSemRelJob3.class.getResourceAsStream( - "/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + SparkResultToProjectThroughSemRelJob3.class + .getResourceAsStream( + "/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); - final String potentialUpdatePath = parser.get("potentialUpdatePath"); - log.info("potentialUpdatePath {}: ", potentialUpdatePath); + final String potentialUpdatePath = parser.get("potentialUpdatePath"); + log.info("potentialUpdatePath {}: ", potentialUpdatePath); - final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); + final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); - final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); + log.info("saveGraph: {}", saveGraph); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - execPropagation( - spark, outputPath, alreadyLinkedPath, potentialUpdatePath, saveGraph); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + execPropagation( + spark, outputPath, alreadyLinkedPath, potentialUpdatePath, saveGraph); + }); + } - private static void execPropagation( - SparkSession spark, - String outputPath, - String alreadyLinkedPath, - String potentialUpdatePath, - Boolean saveGraph) { + private static void execPropagation( + SparkSession spark, + String outputPath, + String alreadyLinkedPath, + String potentialUpdatePath, + Boolean saveGraph) { - Dataset toaddrelations = - readAssocResultProjects(spark, potentialUpdatePath); - Dataset alreadyLinked = readAssocResultProjects(spark, alreadyLinkedPath); + Dataset toaddrelations = readAssocResultProjects(spark, potentialUpdatePath); + Dataset alreadyLinked = readAssocResultProjects(spark, alreadyLinkedPath); - if (saveGraph) { - getNewRelations(alreadyLinked, toaddrelations) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression", "gzip") - .text(outputPath); - } - } + if (saveGraph) { + getNewRelations(alreadyLinked, toaddrelations) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .text(outputPath); + } + } - private static Dataset getNewRelations( - Dataset alreadyLinked, Dataset toaddrelations) { + private static Dataset getNewRelations( + Dataset alreadyLinked, Dataset toaddrelations) { - return toaddrelations - .joinWith( - alreadyLinked, - toaddrelations.col("resultId").equalTo(alreadyLinked.col("resultId")), - "left_outer") - .flatMap( - value -> { - List new_relations = new ArrayList<>(); - ResultProjectSet potential_update = value._1(); - Optional already_linked = - Optional.ofNullable(value._2()); - if (already_linked.isPresent()) { - already_linked.get().getProjectSet().stream() - .forEach( - (p -> { - if (potential_update - .getProjectSet() - .contains(p)) { - potential_update.getProjectSet().remove(p); - } - })); - } - String resId = potential_update.getResultId(); - potential_update.getProjectSet().stream() - .forEach( - pId -> { - new_relations.add( - getRelation( - resId, - pId, - RELATION_RESULT_PROJECT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - new_relations.add( - getRelation( - pId, - resId, - RELATION_PROJECT_RESULT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - }); - return new_relations.iterator(); - }, - Encoders.bean(Relation.class)); - } + return toaddrelations + .joinWith( + alreadyLinked, + toaddrelations.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") + .flatMap( + value -> { + List new_relations = new ArrayList<>(); + ResultProjectSet potential_update = value._1(); + Optional already_linked = Optional.ofNullable(value._2()); + if (already_linked.isPresent()) { + already_linked + .get() + .getProjectSet() + .stream() + .forEach( + (p -> { + if (potential_update + .getProjectSet() + .contains(p)) { + potential_update.getProjectSet().remove(p); + } + })); + } + String resId = potential_update.getResultId(); + potential_update + .getProjectSet() + .stream() + .forEach( + pId -> { + new_relations + .add( + getRelation( + resId, + pId, + RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + new_relations + .add( + getRelation( + pId, + resId, + RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + }); + return new_relations.iterator(); + }, + Encoders.bean(Relation.class)); + } - private static Dataset readAssocResultProjects( - SparkSession spark, String potentialUpdatePath) { - return spark.read() - .textFile(potentialUpdatePath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultProjectSet.class), - Encoders.bean(ResultProjectSet.class)); - } + private static Dataset readAssocResultProjects( + SparkSession spark, String potentialUpdatePath) { + return spark + .read() + .textFile(potentialUpdatePath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultProjectSet.class), + Encoders.bean(ResultProjectSet.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java index 9a42f3f7e..7d786058a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import java.util.ArrayList; @@ -6,15 +7,15 @@ import java.util.List; public class OrganizationMap extends HashMap> { - public OrganizationMap() { - super(); - } + public OrganizationMap() { + super(); + } - public List get(String key) { + public List get(String key) { - if (super.get(key) == null) { - return new ArrayList<>(); - } - return super.get(key); - } + if (super.get(key) == null) { + return new ArrayList<>(); + } + return super.get(key); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index 9e62f9b4f..fbe598e89 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -1,129 +1,133 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; + public class PrepareResultCommunitySet { - private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareResultCommunitySet.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareResultCommunitySet.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final OrganizationMap organizationMap = - new Gson() - .fromJson( - parser.get("organizationtoresultcommunitymap"), - OrganizationMap.class); - log.info("organizationMap: {}", new Gson().toJson(organizationMap)); + final OrganizationMap organizationMap = new Gson() + .fromJson( + parser.get("organizationtoresultcommunitymap"), + OrganizationMap.class); + log.info("organizationMap: {}", new Gson().toJson(organizationMap)); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - prepareInfo(spark, inputPath, outputPath, organizationMap); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo(spark, inputPath, outputPath, organizationMap); + }); + } - private static void prepareInfo( - SparkSession spark, - String inputPath, - String outputPath, - OrganizationMap organizationMap) { - Dataset relation = readRelations(spark, inputPath); - relation.createOrReplaceTempView("relation"); + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + OrganizationMap organizationMap) { + Dataset relation = readRelations(spark, inputPath); + relation.createOrReplaceTempView("relation"); - String query = - "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " - + "FROM (SELECT source, target " - + " FROM relation " - + " WHERE datainfo.deletedbyinference = false " - + " AND relClass = '" - + RELATION_RESULT_ORGANIZATION_REL_CLASS - + "') result_organization " - + "LEFT JOIN (SELECT source, collect_set(target) org_set " - + " FROM relation " - + " WHERE datainfo.deletedbyinference = false " - + " AND relClass = '" - + RELATION_REPRESENTATIVERESULT_RESULT_CLASS - + "' " - + " GROUP BY source) organization_organization " - + "ON result_organization.target = organization_organization.source "; + String query = "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " + + "FROM (SELECT source, target " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_RESULT_ORGANIZATION_REL_CLASS + + "') result_organization " + + "LEFT JOIN (SELECT source, collect_set(target) org_set " + + " FROM relation " + + " WHERE datainfo.deletedbyinference = false " + + " AND relClass = '" + + RELATION_REPRESENTATIVERESULT_RESULT_CLASS + + "' " + + " GROUP BY source) organization_organization " + + "ON result_organization.target = organization_organization.source "; - org.apache.spark.sql.Dataset result_organizationset = - spark.sql(query).as(Encoders.bean(ResultOrganizations.class)); + org.apache.spark.sql.Dataset result_organizationset = spark + .sql(query) + .as(Encoders.bean(ResultOrganizations.class)); - result_organizationset - .map( - value -> { - String rId = value.getResultId(); - Optional> orgs = Optional.ofNullable(value.getMerges()); - String oTarget = value.getOrgId(); - Set communitySet = new HashSet<>(); - if (organizationMap.containsKey(oTarget)) { - communitySet.addAll(organizationMap.get(oTarget)); - } - if (orgs.isPresent()) - // try{ - for (String oId : orgs.get()) { - if (organizationMap.containsKey(oId)) { - communitySet.addAll(organizationMap.get(oId)); - } - } - // }catch(Exception e){ - // - // } - if (communitySet.size() > 0) { - ResultCommunityList rcl = new ResultCommunityList(); - rcl.setResultId(rId); - ArrayList communityList = new ArrayList<>(); - communityList.addAll(communitySet); - rcl.setCommunityList(communityList); - return rcl; - } - return null; - }, - Encoders.bean(ResultCommunityList.class)) - .filter(r -> r != null) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + result_organizationset + .map( + value -> { + String rId = value.getResultId(); + Optional> orgs = Optional.ofNullable(value.getMerges()); + String oTarget = value.getOrgId(); + Set communitySet = new HashSet<>(); + if (organizationMap.containsKey(oTarget)) { + communitySet.addAll(organizationMap.get(oTarget)); + } + if (orgs.isPresent()) + // try{ + for (String oId : orgs.get()) { + if (organizationMap.containsKey(oId)) { + communitySet.addAll(organizationMap.get(oId)); + } + } + // }catch(Exception e){ + // + // } + if (communitySet.size() > 0) { + ResultCommunityList rcl = new ResultCommunityList(); + rcl.setResultId(rId); + ArrayList communityList = new ArrayList<>(); + communityList.addAll(communitySet); + rcl.setCommunityList(communityList); + return rcl; + } + return null; + }, + Encoders.bean(ResultCommunityList.class)) + .filter(r -> r != null) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java index 50d9a6d7a..e3275745d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java @@ -1,25 +1,26 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import java.io.Serializable; import java.util.ArrayList; public class ResultCommunityList implements Serializable { - private String resultId; - private ArrayList communityList; + private String resultId; + private ArrayList communityList; - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public ArrayList getCommunityList() { - return communityList; - } + public ArrayList getCommunityList() { + return communityList; + } - public void setCommunityList(ArrayList communityList) { - this.communityList = communityList; - } + public void setCommunityList(ArrayList communityList) { + this.communityList = communityList; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java index 53a5fccdf..3ea9d41d6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java @@ -1,34 +1,35 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import java.io.Serializable; import java.util.ArrayList; public class ResultOrganizations implements Serializable { - private String resultId; - private String orgId; - private ArrayList merges; + private String resultId; + private String orgId; + private ArrayList merges; - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public String getOrgId() { - return orgId; - } + public String getOrgId() { + return orgId; + } - public void setOrgId(String orgId) { - this.orgId = orgId; - } + public void setOrgId(String orgId) { + this.orgId = orgId; + } - public ArrayList getMerges() { - return merges; - } + public ArrayList getMerges() { + return merges; + } - public void setMerges(ArrayList merges) { - this.merges = merges; - } + public void setMerges(ArrayList merges) { + this.merges = merges; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index 3c5b0a04c..74931a537 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -1,13 +1,12 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Encoders; @@ -16,111 +15,119 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; + public class SparkResultToCommunityFromOrganizationJob2 { - private static final Logger log = - LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkResultToCommunityFromOrganizationJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkResultToCommunityFromOrganizationJob2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String possibleupdatespath = parser.get("preparedInfoPath"); - log.info("preparedInfoPath: {}", possibleupdatespath); + final String possibleupdatespath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", possibleupdatespath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); + }); + } - private static void execPropagation( - SparkSession spark, - String inputPath, - String outputPath, - Class resultClazz, - String possibleUpdatesPath) { - org.apache.spark.sql.Dataset possibleUpdates = - readResultCommunityList(spark, possibleUpdatesPath); - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + Class resultClazz, + String possibleUpdatesPath) { + org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList( + spark, possibleUpdatesPath); + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.joinWith( - possibleUpdates, - result.col("id").equalTo(possibleUpdates.col("resultId")), - "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if (rcl.isPresent()) { - ArrayList communitySet = rcl.get().getCommunityList(); - List contextList = - ret.getContext().stream() - .map(con -> con.getId()) - .collect(Collectors.toList()); - Result res = new Result(); - res.setId(ret.getId()); - List propagatedContexts = new ArrayList<>(); - for (String cId : communitySet) { - if (!contextList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - propagatedContexts.add(newContext); - } - } - res.setContext(propagatedContexts); - ret.mergeFrom(res); - } - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + result + .joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map( + value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + ArrayList communitySet = rcl.get().getCommunityList(); + List contextList = ret + .getContext() + .stream() + .map(con -> con.getId()) + .collect(Collectors.toList()); + Result res = new Result(); + res.setId(ret.getId()); + List propagatedContexts = new ArrayList<>(); + for (String cId : communitySet) { + if (!contextList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); + propagatedContexts.add(newContext); + } + } + res.setContext(propagatedContexts); + ret.mergeFrom(res); + } + return ret; + }, + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index cbb9b580e..5aef1c370 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -1,17 +1,12 @@ + package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.QueryInformationSystem; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Result; import java.util.Arrays; import java.util.List; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -23,154 +18,158 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.QueryInformationSystem; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; + public class PrepareResultCommunitySetStep1 { - private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep1.class); + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep1.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareResultCommunitySetStep1.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json")); + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareResultCommunitySetStep1.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); - log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); + final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); + log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - final String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); - final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); - log.info("communityIdList: {}", new Gson().toJson(communityIdList)); + final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); + log.info("communityIdList: {}", new Gson().toJson(communityIdList)); - final String resultType = - resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - prepareInfo( - spark, - inputPath, - outputPath, - allowedsemrel, - resultClazz, - resultType, - communityIdList); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + prepareInfo( + spark, + inputPath, + outputPath, + allowedsemrel, + resultClazz, + resultType, + communityIdList); + }); + } - private static void prepareInfo( - SparkSession spark, - String inputPath, - String outputPath, - List allowedsemrel, - Class resultClazz, - String resultType, - List communityIdList) { - // read the relation table and the table related to the result it is using - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = - spark.createDataset( - sc.textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); - relation.createOrReplaceTempView("relation"); + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + List allowedsemrel, + Class resultClazz, + String resultType, + List communityIdList) { + // read the relation table and the table related to the result it is using + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + org.apache.spark.sql.Dataset relation = spark + .createDataset( + sc + .textFile(inputPath + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); + relation.createOrReplaceTempView("relation"); - log.info("Reading Graph table from: {}", inputPath + "/" + resultType); - Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); + log.info("Reading Graph table from: {}", inputPath + "/" + resultType); + Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); - result.createOrReplaceTempView("result"); + result.createOrReplaceTempView("result"); - getPossibleResultcommunityAssociation( - spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); - } + getPossibleResultcommunityAssociation( + spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); + } - private static void getPossibleResultcommunityAssociation( - SparkSession spark, - List allowedsemrel, - String outputPath, - List communityIdList) { + private static void getPossibleResultcommunityAssociation( + SparkSession spark, + List allowedsemrel, + String outputPath, + List communityIdList) { - String communitylist = getConstraintList(" co.id = '", communityIdList); - String semrellist = getConstraintList(" relClass = '", allowedsemrel); + String communitylist = getConstraintList(" co.id = '", communityIdList); + String semrellist = getConstraintList(" relClass = '", allowedsemrel); - /* - associates to each result the set of community contexts they are associated to - select id, collect_set(co.id) community_context " + - " from result " + - " lateral view explode (context) c as co " + - " where datainfo.deletedbyinference = false "+ communitylist + - " group by id + /* + * associates to each result the set of community contexts they are associated to select id, collect_set(co.id) + * community_context " + " from result " + " lateral view explode (context) c as co " + + * " where datainfo.deletedbyinference = false "+ communitylist + " group by id associates to each target + * of a relation with allowed semantics the set of community context it could possibly inherit from the source + * of the relation + */ + String query = "Select target resultId, community_context " + + "from (select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false " + + communitylist + + " group by id) p " + + "JOIN " + + "(select source, target " + + "from relation " + + "where datainfo.deletedbyinference = false " + + semrellist + + ") r " + + "ON p.id = r.source"; - associates to each target of a relation with allowed semantics the set of community context it could possibly - inherit from the source of the relation - */ - String query = - "Select target resultId, community_context " - + "from (select id, collect_set(co.id) community_context " - + " from result " - + " lateral view explode (context) c as co " - + " where datainfo.deletedbyinference = false " - + communitylist - + " group by id) p " - + "JOIN " - + "(select source, target " - + "from relation " - + "where datainfo.deletedbyinference = false " - + semrellist - + ") r " - + "ON p.id = r.source"; + org.apache.spark.sql.Dataset result_context = spark.sql(query); + result_context.createOrReplaceTempView("result_context"); - org.apache.spark.sql.Dataset result_context = spark.sql(query); - result_context.createOrReplaceTempView("result_context"); + // ( target, (mes, dh-ch-, ni)) + /* + * a dataset for example could be linked to more than one publication. For each publication linked to that + * dataset the previous query will produce a row: targetId set of community context the target could possibly + * inherit with the following query there will be a single row for each result linked to more than one result of + * the result type currently being used + */ + query = "select resultId , collect_set(co) communityList " + + "from result_context " + + "lateral view explode (community_context) c as co " + + "where length(co) > 0 " + + "group by resultId"; - // ( target, (mes, dh-ch-, ni)) - /* - a dataset for example could be linked to more than one publication. For each publication linked to that dataset - the previous query will produce a row: targetId set of community context the target could possibly inherit - with the following query there will be a single row for each result linked to more than one result of the result type - currently being used - */ - query = - "select resultId , collect_set(co) communityList " - + "from result_context " - + "lateral view explode (community_context) c as co " - + "where length(co) > 0 " - + "group by resultId"; - - spark.sql(query) - .as(Encoders.bean(ResultCommunityList.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); - } + spark + .sql(query) + .as(Encoders.bean(ResultCommunityList.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java index 3579db9e6..cbd7e5e50 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -1,13 +1,12 @@ + package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import java.util.HashSet; import java.util.Set; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -16,89 +15,98 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import scala.Tuple2; public class PrepareResultCommunitySetStep2 { - private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep2.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareResultCommunitySetStep2.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + PrepareResultCommunitySetStep2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - SparkConf conf = new SparkConf(); + SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - mergeInfo(spark, inputPath, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + mergeInfo(spark, inputPath, outputPath); + }); + } - private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { + private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssocCommunityList = - readResultCommunityList(spark, inputPath + "/publication") - .union(readResultCommunityList(spark, inputPath + "/dataset")) - .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) - .union(readResultCommunityList(spark, inputPath + "/software")); + Dataset resultOrcidAssocCommunityList = readResultCommunityList( + spark, inputPath + "/publication") + .union(readResultCommunityList(spark, inputPath + "/dataset")) + .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) + .union(readResultCommunityList(spark, inputPath + "/software")); - resultOrcidAssocCommunityList - .toJavaRDD() - .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) - .reduceByKey( - (a, b) -> { - if (a == null) { - return b; - } - if (b == null) { - return a; - } - Set community_set = new HashSet<>(); + resultOrcidAssocCommunityList + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getResultId(), r)) + .reduceByKey( + (a, b) -> { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + Set community_set = new HashSet<>(); - a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); + a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); - b.getCommunityList().stream() - .forEach( - aa -> { - if (!community_set.contains(aa)) { - a.getCommunityList().add(aa); - community_set.add(aa); - } - }); - return a; - }) - .map(c -> c._2()) - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); - } + b + .getCommunityList() + .stream() + .forEach( + aa -> { + if (!community_set.contains(aa)) { + a.getCommunityList().add(aa); + community_set.add(aa); + } + }); + return a; + }) + .map(c -> c._2()) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } - private static Dataset readResultCommunityList( - SparkSession spark, String relationPath) { - return spark.read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), - Encoders.bean(ResultCommunityList.class)); - } + private static Dataset readResultCommunityList( + SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), + Encoders.bean(ResultCommunityList.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java index 4e72fac27..b513ddd79 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java @@ -1,15 +1,12 @@ + package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.ximpleware.extended.xpath.parser; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Encoders; @@ -18,119 +15,130 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ximpleware.extended.xpath.parser; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.schema.oaf.*; + public class SparkResultToCommunityThroughSemRelJob4 { - private static final Logger log = - LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob4.class); + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob4.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkResultToCommunityThroughSemRelJob4.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + SparkResultToCommunityThroughSemRelJob4.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String preparedInfoPath = parser.get("preparedInfoPath"); - log.info("preparedInfoPath: {}", preparedInfoPath); + final String preparedInfoPath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", preparedInfoPath); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - if (saveGraph) { - execPropagation( - spark, inputPath, outputPath, preparedInfoPath, resultClazz); - } - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + if (saveGraph) { + execPropagation( + spark, inputPath, outputPath, preparedInfoPath, resultClazz); + } + }); + } - private static void execPropagation( - SparkSession spark, - String inputPath, - String outputPath, - String preparedInfoPath, - Class resultClazz) { + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + String preparedInfoPath, + Class resultClazz) { - org.apache.spark.sql.Dataset possibleUpdates = - readResultCommunityList(spark, preparedInfoPath); - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList( + spark, preparedInfoPath); + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.joinWith( - possibleUpdates, - result.col("id").equalTo(possibleUpdates.col("resultId")), - "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if (rcl.isPresent()) { - Set context_set = new HashSet<>(); - ret.getContext().stream().forEach(c -> context_set.add(c.getId())); - List contextList = - rcl.get().getCommunityList().stream() - .map( - c -> { - if (!context_set.contains(c)) { - Context newContext = new Context(); - newContext.setId(c); - newContext.setDataInfo( - Arrays.asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - return newContext; - } - return null; - }) - .filter(c -> c != null) - .collect(Collectors.toList()); - Result r = new Result(); - r.setId(ret.getId()); - r.setContext(contextList); - ret.mergeFrom(r); - } + result + .joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map( + value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + Set context_set = new HashSet<>(); + ret.getContext().stream().forEach(c -> context_set.add(c.getId())); + List contextList = rcl + .get() + .getCommunityList() + .stream() + .map( + c -> { + if (!context_set.contains(c)) { + Context newContext = new Context(); + newContext.setId(c); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + return newContext; + } + return null; + }) + .filter(c -> c != null) + .collect(Collectors.toList()); + Result r = new Result(); + r.setId(ret.getId()); + r.setContext(contextList); + ret.mergeFrom(r); + } - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + return ret; + }, + Encoders.bean(resultClazz)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java index e13e2a68d..e6b13dfa4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java @@ -1,25 +1,26 @@ + package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import java.io.Serializable; public class DatasourceOrganization implements Serializable { - private String datasourceId; - private String organizationId; + private String datasourceId; + private String organizationId; - public String getDatasourceId() { - return datasourceId; - } + public String getDatasourceId() { + return datasourceId; + } - public void setDatasourceId(String datasourceId) { - this.datasourceId = datasourceId; - } + public void setDatasourceId(String datasourceId) { + this.datasourceId = datasourceId; + } - public String getOrganizationId() { - return organizationId; - } + public String getOrganizationId() { + return organizationId; + } - public void setOrganizationId(String organizationId) { - this.organizationId = organizationId; - } + public void setOrganizationId(String organizationId) { + this.organizationId = organizationId; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 095f476cf..02faf0086 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -1,13 +1,9 @@ + package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Datasource; -import eu.dnetlib.dhp.schema.oaf.Organization; -import eu.dnetlib.dhp.schema.oaf.Relation; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -18,121 +14,131 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Datasource; +import eu.dnetlib.dhp.schema.oaf.Organization; +import eu.dnetlib.dhp.schema.oaf.Relation; + public class PrepareResultInstRepoAssociation { - private static final Logger log = - LoggerFactory.getLogger(PrepareResultInstRepoAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareResultInstRepoAssociation.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - PrepareResultInstRepoAssociation.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + PrepareResultInstRepoAssociation.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String datasourceOrganizationPath = parser.get("datasourceOrganizationPath"); - log.info("datasourceOrganizationPath {}: ", datasourceOrganizationPath); + final String datasourceOrganizationPath = parser.get("datasourceOrganizationPath"); + log.info("datasourceOrganizationPath {}: ", datasourceOrganizationPath); - final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); + final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - readNeededResources(spark, inputPath); - prepareDatasourceOrganizationAssociations( - spark, datasourceOrganizationPath, alreadyLinkedPath); - prepareAlreadyLinkedAssociation(spark, alreadyLinkedPath); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + readNeededResources(spark, inputPath); + prepareDatasourceOrganizationAssociations( + spark, datasourceOrganizationPath, alreadyLinkedPath); + prepareAlreadyLinkedAssociation(spark, alreadyLinkedPath); + }); + } - private static void prepareAlreadyLinkedAssociation( - SparkSession spark, String alreadyLinkedPath) { - String query = - "Select source resultId, collect_set(target) organizationSet " - + "from relation " - + "where datainfo.deletedbyinference = false " - + "and relClass = '" - + RELATION_RESULT_ORGANIZATION_REL_CLASS - + "' " - + "group by source"; + private static void prepareAlreadyLinkedAssociation( + SparkSession spark, String alreadyLinkedPath) { + String query = "Select source resultId, collect_set(target) organizationSet " + + "from relation " + + "where datainfo.deletedbyinference = false " + + "and relClass = '" + + RELATION_RESULT_ORGANIZATION_REL_CLASS + + "' " + + "group by source"; - spark.sql(query) - .as(Encoders.bean(ResultOrganizationSet.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); - } + spark + .sql(query) + .as(Encoders.bean(ResultOrganizationSet.class)) + .toJavaRDD() + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + } - private static void readNeededResources(SparkSession spark, String inputPath) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + private static void readNeededResources(SparkSession spark, String inputPath) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset datasource = - spark.createDataset( - sc.textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)) - .rdd(), - Encoders.bean(Datasource.class)); + org.apache.spark.sql.Dataset datasource = spark + .createDataset( + sc + .textFile(inputPath + "/datasource") + .map(item -> new ObjectMapper().readValue(item, Datasource.class)) + .rdd(), + Encoders.bean(Datasource.class)); - org.apache.spark.sql.Dataset relation = - spark.createDataset( - sc.textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + org.apache.spark.sql.Dataset relation = spark + .createDataset( + sc + .textFile(inputPath + "/relation") + .map(item -> new ObjectMapper().readValue(item, Relation.class)) + .rdd(), + Encoders.bean(Relation.class)); - org.apache.spark.sql.Dataset organization = - spark.createDataset( - sc.textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)) - .rdd(), - Encoders.bean(Organization.class)); + org.apache.spark.sql.Dataset organization = spark + .createDataset( + sc + .textFile(inputPath + "/organization") + .map(item -> new ObjectMapper().readValue(item, Organization.class)) + .rdd(), + Encoders.bean(Organization.class)); - datasource.createOrReplaceTempView("datasource"); - relation.createOrReplaceTempView("relation"); - organization.createOrReplaceTempView("organization"); - } + datasource.createOrReplaceTempView("datasource"); + relation.createOrReplaceTempView("relation"); + organization.createOrReplaceTempView("organization"); + } - private static void prepareDatasourceOrganizationAssociations( - SparkSession spark, String datasourceOrganizationPath, String alreadyLinkedPath) { + private static void prepareDatasourceOrganizationAssociations( + SparkSession spark, String datasourceOrganizationPath, String alreadyLinkedPath) { - String query = - "SELECT source datasourceId, target organizationId " - + "FROM ( SELECT id " - + "FROM datasource " - + "WHERE datasourcetype.classid = '" - + INSTITUTIONAL_REPO_TYPE - + "' " - + "AND datainfo.deletedbyinference = false ) d " - + "JOIN ( SELECT source, target " - + "FROM relation " - + "WHERE relclass = '" - + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS - + "' " - + "AND datainfo.deletedbyinference = false ) rel " - + "ON d.id = rel.source "; + String query = "SELECT source datasourceId, target organizationId " + + "FROM ( SELECT id " + + "FROM datasource " + + "WHERE datasourcetype.classid = '" + + INSTITUTIONAL_REPO_TYPE + + "' " + + "AND datainfo.deletedbyinference = false ) d " + + "JOIN ( SELECT source, target " + + "FROM relation " + + "WHERE relclass = '" + + RELATION_DATASOURCE_ORGANIZATION_REL_CLASS + + "' " + + "AND datainfo.deletedbyinference = false ) rel " + + "ON d.id = rel.source "; - spark.sql(query) - .as(Encoders.bean(DatasourceOrganization.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(datasourceOrganizationPath); - } + spark + .sql(query) + .as(Encoders.bean(DatasourceOrganization.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(datasourceOrganizationPath); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java index bad581c1d..3bce14cdb 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java @@ -1,25 +1,26 @@ + package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import java.io.Serializable; import java.util.ArrayList; public class ResultOrganizationSet implements Serializable { - private String resultId; - private ArrayList organizationSet; + private String resultId; + private ArrayList organizationSet; - public String getResultId() { - return resultId; - } + public String getResultId() { + return resultId; + } - public void setResultId(String resultId) { - this.resultId = resultId; - } + public void setResultId(String resultId) { + this.resultId = resultId; + } - public ArrayList getOrganizationSet() { - return organizationSet; - } + public ArrayList getOrganizationSet() { + return organizationSet; + } - public void setOrganizationSet(ArrayList organizationSet) { - this.organizationSet = organizationSet; - } + public void setOrganizationSet(ArrayList organizationSet) { + this.organizationSet = organizationSet; + } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index db8b99ac7..6bdfa36dd 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -1,12 +1,11 @@ + package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; @@ -16,277 +15,279 @@ import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; public class SparkResultToOrganizationFromIstRepoJob2 { - private static final Logger log = - LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); + private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = - IOUtils.toString( - SparkResultToOrganizationFromIstRepoJob2.class.getResourceAsStream( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + SparkResultToOrganizationFromIstRepoJob2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String datasourceorganization = parser.get("datasourceOrganizationPath"); - log.info("datasourceOrganizationPath: {}", datasourceorganization); + final String datasourceorganization = parser.get("datasourceOrganizationPath"); + log.info("datasourceOrganizationPath: {}", datasourceorganization); - final String alreadylinked = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath: {}", alreadylinked); + final String alreadylinked = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath: {}", alreadylinked); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); - final String resultType = - resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); - final Boolean writeUpdates = - Optional.ofNullable(parser.get("writeUpdate")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("writeUpdate: {}", writeUpdates); + final Boolean writeUpdates = Optional + .ofNullable(parser.get("writeUpdate")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = - Optional.ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); - Class resultClazz = - (Class) Class.forName(resultClassName); + Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - execPropagation( - spark, - datasourceorganization, - alreadylinked, - inputPath, - outputPath, - resultClazz, - resultType, - writeUpdates, - saveGraph); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + execPropagation( + spark, + datasourceorganization, + alreadylinked, + inputPath, + outputPath, + resultClazz, + resultType, + writeUpdates, + saveGraph); + }); + } - private static void execPropagation( - SparkSession spark, - String datasourceorganization, - String alreadylinked, - String inputPath, - String outputPath, - Class resultClazz, - String resultType, - Boolean writeUpdates, - Boolean saveGraph) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + private static void execPropagation( + SparkSession spark, + String datasourceorganization, + String alreadylinked, + String inputPath, + String outputPath, + Class resultClazz, + String resultType, + Boolean writeUpdates, + Boolean saveGraph) { + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset datasourceorganizationassoc = - readAssocDatasourceOrganization(spark, datasourceorganization); + org.apache.spark.sql.Dataset datasourceorganizationassoc = readAssocDatasourceOrganization( + spark, datasourceorganization); - // broadcasting the result of the preparation step - Broadcast> - broadcast_datasourceorganizationassoc = sc.broadcast(datasourceorganizationassoc); + // broadcasting the result of the preparation step + Broadcast> broadcast_datasourceorganizationassoc = sc + .broadcast(datasourceorganizationassoc); - org.apache.spark.sql.Dataset potentialUpdates = - getPotentialRelations( - spark, - inputPath, - resultClazz, - broadcast_datasourceorganizationassoc) - .as(Encoders.bean(ResultOrganizationSet.class)); + org.apache.spark.sql.Dataset potentialUpdates = getPotentialRelations( + spark, + inputPath, + resultClazz, + broadcast_datasourceorganizationassoc) + .as(Encoders.bean(ResultOrganizationSet.class)); - if (writeUpdates) { - createUpdateForRelationWrite(potentialUpdates, outputPath + "/" + resultType); - } + if (writeUpdates) { + createUpdateForRelationWrite(potentialUpdates, outputPath + "/" + resultType); + } - if (saveGraph) { - getNewRelations( - spark.read() - .textFile(alreadylinked) - .map( - value -> - OBJECT_MAPPER.readValue( - value, ResultOrganizationSet.class), - Encoders.bean(ResultOrganizationSet.class)), - potentialUpdates) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression", "gzip") - .text(outputPath); - } - } + if (saveGraph) { + getNewRelations( + spark + .read() + .textFile(alreadylinked) + .map( + value -> OBJECT_MAPPER + .readValue( + value, ResultOrganizationSet.class), + Encoders.bean(ResultOrganizationSet.class)), + potentialUpdates) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .text(outputPath); + } + } - private static Dataset getNewRelations( - Dataset alreadyLinked, - Dataset potentialUpdates) { + private static Dataset getNewRelations( + Dataset alreadyLinked, + Dataset potentialUpdates) { - return potentialUpdates - .joinWith( - alreadyLinked, - potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), - "left_outer") - .flatMap( - (FlatMapFunction< - Tuple2, - Relation>) - value -> { - List new_relations = new ArrayList<>(); - ResultOrganizationSet potential_update = value._1(); - Optional already_linked = - Optional.ofNullable(value._2()); - List organization_list = - potential_update.getOrganizationSet(); - if (already_linked.isPresent()) { - already_linked.get().getOrganizationSet().stream() - .forEach( - rId -> { - if (organization_list.contains(rId)) { - organization_list.remove(rId); - } - }); - } - String resultId = potential_update.getResultId(); - organization_list.stream() - .forEach( - orgId -> { - new_relations.add( - getRelation( - orgId, - resultId, - RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - new_relations.add( - getRelation( - resultId, - orgId, - RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - }); - return new_relations.iterator(); - }, - Encoders.bean(Relation.class)); - } + return potentialUpdates + .joinWith( + alreadyLinked, + potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") + .flatMap( + (FlatMapFunction, Relation>) value -> { + List new_relations = new ArrayList<>(); + ResultOrganizationSet potential_update = value._1(); + Optional already_linked = Optional.ofNullable(value._2()); + List organization_list = potential_update.getOrganizationSet(); + if (already_linked.isPresent()) { + already_linked + .get() + .getOrganizationSet() + .stream() + .forEach( + rId -> { + if (organization_list.contains(rId)) { + organization_list.remove(rId); + } + }); + } + String resultId = potential_update.getResultId(); + organization_list + .stream() + .forEach( + orgId -> { + new_relations + .add( + getRelation( + orgId, + resultId, + RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + new_relations + .add( + getRelation( + resultId, + orgId, + RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + }); + return new_relations.iterator(); + }, + Encoders.bean(Relation.class)); + } - private static - org.apache.spark.sql.Dataset getPotentialRelations( - SparkSession spark, - String inputPath, - Class resultClazz, - Broadcast> - broadcast_datasourceorganizationassoc) { - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.createOrReplaceTempView("result"); - createCfHbforresult(spark); + private static org.apache.spark.sql.Dataset getPotentialRelations( + SparkSession spark, + String inputPath, + Class resultClazz, + Broadcast> broadcast_datasourceorganizationassoc) { + org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + createCfHbforresult(spark); - return organizationPropagationAssoc(spark, broadcast_datasourceorganizationassoc); - } + return organizationPropagationAssoc(spark, broadcast_datasourceorganizationassoc); + } - private static org.apache.spark.sql.Dataset - readAssocDatasourceOrganization( - SparkSession spark, String datasourcecountryorganization) { - return spark.read() - .textFile(datasourcecountryorganization) - .map( - value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), - Encoders.bean(DatasourceOrganization.class)); - } + private static org.apache.spark.sql.Dataset readAssocDatasourceOrganization( + SparkSession spark, String datasourcecountryorganization) { + return spark + .read() + .textFile(datasourcecountryorganization) + .map( + value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), + Encoders.bean(DatasourceOrganization.class)); + } - private static void createUpdateForRelationWrite( - Dataset toupdaterelation, String outputPath) { - toupdaterelation - .flatMap( - s -> { - List relationList = new ArrayList<>(); - List orgs = s.getOrganizationSet(); - String resId = s.getResultId(); - for (String org : orgs) { - relationList.add( - getRelation( - org, - resId, - RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList.add( - getRelation( - resId, - org, - RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - } - return relationList.iterator(); - }, - Encoders.bean(Relation.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } + private static void createUpdateForRelationWrite( + Dataset toupdaterelation, String outputPath) { + toupdaterelation + .flatMap( + s -> { + List relationList = new ArrayList<>(); + List orgs = s.getOrganizationSet(); + String resId = s.getResultId(); + for (String org : orgs) { + relationList + .add( + getRelation( + org, + resId, + RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + relationList + .add( + getRelation( + resId, + org, + RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + } + return relationList.iterator(); + }, + Encoders.bean(Relation.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } - private static org.apache.spark.sql.Dataset organizationPropagationAssoc( - SparkSession spark, - Broadcast> - broadcast_datasourceorganizationassoc) { - org.apache.spark.sql.Dataset datasourceorganization = - broadcast_datasourceorganizationassoc.value(); - datasourceorganization.createOrReplaceTempView("rels"); - String query = - "SELECT id resultId, collect_set(organizationId) organizationSet " - + "FROM ( SELECT id, organizationId " - + "FROM rels " - + "JOIN cfhb " - + " ON cf = datasourceId " - + "UNION ALL " - + "SELECT id , organizationId " - + "FROM rels " - + "JOIN cfhb " - + " ON hb = datasourceId ) tmp " - + "GROUP BY id"; - return spark.sql(query).as(Encoders.bean(ResultOrganizationSet.class)); - } + private static org.apache.spark.sql.Dataset organizationPropagationAssoc( + SparkSession spark, + Broadcast> broadcast_datasourceorganizationassoc) { + org.apache.spark.sql.Dataset datasourceorganization = broadcast_datasourceorganizationassoc + .value(); + datasourceorganization.createOrReplaceTempView("rels"); + String query = "SELECT id resultId, collect_set(organizationId) organizationSet " + + "FROM ( SELECT id, organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON cf = datasourceId " + + "UNION ALL " + + "SELECT id , organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON hb = datasourceId ) tmp " + + "GROUP BY id"; + return spark.sql(query).as(Encoders.bean(ResultOrganizationSet.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 675bb3917..3bc499233 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -1,13 +1,12 @@ + package eu.dnetlib.dhp.countrypropagation; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Country; -import eu.dnetlib.dhp.schema.oaf.Software; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; import java.util.List; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -19,231 +18,252 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Country; +import eu.dnetlib.dhp.schema.oaf.Software; import scala.Tuple2; public class CountryPropagationJobTest { - private static final Logger log = LoggerFactory.getLogger(CountryPropagationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(CountryPropagationJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = CountryPropagationJobTest.class.getClassLoader(); + private static final ClassLoader cl = CountryPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(CountryPropagationJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(CountryPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName(CountryPropagationJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(CountryPropagationJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(CountryPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(CountryPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - @Test - public void testCountryPropagationSoftware() throws Exception { - SparkCountryPropagationJob2.main( - new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/countrypropagation/sample/software") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", - workingDir.toString() + "/software", - "-preparedInfoPath", - getClass() - .getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo") - .getPath(), - }); + @Test + public void testCountryPropagationSoftware() throws Exception { + SparkCountryPropagationJob2 + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/sample/software") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath", + workingDir.toString() + "/software", + "-preparedInfoPath", + getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/software") - .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/software") + .map(item -> OBJECT_MAPPER.readValue(item, Software.class)); - // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); - Assertions.assertEquals(10, tmp.count()); + Assertions.assertEquals(10, tmp.count()); - Dataset verificationDs = - spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Software.class)); - Assertions.assertEquals(6, verificationDs.filter("size(country) > 0").count()); - Assertions.assertEquals(3, verificationDs.filter("size(country) = 1").count()); - Assertions.assertEquals(3, verificationDs.filter("size(country) = 2").count()); - Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); + Assertions.assertEquals(6, verificationDs.filter("size(country) > 0").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 1").count()); + Assertions.assertEquals(3, verificationDs.filter("size(country) = 2").count()); + Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); - Dataset countryExploded = - verificationDs - .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) - .map(c -> c.getClassid(), Encoders.STRING()); + Dataset countryExploded = verificationDs + .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) + .map(c -> c.getClassid(), Encoders.STRING()); - Assertions.assertEquals(9, countryExploded.count()); + Assertions.assertEquals(9, countryExploded.count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'FR'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'TR'").count()); - Assertions.assertEquals(2, countryExploded.filter("value = 'IT'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'US'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'MX'").count()); - Assertions.assertEquals(1, countryExploded.filter("value = 'CH'").count()); - Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'FR'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'TR'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'IT'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'US'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'MX'").count()); + Assertions.assertEquals(1, countryExploded.filter("value = 'CH'").count()); + Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); - Dataset> countryExplodedWithCountryclassid = - verificationDs.flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream() - .forEach( - c -> - prova.add( - new Tuple2<>( - row.getId(), c.getClassid()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + Dataset> countryExplodedWithCountryclassid = verificationDs + .flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), c.getClassid()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); + Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); - countryExplodedWithCountryclassid.show(false); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'FR' ") - .count()); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'TR' ") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'IT' or _2 = 'MX') ") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'IT' or _2 = 'US') ") - .count()); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'JP'") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassid - .filter( - "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'CH' or _2 = 'JP') ") - .count()); + countryExplodedWithCountryclassid.show(false); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'FR' ") + .count()); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'TR' ") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'IT' or _2 = 'MX') ") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'IT' or _2 = 'US') ") + .count()); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'JP'") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassid + .filter( + "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'CH' or _2 = 'JP') ") + .count()); - Dataset> countryExplodedWithCountryclassname = - verificationDs.flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream() - .forEach( - c -> - prova.add( - new Tuple2<>( - row.getId(), - c.getClassname()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + Dataset> countryExplodedWithCountryclassname = verificationDs + .flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c.getClassname()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - countryExplodedWithCountryclassname.show(false); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'France' ") - .count()); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'Turkey' ") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'Italy' or _2 = 'Mexico') ") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'Italy' or _2 = 'United States') ") - .count()); - Assertions.assertEquals( - 1, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'Japan' ") - .count()); - Assertions.assertEquals( - 2, - countryExplodedWithCountryclassname - .filter( - "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'Switzerland' or _2 = 'Japan') ") - .count()); + countryExplodedWithCountryclassname.show(false); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od______1582::6e7a9b21a2feef45673890432af34244' and _2 = 'France' ") + .count()); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523' and _2 = 'Turkey' ") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od______1106::2b7ca9726230be8e862be224fd463ac4' and (_2 = 'Italy' or _2 = 'Mexico') ") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od_______935::46a0ad9964171c3dd13373f5427b9a1c' and (_2 = 'Italy' or _2 = 'United States') ") + .count()); + Assertions + .assertEquals( + 1, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and _2 = 'Japan' ") + .count()); + Assertions + .assertEquals( + 2, + countryExplodedWithCountryclassname + .filter( + "_1 = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6' and (_2 = 'Switzerland' or _2 = 'Japan') ") + .count()); - Dataset> countryExplodedWithCountryProvenance = - verificationDs.flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list.stream() - .forEach( - c -> - prova.add( - new Tuple2<>( - row.getId(), - c.getDataInfo() - .getInferenceprovenance()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + Dataset> countryExplodedWithCountryProvenance = verificationDs + .flatMap( + row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c + .getDataInfo() + .getInferenceprovenance()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); - Assertions.assertEquals( - 7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); - } + Assertions + .assertEquals( + 7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index 4292f3b05..d18acd550 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -1,10 +1,10 @@ + package eu.dnetlib.dhp.orcidtoresultfromsemrel; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Dataset; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -19,234 +19,242 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Dataset; + public class OrcidPropagationJobTest { - private static final Logger log = LoggerFactory.getLogger(OrcidPropagationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(OrcidPropagationJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = OrcidPropagationJobTest.class.getClassLoader(); + private static final ClassLoader cl = OrcidPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(OrcidPropagationJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(OrcidPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName(OrcidPropagationJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(OrcidPropagationJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(OrcidPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - @Test - public void noUpdateTest() throws Exception { - SparkOrcidToResultFromSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-possibleUpdatesPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") - .getPath() - }); + @Test + public void noUpdateTest() throws Exception { + SparkOrcidToResultFromSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); - Assertions.assertEquals(10, tmp.count()); + Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - verificationDataset.createOrReplaceTempView("dataset"); + verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id " - + "from dataset " - + "lateral view explode(author) a as MyT " - + "lateral view explode(MyT.pid) p as MyP " - + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = "select id " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; - Assertions.assertEquals(0, spark.sql(query).count()); - } + Assertions.assertEquals(0, spark.sql(query).count()); + } - @Test - public void oneUpdateTest() throws Exception { - SparkOrcidToResultFromSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-possibleUpdatesPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") - .getPath() - }); + @Test + public void oneUpdateTest() throws Exception { + SparkOrcidToResultFromSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); + // tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s)); - Assertions.assertEquals(10, tmp.count()); + Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - verificationDataset.createOrReplaceTempView("dataset"); + verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " - + "from dataset " - + "lateral view explode(author) a as MyT " - + "lateral view explode(MyT.pid) p as MyP " - + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; - org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); + org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); - Assertions.assertEquals(1, propagatedAuthors.count()); + Assertions.assertEquals(1, propagatedAuthors.count()); - Assertions.assertEquals( - 1, - propagatedAuthors - .filter( - "id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " - + "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'") - .count()); + Assertions + .assertEquals( + 1, + propagatedAuthors + .filter( + "id = '50|dedup_wf_001::95b033c0c3961f6a1cdcd41a99a9632e' " + + "and name = 'Vajinder' and surname = 'Kumar' and pidType = 'ORCID'") + .count()); - Assertions.assertEquals(1, propagatedAuthors.filter("pid = '0000-0002-8825-3517'").count()); - } + Assertions.assertEquals(1, propagatedAuthors.filter("pid = '0000-0002-8825-3517'").count()); + } - @Test - public void twoUpdatesTest() throws Exception { - SparkOrcidToResultFromSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-possibleUpdatesPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") - .getPath() - }); + @Test + public void twoUpdatesTest() throws Exception { + SparkOrcidToResultFromSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-possibleUpdatesPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath() + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - Assertions.assertEquals(10, tmp.count()); + Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - verificationDataset.createOrReplaceTempView("dataset"); + verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " - + "from dataset " - + "lateral view explode(author) a as MyT " - + "lateral view explode(MyT.pid) p as MyP " - + "where MyP.datainfo.inferenceprovenance = 'propagation'"; + String query = "select id, MyT.name name, MyT.surname surname, MyP.value pid, MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP " + + "where MyP.datainfo.inferenceprovenance = 'propagation'"; - org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); + org.apache.spark.sql.Dataset propagatedAuthors = spark.sql(query); - Assertions.assertEquals(2, propagatedAuthors.count()); + Assertions.assertEquals(2, propagatedAuthors.count()); - Assertions.assertEquals( - 1, propagatedAuthors.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); - Assertions.assertEquals( - 1, propagatedAuthors.filter("name = 'Ruediger' and surname = 'Beckhaus'").count()); + Assertions + .assertEquals( + 1, propagatedAuthors.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions + .assertEquals( + 1, propagatedAuthors.filter("name = 'Ruediger' and surname = 'Beckhaus'").count()); - query = - "select id, MyT.name name, MyT.surname surname, MyP.value pid ,MyP.qualifier.classid pidType " - + "from dataset " - + "lateral view explode(author) a as MyT " - + "lateral view explode(MyT.pid) p as MyP "; + query = "select id, MyT.name name, MyT.surname surname, MyP.value pid ,MyP.qualifier.classid pidType " + + "from dataset " + + "lateral view explode(author) a as MyT " + + "lateral view explode(MyT.pid) p as MyP "; - org.apache.spark.sql.Dataset authorsExplodedPids = spark.sql(query); + org.apache.spark.sql.Dataset authorsExplodedPids = spark.sql(query); - Assertions.assertEquals( - 2, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); - Assertions.assertEquals( - 1, - authorsExplodedPids - .filter( - "name = 'Marc' and surname = 'Schmidtmann' and pidType = 'MAG Identifier'") - .count()); - } + Assertions + .assertEquals( + 2, authorsExplodedPids.filter("name = 'Marc' and surname = 'Schmidtmann'").count()); + Assertions + .assertEquals( + 1, + authorsExplodedPids + .filter( + "name = 'Marc' and surname = 'Schmidtmann' and pidType = 'MAG Identifier'") + .count()); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index 7a742e4db..ac28e9d4b 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -1,10 +1,10 @@ + package eu.dnetlib.dhp.projecttoresult; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -19,244 +19,252 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Relation; + public class ProjectPropagationJobTest { - private static final Logger log = LoggerFactory.getLogger(ProjectPropagationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(ProjectPropagationJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = ProjectPropagationJobTest.class.getClassLoader(); + private static final ClassLoader cl = ProjectPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(ProjectPropagationJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(ProjectPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName(ProjectPropagationJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(ProjectPropagationJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(ProjectPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(ProjectPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - /** - * There are no new relations to be added. All the possible relations have already been linked - * with the project in the graph - * - * @throws Exception - */ - @Test - public void NoUpdateTest() throws Exception { + /** + * There are no new relations to be added. All the possible relations have already been linked with the project in + * the graph + * + * @throws Exception + */ + @Test + public void NoUpdateTest() throws Exception { - SparkResultToProjectThroughSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), - }); + SparkResultToProjectThroughSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - Assertions.assertEquals(0, tmp.count()); - } + Assertions.assertEquals(0, tmp.count()); + } - /** - * All the possible updates will produce a new relation. No relations are already linked in the - * grpha - * - * @throws Exception - */ - @Test - public void UpdateTenTest() throws Exception { - SparkResultToProjectThroughSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), - }); + /** + * All the possible updates will produce a new relation. No relations are already linked in the grpha + * + * @throws Exception + */ + @Test + public void UpdateTenTest() throws Exception { + SparkResultToProjectThroughSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - // got 20 new relations because "produces" and "isProducedBy" are added - Assertions.assertEquals(10, tmp.count()); + // got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(10, tmp.count()); - Dataset verificationDs = - spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals(5, verificationDs.filter("relClass = 'produces'").count()); - Assertions.assertEquals(5, verificationDs.filter("relClass = 'isProducedBy'").count()); + Assertions.assertEquals(5, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(5, verificationDs.filter("relClass = 'isProducedBy'").count()); - Assertions.assertEquals( - 5, - verificationDs - .filter( - r -> - r.getSource().substring(0, 2).equals("50") - && r.getTarget().substring(0, 2).equals("40") - && r.getRelClass().equals("isProducedBy")) - .count()); - Assertions.assertEquals( - 5, - verificationDs - .filter( - r -> - r.getSource().substring(0, 2).equals("40") - && r.getTarget().substring(0, 2).equals("50") - && r.getRelClass().equals("produces")) - .count()); + Assertions + .assertEquals( + 5, + verificationDs + .filter( + r -> r.getSource().substring(0, 2).equals("50") + && r.getTarget().substring(0, 2).equals("40") + && r.getRelClass().equals("isProducedBy")) + .count()); + Assertions + .assertEquals( + 5, + verificationDs + .filter( + r -> r.getSource().substring(0, 2).equals("40") + && r.getTarget().substring(0, 2).equals("50") + && r.getRelClass().equals("produces")) + .count()); - verificationDs.createOrReplaceTempView("temporary"); + verificationDs.createOrReplaceTempView("temporary"); - Assertions.assertEquals( - 10, - spark.sql( - "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") - .count()); - } + Assertions + .assertEquals( + 10, + spark + .sql( + "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") + .count()); + } - /** - * One of the relations in the possible updates is already linked to the project in the graph. - * All the others are not. There will be 9 new associations leading to 18 new relations - * - * @throws Exception - */ - @Test - public void UpdateMixTest() throws Exception { - SparkResultToProjectThroughSemRelJob3.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), - }); + /** + * One of the relations in the possible updates is already linked to the project in the graph. All the others are + * not. There will be 9 new associations leading to 18 new relations + * + * @throws Exception + */ + @Test + public void UpdateMixTest() throws Exception { + SparkResultToProjectThroughSemRelJob3 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + // "-sourcePath", + // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-potentialUpdatePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - // JavaRDD tmp = sc.textFile("/tmp/relation") - // .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + // JavaRDD tmp = sc.textFile("/tmp/relation") + // .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - // got 20 new relations because "produces" and "isProducedBy" are added - Assertions.assertEquals(8, tmp.count()); + // got 20 new relations because "produces" and "isProducedBy" are added + Assertions.assertEquals(8, tmp.count()); - Dataset verificationDs = - spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals(4, verificationDs.filter("relClass = 'produces'").count()); - Assertions.assertEquals(4, verificationDs.filter("relClass = 'isProducedBy'").count()); + Assertions.assertEquals(4, verificationDs.filter("relClass = 'produces'").count()); + Assertions.assertEquals(4, verificationDs.filter("relClass = 'isProducedBy'").count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - r -> - r.getSource().substring(0, 2).equals("50") - && r.getTarget().substring(0, 2).equals("40") - && r.getRelClass().equals("isProducedBy")) - .count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - r -> - r.getSource().substring(0, 2).equals("40") - && r.getTarget().substring(0, 2).equals("50") - && r.getRelClass().equals("produces")) - .count()); + Assertions + .assertEquals( + 4, + verificationDs + .filter( + r -> r.getSource().substring(0, 2).equals("50") + && r.getTarget().substring(0, 2).equals("40") + && r.getRelClass().equals("isProducedBy")) + .count()); + Assertions + .assertEquals( + 4, + verificationDs + .filter( + r -> r.getSource().substring(0, 2).equals("40") + && r.getTarget().substring(0, 2).equals("50") + && r.getRelClass().equals("produces")) + .count()); - verificationDs.createOrReplaceTempView("temporary"); + verificationDs.createOrReplaceTempView("temporary"); - Assertions.assertEquals( - 8, - spark.sql( - "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") - .count()); - } + Assertions + .assertEquals( + 8, + spark + .sql( + "Select * from temporary where datainfo.inferenceprovenance = 'propagation'") + .count()); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index 78b311bc1..0dd8c6bd4 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -1,13 +1,12 @@ + package eu.dnetlib.dhp.resulttocommunityfromorganization; import static org.apache.spark.sql.functions.*; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; -import eu.dnetlib.dhp.schema.oaf.Dataset; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -22,295 +21,320 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.schema.oaf.Dataset; + public class ResultToCommunityJobTest { - private static final Logger log = LoggerFactory.getLogger(ResultToCommunityJobTest.class); + private static final Logger log = LoggerFactory.getLogger(ResultToCommunityJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = ResultToCommunityJobTest.class.getClassLoader(); + private static final ClassLoader cl = ResultToCommunityJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(ResultToCommunityJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(ResultToCommunityJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName(ResultToCommunityJobTest.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(ResultToCommunityJobTest.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(OrcidPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - @Test - public void test1() throws Exception { - SparkResultToCommunityFromOrganizationJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-preparedInfoPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") - .getPath() - }); + @Test + public void test1() throws Exception { + SparkResultToCommunityFromOrganizationJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") + .getPath(), + "-hive_metastore_uris", + "", + "-saveGraph", + "true", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", + workingDir.toString() + "/dataset", + "-preparedInfoPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") + .getPath() + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - verificationDataset.createOrReplaceTempView("dataset"); + verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'propagation'"; + String query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'propagation'"; - org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); - Assertions.assertEquals(5, resultExplodedProvenance.count()); - Assertions.assertEquals( - 0, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") - .count()); - Assertions.assertEquals( - 1, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") - .count()); - Assertions.assertEquals( - "beopen", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b")) - .collectAsList() - .get(0) - .getString(0)); + org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); + Assertions.assertEquals(5, resultExplodedProvenance.count()); + Assertions + .assertEquals( + 0, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") + .count()); + Assertions + .assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") + .count()); + Assertions + .assertEquals( + "beopen", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b")) + .collectAsList() + .get(0) + .getString(0)); - Assertions.assertEquals( - 2, - resultExplodedProvenance - .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") - .count()); - Assertions.assertEquals( - "mes", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) - .sort(desc("community")) - .collectAsList() - .get(0) - .getString(0)); - Assertions.assertEquals( - "euromarine", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) - .sort(desc("community")) - .collectAsList() - .get(1) - .getString(0)); + Assertions + .assertEquals( + 2, + resultExplodedProvenance + .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") + .count()); + Assertions + .assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "euromarine", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(1) + .getString(0)); - Assertions.assertEquals( - 1, - resultExplodedProvenance - .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") - .count()); - Assertions.assertEquals( - "mes", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) - .sort(desc("community")) - .collectAsList() - .get(0) - .getString(0)); + Assertions + .assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions + .assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); - Assertions.assertEquals( - 1, - resultExplodedProvenance - .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") - .count()); - Assertions.assertEquals( - "mes", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) - .sort(desc("community")) - .collectAsList() - .get(0) - .getString(0)); + Assertions + .assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") + .count()); + Assertions + .assertEquals( + "mes", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); - /* - {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8"} "context" ["id": euromarine] updates = 1 - {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6"} context = [ni, euromarine] updates = 1 + /* + * {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8"} + * "context" ["id": euromarine] updates = 1 + * {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6"} context + * = [ni, euromarine] updates = 1 + */ - */ + query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; - query = - "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD "; + org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); - org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); + Assertions.assertEquals(10, resultCommunityId.count()); - Assertions.assertEquals(10, resultCommunityId.count()); + Assertions + .assertEquals( + 1, + resultCommunityId + .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") + .count()); + Assertions + .assertEquals( + "beopen", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe")) + .collectAsList() + .get(0) + .getString(0)); - Assertions.assertEquals( - 1, - resultCommunityId - .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") - .count()); - Assertions.assertEquals( - "beopen", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe")) - .collectAsList() - .get(0) - .getString(0)); + Assertions + .assertEquals( + 1, + resultCommunityId + .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") + .count()); - Assertions.assertEquals( - 1, - resultCommunityId - .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") - .count()); + Assertions + .assertEquals( + 3, + resultCommunityId + .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") + .count()); + Assertions + .assertEquals( + "beopen", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) + .sort(desc("community")) + .collectAsList() + .get(2) + .getString(0)); - Assertions.assertEquals( - 3, - resultCommunityId - .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") - .count()); - Assertions.assertEquals( - "beopen", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) - .sort(desc("community")) - .collectAsList() - .get(2) - .getString(0)); + Assertions + .assertEquals( + 2, + resultCommunityId + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions + .assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .sort(desc("community")) + .collectAsList() + .get(1) + .getString(0)); - Assertions.assertEquals( - 2, - resultCommunityId - .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") - .count()); - Assertions.assertEquals( - "euromarine", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) - .sort(desc("community")) - .collectAsList() - .get(1) - .getString(0)); - - Assertions.assertEquals( - 3, - resultCommunityId - .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") - .count()); - Assertions.assertEquals( - "euromarine", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) - .sort(desc("community")) - .collectAsList() - .get(2) - .getString(0)); - Assertions.assertEquals( - "ni", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) - .sort(desc("community")) - .collectAsList() - .get(0) - .getString(0)); - } + Assertions + .assertEquals( + 3, + resultCommunityId + .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") + .count()); + Assertions + .assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(2) + .getString(0)); + Assertions + .assertEquals( + "ni", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) + .sort(desc("community")) + .collectAsList() + .get(0) + .getString(0)); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index f8806d8bb..e0ee12be6 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -1,14 +1,13 @@ + package eu.dnetlib.dhp.resulttocommunityfromsemrel; import static org.apache.spark.sql.functions.desc; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; -import eu.dnetlib.dhp.schema.oaf.Dataset; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.List; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -23,258 +22,269 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.schema.oaf.Dataset; + public class ResultToCommunityJobTest { - private static final Logger log = - LoggerFactory.getLogger( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class); + private static final Logger log = LoggerFactory + .getLogger( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getClassLoader(); + private static final ClassLoader cl = eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = - Files.createTempDirectory( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files + .createTempDirectory( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getSimpleName()); + SparkConf conf = new SparkConf(); + conf + .setAppName( + eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class + .getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(OrcidPropagationJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - @Test - public void test1() throws Exception { - SparkResultToCommunityThroughSemRelJob4.main( - new String[] { - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample") - .getPath(), - "-hive_metastore_uris", "", - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-preparedInfoPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo") - .getPath() - }); + @Test + public void test1() throws Exception { + SparkResultToCommunityThroughSemRelJob4 + .main( + new String[] { + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample") + .getPath(), + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-preparedInfoPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo") + .getPath() + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - Assertions.assertEquals(10, tmp.count()); - org.apache.spark.sql.Dataset verificationDataset = - spark.createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); - verificationDataset.createOrReplaceTempView("dataset"); + verificationDataset.createOrReplaceTempView("dataset"); - String query = - "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'propagation'"; + String query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'propagation'"; - org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); - Assertions.assertEquals(5, resultExplodedProvenance.count()); + org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); + Assertions.assertEquals(5, resultExplodedProvenance.count()); - Assertions.assertEquals( - 0, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") - .count()); + Assertions + .assertEquals( + 0, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") + .count()); - Assertions.assertEquals( - 1, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") - .count()); - Assertions.assertEquals( - "dh-ch", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) - .collectAsList() - .get(0) - .getString(0)); + Assertions + .assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") + .count()); + Assertions + .assertEquals( + "dh-ch", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .collectAsList() + .get(0) + .getString(0)); - Assertions.assertEquals( - 3, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") - .count()); - List rowList = - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("mes", rowList.get(0).getString(0)); - Assertions.assertEquals("fam", rowList.get(1).getString(0)); - Assertions.assertEquals("ee", rowList.get(2).getString(0)); + Assertions + .assertEquals( + 3, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") + .count()); + List rowList = resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); - Assertions.assertEquals( - 1, - resultExplodedProvenance - .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") - .count()); - Assertions.assertEquals( - "aginfra", - resultExplodedProvenance - .select("community") - .where( - resultExplodedProvenance - .col("id") - .equalTo( - "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) - .collectAsList() - .get(0) - .getString(0)); + Assertions + .assertEquals( + 1, + resultExplodedProvenance + .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") + .count()); + Assertions + .assertEquals( + "aginfra", + resultExplodedProvenance + .select("community") + .where( + resultExplodedProvenance + .col("id") + .equalTo( + "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .collectAsList() + .get(0) + .getString(0)); - query = - "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD "; + query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD "; - org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); + org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); - Assertions.assertEquals(10, resultCommunityId.count()); + Assertions.assertEquals(10, resultCommunityId.count()); - Assertions.assertEquals( - 2, - resultCommunityId - .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") - .count()); - rowList = - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("dh-ch", rowList.get(0).getString(0)); - Assertions.assertEquals("beopen", rowList.get(1).getString(0)); + Assertions + .assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::0489ae524201eedaa775da282dce35e7'") + .count()); + rowList = resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0489ae524201eedaa775da282dce35e7")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("dh-ch", rowList.get(0).getString(0)); + Assertions.assertEquals("beopen", rowList.get(1).getString(0)); - Assertions.assertEquals( - 3, - resultCommunityId - .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") - .count()); - rowList = - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("mes", rowList.get(0).getString(0)); - Assertions.assertEquals("fam", rowList.get(1).getString(0)); - Assertions.assertEquals("ee", rowList.get(2).getString(0)); + Assertions + .assertEquals( + 3, + resultCommunityId + .filter("id = '50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28'") + .count()); + rowList = resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0a60e33b4f0986ebd9819451f2d87a28")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("mes", rowList.get(0).getString(0)); + Assertions.assertEquals("fam", rowList.get(1).getString(0)); + Assertions.assertEquals("ee", rowList.get(2).getString(0)); - Assertions.assertEquals( - 2, - resultCommunityId - .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") - .count()); - rowList = - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("beopen", rowList.get(0).getString(0)); - Assertions.assertEquals("aginfra", rowList.get(1).getString(0)); + Assertions + .assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc'") + .count()); + rowList = resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::0ae02edb5598a5545d10b107fcf48dcc")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("beopen", rowList.get(0).getString(0)); + Assertions.assertEquals("aginfra", rowList.get(1).getString(0)); - Assertions.assertEquals( - 2, - resultCommunityId - .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") - .count()); - rowList = - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b")) - .sort(desc("community")) - .collectAsList(); - Assertions.assertEquals("euromarine", rowList.get(1).getString(0)); - Assertions.assertEquals("ni", rowList.get(0).getString(0)); + Assertions + .assertEquals( + 2, + resultCommunityId + .filter("id = '50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b'") + .count()); + rowList = resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|dedup_wf_001::2305908abeca9da37eaf3bddcaf81b7b")) + .sort(desc("community")) + .collectAsList(); + Assertions.assertEquals("euromarine", rowList.get(1).getString(0)); + Assertions.assertEquals("ni", rowList.get(0).getString(0)); - Assertions.assertEquals( - 1, - resultCommunityId - .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") - .count()); - Assertions.assertEquals( - "euromarine", - resultCommunityId - .select("community") - .where( - resultCommunityId - .col("id") - .equalTo( - "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) - .collectAsList() - .get(0) - .getString(0)); - } + Assertions + .assertEquals( + 1, + resultCommunityId + .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") + .count()); + Assertions + .assertEquals( + "euromarine", + resultCommunityId + .select("community") + .where( + resultCommunityId + .col("id") + .equalTo( + "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) + .collectAsList() + .get(0) + .getString(0)); + } } diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index 2b2f64a33..01bf64dd9 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -1,10 +1,10 @@ + package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Relation; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -19,269 +19,286 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Relation; + public class Result2OrganizationJobTest { - private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); + private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); - private static SparkSession spark; + private static SparkSession spark; - private static Path workingDir; + private static Path workingDir; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = - Files.createTempDirectory( - SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); - log.info("using work dir {}", workingDir); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files + .createTempDirectory( + SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + log.info("using work dir {}", workingDir); - SparkConf conf = new SparkConf(); - conf.setAppName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + SparkConf conf = new SparkConf(); + conf.setAppName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); - conf.setMaster("local[*]"); - conf.set("spark.driver.host", "localhost"); - conf.set("hive.metastore.local", "true"); - conf.set("spark.ui.enabled", "false"); - conf.set("spark.sql.warehouse.dir", workingDir.toString()); - conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); - spark = - SparkSession.builder() - .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } + spark = SparkSession + .builder() + .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } - /** - * No modifications done to the sample sets, so that no possible updates are created - * - * @throws Exception - */ - @Test - public void NoUpdateTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") - .getPath(), - }); + /** + * No modifications done to the sample sets, so that no possible updates are created + * + * @throws Exception + */ + @Test + public void NoUpdateTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - Assertions.assertEquals(0, tmp.count()); - } + Assertions.assertEquals(0, tmp.count()); + } - /** - * Testing set with modified association between datasource and organization. Copied some - * hostedby collectedfrom from the software sample set. No intersection with the already linked - * (all the possible new relations, will became new relations) - * - * @throws Exception - */ - @Test - public void UpdateNoMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") - .getPath(), - }); + /** + * Testing set with modified association between datasource and organization. Copied some hostedby collectedfrom + * from the software sample set. No intersection with the already linked (all the possible new relations, will + * became new relations) + * + * @throws Exception + */ + @Test + public void UpdateNoMixTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - Assertions.assertEquals(20, tmp.count()); + Assertions.assertEquals(20, tmp.count()); - Dataset verificationDs = - spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals( - 8, - verificationDs - .filter("target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2'") - .count()); - Assertions.assertEquals( - 1, - verificationDs - .filter("target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091'") - .count()); - Assertions.assertEquals( - 1, - verificationDs - .filter("target = '20|opendoar____::4429502fa1936b0941f4647b69b844c8'") - .count()); + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Assertions + .assertEquals( + 8, + verificationDs + .filter("target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2'") + .count()); + Assertions + .assertEquals( + 1, + verificationDs + .filter("target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091'") + .count()); + Assertions + .assertEquals( + 1, + verificationDs + .filter("target = '20|opendoar____::4429502fa1936b0941f4647b69b844c8'") + .count()); - Assertions.assertEquals( - 2, - verificationDs - .filter( - "source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and " - + "(target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091' " - + "or target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2')") - .count()); - } + Assertions + .assertEquals( + 2, + verificationDs + .filter( + "source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218' and " + + "(target = '20|opendoar____::124266ebc4ece2934eb80edfda3f2091' " + + "or target = '20|dedup_wf_001::5168917a6aeeea55269daeac1af2ecd2')") + .count()); + } - @Test - public void UpdateMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2.main( - new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") - .getPath(), - }); + @Test + public void UpdateMixTest() throws Exception { + SparkResultToOrganizationFromIstRepoJob2 + .main( + new String[] { + "-isTest", + Boolean.TRUE.toString(), + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") + .getPath(), + "-hive_metastore_uris", + "", + "-resultTableName", + "eu.dnetlib.dhp.schema.oaf.Software", + "-writeUpdate", + "false", + "-saveGraph", + "true", + "-outputPath", + workingDir.toString() + "/relation", + "-datasourceOrganizationPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") + .getPath(), + "-alreadyLinkedPath", + getClass() + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") + .getPath(), + }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaRDD tmp = - sc.textFile(workingDir.toString() + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); - Dataset verificationDs = - spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); + Dataset verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class)); - Assertions.assertEquals(8, verificationDs.count()); + Assertions.assertEquals(8, verificationDs.count()); - Assertions.assertEquals( - 2, - verificationDs - .filter("source = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6'") - .count()); - Assertions.assertEquals( - 1, - verificationDs - .filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218'") - .count()); - Assertions.assertEquals( - 1, - verificationDs - .filter("source = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523'") - .count()); + Assertions + .assertEquals( + 2, + verificationDs + .filter("source = '50|od_______109::f375befa62a741e9250e55bcfa88f9a6'") + .count()); + Assertions + .assertEquals( + 1, + verificationDs + .filter("source = '50|dedup_wf_001::b67bc915603fc01e445f2b5888ba7218'") + .count()); + Assertions + .assertEquals( + 1, + verificationDs + .filter("source = '50|dedup_wf_001::40ea2f24181f6ae77b866ebcbffba523'") + .count()); - Assertions.assertEquals( - 1, - verificationDs - .filter("source = '20|wt__________::a72760363ca885e6bef165804770e00c'") - .count()); + Assertions + .assertEquals( + 1, + verificationDs + .filter("source = '20|wt__________::a72760363ca885e6bef165804770e00c'") + .count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - "relclass = 'hasAuthorInstitution' and substring(source, 1,2) = '50'") - .count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - "relclass = 'isAuthorInstitutionOf' and substring(source, 1,2) = '20'") - .count()); + Assertions + .assertEquals( + 4, + verificationDs + .filter( + "relclass = 'hasAuthorInstitution' and substring(source, 1,2) = '50'") + .count()); + Assertions + .assertEquals( + 4, + verificationDs + .filter( + "relclass = 'isAuthorInstitutionOf' and substring(source, 1,2) = '20'") + .count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - "relclass = 'hasAuthorInstitution' and " - + "substring(source, 1,2) = '50' and substring(target, 1, 2) = '20'") - .count()); - Assertions.assertEquals( - 4, - verificationDs - .filter( - "relclass = 'isAuthorInstitutionOf' and " - + "substring(source, 1,2) = '20' and substring(target, 1, 2) = '50'") - .count()); - } + Assertions + .assertEquals( + 4, + verificationDs + .filter( + "relclass = 'hasAuthorInstitution' and " + + "substring(source, 1,2) = '50' and substring(target, 1, 2) = '20'") + .count()); + Assertions + .assertEquals( + 4, + verificationDs + .filter( + "relclass = 'isAuthorInstitutionOf' and " + + "substring(source, 1,2) = '20' and substring(target, 1, 2) = '50'") + .count()); + } } From c89fe762b166a4e5d67622776c5382a195428dca Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 11:17:03 +0200 Subject: [PATCH 189/259] modified relation datasource organization --- .../src/main/java/eu/dnetlib/dhp/PropagationConstant.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index a2fb0c0ef..5bab36a5f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -47,7 +47,7 @@ public class PropagationConstant { public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "authorpid:result"; public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of authors pid to result through semantic relations"; - public static final String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "provides"; + public static final String RELATION_DATASOURCE_ORGANIZATION_REL_CLASS = "isProvidedBy"; public static final String RELATION_RESULTORGANIZATION_REL_TYPE = "resultOrganization"; public static final String RELATION_RESULTORGANIZATION_SUBREL_TYPE = "affiliation"; From 026f297e4924acb6f6a7c77b9d4ce6363f303853 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 11:31:59 +0200 Subject: [PATCH 190/259] removed the writeUpdate oprion --- ...rkResultToOrganizationFromIstRepoJob2.java | 54 ------------------- 1 file changed, 54 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index 6bdfa36dd..8c9f270e6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -61,12 +61,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); log.info("resultType: {}", resultType); - final Boolean writeUpdates = Optional - .ofNullable(parser.get("writeUpdate")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("writeUpdate: {}", writeUpdates); - final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) @@ -93,7 +87,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { outputPath, resultClazz, resultType, - writeUpdates, saveGraph); }); } @@ -106,7 +99,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { String outputPath, Class resultClazz, String resultType, - Boolean writeUpdates, Boolean saveGraph) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); @@ -124,10 +116,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { broadcast_datasourceorganizationassoc) .as(Encoders.bean(ResultOrganizationSet.class)); - if (writeUpdates) { - createUpdateForRelationWrite(potentialUpdates, outputPath + "/" + resultType); - } - if (saveGraph) { getNewRelations( spark @@ -229,48 +217,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { Encoders.bean(DatasourceOrganization.class)); } - private static void createUpdateForRelationWrite( - Dataset toupdaterelation, String outputPath) { - toupdaterelation - .flatMap( - s -> { - List relationList = new ArrayList<>(); - List orgs = s.getOrganizationSet(); - String resId = s.getResultId(); - for (String org : orgs) { - relationList - .add( - getRelation( - org, - resId, - RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - relationList - .add( - getRelation( - resId, - org, - RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - } - return relationList.iterator(); - }, - Encoders.bean(Relation.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); - } - private static org.apache.spark.sql.Dataset organizationPropagationAssoc( SparkSession spark, Broadcast> broadcast_datasourceorganizationassoc) { From 951517f9ec3d185b1eec0e424a17548758462ba0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 11:32:50 +0200 Subject: [PATCH 191/259] new input parameters and workflow definition to be used in the provision pipeline --- ...sulaffiliationfrominstrepo_parameters.json | 8 +- .../oozie_app/workflow.xml | 123 +++++++++++++----- 2 files changed, 89 insertions(+), 42 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json index 9da25874a..d2b076c82 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json @@ -11,17 +11,11 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"wu", - "paramLongName":"writeUpdate", - "paramDescription": "true if the update must be writte. No double check if information is already present", - "paramRequired": true - }, { "paramName":"sg", "paramLongName":"saveGraph", "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": true + "paramRequired": false }, { "paramName":"dop", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 245c4f556..f1495e03b 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -5,12 +5,8 @@ the source path - writeUpdate - writes the information found for the update. No double check done if the information is already present - - - saveGraph - writes new version of the graph after the propagation step + outputPath + sets the outputPath @@ -22,18 +18,29 @@ - + + + + + + + + + - + - + + + + @@ -41,18 +48,17 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/resulttoorganization_propagation/relation + ${nameNode}/${outputPath}/relation - ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/publication - ${nameNode}/${workingDir}/resulttoorganization_propagation/publication + ${nameNode}/${outputPath}/publication @@ -63,7 +69,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/dataset - ${nameNode}/${workingDir}/resulttoorganization_propagation/dataset + ${nameNode}/${outputPath}/dataset @@ -74,7 +80,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/otherresearchproduct - ${nameNode}/${workingDir}/resulttoorganization_propagation/otherresearchproduct + ${nameNode}/${outputPath}/otherresearchproduct @@ -84,11 +90,44 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/software - ${nameNode}/${workingDir}/resulttoorganization_propagation/software + ${nameNode}/${outputPath}/software + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + @@ -108,11 +147,11 @@ --sourcePath${sourcePath} --hive_metastore_uris${hive_metastore_uris} - --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - + @@ -142,12 +181,11 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/resulttoorganization_propagation/relation - --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + --outputPath${outputPath}/relation + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked @@ -173,17 +211,15 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/resulttoorganization_propagation/relation - --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + --outputPath${outputPath}/relation + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - yarn @@ -204,12 +240,11 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/resulttoorganization_propagation/relation - --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + --outputPath${outputPath}/relation + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked @@ -235,12 +270,30 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} - --writeUpdate${writeUpdate} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/resulttoorganization_propagation/relation - --datasourceOrganizationPath${workingDir}/resulttoorganization_propagation/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/resulttoorganization_propagation/preparedInfo/alreadyLinked + --outputPath${outputPath}/relation + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --saveGraph${saveGraph} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${outputPath}/relation + --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked From 2349bfd8b818bdf362abd9ac5a921d350c0d2698 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 11:43:33 +0200 Subject: [PATCH 192/259] changed the job test to remove the writeUpdate option --- .../Result2OrganizationJobTest.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index 01bf64dd9..ccc1a452a 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -88,8 +88,6 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", "-saveGraph", "true", "-outputPath", @@ -140,8 +138,6 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", "-saveGraph", "true", "-outputPath", @@ -215,8 +211,6 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-writeUpdate", - "false", "-saveGraph", "true", "-outputPath", From 4b0bd91012179a1f29de3142cbbf85629ba17568 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 12:45:28 +0200 Subject: [PATCH 193/259] - --- : | 7 ++ .../dhp/schema/common/ModelSupport.java | 7 +- .../raw/MigrateDbEntitiesApplication.java | 85 ++++++++++++------- ...rkResultToOrganizationFromIstRepoJob2.java | 1 + .../input_communitytoresult_parameters.json | 6 ++ .../oozie_app/workflow.xml | 76 +++++++++++++++-- .../Result2OrganizationJobTest.java | 3 + 7 files changed, 144 insertions(+), 41 deletions(-) create mode 100644 : diff --git a/: b/: new file mode 100644 index 000000000..7768c1527 --- /dev/null +++ b/: @@ -0,0 +1,7 @@ +Merge remote-tracking branch 'upstream/master' + +# Please enter a commit message to explain why this merge is necessary, +# especially if it merges an updated upstream into a topic branch. +# +# Lines starting with '#' will be ignored, and an empty message aborts +# the commit. diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java index 7838cc0cf..1fd2ef2da 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java @@ -1,11 +1,14 @@ + package eu.dnetlib.dhp.schema.common; -import com.google.common.collect.Maps; -import eu.dnetlib.dhp.schema.oaf.*; import java.util.Map; import java.util.Optional; import java.util.function.Function; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.schema.oaf.*; + /** Oaf model utility methods. */ public class ModelSupport { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 997cb8f03..f5ac56b78 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -51,7 +51,7 @@ import eu.dnetlib.dhp.schema.oaf.Software; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; public class MigrateDbEntitiesApplication extends AbstractMigrationApplication - implements Closeable { + implements Closeable { private static final Log log = LogFactory.getLog(MigrateDbEntitiesApplication.class); @@ -61,9 +61,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication public static void main(final String[] args) throws Exception { final ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString(MigrateDbEntitiesApplication.class - .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/migrate_db_entities_parameters.json"))); + IOUtils + .toString( + MigrateDbEntitiesApplication.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/migrate_db_entities_parameters.json"))); parser.parseArgument(args); @@ -76,7 +77,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final boolean processClaims = parser.get("action") != null && parser.get("action").equalsIgnoreCase("claims"); try (final MigrateDbEntitiesApplication smdbe = new MigrateDbEntitiesApplication(hdfsPath, dbUrl, dbUser, - dbPassword)) { + dbPassword)) { if (processClaims) { log.info("Processing claims..."); smdbe.execute("queryClaims.sql", smdbe::processClaims); @@ -107,15 +108,15 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication } public MigrateDbEntitiesApplication( - final String hdfsPath, final String dbUrl, final String dbUser, final String dbPassword) - throws Exception { + final String hdfsPath, final String dbUrl, final String dbUser, final String dbPassword) + throws Exception { super(hdfsPath); this.dbClient = new DbClient(dbUrl, dbUser, dbPassword); this.lastUpdateTimestamp = new Date().getTime(); } public void execute(final String sqlFile, final Function> producer) - throws Exception { + throws Exception { final String sql = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/sql/" + sqlFile)); final Consumer consumer = rs -> producer.apply(rs).forEach(oaf -> emitOaf(oaf)); @@ -134,7 +135,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication ds.setId(createOpenaireId(10, rs.getString("datasourceid"), true)); ds.setOriginalId(Arrays.asList(rs.getString("datasourceid"))); ds - .setCollectedfrom(listKeyValues(createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname"))); + .setCollectedfrom( + listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), + rs.getString("collectedfromname"))); ds.setPid(new ArrayList<>()); ds.setDateofcollection(asString(rs.getDate("dateofcollection"))); ds.setDateoftransformation(null); // Value not returned by the SQL query @@ -175,7 +179,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication ds.setCertificates(field(rs.getString("certificates"), info)); ds.setPolicies(new ArrayList<>()); // The sql query returns an empty array ds - .setJournal(prepareJournal(rs.getString("officialname"), rs.getString("journal"), info)); // Journal + .setJournal(prepareJournal(rs.getString("officialname"), rs.getString("journal"), info)); // Journal ds.setDataInfo(info); ds.setLastupdatetimestamp(lastUpdateTimestamp); @@ -195,7 +199,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication p.setId(createOpenaireId(40, rs.getString("projectid"), true)); p.setOriginalId(Arrays.asList(rs.getString("projectid"))); p - .setCollectedfrom(listKeyValues(createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname"))); + .setCollectedfrom( + listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), + rs.getString("collectedfromname"))); p.setPid(new ArrayList<>()); p.setDateofcollection(asString(rs.getDate("dateofcollection"))); p.setDateoftransformation(asString(rs.getDate("dateoftransformation"))); @@ -212,7 +219,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication p.setDuration(field(Integer.toString(rs.getInt("duration")), info)); p.setEcsc39(field(Boolean.toString(rs.getBoolean("ecsc39")), info)); p - .setOamandatepublications(field(Boolean.toString(rs.getBoolean("oamandatepublications")), info)); + .setOamandatepublications(field(Boolean.toString(rs.getBoolean("oamandatepublications")), info)); p.setEcarticle29_3(field(Boolean.toString(rs.getBoolean("ecarticle29_3")), info)); p.setSubjects(prepareListOfStructProps(rs.getArray("subjects"), info)); p.setFundingtree(prepareListFields(rs.getArray("fundingtree"), info)); @@ -249,7 +256,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication o.setId(createOpenaireId(20, rs.getString("organizationid"), true)); o.setOriginalId(Arrays.asList(rs.getString("organizationid"))); o - .setCollectedfrom(listKeyValues(createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname"))); + .setCollectedfrom( + listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), + rs.getString("collectedfromname"))); o.setPid(new ArrayList<>()); o.setDateofcollection(asString(rs.getDate("dateofcollection"))); o.setDateoftransformation(asString(rs.getDate("dateoftransformation"))); @@ -264,12 +274,14 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication o.setEclegalperson(field(Boolean.toString(rs.getBoolean("eclegalperson")), info)); o.setEcnonprofit(field(Boolean.toString(rs.getBoolean("ecnonprofit")), info)); o - .setEcresearchorganization(field(Boolean.toString(rs.getBoolean("ecresearchorganization")), info)); + .setEcresearchorganization(field(Boolean.toString(rs.getBoolean("ecresearchorganization")), info)); o.setEchighereducation(field(Boolean.toString(rs.getBoolean("echighereducation")), info)); o - .setEcinternationalorganizationeurinterests(field(Boolean.toString(rs.getBoolean("ecinternationalorganizationeurinterests")), info)); + .setEcinternationalorganizationeurinterests( + field(Boolean.toString(rs.getBoolean("ecinternationalorganizationeurinterests")), info)); o - .setEcinternationalorganization(field(Boolean.toString(rs.getBoolean("ecinternationalorganization")), info)); + .setEcinternationalorganization( + field(Boolean.toString(rs.getBoolean("ecinternationalorganization")), info)); o.setEcenterprise(field(Boolean.toString(rs.getBoolean("ecenterprise")), info)); o.setEcsmevalidated(field(Boolean.toString(rs.getBoolean("ecsmevalidated")), info)); o.setEcnutscode(field(Boolean.toString(rs.getBoolean("ecnutscode")), info)); @@ -288,7 +300,8 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final DataInfo info = prepareDataInfo(rs); final String orgId = createOpenaireId(20, rs.getString("organization"), true); final String dsId = createOpenaireId(10, rs.getString("datasource"), true); - final List collectedFrom = listKeyValues(createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname")); + final List collectedFrom = listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname")); final Relation r1 = new Relation(); r1.setRelType("datasourceOrganization"); @@ -321,7 +334,8 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final DataInfo info = prepareDataInfo(rs); final String orgId = createOpenaireId(20, rs.getString("resporganization"), true); final String projectId = createOpenaireId(40, rs.getString("project"), true); - final List collectedFrom = listKeyValues(createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname")); + final List collectedFrom = listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname")); final Relation r1 = new Relation(); r1.setRelType("projectOrganization"); @@ -351,10 +365,12 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication public List processClaims(final ResultSet rs) { - final DataInfo info = - dataInfo(false, null, false, false, qualifier("user:claim", "user:claim", "dnet:provenanceActions", "dnet:provenanceActions"), "0.9"); + final DataInfo info = dataInfo( + false, null, false, false, + qualifier("user:claim", "user:claim", "dnet:provenanceActions", "dnet:provenanceActions"), "0.9"); - final List collectedFrom = listKeyValues(createOpenaireId(10, "infrastruct_::openaire", true), "OpenAIRE"); + final List collectedFrom = listKeyValues( + createOpenaireId(10, "infrastruct_::openaire", true), "OpenAIRE"); try { @@ -440,11 +456,15 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final String inferenceprovenance = rs.getString("inferenceprovenance"); final Boolean inferred = rs.getBoolean("inferred"); final String trust = rs.getString("trust"); - return dataInfo(deletedbyinference, inferenceprovenance, inferred, false, MigrationConstants.ENTITYREGISTRY_PROVENANCE_ACTION, trust); + return dataInfo( + deletedbyinference, inferenceprovenance, inferred, false, + MigrationConstants.ENTITYREGISTRY_PROVENANCE_ACTION, trust); } private Qualifier prepareQualifierSplitting(final String s) { - if (StringUtils.isBlank(s)) { return null; } + if (StringUtils.isBlank(s)) { + return null; + } final String[] arr = s.split("@@@"); return arr.length == 4 ? qualifier(arr[0], arr[1], arr[2], arr[3]) : null; } @@ -458,19 +478,23 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication } private StructuredProperty prepareStructProp(final String s, final DataInfo dataInfo) { - if (StringUtils.isBlank(s)) { return null; } + if (StringUtils.isBlank(s)) { + return null; + } final String[] parts = s.split("###"); if (parts.length == 2) { final String value = parts[0]; final String[] arr = parts[1].split("@@@"); - if (arr.length == 4) { return structuredProperty(value, arr[0], arr[1], arr[2], arr[3], dataInfo); } + if (arr.length == 4) { + return structuredProperty(value, arr[0], arr[1], arr[2], arr[3], dataInfo); + } } return null; } private List prepareListOfStructProps( - final Array array, - final DataInfo dataInfo) throws SQLException { + final Array array, + final DataInfo dataInfo) throws SQLException { final List res = new ArrayList<>(); if (array != null) { for (final String s : (String[]) array.getArray()) { @@ -489,8 +513,10 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final String[] arr = sj.split("@@@"); if (arr.length == 3) { final String issn = StringUtils.isNotBlank(arr[0]) ? arr[0].trim() : null; - final String eissn = StringUtils.isNotBlank(arr[1]) ? arr[1].trim() : null;; - final String lissn = StringUtils.isNotBlank(arr[2]) ? arr[2].trim() : null;; + final String eissn = StringUtils.isNotBlank(arr[1]) ? arr[1].trim() : null; + ; + final String lissn = StringUtils.isNotBlank(arr[2]) ? arr[2].trim() : null; + ; if (issn != null || eissn != null || lissn != null) { return journal(name, issn, eissn, eissn, null, null, null, null, null, null, null, info); } @@ -504,5 +530,4 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication super.close(); dbClient.close(); } - } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index 8c9f270e6..3d1603c7e 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -99,6 +99,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { String outputPath, Class resultClazz, String resultType, + Boolean saveGraph) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json index 2baec0e68..a40ce375e 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json @@ -6,6 +6,12 @@ "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, + { + "paramName":"sg", + "paramLongName":"saveGraph", + "paramDescription": "true if the new version of the graph must be saved", + "paramRequired": false + }, { "paramName":"h", "paramLongName":"hive_metastore_uris", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index d320bc9eb..e033e0e34 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -12,6 +12,10 @@ isLookupUrl the isLookup service endpoint + + outputPath + the output path + @@ -21,22 +25,72 @@ - + + + + + + + + + - + + + + + + + + ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/projecttoresult_propagation/relation + ${nameNode}/${outputPath}/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + @@ -67,7 +121,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${workingDir}/preparedInfo/targetCommunityAssoc --allowedsemrels${allowedsemrels} - --isLookupUrl${isLookupUrl} + --isLookupUrl${isLookUpUrl} @@ -215,7 +269,8 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/publication + --outputPath${outputPath}/publication + --saveGrap${saveGraph @@ -242,7 +297,8 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/dataset + --outputPath${outputPath}/dataset + --saveGrap${saveGraph @@ -269,7 +325,8 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/otherresearchproduct + --outputPath${outputPath}/otherresearchproduct + --saveGrap${saveGraph @@ -296,7 +353,8 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/software + --outputPath${outputPath}/software + --saveGrap${saveGraph diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index ccc1a452a..447cf47b0 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -88,6 +88,7 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", "-outputPath", @@ -138,6 +139,7 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", "-outputPath", @@ -211,6 +213,7 @@ public class Result2OrganizationJobTest { "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", "-outputPath", From ce8b1d0bc3efa1e72f46bb34afeb89ddffd37e82 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 14:38:54 +0200 Subject: [PATCH 194/259] new workflow definition to be inserted in the provision pipeline --- .../dhp/bulktag/oozie_app/workflow.xml | 94 +++++++++++++------ .../countrypropagation/oozie_app/workflow.xml | 88 +++++++++++------ .../oozie_app/workflow.xml | 76 ++++++++++++--- .../projecttoresult/oozie_app/workflow.xml | 83 ++++++++++------ ...t_preparecommunitytoresult_parameters.json | 6 -- .../oozie_app/workflow.xml | 76 ++++++++++++--- .../oozie_app/workflow.xml | 19 ---- 7 files changed, 302 insertions(+), 140 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 02efeb7ae..4f3d050b3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -4,18 +4,6 @@ sourcePath the source path - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - isLookupUrl the isLookup service endpoint @@ -24,6 +12,10 @@ protoMap the json path associated to each selection field + + outputPath + the output path + @@ -34,27 +26,73 @@ - - - - - + + + + + + + + - + + + + + + + + ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/relation + ${nameNode}/${outputPath}/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + + + @@ -81,11 +119,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath}/publication - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/publication + --outputPath${outputPath}/publication --protoMap${protoMap} --isLookupUrl${isLookupUrl} + --saveGraph${saveGraph} @@ -110,11 +148,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath}/dataset - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/dataset + --outputPath${outputPath}/dataset --protoMap${protoMap} --isLookupUrl${isLookupUrl} + --saveGraph${saveGraph} @@ -139,11 +177,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath}/otherresearchproduct - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/otherresearchproduct + --outputPath${outputPath}/otherresearchproduct --protoMap${protoMap} --isLookupUrl${isLookupUrl} + --saveGraph${saveGraph} @@ -168,11 +206,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath}/software - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/software + --outputPath${outputPath}/software --protoMap${protoMap} --isLookupUrl${isLookupUrl} + --saveGraph${saveGraph} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index bd2473308..d5fb199cd 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -13,24 +13,8 @@ the allowed types - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - sparkExecutorNumber - number of executors used - - - saveGraph - writes new version of the graph after the propagation step + outputPath + the output path @@ -43,26 +27,70 @@ - - - - - + + + + + + + + - + + + + + + + ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/relation + ${nameNode}/${outputPath}/relation - + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + yarn @@ -117,7 +145,7 @@ --hive_metastore_uris${hive_metastore_uris} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/publication + --outputPath${outputPath}/publication --preparedInfoPath${workingDir}/preparedInfo @@ -146,7 +174,7 @@ --hive_metastore_uris${hive_metastore_uris} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/dataset + --outputPath${outputPath}/dataset --preparedInfoPath${workingDir}/preparedInfo @@ -175,7 +203,7 @@ --hive_metastore_uris${hive_metastore_uris} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/otherresearchproduct + --outputPath${outputPath}/otherresearchproduct --preparedInfoPath${workingDir}/preparedInfo @@ -204,7 +232,7 @@ --hive_metastore_uris${hive_metastore_uris} --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/software + --outputPath${outputPath}/software --preparedInfoPath${workingDir}/preparedInfo diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index e26c8f28a..ac25b6728 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -9,12 +9,8 @@ the semantic relationships allowed for propagation - writeUpdate - writes the information found for the update. No double check done if the information is already present - - - saveGraph - writes new version of the graph after the propagation step + outputPath + the output path @@ -27,24 +23,72 @@ - + + + + + + + + - + + + + + + + + ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/orcid_propagation/relation + ${nameNode}/${outputPath}/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + @@ -222,7 +266,8 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/orcid_propagation/publication + --outputPath${outputPath}/publication + --saveGraph${saveGraph} @@ -249,7 +294,8 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/orcid_propagation/dataset + --outputPath${outputPath}/dataset + --saveGraph${saveGraph} @@ -276,7 +322,8 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/orcid_propagation/otherresearchproduct + --outputPath${outputPath}/otherresearchproduct + --saveGraph${saveGraph} @@ -303,7 +350,8 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/orcid_propagation/software + --outputPath${outputPath}/software + --saveGraph${saveGraph} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index e4f791dbc..17bf9adf2 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -8,22 +8,10 @@ allowedsemrels the allowed semantics - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - saveGraph - writes new version of the graph after the propagation step - + + outputPath + the output path + @@ -35,21 +23,27 @@ - - - - - + + + + + + + + - + - + + + + @@ -57,7 +51,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/relation + ${nameNode}/${outputPath}/relation @@ -68,7 +62,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/publication - ${nameNode}/${workingDir}/publication + ${nameNode}/${outputPath}/publication @@ -79,7 +73,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/dataset - ${nameNode}/${workingDir}/dataset + ${nameNode}/${outputPath}/dataset @@ -90,7 +84,7 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/otherresearchproduct - ${nameNode}/${workingDir}/otherresearchproduct + ${nameNode}/${outputPath}/otherresearchproduct @@ -101,11 +95,42 @@ ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/software - ${nameNode}/${workingDir}/software + ${nameNode}/${outputPath}/software + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + @@ -156,7 +181,7 @@ --saveGraph${saveGraph} --hive_metastore_uris${hive_metastore_uris} - --outputPath${workingDir}/relation + --outputPath${outputPath}/relation --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json index de472417d..8df509abf 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -23,12 +23,6 @@ "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false }, - { - "paramName":"test", - "paramLongName":"isTest", - "paramDescription": "true if it is executing a test", - "paramRequired": false - }, { "paramName": "out", "paramLongName": "outputPath", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index 20ce6ddda..bf200e242 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -9,12 +9,8 @@ organization community map - writeUpdate - writes the information found for the update. No double check done if the information is already present - - - saveGraph - writes new version of the graph after the propagation step + outputPath + the output path @@ -26,23 +22,71 @@ - + + + + + + + + + - + + + + + + + + ${jobTracker} ${nameNode} ${nameNode}/${sourcePath}/relation - ${nameNode}/${workingDir}/projecttoresult_propagation/relation + ${nameNode}/${outputPath}/relation - + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + yarn @@ -100,7 +144,8 @@ --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/publication + --outputPath${outputPath}/publication + --saveGraph${saveGraph} @@ -127,7 +172,8 @@ --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/dataset + --outputPath${outputPath}/dataset + --saveGraph${saveGraph} @@ -154,7 +200,8 @@ --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/otherresearchproduct + --outputPath${outputPath}/otherresearchproduct + --saveGraph${saveGraph} @@ -181,7 +228,8 @@ --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/software + --outputPath${outputPath}/software + --saveGraph${saveGraph} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index f1495e03b..7e124f843 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -275,25 +275,6 @@ --outputPath${outputPath}/relation --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - dhp-propagation-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --sourcePath${sourcePath}/software - --hive_metastore_uris${hive_metastore_uris} - --saveGraph${saveGraph} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${outputPath}/relation - --datasourceOrganizationPath${workingDir}/preparedInfo/datasourceOrganization - --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked From 418595fec29d7774a608dfb996d2f4718e50c551 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 14:45:00 +0200 Subject: [PATCH 195/259] removed the saveGraph parameter --- .../resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 4f3d050b3..fe8f706e7 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -123,7 +123,6 @@ --outputPath${outputPath}/publication --protoMap${protoMap} --isLookupUrl${isLookupUrl} - --saveGraph${saveGraph} @@ -152,7 +151,6 @@ --outputPath${outputPath}/dataset --protoMap${protoMap} --isLookupUrl${isLookupUrl} - --saveGraph${saveGraph} @@ -181,7 +179,6 @@ --outputPath${outputPath}/otherresearchproduct --protoMap${protoMap} --isLookupUrl${isLookupUrl} - --saveGraph${saveGraph} @@ -210,7 +207,6 @@ --outputPath${outputPath}/software --protoMap${protoMap} --isLookupUrl${isLookupUrl} - --saveGraph${saveGraph} From 65a5d67b8b543cb4287e85e4e16568e96c6b857e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 14:45:27 +0200 Subject: [PATCH 196/259] minor modifications --- ...ResultToCommunityFromOrganizationJob2.java | 3 +- ...rkResultToOrganizationFromIstRepoJob2.java | 31 ++++++++----------- 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index 74931a537..30908012b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -70,7 +70,8 @@ public class SparkResultToCommunityFromOrganizationJob2 { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); + if(saveGraph) + execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); }); } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index 3d1603c7e..b69158c87 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -58,8 +58,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); + final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) @@ -79,16 +78,15 @@ public class SparkResultToOrganizationFromIstRepoJob2 { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - execPropagation( - spark, - datasourceorganization, - alreadylinked, - inputPath, - outputPath, - resultClazz, - resultType, - saveGraph); - }); + if(saveGraph) + execPropagation( + spark, + datasourceorganization, + alreadylinked, + inputPath, + outputPath, + resultClazz); + }); } private static void execPropagation( @@ -97,10 +95,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { String alreadylinked, String inputPath, String outputPath, - Class resultClazz, - String resultType, - - Boolean saveGraph) { + Class resultClazz) { final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); org.apache.spark.sql.Dataset datasourceorganizationassoc = readAssocDatasourceOrganization( @@ -117,7 +112,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { broadcast_datasourceorganizationassoc) .as(Encoders.bean(ResultOrganizationSet.class)); - if (saveGraph) { + getNewRelations( spark .read() @@ -133,7 +128,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { .mode(SaveMode.Append) .option("compression", "gzip") .text(outputPath); - } + } private static Dataset getNewRelations( From 276b95b7b375a5369156d7b12ba006423291eda6 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 15:05:17 +0200 Subject: [PATCH 197/259] add create file instruction --- .../main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index 4f66d2f10..305b476b0 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -119,6 +119,7 @@ public class ReadBlacklistFromDB implements Closeable { this.conf.set("fs.defaultFS", hdfsNameNode); FileSystem fileSystem = FileSystem.get(this.conf); Path hdfsWritePath = new Path(hdfsPath); + fileSystem.create(hdfsWritePath); FSDataOutputStream fsDataOutputStream = fileSystem.append(hdfsWritePath); this.writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); } From 13f30664ea02578c9d9f8803ec88399c700c6132 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 15:23:49 +0200 Subject: [PATCH 198/259] minor --- ...ResultToCommunityFromOrganizationJob2.java | 2 +- ...rkResultToOrganizationFromIstRepoJob2.java | 37 +++++++++---------- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java index 30908012b..7cc3d6d59 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java @@ -70,7 +70,7 @@ public class SparkResultToCommunityFromOrganizationJob2 { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - if(saveGraph) + if (saveGraph) execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); }); } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java index b69158c87..72c75e8a6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java @@ -58,8 +58,6 @@ public class SparkResultToOrganizationFromIstRepoJob2 { final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); - - final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) @@ -78,7 +76,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { if (isTest(parser)) { removeOutputDir(spark, outputPath); } - if(saveGraph) + if (saveGraph) execPropagation( spark, datasourceorganization, @@ -86,7 +84,7 @@ public class SparkResultToOrganizationFromIstRepoJob2 { inputPath, outputPath, resultClazz); - }); + }); } private static void execPropagation( @@ -112,22 +110,21 @@ public class SparkResultToOrganizationFromIstRepoJob2 { broadcast_datasourceorganizationassoc) .as(Encoders.bean(ResultOrganizationSet.class)); - - getNewRelations( - spark - .read() - .textFile(alreadylinked) - .map( - value -> OBJECT_MAPPER - .readValue( - value, ResultOrganizationSet.class), - Encoders.bean(ResultOrganizationSet.class)), - potentialUpdates) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression", "gzip") - .text(outputPath); + getNewRelations( + spark + .read() + .textFile(alreadylinked) + .map( + value -> OBJECT_MAPPER + .readValue( + value, ResultOrganizationSet.class), + Encoders.bean(ResultOrganizationSet.class)), + potentialUpdates) + .toJSON() + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .text(outputPath); } From d6d6ebeae5a998316f20faf7c3fe81c42e85e40e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:25:33 +0200 Subject: [PATCH 199/259] preparation step: creates the subset of the merges relations --- .../blacklist/PrepareMergedRelationJob.java | 91 +++++++++++++++++++ .../SparkRemoveBlacklistedRelationJob.java | 4 + .../input_preparerelation_parameters.json | 26 ++++++ .../blacklist/sparkblacklist_parameters.json | 32 +++++++ 4 files changed, 153 insertions(+) create mode 100644 dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java create mode 100644 dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java create mode 100644 dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json create mode 100644 dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java new file mode 100644 index 000000000..3d4d4315e --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -0,0 +1,91 @@ +package eu.dnetlib.dhp.blacklist; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +public class PrepareMergedRelationJob { + + private static final Logger log = LoggerFactory.getLogger(PrepareMergedRelationJob.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + PrepareMergedRelationJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + String outputPath = parser.get("outputPath"); + log.info("outputPath: {} " , outputPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + // removeOutputDir(spark, potentialUpdatePath); + // removeOutputDir(spark, alreadyLinkedPath); + selectMergesRelations( + spark, + inputPath, + outputPath); + }); + } + + private static void selectMergesRelations(SparkSession spark, String inputPath, String outputPath) { + + Dataset relation = readRelations(spark, inputPath); + relation.createOrReplaceTempView("relation"); + + spark.sql("Select * from relation where relclass = 'merges' and datainfo.deletedbyinference = false") + .as(Encoders.bean(Relation.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression","gzip") + .text(outputPath); + } + + public static org.apache.spark.sql.Dataset readRelations( + SparkSession spark, String inputPath) { + return spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), + Encoders.bean(Relation.class)); + } +} diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java new file mode 100644 index 000000000..348468f51 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.blacklist; + +public class RemoveBlacklistedRelationSparkJob { +} diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json new file mode 100644 index 000000000..69de24213 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "p", + "paramLongName": "hdfsPath", + "paramDescription": "the path where storing the sequential file", + "paramRequired": true + }, + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the path to the graph used to remove the relations ", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path where to store the temporary result ", + "paramRequired": true + }, + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json new file mode 100644 index 000000000..9a2eadaa7 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "p", + "paramLongName": "hdfsPath", + "paramDescription": "the path where storing the sequential file", + "paramRequired": true + }, + { + "paramName": "nn", + "paramLongName": "hdfsNameNode", + "paramDescription": "the name node on hdfs", + "paramRequired": true + }, + { + "paramName": "pgurl", + "paramLongName": "postgresUrl", + "paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb", + "paramRequired": true + }, + { + "paramName": "pguser", + "paramLongName": "postgresUser", + "paramDescription": "postgres user", + "paramRequired": false + }, + { + "paramName": "pgpasswd", + "paramLongName": "postgresPassword", + "paramDescription": "postgres password", + "paramRequired": false + } +] \ No newline at end of file From b9d56b3cedfec55c0fe6ebc317a43c1b368358a1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:26:25 +0200 Subject: [PATCH 200/259] applies the actual removal of the relations --- .../SparkRemoveBlacklistedRelationJob.java | 132 +++++++++++++++++- 1 file changed, 131 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 348468f51..2dacb542b 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -1,4 +1,134 @@ + package eu.dnetlib.dhp.blacklist; -public class RemoveBlacklistedRelationSparkJob { +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class SparkRemoveBlacklistedRelationJob { + private static final Logger log = LoggerFactory.getLogger(SparkRemoveBlacklistedRelationJob.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkRemoveBlacklistedRelationJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + final String blacklistPath = parser.get("hdfsPath"); + log.info("blacklistPath {}: ", blacklistPath); + + final String mergesPath = parser.get("mergesPath"); + log.info("mergesPath {}: ", mergesPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeBlacklistedRelations( + spark, + inputPath, + blacklistPath, + outputPath, + mergesPath); + }); + } + + private static void removeBlacklistedRelations(SparkSession spark, String blacklistPath, String inputPath, + String outputPath, String mergesPath) { + Dataset blackListed = readRelations(spark, blacklistPath); + Dataset inputRelation = readRelations(spark, inputPath); + Dataset mergesRelation = readRelations(spark, mergesPath); + + Dataset dedupSource = blackListed + .joinWith(mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), "left_outer") + // .joinWith(inputRelation,blackListed.col("target").equalTo(inputRelation.col("target")),"left_outer") + .map(c -> { + Optional merged = Optional.ofNullable(c._2()); + Relation bl = c._1(); + if (merged.isPresent()) { + bl.setSource(merged.get().getSource()); + } + return bl; + }, Encoders.bean(Relation.class)); + + Dataset dedupBL = dedupSource + .joinWith(mergesRelation, dedupSource.col("target").equalTo(mergesRelation.col("target")), "left_outer") + .map(c -> { + Optional merged = Optional.ofNullable(c._2()); + Relation bl = c._1(); + if (merged.isPresent()) { + bl.setTarget(merged.get().getSource()); + } + return bl; + }, Encoders.bean(Relation.class)); + + inputRelation + .joinWith(dedupBL, inputRelation.col("source").equalTo(dedupBL.col("source")), "left_outer") + .map(c -> { + Relation ir = c._1(); + Optional obl = Optional.ofNullable(c._2()); + if (obl.isPresent()) { + if (ir.equals(obl.get())) { + return null; + } + } + return ir; + + }, Encoders.bean(Relation.class)) + .filter(r -> !(r == null)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("conpression", "gzip") + .text(outputPath); + + } + + public static org.apache.spark.sql.Dataset readRelations( + SparkSession spark, String inputPath) { + return spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), + Encoders.bean(Relation.class)); + } + } From 1070790c198b8eb23ae06657c65aacd1e2788a90 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:26:58 +0200 Subject: [PATCH 201/259] minor --- .../dnetlib/dhp/blacklist/ReadBlacklistFromDB.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index 305b476b0..5e7191834 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -34,6 +34,7 @@ public class ReadBlacklistFromDB implements Closeable { private static final Log log = LogFactory.getLog(ReadBlacklistFromDB.class); private final Configuration conf; private final BufferedWriter writer; + private final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private final static String query = "SELECT source_type, unnest(original_source_objects) as source, " + "target_type, unnest(original_target_objects) as target, " + @@ -119,14 +120,19 @@ public class ReadBlacklistFromDB implements Closeable { this.conf.set("fs.defaultFS", hdfsNameNode); FileSystem fileSystem = FileSystem.get(this.conf); Path hdfsWritePath = new Path(hdfsPath); - fileSystem.create(hdfsWritePath); - FSDataOutputStream fsDataOutputStream = fileSystem.append(hdfsWritePath); + FSDataOutputStream fsDataOutputStream = null; + if (fileSystem.exists(hdfsWritePath)) { + fsDataOutputStream = fileSystem.append(hdfsWritePath); + } else { + fsDataOutputStream = fileSystem.create(hdfsWritePath); + } + this.writer = new BufferedWriter(new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); } protected void writeRelation(final Relation r) { try { - writer.write(new ObjectMapper().writeValueAsString(r)); + writer.write(OBJECT_MAPPER.writeValueAsString(r)); writer.newLine(); } catch (final Exception e) { throw new RuntimeException(e); From 95433e10872849fa535cd9d731bb98db809be627 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:28:13 +0200 Subject: [PATCH 202/259] parameters for the preparation phase and blacklist phase --- .../input_preparerelation_parameters.json | 12 +++++----- .../blacklist/sparkblacklist_parameters.json | 24 +++++++------------ 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json index 69de24213..4a3d21f4d 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json @@ -1,10 +1,4 @@ [ - { - "paramName": "p", - "paramLongName": "hdfsPath", - "paramDescription": "the path where storing the sequential file", - "paramRequired": true - }, { "paramName": "s", "paramLongName": "sourcePath", @@ -22,5 +16,11 @@ "paramLongName": "isSparkSessionManaged", "paramDescription": "true if the spark session is managed", "paramRequired": false + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json index 9a2eadaa7..69de24213 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json @@ -6,27 +6,21 @@ "paramRequired": true }, { - "paramName": "nn", - "paramLongName": "hdfsNameNode", - "paramDescription": "the name node on hdfs", + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the path to the graph used to remove the relations ", "paramRequired": true }, { - "paramName": "pgurl", - "paramLongName": "postgresUrl", - "paramDescription": "postgres url, example: jdbc:postgresql://localhost:5432/testdb", + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path where to store the temporary result ", "paramRequired": true }, { - "paramName": "pguser", - "paramLongName": "postgresUser", - "paramDescription": "postgres user", - "paramRequired": false - }, - { - "paramName": "pgpasswd", - "paramLongName": "postgresPassword", - "paramDescription": "postgres password", + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed", "paramRequired": false } ] \ No newline at end of file From 38ecfd57855b212cda31863ab78bbd2b5ae2e418 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:28:46 +0200 Subject: [PATCH 203/259] the wf with all the three steps for blacklisting relations --- .../dhp/blacklist/oozie_app/workflow.xml | 53 ++++++++++++++++++- 1 file changed, 52 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index e38d721b9..48351e3fb 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -12,6 +12,10 @@ postgresPassword the postgres password + + sourcePath + the source path + @@ -38,9 +42,56 @@ --postgresUser${postgresUser} --postgresPassword${postgresPassword} - + + + + yarn + cluster + PrepareMergedRelation + eu.dnetlib.dhp.blacklist.PrepareMergedRelationJob + dhp-blacklist-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --outputPath${workingDir}/relation + --hive_metastore_uris${hive_metastore_uris} + + + + + + + + yarn + cluster + ApplyBlacklist + eu.dnetlib.dhp.blacklist.SparkRemoveBlacklistedRelationJob + dhp-blacklist-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --outputPath${workingDir}/relation + --hdfsPath${workingDir}/blacklist + + + + \ No newline at end of file From 0631fe548acbb69d78d9591a69151b4e0e3c2efa Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:29:46 +0200 Subject: [PATCH 204/259] pom.xml --- dhp-workflows/dhp-blacklist/pom.xml | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 1c6c00e9a..6854ea98e 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -18,14 +18,23 @@ compile - com.fasterxml.jackson.core - jackson-databind - compile + eu.dnetlib.dhp + dhp-common + ${project.version} - org.apache.hadoop - hadoop-common - compile + eu.dnetlib.dhp + dhp-schemas + ${project.version} + + + + org.apache.spark + spark-core_2.11 + + + org.apache.spark + spark-sql_2.11 From 43053a286dd96c4a52c4893177e028f33617e18b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 18:30:21 +0200 Subject: [PATCH 205/259] workflow pom with added blacklist module --- dhp-workflows/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index 852b64483..692a6c90f 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -25,6 +25,7 @@ dhp-graph-provision dhp-dedup-scholexplorer dhp-graph-provision-scholexplorer + dhp-blacklist From dbf3ba051a613f40984bc721d1c467c256d53d4c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 30 Apr 2020 20:22:07 +0200 Subject: [PATCH 206/259] minor --- dhp-workflows/dhp-blacklist/pom.xml | 1 + .../blacklist/PrepareMergedRelationJob.java | 123 +++++++++--------- .../SparkRemoveBlacklistedRelationJob.java | 24 ++-- .../dhp/blacklist/oozie_app/workflow.xml | 3 +- .../blacklist/sparkblacklist_parameters.json | 7 + 5 files changed, 84 insertions(+), 74 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 6854ea98e..497516d85 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -11,6 +11,7 @@ dhp-blacklist + eu.dnetlib.dhp dhp-graph-mapper diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java index 3d4d4315e..296c49a42 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -1,9 +1,10 @@ + package eu.dnetlib.dhp.blacklist; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -14,78 +15,78 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; +import com.fasterxml.jackson.databind.ObjectMapper; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; public class PrepareMergedRelationJob { - private static final Logger log = LoggerFactory.getLogger(PrepareMergedRelationJob.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(PrepareMergedRelationJob.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - PrepareMergedRelationJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + PrepareMergedRelationJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/blacklist/input_preparerelation_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {} " , outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {} ", outputPath); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - // removeOutputDir(spark, potentialUpdatePath); - // removeOutputDir(spark, alreadyLinkedPath); - selectMergesRelations( - spark, - inputPath, - outputPath); - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + selectMergesRelations( + spark, + inputPath, + outputPath); + }); + } - private static void selectMergesRelations(SparkSession spark, String inputPath, String outputPath) { + private static void selectMergesRelations(SparkSession spark, String inputPath, String outputPath) { - Dataset relation = readRelations(spark, inputPath); - relation.createOrReplaceTempView("relation"); + Dataset relation = readRelations(spark, inputPath); + relation.createOrReplaceTempView("relation"); - spark.sql("Select * from relation where relclass = 'merges' and datainfo.deletedbyinference = false") - .as(Encoders.bean(Relation.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression","gzip") - .text(outputPath); - } + spark + .sql("Select * from relation " + + "where relclass = 'merges' " + + "and datainfo.deletedbyinference = false") + .as(Encoders.bean(Relation.class)) + .toJSON() + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(outputPath); + } - public static org.apache.spark.sql.Dataset readRelations( - SparkSession spark, String inputPath) { - return spark - .read() - .textFile(inputPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), - Encoders.bean(Relation.class)); - } + public static org.apache.spark.sql.Dataset readRelations( + SparkSession spark, String inputPath) { + return spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), + Encoders.bean(Relation.class)); + } } diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 2dacb542b..86dd0fbd3 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -55,19 +55,20 @@ public class SparkRemoveBlacklistedRelationJob { log.info("mergesPath {}: ", mergesPath); SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeBlacklistedRelations( - spark, - inputPath, - blacklistPath, - outputPath, - mergesPath); - }); + conf, + isSparkSessionManaged, + spark -> { + removeBlacklistedRelations( + spark, + inputPath, + blacklistPath, + outputPath, + mergesPath); + }); + + } private static void removeBlacklistedRelations(SparkSession spark, String blacklistPath, String inputPath, @@ -78,7 +79,6 @@ public class SparkRemoveBlacklistedRelationJob { Dataset dedupSource = blackListed .joinWith(mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), "left_outer") - // .joinWith(inputRelation,blackListed.col("target").equalTo(inputRelation.col("target")),"left_outer") .map(c -> { Optional merged = Optional.ofNullable(c._2()); Relation bl = c._1(); diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 48351e3fb..78bac5eaf 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -63,7 +63,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath} - --outputPath${workingDir}/relation + --outputPath${workingDir}/mergesRelation --hive_metastore_uris${hive_metastore_uris} @@ -89,6 +89,7 @@ --sourcePath${sourcePath} --outputPath${workingDir}/relation --hdfsPath${workingDir}/blacklist + --mergesPath${workingDir}/mergesRelation diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json index 69de24213..91a87b8b5 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/sparkblacklist_parameters.json @@ -22,5 +22,12 @@ "paramLongName": "isSparkSessionManaged", "paramDescription": "true if the spark session is managed", "paramRequired": false + }, + { + "paramName": "m", + "paramLongName": "mergesPath", + "paramDescription": "true if the spark session is managed", + "paramRequired": true + } ] \ No newline at end of file From b7dd400e5175ca7366ff8d98378ec8bfc8cf9ba3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 1 May 2020 15:09:02 +0200 Subject: [PATCH 207/259] added check if author.pid exists or is null --- .../SparkOrcidToResultFromSemRelJob3.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index 997b58bf2..4611eac39 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -187,7 +187,11 @@ public class SparkOrcidToResultFromSemRelJob3 { } private static boolean containsAllowedPid(Author a) { - for (StructuredProperty pid : a.getPid()) { + Optional> pids = Optional.ofNullable(a.getPid()); + if (!pids.isPresent()) { + return false; + } + for (StructuredProperty pid : pids.get()) { if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) { return true; } From 31ea05297dbfadf79dd8c1809b9b297cfcf91709 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 4 May 2020 12:22:28 +0200 Subject: [PATCH 208/259] moved the DbClient to common and added needed dependency to pom --- dhp-common/pom.xml | 4 ++ .../java/eu/dnetlib/dhp/common/DbClient.java | 62 +++++++++++++++++++ .../dhp/oa/graph/raw/common/DbClient.java | 61 ------------------ 3 files changed, 66 insertions(+), 61 deletions(-) create mode 100644 dhp-common/src/main/java/eu/dnetlib/dhp/common/DbClient.java delete mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java diff --git a/dhp-common/pom.xml b/dhp-common/pom.xml index 7b073397f..0daee9c8b 100644 --- a/dhp-common/pom.xml +++ b/dhp-common/pom.xml @@ -81,6 +81,10 @@ com.jayway.jsonpath json-path + + org.postgresql + postgresql + diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/DbClient.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/DbClient.java new file mode 100644 index 000000000..cedc9bd4d --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/DbClient.java @@ -0,0 +1,62 @@ + +package eu.dnetlib.dhp.common; + +import java.io.Closeable; +import java.io.IOException; +import java.sql.*; +import java.util.function.Consumer; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +public class DbClient implements Closeable { + + private static final Log log = LogFactory.getLog(DbClient.class); + + private Connection connection; + + public DbClient(final String address, final String login, final String password) { + + try { + Class.forName("org.postgresql.Driver"); + + this.connection = StringUtils.isNoneBlank(login, password) + ? DriverManager.getConnection(address, login, password) + : DriverManager.getConnection(address); + this.connection.setAutoCommit(false); + } catch (final Exception e) { + log.error("Connection to postgresDB failed"); + throw new RuntimeException("Connection to postgresDB failed", e); + } + log.info("Opened database successfully"); + } + + public void processResults(final String sql, final Consumer consumer) { + + try (final Statement stmt = connection.createStatement()) { + stmt.setFetchSize(100); + + try (final ResultSet rs = stmt.executeQuery(sql)) { + while (rs.next()) { + consumer.accept(rs); + } + } catch (final SQLException e) { + log.error("Error executing sql query: " + sql, e); + throw new RuntimeException("Error executing sql query", e); + } + } catch (final SQLException e1) { + log.error("Error preparing sql statement", e1); + throw new RuntimeException("Error preparing sql statement", e1); + } + } + + @Override + public void close() throws IOException { + try { + connection.close(); + } catch (final SQLException e) { + throw new RuntimeException(e); + } + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java deleted file mode 100644 index ca7c9fffb..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/DbClient.java +++ /dev/null @@ -1,61 +0,0 @@ -package eu.dnetlib.dhp.oa.graph.raw.common; - -import java.io.Closeable; -import java.io.IOException; -import java.sql.*; -import java.util.function.Consumer; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -public class DbClient implements Closeable { - - private static final Log log = LogFactory.getLog(DbClient.class); - - private Connection connection; - - public DbClient(final String address, final String login, final String password) { - - try { - Class.forName("org.postgresql.Driver"); - - this.connection = - StringUtils.isNoneBlank(login, password) - ? DriverManager.getConnection(address, login, password) - : DriverManager.getConnection(address); - this.connection.setAutoCommit(false); - } catch (final Exception e) { - log.error("Connection to postgresDB failed"); - throw new RuntimeException("Connection to postgresDB failed", e); - } - log.info("Opened database successfully"); - } - - public void processResults(final String sql, final Consumer consumer) { - - try (final Statement stmt = connection.createStatement()) { - stmt.setFetchSize(100); - - try (final ResultSet rs = stmt.executeQuery(sql)) { - while (rs.next()) { - consumer.accept(rs); - } - } catch (final SQLException e) { - log.error("Error executing sql query: " + sql, e); - throw new RuntimeException("Error executing sql query", e); - } - } catch (final SQLException e1) { - log.error("Error preparing sql statement", e1); - throw new RuntimeException("Error preparing sql statement", e1); - } - } - - @Override - public void close() throws IOException { - try { - connection.close(); - } catch (final SQLException e) { - throw new RuntimeException(e); - } - } -} From e218360f8acc9a5fbf601ab25eed00fae0bea8bb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 4 May 2020 12:26:17 +0200 Subject: [PATCH 209/259] changed code for the mode of DbClient and also removed the dependency to graph-mapper --- dhp-workflows/dhp-blacklist/pom.xml | 7 ------- .../blacklist/PrepareMergedRelationJob.java | 3 ++- .../dhp/blacklist/ReadBlacklistFromDB.java | 2 +- .../SparkRemoveBlacklistedRelationJob.java | 21 +++++++++---------- 4 files changed, 13 insertions(+), 20 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 497516d85..309f62a19 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -11,13 +11,6 @@ dhp-blacklist - - - eu.dnetlib.dhp - dhp-graph-mapper - 1.1.7-SNAPSHOT - compile - eu.dnetlib.dhp dhp-common diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java index 296c49a42..fbefc1c87 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -69,7 +69,8 @@ public class PrepareMergedRelationJob { relation.createOrReplaceTempView("relation"); spark - .sql("Select * from relation " + + .sql( + "Select * from relation " + "where relclass = 'merges' " + "and datainfo.deletedbyinference = false") .as(Encoders.bean(Relation.class)) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index 5e7191834..fc20eabe0 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; +import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.common.RelationInverse; import eu.dnetlib.dhp.schema.oaf.Relation; diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 86dd0fbd3..d25272263 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -57,17 +57,16 @@ public class SparkRemoveBlacklistedRelationJob { SparkConf conf = new SparkConf(); runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeBlacklistedRelations( - spark, - inputPath, - blacklistPath, - outputPath, - mergesPath); - }); - + conf, + isSparkSessionManaged, + spark -> { + removeBlacklistedRelations( + spark, + inputPath, + blacklistPath, + outputPath, + mergesPath); + }); } From 3957c815b9ab834c075f17d9c4242f0f2fdb14e2 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 4 May 2020 13:58:52 +0200 Subject: [PATCH 210/259] changed the name of some parameters --- .../dnetlib/dhp/bulktag/SparkBulkTagJob2.java | 6 ++-- .../dhp/bulktag/input_bulkTag_parameters.json | 4 +-- .../dhp/bulktag/oozie_app/workflow.xml | 20 ++++++------ .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 32 +++++++++---------- 4 files changed, 31 insertions(+), 31 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java index 7a8ae0bd0..a6e49e93b 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java @@ -57,9 +57,9 @@ public class SparkBulkTagJob2 { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - ProtoMap protoMappingParams = new Gson().fromJson(parser.get("protoMap"), ProtoMap.class); + ProtoMap protoMappingParams = new Gson().fromJson(parser.get("pathMap"), ProtoMap.class); ; - log.info("protoMap: {}", new Gson().toJson(protoMappingParams)); + log.info("pathMap: {}", new Gson().toJson(protoMappingParams)); final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); @@ -80,7 +80,7 @@ public class SparkBulkTagJob2 { if (isTest) { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { - cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookupUrl")); + cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); } runWithSparkSession( diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json index b370467ae..a37d7d168 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json @@ -1,7 +1,7 @@ [ { "paramName":"is", - "paramLongName":"isLookupUrl", + "paramLongName":"isLookUpUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true }, @@ -13,7 +13,7 @@ }, { "paramName": "pm", - "paramLongName":"protoMap", + "paramLongName":"pathMap", "paramDescription": "the json path associated to each selection field", "paramRequired": true }, diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index fe8f706e7..718ad40ec 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -5,11 +5,11 @@ the source path - isLookupUrl + isLookUpUrl the isLookup service endpoint - protoMap + pathMap the json path associated to each selection field @@ -121,8 +121,8 @@ --sourcePath${sourcePath}/publication --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${outputPath}/publication - --protoMap${protoMap} - --isLookupUrl${isLookupUrl} + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} @@ -149,8 +149,8 @@ --sourcePath${sourcePath}/dataset --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${outputPath}/dataset - --protoMap${protoMap} - --isLookupUrl${isLookupUrl} + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} @@ -177,8 +177,8 @@ --sourcePath${sourcePath}/otherresearchproduct --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${outputPath}/otherresearchproduct - --protoMap${protoMap} - --isLookupUrl${isLookupUrl} + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} @@ -205,8 +205,8 @@ --sourcePath${sourcePath}/software --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${outputPath}/software - --protoMap${protoMap} - --isLookupUrl${isLookupUrl} + --pathMap${pathMap} + --isLookUpUrl${isLookUpUrl} diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index 6873f2df9..cb5015ba1 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -99,9 +99,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -151,9 +151,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -258,9 +258,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -349,9 +349,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Publication", "-outputPath", workingDir.toString() + "/publication", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -433,9 +433,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", "-outputPath", workingDir.toString() + "/orp", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -565,9 +565,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -703,9 +703,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Software", "-outputPath", workingDir.toString() + "/software", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," @@ -814,9 +814,9 @@ public class BulkTagJobTest { "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-isLookupUrl", + "-isLookUpUrl", "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-protoMap", + "-pathMap", "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," From 78578c3ccf5ed83f619b2a6090cdfc6c152e89a4 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 4 May 2020 15:46:24 +0200 Subject: [PATCH 211/259] fixed wrong trnasition name in workflow --- .../eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 17bf9adf2..72ced0905 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -107,7 +107,7 @@ ${nameNode}/${sourcePath}/organization ${nameNode}/${outputPath}/organization - + @@ -117,7 +117,7 @@ ${nameNode}/${sourcePath}/project ${nameNode}/${outputPath}/project - + @@ -128,7 +128,7 @@ ${nameNode}/${sourcePath}/datasource ${nameNode}/${outputPath}/datasource - + From 252b219dd54082fa0420212b5cfc805e55361772 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 5 May 2020 10:03:32 +0200 Subject: [PATCH 212/259] chanced the name of some properties --- .../PrepareResultCommunitySetStep1.java | 2 +- ...ut_preparecommunitytoresult_parameters.json | 2 +- .../oozie_app/workflow.xml | 18 +++++++++--------- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index 5aef1c370..95fad98d7 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -61,7 +61,7 @@ public class PrepareResultCommunitySetStep1 { final List allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";")); log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel)); - final String isLookupUrl = parser.get("isLookupUrl"); + final String isLookupUrl = parser.get("isLookUpUrl"); log.info("isLookupUrl: {}", isLookupUrl); final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json index a5dfefc0a..8c99da673 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json @@ -1,7 +1,7 @@ [ { "paramName":"is", - "paramLongName":"isLookupUrl", + "paramLongName":"isLookUpUrl", "paramDescription": "URL of the isLookUp Service", "paramRequired": true }, diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index e033e0e34..977e4838c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -9,7 +9,7 @@ the semantic relationships allowed for propagation - isLookupUrl + isLookUpUrl the isLookup service endpoint @@ -121,7 +121,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${workingDir}/preparedInfo/targetCommunityAssoc --allowedsemrels${allowedsemrels} - --isLookupUrl${isLookUpUrl} + --isLookUpUrl${isLookUpUrl} @@ -149,7 +149,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${workingDir}/preparedInfo/targetCommunityAssoc --allowedsemrels${allowedsemrels} - --isLookupUrl${isLookupUrl} + --isLookUpUrl${isLookUpUrl} @@ -177,7 +177,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${workingDir}/preparedInfo/targetCommunityAssoc --allowedsemrels${allowedsemrels} - --isLookupUrl${isLookupUrl} + --isLookUpUrl${isLookUpUrl} @@ -205,7 +205,7 @@ --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${workingDir}/preparedInfo/targetCommunityAssoc --allowedsemrels${allowedsemrels} - --isLookupUrl${isLookupUrl} + --isLookUpUrl${isLookUpUrl} @@ -270,7 +270,7 @@ --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${outputPath}/publication - --saveGrap${saveGraph + --saveGraph${saveGraph} @@ -298,7 +298,7 @@ --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${outputPath}/dataset - --saveGrap${saveGraph + --saveGraph${saveGraph} @@ -326,7 +326,7 @@ --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${outputPath}/otherresearchproduct - --saveGrap${saveGraph + --saveGraph${saveGraph} @@ -354,7 +354,7 @@ --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${outputPath}/software - --saveGrap${saveGraph + --saveGraph${saveGraph} From dd2e698a724de08c381171b67f55499f20fa71e0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 5 May 2020 17:03:43 +0200 Subject: [PATCH 213/259] added a sequentialization step on the spark job. Addedd new parameter --- .../PrepareResultCountrySet.java | 4 + .../SparkCountryPropagationJob2.java | 160 +++++++++++++----- .../countrypropagation/oozie_app/workflow.xml | 4 + .../oozie_app/workflow.xml | 12 ++ 4 files changed, 133 insertions(+), 47 deletions(-) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java new file mode 100644 index 000000000..9a8df9d3e --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.countrypropagation; + +public class PrepareResultCountrySet { +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 15f9e9b60..58bc741de 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -3,8 +3,10 @@ package eu.dnetlib.dhp.countrypropagation; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import static jdk.nashorn.internal.objects.NativeDebug.map; import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -52,6 +54,11 @@ public class SparkCountryPropagationJob2 { final String datasourcecountrypath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", datasourcecountrypath); + final String possibleUpdatesPath = datasourcecountrypath + .substring(0, datasourcecountrypath.lastIndexOf("/") + 1) + + "possibleUpdates"; + log.info("possibleUpdatesPath: {}", possibleUpdatesPath); + final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); @@ -70,13 +77,14 @@ public class SparkCountryPropagationJob2 { conf, isSparkSessionManaged, spark -> { + removeOutputDir(spark, possibleUpdatesPath); execPropagation( spark, datasourcecountrypath, inputPath, outputPath, resultClazz, - saveGraph); + saveGraph, possibleUpdatesPath); }); } @@ -86,19 +94,30 @@ public class SparkCountryPropagationJob2 { String inputPath, String outputPath, Class resultClazz, - boolean saveGraph) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + boolean saveGraph, String possilbeUpdatesPath) { + // final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); // Load file with preprocessed association datasource - country Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); // broadcasting the result of the preparation step - Broadcast> broadcast_datasourcecountryassoc = sc.broadcast(datasourcecountryassoc); + // Broadcast> broadcast_datasourcecountryassoc = + // sc.broadcast(datasourcecountryassoc); Dataset potentialUpdates = getPotentialResultToUpdate( - spark, inputPath, resultClazz, broadcast_datasourcecountryassoc) + spark, inputPath, resultClazz, datasourcecountryassoc) .as(Encoders.bean(ResultCountrySet.class)); + potentialUpdates.write().option("compression", "gzip").mode(SaveMode.Overwrite).json(possilbeUpdatesPath); + if (saveGraph) { + // updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); + potentialUpdates = spark + .read() + .textFile(possilbeUpdatesPath) + .map( + (MapFunction) value -> OBJECT_MAPPER + .readValue(value, ResultCountrySet.class), + Encoders.bean(ResultCountrySet.class)); updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); } } @@ -113,69 +132,116 @@ public class SparkCountryPropagationJob2 { log.info("Reading Graph table from: {}", inputPath); Dataset result = readPathEntity(spark, inputPath, resultClazz); - Dataset> result_pair = result - .map( - r -> new Tuple2<>(r.getId(), r), - Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); - - Dataset new_table = result_pair + Dataset new_table = result .joinWith( - potentialUpdates, - result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), + potentialUpdates, result + .col("id") + .equalTo(potentialUpdates.col("resultId")), "left_outer") - .map( - (MapFunction, ResultCountrySet>, R>) value -> { - R r = value._1()._2(); - Optional potentialNewCountries = Optional.ofNullable(value._2()); - if (potentialNewCountries.isPresent()) { - HashSet countries = new HashSet<>(); - for (Qualifier country : r.getCountry()) { - countries.add(country.getClassid()); - } - Result res = new Result(); - res.setId(r.getId()); - List countryList = new ArrayList<>(); - for (CountrySbs country : potentialNewCountries - .get() - .getCountrySet()) { - if (!countries.contains(country.getClassid())) { - countryList - .add( - getCountry( - country.getClassid(), - country.getClassname())); - } - } - res.setCountry(countryList); - r.mergeFrom(res); - } - return r; - }, - Encoders.bean(resultClazz)); + .map((MapFunction, R>) value -> { + R r = value._1(); + Optional potentialNewCountries = Optional.ofNullable(value._2()); + if (potentialNewCountries.isPresent()) { + HashSet countries = r + .getCountry() + .stream() + .map(c -> c.getClassid()) + .collect(Collectors.toCollection(HashSet::new)); + + r + .getCountry() + .addAll( + potentialNewCountries + .get() + .getCountrySet() + .stream() + .filter(c -> !countries.contains(c.getClassid())) + .map(c -> getCountry(c.getClassid(), c.getClassname())) + .collect(Collectors.toList())); + +// Result res = new Result(); +// res.setId(r.getId()); +// List countryList = new ArrayList<>(); +// for (CountrySbs country : potentialNewCountries +// .get() +// .getCountrySet()) { +// if (!countries.contains(country.getClassid())) { +// countryList +// .add( +// getCountry( +// country.getClassid(), +// country.getClassname())); +// } +// } +// res.setCountry(countryList); +// r.mergeFrom(res); + } + return r; + }, Encoders.bean(resultClazz)); +// Dataset> result_pair = result +// .map( +// r -> new Tuple2<>(r.getId(), r), +// Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); +// +// Dataset new_table = result_pair +// .joinWith( +// potentialUpdates, +// result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), +// "left_outer") +// .map( +// (MapFunction, ResultCountrySet>, R>) value -> { +// R r = value._1()._2(); +// Optional potentialNewCountries = Optional.ofNullable(value._2()); +// if (potentialNewCountries.isPresent()) { +// HashSet countries = new HashSet<>(); +// for (Qualifier country : r.getCountry()) { +// countries.add(country.getClassid()); +// } +// Result res = new Result(); +// res.setId(r.getId()); +// List countryList = new ArrayList<>(); +// for (CountrySbs country : potentialNewCountries +// .get() +// .getCountrySet()) { +// if (!countries.contains(country.getClassid())) { +// countryList +// .add( +// getCountry( +// country.getClassid(), +// country.getClassname())); +// } +// } +// res.setCountry(countryList); +// r.mergeFrom(res); +// } +// return r; +// }, +// Encoders.bean(resultClazz)); log.info("Saving graph table to path: {}", outputPath); // log.info("number of saved recordsa: {}", new_table.count()); - new_table.toJSON().write().option("compression", "gzip").text(outputPath); + new_table.write().option("compression", "gzip").mode(SaveMode.Overwrite).json(outputPath); + } private static Dataset getPotentialResultToUpdate( SparkSession spark, String inputPath, Class resultClazz, - Broadcast> broadcast_datasourcecountryassoc) { + Dataset datasourcecountryassoc) { Dataset result = readPathEntity(spark, inputPath, resultClazz); result.createOrReplaceTempView("result"); // log.info("number of results: {}", result.count()); createCfHbforresult(spark); - return countryPropagationAssoc(spark, broadcast_datasourcecountryassoc); + return countryPropagationAssoc(spark, datasourcecountryassoc); } private static Dataset countryPropagationAssoc( SparkSession spark, - Broadcast> broadcast_datasourcecountryassoc) { + Dataset datasource_country) { - Dataset datasource_country = broadcast_datasourcecountryassoc.value(); + // Dataset datasource_country = broadcast_datasourcecountryassoc.value(); datasource_country.createOrReplaceTempView("datasource_country"); log.info("datasource_country number : {}", datasource_country.count()); diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index d5fb199cd..70ac77434 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -140,6 +140,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false --sourcePath${sourcePath}/publication --hive_metastore_uris${hive_metastore_uris} @@ -169,6 +170,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false --sourcePath${sourcePath}/dataset --hive_metastore_uris${hive_metastore_uris} @@ -198,6 +200,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false --sourcePath${sourcePath}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} @@ -227,6 +230,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false --sourcePath${sourcePath}/software --hive_metastore_uris${hive_metastore_uris} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index ac25b6728..798717c3f 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -261,6 +261,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc --sourcePath${sourcePath}/publication @@ -289,6 +292,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc --sourcePath${sourcePath}/dataset @@ -317,6 +323,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc --sourcePath${sourcePath}/otherresearchproduct @@ -345,6 +354,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc --sourcePath${sourcePath}/software From 42ad51577ae2daa251ca58910bf93549799535e1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 09:57:49 +0200 Subject: [PATCH 214/259] new implementation with one more serialization step --- .../SparkCountryPropagationJob3.java | 145 ++++++++++++++++++ ...input_prepareresultcountry_parameters.json | 32 ++++ 2 files changed, 177 insertions(+) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java create mode 100644 dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java new file mode 100644 index 000000000..9e2cac757 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java @@ -0,0 +1,145 @@ + +package eu.dnetlib.dhp.countrypropagation; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Country; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Result; +import scala.Tuple2; + +public class SparkCountryPropagationJob3 { + + private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob3.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkCountryPropagationJob3.class + .getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + + execPropagation( + spark, + inputPath, + outputPath, + resultClazz, + saveGraph); + }); + } + + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + Class resultClazz, + boolean saveGraph) { + + if (saveGraph) { + // updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); + log.info("Reading Graph table from: {}", inputPath); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + JavaPairRDD results = sc + .textFile(inputPath) + .map(r -> OBJECT_MAPPER.readValue(r, resultClazz)) + .mapToPair(r -> new Tuple2<>(r.getId(), r)); + + JavaPairRDD tmp = results.reduceByKey((r1, r2) -> { + if (r1 == null) { + return r2; + } + if (r2 == null) { + return r1; + } + if (Optional.ofNullable(r1.getCollectedfrom()).isPresent()) { + r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); + return r1; + } + if (Optional.ofNullable(r2.getCollectedfrom()).isPresent()) { + r2.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); + return r2; + } + r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); + return r1; + }); + + tmp + .map(c -> c._2()) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath, GzipCodec.class); + } + } + + private static List getUnionCountries(List country, List country1) { + HashSet countries = country + .stream() + .map(c -> c.getClassid()) + .collect(Collectors.toCollection(HashSet::new)); + country + .addAll( + country1 + .stream() + .filter(c -> !(countries.contains(c.getClassid()))) + .collect(Collectors.toList())); + return country; + } + +} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json new file mode 100644 index 000000000..9956f3474 --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + { + "paramName":"h", + "paramLongName":"hive_metastore_uris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName":"tn", + "paramLongName":"resultTableName", + "paramDescription": "the name of the result table we are currently working on", + "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + } +] \ No newline at end of file From 5a476c7a13d697d6d5b717df7fe0154505603e9e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 09:58:17 +0200 Subject: [PATCH 215/259] chenged the xquery for the cfhb table --- .../java/eu/dnetlib/dhp/PropagationConstant.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 5bab36a5f..ebd25fa35 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -157,11 +157,16 @@ public class PropagationConstant { public static void createCfHbforresult(SparkSession spark) { String query; - query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " - + "FROM ( SELECT id, instance " - + "FROM result " - + " WHERE datainfo.deletedbyinference = false) ds " - + "LATERAL VIEW EXPLODE(instance) i AS inst"; +// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " +// + "FROM ( SELECT id, instance " +// + "FROM result " +// + " WHERE datainfo.deletedbyinference = false) ds " +// + "LATERAL VIEW EXPLODE(instance) i AS inst"; + query = "select distinct r.id, inst.collectedfrom.key cf, inst.hostedby.key hb " + + "from result r " + + "lateral view explode(instance) i as inst " + + "where r.datainfo.deletedbyinference=false"; + org.apache.spark.sql.Dataset cfhb = spark.sql(query); cfhb.createOrReplaceTempView("cfhb"); } From 16193cf0ba71d64489a5a5dfbc22f75937e63122 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 09:59:58 +0200 Subject: [PATCH 216/259] new workflow and parameter for country propagation --- .../PrepareResultCountrySet.java | 130 ++++++++++ .../input_countrypropagation_parameters.json | 4 +- .../countrypropagation/oozie_app/workflow.xml | 223 ++++++++++++++++-- 3 files changed, 335 insertions(+), 22 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java index 9a8df9d3e..fec4a08ce 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java @@ -1,4 +1,134 @@ + package eu.dnetlib.dhp.countrypropagation; +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.util.Arrays; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; + public class PrepareResultCountrySet { + private static final Logger log = LoggerFactory.getLogger(PrepareResultCountrySet.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkCountryPropagationJob2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String datasourcecountrypath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", datasourcecountrypath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + getPotentialResultToUpdate( + spark, + inputPath, + datasourcecountrypath, + resultClazz); + }); + + } + + private static void getPotentialResultToUpdate( + SparkSession spark, + String inputPath, + String datasourcecountrypath, + Class resultClazz) { + + Dataset result = readPathEntity(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + // log.info("number of results: {}", result.count()); + createCfHbforresult(spark); + Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); + countryPropagationAssoc(spark, datasourcecountryassoc) + .map((MapFunction) value -> { + R ret = resultClazz.newInstance(); + ret.setId(value.getResultId()); + ret + .setCountry( + value + .getCountrySet() + .stream() + .map(c -> getCountry(c.getClassid(), c.getClassname())) + .collect(Collectors.toList())); + return ret; + }, Encoders.bean(resultClazz)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Append) + .json(inputPath); + } + + private static Dataset countryPropagationAssoc( + SparkSession spark, + Dataset datasource_country) { + + // Dataset datasource_country = broadcast_datasourcecountryassoc.value(); + datasource_country.createOrReplaceTempView("datasource_country"); + log.info("datasource_country number : {}", datasource_country.count()); + + String query = "SELECT id resultId, collect_set(country) countrySet " + + "FROM ( SELECT id, country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON cf = dataSourceId " + + "UNION ALL " + + "SELECT id , country " + + "FROM datasource_country " + + "JOIN cfhb " + + " ON hb = dataSourceId ) tmp " + + "GROUP BY id"; + Dataset potentialUpdates = spark + .sql(query) + .as(Encoders.bean(ResultCountrySet.class)); + // log.info("potential update number : {}", potentialUpdates.count()); + return potentialUpdates; + } + + private static Dataset readAssocDatasourceCountry( + SparkSession spark, String relationPath) { + return spark + .read() + .textFile(relationPath) + .map( + value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), + Encoders.bean(DatasourceCountry.class)); + } } diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json index 912c88743..984b40774 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json @@ -9,7 +9,7 @@ "paramName":"h", "paramLongName":"hive_metastore_uris", "paramDescription": "the hive metastore uris", - "paramRequired": true + "paramRequired": false }, { "paramName":"sg", @@ -33,7 +33,7 @@ "paramName": "p", "paramLongName": "preparedInfoPath", "paramDescription": "the path where prepared info have been stored", - "paramRequired": true + "paramRequired": false }, { "paramName": "ssm", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 70ac77434..469aa2562 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -27,6 +27,10 @@ + + + + @@ -45,6 +49,10 @@ + + + + @@ -89,6 +97,50 @@ + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/publication + ${nameNode}/${workingDir}/publication + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/dataset + ${nameNode}/${workingDir}/dataset + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/otherresearchproduct + ${nameNode}/${workingDir}/otherresearchproduct + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/software + ${nameNode}/${workingDir}/software + + + + + @@ -113,9 +165,139 @@ --hive_metastore_uris${hive_metastore_uris} --outputPath${workingDir}/preparedInfo - + + + + + + + + + + + + yarn + cluster + prepareResultCountry-Publication + eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + + --sourcePath${workingDir}/publication + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --preparedInfoPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + prepareResultCountry-Dataset + eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + + --sourcePath${workingDir}/dataset + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --preparedInfoPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + prepareResultCountry-ORP + eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + + --sourcePath${workingDir}/otherresearchproduct + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --preparedInfoPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + prepareResultCountry-Software + eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + + --sourcePath${workingDir}/software + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --preparedInfoPath${workingDir}/preparedInfo + + + + + + + @@ -128,26 +310,27 @@ yarn cluster countryPropagationForPublications - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 dhp-propagation-${projectVersion}.jar - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} + + --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 - --sourcePath${sourcePath}/publication - --hive_metastore_uris${hive_metastore_uris} + --sourcePath${workingDir}/publication --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${outputPath}/publication - --preparedInfoPath${workingDir}/preparedInfo @@ -158,7 +341,7 @@ yarn cluster countryPropagationForDataset - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -171,13 +354,13 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false - --sourcePath${sourcePath}/dataset - --hive_metastore_uris${hive_metastore_uris} + --sourcePath${workingDir}/dataset --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${outputPath}/dataset - --preparedInfoPath${workingDir}/preparedInfo @@ -188,7 +371,7 @@ yarn cluster countryPropagationForORP - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -201,13 +384,13 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false - --sourcePath${sourcePath}/otherresearchproduct - --hive_metastore_uris${hive_metastore_uris} + --sourcePath${workingDir}/otherresearchproduct --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${outputPath}/otherresearchproduct - --preparedInfoPath${workingDir}/preparedInfo @@ -218,7 +401,7 @@ yarn cluster countryPropagationForSoftware - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob2 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -231,13 +414,13 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false - --sourcePath${sourcePath}/software - --hive_metastore_uris${hive_metastore_uris} + --sourcePath${workingDir}/software --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${outputPath}/software - --preparedInfoPath${workingDir}/preparedInfo From 55e825acd43372cd7e3e5302afb7ab11cb048ee5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 10:01:00 +0200 Subject: [PATCH 217/259] chenged the test according to changes in SparkCOuntryPropagationJob2 --- .../dhp/countrypropagation/CountryPropagationJobTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 3bc499233..6c66606e1 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -266,4 +266,5 @@ public class CountryPropagationJobTest { .assertEquals( 7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count()); } + } From 29bc8c44b101fe520ba234a71f0c842f5b9805a8 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 10:01:34 +0200 Subject: [PATCH 218/259] changes in the construction of new country set --- .../SparkCountryPropagationJob2.java | 36 +++++++++++++------ 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java index 58bc741de..059e388db 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java @@ -54,14 +54,17 @@ public class SparkCountryPropagationJob2 { final String datasourcecountrypath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", datasourcecountrypath); - final String possibleUpdatesPath = datasourcecountrypath - .substring(0, datasourcecountrypath.lastIndexOf("/") + 1) - + "possibleUpdates"; - log.info("possibleUpdatesPath: {}", possibleUpdatesPath); - final String resultClassName = parser.get("resultTableName"); log.info("resultTableName: {}", resultClassName); + final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); + log.info("resultType: {}", resultType); + + final String possibleUpdatesPath = datasourcecountrypath + .substring(0, datasourcecountrypath.lastIndexOf("/") + 1) + + "possibleUpdates/" + resultType; + log.info("possibleUpdatesPath: {}", possibleUpdatesPath); + final Boolean saveGraph = Optional .ofNullable(parser.get("saveGraph")) .map(Boolean::valueOf) @@ -219,12 +222,12 @@ public class SparkCountryPropagationJob2 { // Encoders.bean(resultClazz)); log.info("Saving graph table to path: {}", outputPath); - // log.info("number of saved recordsa: {}", new_table.count()); + log.info("number of saved recordsa: {}", new_table.count()); new_table.write().option("compression", "gzip").mode(SaveMode.Overwrite).json(outputPath); } - private static Dataset getPotentialResultToUpdate( + private static Dataset getPotentialResultToUpdate( SparkSession spark, String inputPath, Class resultClazz, @@ -237,7 +240,7 @@ public class SparkCountryPropagationJob2 { return countryPropagationAssoc(spark, datasourcecountryassoc); } - private static Dataset countryPropagationAssoc( + private static Dataset countryPropagationAssoc( SparkSession spark, Dataset datasource_country) { @@ -256,7 +259,19 @@ public class SparkCountryPropagationJob2 { + "JOIN cfhb " + " ON hb = dataSourceId ) tmp " + "GROUP BY id"; - Dataset potentialUpdates = spark.sql(query); + + Dataset potentialUpdates = spark + .sql(query) + .as(Encoders.bean(ResultCountrySet.class)) + .map((MapFunction) r -> { + final ArrayList c = r + .getCountrySet() + .stream() + .limit(100) + .collect(Collectors.toCollection(ArrayList::new)); + r.setCountrySet(c); + return r; + }, Encoders.bean(ResultCountrySet.class)); // log.info("potential update number : {}", potentialUpdates.count()); return potentialUpdates; } @@ -267,7 +282,8 @@ public class SparkCountryPropagationJob2 { .read() .textFile(relationPath) .map( - value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), + (MapFunction) value -> OBJECT_MAPPER + .readValue(value, DatasourceCountry.class), Encoders.bean(DatasourceCountry.class)); } } From b2fec32c87e8e36825489a3791b9cecc6b373e16 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 10:01:57 +0200 Subject: [PATCH 219/259] new workflow for job3 --- .../dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index 798717c3f..4c4b74b52 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -114,6 +114,10 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.sql.shuffle.partitions=3840 + --conf spark.speculation=false + --conf spark.hadoop.mapreduce.map.speculative=false + --conf spark.hadoop.mapreduce.reduce.speculative=false --sourcePath${sourcePath} --hive_metastore_uris${hive_metastore_uris} @@ -264,6 +268,7 @@ --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc --sourcePath${sourcePath}/publication From 128c3bf1c8d7497c252f2d9a9c204ed74bcfa7b1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 11:14:56 +0200 Subject: [PATCH 220/259] restored Author bean with simple getter/setter, author pid addition moved into dedicated implementation SparkOrcidToResultFromSemRelJob3 --- .../main/java/eu/dnetlib/dhp/schema/oaf/Author.java | 11 ----------- .../SparkOrcidToResultFromSemRelJob3.java | 11 ++++++++++- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java index 89398455e..231fb1e60 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/oaf/Author.java @@ -86,15 +86,4 @@ public class Author implements Serializable { return Objects.hash(fullname, name, surname, rank, pid, affiliation); } - public void addPid(StructuredProperty pid) { - - if (pid == null) - return; - - if (this.pid == null) { - this.pid = Arrays.asList(pid); - } else { - this.pid.add(pid); - } - } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java index 4611eac39..6214e18ca 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java @@ -6,6 +6,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import java.util.List; import java.util.Optional; +import java.util.function.Consumer; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; @@ -18,6 +19,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Author; @@ -181,7 +183,14 @@ public class SparkOrcidToResultFromSemRelJob3 { PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME)); - author.addPid(p); + + Optional> authorPid = Optional.ofNullable(author.getPid()); + if (authorPid.isPresent()) { + authorPid.get().add(p); + } else { + author.setPid(Lists.newArrayList(p)); + } + } return toaddpid; } From 73243793b2a67eefae0ccb9720c13298b92e6b31 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 11:15:24 +0200 Subject: [PATCH 221/259] Dataset based implementation for SparkCountryPropagationJob3 --- .../SparkCountryPropagationJob3.java | 90 ++++++++++--------- .../countrypropagation/oozie_app/workflow.xml | 34 ++++--- 2 files changed, 61 insertions(+), 63 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java index 9e2cac757..3d526ec69 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java @@ -4,10 +4,8 @@ package eu.dnetlib.dhp.countrypropagation; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; +import java.util.*; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; @@ -17,6 +15,7 @@ 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.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; @@ -27,6 +26,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Country; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Result; @@ -71,6 +71,7 @@ public class SparkCountryPropagationJob3 { Class resultClazz = (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); runWithSparkSession( conf, @@ -96,50 +97,51 @@ public class SparkCountryPropagationJob3 { if (saveGraph) { // updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); log.info("Reading Graph table from: {}", inputPath); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - JavaPairRDD results = sc - .textFile(inputPath) - .map(r -> OBJECT_MAPPER.readValue(r, resultClazz)) - .mapToPair(r -> new Tuple2<>(r.getId(), r)); - JavaPairRDD tmp = results.reduceByKey((r1, r2) -> { - if (r1 == null) { - return r2; - } - if (r2 == null) { - return r1; - } - if (Optional.ofNullable(r1.getCollectedfrom()).isPresent()) { - r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); - return r1; - } - if (Optional.ofNullable(r2.getCollectedfrom()).isPresent()) { - r2.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); - return r2; - } - r1.setCountry(getUnionCountries(r1.getCountry(), r2.getCountry())); - return r1; - }); - - tmp - .map(c -> c._2()) - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); + spark + .read() + .json(inputPath) + .as(Encoders.kryo(resultClazz)) + .groupByKey((MapFunction) result1 -> result1.getId(), Encoders.STRING()) + .mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz)) + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(outputPath); } } - private static List getUnionCountries(List country, List country1) { - HashSet countries = country - .stream() - .map(c -> c.getClassid()) - .collect(Collectors.toCollection(HashSet::new)); - country - .addAll( - country1 - .stream() - .filter(c -> !(countries.contains(c.getClassid()))) - .collect(Collectors.toList())); - return country; + private static MapGroupsFunction getCountryMergeFn(Class resultClazz) { + return (MapGroupsFunction) (key, values) -> { + R res = resultClazz.newInstance(); + List countries = new ArrayList<>(); + values.forEachRemaining(r -> { + res.mergeFrom(r); + countries.addAll(r.getCountry()); + }); + res + .setCountry( + countries + .stream() + .collect( + Collectors + .toMap( + Country::getClassid, + Function.identity(), + (c1, c2) -> { + if (Optional + .ofNullable( + c1.getDataInfo().getInferenceprovenance()) + .isPresent()) { + return c2; + } + return c1; + })) + .values() + .stream() + .collect(Collectors.toList())); + return res; + }; } } diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 469aa2562..ac0fff2c0 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -24,6 +24,7 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + @@ -158,6 +159,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=300 --sourcePath${sourcePath} --whitelist${whitelist} @@ -191,11 +193,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/publication --hive_metastore_uris${hive_metastore_uris} @@ -221,11 +222,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/dataset --hive_metastore_uris${hive_metastore_uris} @@ -251,11 +251,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} @@ -281,11 +280,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/software --hive_metastore_uris${hive_metastore_uris} @@ -313,15 +311,13 @@ eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 dhp-propagation-${projectVersion}.jar - - + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false @@ -351,11 +347,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/dataset --saveGraph${saveGraph} @@ -381,11 +376,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/otherresearchproduct --saveGraph${saveGraph} @@ -411,11 +405,10 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --conf spark.speculation=false --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false + --conf spark.sql.shuffle.partitions=3840 --sourcePath${workingDir}/software --saveGraph${saveGraph} @@ -425,6 +418,9 @@ - + + + + \ No newline at end of file From 5b3f8a0e909e020d5fe0b7aba89c798cc22854c8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 11:41:41 +0200 Subject: [PATCH 222/259] using Encoders.bean instead of kryo --- .../SparkCountryPropagationJob3.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java index 3d526ec69..4d1dcfbd5 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java @@ -71,20 +71,15 @@ public class SparkCountryPropagationJob3 { Class resultClazz = (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - runWithSparkSession( conf, isSparkSessionManaged, - spark -> { - - execPropagation( - spark, - inputPath, - outputPath, - resultClazz, - saveGraph); - }); + spark -> execPropagation( + spark, + inputPath, + outputPath, + resultClazz, + saveGraph)); } private static void execPropagation( @@ -101,7 +96,7 @@ public class SparkCountryPropagationJob3 { spark .read() .json(inputPath) - .as(Encoders.kryo(resultClazz)) + .as(Encoders.bean(resultClazz)) .groupByKey((MapFunction) result1 -> result1.getId(), Encoders.STRING()) .mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz)) .write() From e07feb4c5f230bc7e1a130c691afa5935ffa607e Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 11:42:46 +0200 Subject: [PATCH 223/259] removed spurious file --- : | 7 ------- 1 file changed, 7 deletions(-) delete mode 100644 : diff --git a/: b/: deleted file mode 100644 index 7768c1527..000000000 --- a/: +++ /dev/null @@ -1,7 +0,0 @@ -Merge remote-tracking branch 'upstream/master' - -# Please enter a commit message to explain why this merge is necessary, -# especially if it merges an updated upstream into a topic branch. -# -# Lines starting with '#' will be ignored, and an empty message aborts -# the commit. From 207b899d6d70c54348f21bf4fa2c6e61f74586eb Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 11:43:53 +0200 Subject: [PATCH 224/259] merged with upstream --- .../dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 8d2bda93a..b64642e02 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -522,7 +522,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication final String eissn = StringUtils.isNotBlank(arr[1]) ? arr[1].trim() : null; final String lissn = StringUtils.isNotBlank(arr[2]) ? arr[2].trim() : null; - + if (issn != null || eissn != null || lissn != null) { return journal(name, issn, eissn, eissn, null, null, null, null, null, null, null, info); } From 9f8855991c79a73fa513b59a83e787e77b269319 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 11:44:35 +0200 Subject: [PATCH 225/259] changed Encorders.bean to Encoders.kryo --- .../dhp/countrypropagation/SparkCountryPropagationJob3.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java index 3d526ec69..3e8fa7f64 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java @@ -71,7 +71,6 @@ public class SparkCountryPropagationJob3 { Class resultClazz = (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); runWithSparkSession( conf, @@ -101,7 +100,7 @@ public class SparkCountryPropagationJob3 { spark .read() .json(inputPath) - .as(Encoders.kryo(resultClazz)) + .as(Encoders.bean(resultClazz)) .groupByKey((MapFunction) result1 -> result1.getId(), Encoders.STRING()) .mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz)) .write() From 5111671e629121840d6fdfef448c681ea62c9713 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 11:47:00 +0200 Subject: [PATCH 226/259] celanup --- .../dhp/countrypropagation/SparkCountryPropagationJob3.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java index 4d1dcfbd5..8f12fcf3d 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java @@ -97,7 +97,7 @@ public class SparkCountryPropagationJob3 { .read() .json(inputPath) .as(Encoders.bean(resultClazz)) - .groupByKey((MapFunction) result1 -> result1.getId(), Encoders.STRING()) + .groupByKey((MapFunction) r -> r.getId(), Encoders.STRING()) .mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz)) .write() .option("compression", "gzip") From 166aafd9366a436ddc876329f479f1c432daa10a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 May 2020 18:22:26 +0200 Subject: [PATCH 227/259] heavy cleanup --- ...kBulkTagJob2.java => SparkBulkTagJob.java} | 74 +---- .../dhp/bulktag/oozie_app/workflow.xml | 8 +- .../java/eu/dnetlib/dhp/BulkTagJobTest.java | 18 +- .../eu/dnetlib/dhp/PropagationConstant.java | 63 +--- .../dnetlib/dhp/QueryInformationSystem.java | 20 -- .../PrepareDatasourceCountryAssociation.java | 38 +-- .../PrepareResultCountrySet.java | 86 ++---- ...3.java => SparkCountryPropagationJob.java} | 90 +++--- .../SparkCountryPropagationJob2.java | 289 ------------------ .../AutoritativeAuthor.java | 10 +- .../PrepareResultOrcidAssociationStep1.java | 62 ++-- .../PrepareResultOrcidAssociationStep2.java | 18 +- ...a => SparkOrcidToResultFromSemRelJob.java} | 61 ++-- .../PrepareProjectResultsAssociation.java | 62 +--- .../SparkResultToProjectThroughSemRelJob.java | 147 +++++++++ ...SparkResultToProjectThroughSemRelJob3.java | 159 ---------- .../PrepareResultCommunitySet.java | 73 +++-- ...ResultToCommunityFromOrganizationJob.java} | 93 +++--- .../PrepareResultCommunitySetStep1.java | 138 ++++----- .../PrepareResultCommunitySetStep2.java | 21 +- ...arkResultToCommunityThroughSemRelJob.java} | 103 ++++--- .../PrepareResultInstRepoAssociation.java | 44 +-- ...arkResultToOrganizationFromIstRepoJob.java | 193 ++++++++++++ ...rkResultToOrganizationFromIstRepoJob2.java | 232 -------------- ...input_prepareresultcountry_parameters.json | 6 + .../countrypropagation/oozie_app/workflow.xml | 108 ++----- .../oozie_app/workflow.xml | 8 +- .../projecttoresult/oozie_app/workflow.xml | 2 +- .../oozie_app/workflow.xml | 8 +- .../oozie_app/workflow.xml | 8 +- .../oozie_app/workflow.xml | 8 +- .../CountryPropagationJobTest.java | 35 +-- .../OrcidPropagationJobTest.java | 44 ++- .../ProjectPropagationJobTest.java | 6 +- .../ResultToCommunityJobTest.java | 4 +- .../ResultToCommunityJobTest.java | 2 +- .../Result2OrganizationJobTest.java | 12 +- 37 files changed, 846 insertions(+), 1507 deletions(-) rename dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/{SparkBulkTagJob2.java => SparkBulkTagJob.java} (58%) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/{SparkCountryPropagationJob3.java => SparkCountryPropagationJob.java} (59%) delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/{SparkOrcidToResultFromSemRelJob3.java => SparkOrcidToResultFromSemRelJob.java} (81%) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/{SparkResultToCommunityFromOrganizationJob2.java => SparkResultToCommunityFromOrganizationJob.java} (61%) rename dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/{SparkResultToCommunityThroughSemRelJob4.java => SparkResultToCommunityThroughSemRelJob.java} (61%) create mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java delete mode 100644 dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java similarity index 58% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java rename to dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index a6e49e93b..a6662b9fc 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob2.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -22,21 +22,18 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.community.*; import eu.dnetlib.dhp.schema.oaf.*; -public class SparkBulkTagJob2 { +public class SparkBulkTagJob { - private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob2.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob.class); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( - SparkBulkTagJob2.class + SparkBulkTagJob.class .getResourceAsStream( "/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); Boolean isSparkSessionManaged = Optional @@ -58,7 +55,6 @@ public class SparkBulkTagJob2 { log.info("outputPath: {}", outputPath); ProtoMap protoMappingParams = new Gson().fromJson(parser.get("pathMap"), ProtoMap.class); - ; log.info("pathMap: {}", new Gson().toJson(protoMappingParams)); final String resultClassName = parser.get("resultTableName"); @@ -89,45 +85,6 @@ public class SparkBulkTagJob2 { spark -> { execBulkTag(spark, inputPath, outputPath, protoMappingParams, resultClazz, cc); }); - - // runWithSparkSession(conf, isSparkSessionManaged, - // spark -> { - // if(isTest(parser)) { - // removeOutputDir(spark, outputPath); - // } - // if(saveGraph) - // execPropagation(spark, possibleUpdates, inputPath, outputPath, - // resultClazz); - // }); - // - // - // - // - // - // - // sc.textFile(inputPath + "/publication") - // .map(item -> new ObjectMapper().readValue(item, Publication.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/publication"); - // sc.textFile(inputPath + "/dataset") - // .map(item -> new ObjectMapper().readValue(item, Dataset.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/dataset"); - // sc.textFile(inputPath + "/software") - // .map(item -> new ObjectMapper().readValue(item, Software.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/software"); - // sc.textFile(inputPath + "/otherresearchproduct") - // .map(item -> new ObjectMapper().readValue(item, - // OtherResearchProduct.class)) - // .map(p -> resultTagger.enrichContextCriteria(p, cc, protoMappingParams)) - // .map(p -> new ObjectMapper().writeValueAsString(p)) - // .saveAsTextFile(outputPath+"/otherresearchproduct"); - // - } private static void execBulkTag( @@ -139,28 +96,23 @@ public class SparkBulkTagJob2 { CommunityConfiguration communityConfiguration) { ResultTagger resultTagger = new ResultTagger(); - Dataset result = readPathEntity(spark, inputPath, resultClazz); - result - .map( - value -> resultTagger + readPath(spark, inputPath, resultClazz) + .map((MapFunction) value -> resultTagger .enrichContextCriteria( - value, communityConfiguration, protoMappingParams), - Encoders.bean(resultClazz)) - .toJSON() + value, communityConfiguration, protoMappingParams), + Encoders.bean(resultClazz)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(outputPath); + .json(outputPath); } - private static org.apache.spark.sql.Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class resultClazz) { - + private static Dataset readPath( + SparkSession spark, String inputEntityPath, Class clazz) { return spark .read() - .textFile(inputEntityPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), - Encoders.bean(resultClazz)); + .json(inputEntityPath) + .as(Encoders.bean(clazz)); } + } diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 718ad40ec..2fea9ff41 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -106,7 +106,7 @@ yarn-cluster cluster bulkTagging-publication - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + eu.dnetlib.dhp.bulktag.SparkBulkTagJob dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -134,7 +134,7 @@ yarn-cluster cluster bulkTagging-dataset - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + eu.dnetlib.dhp.bulktag.SparkBulkTagJob dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -162,7 +162,7 @@ yarn-cluster cluster bulkTagging-orp - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + eu.dnetlib.dhp.bulktag.SparkBulkTagJob dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} @@ -190,7 +190,7 @@ yarn-cluster cluster bulkTagging-software - eu.dnetlib.dhp.bulktag.SparkBulkTagJob2 + eu.dnetlib.dhp.bulktag.SparkBulkTagJob dhp-bulktag-${projectVersion}.jar --num-executors=${sparkExecutorNumber} diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java index cb5015ba1..75ecb0298 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java @@ -24,7 +24,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.bulktag.SparkBulkTagJob2; +import eu.dnetlib.dhp.bulktag.SparkBulkTagJob; import eu.dnetlib.dhp.schema.oaf.Dataset; import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; import eu.dnetlib.dhp.schema.oaf.Publication; @@ -84,7 +84,7 @@ public class BulkTagJobTest { @Test public void noUpdatesTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -134,7 +134,7 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectNoPreviousContextTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -240,7 +240,7 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -332,7 +332,7 @@ public class BulkTagJobTest { @Test public void bulktagByDatasourceTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -415,7 +415,7 @@ public class BulkTagJobTest { @Test public void bulktagByZenodoCommunityTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -548,7 +548,7 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectDatasourceTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -688,7 +688,7 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectDatasourceZenodoCommunityTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", @@ -796,7 +796,7 @@ public class BulkTagJobTest { @Test public void bulktagDatasourcewithConstraintsTest() throws Exception { - SparkBulkTagJob2 + SparkBulkTagJob .main( new String[] { "-isTest", diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index ebd25fa35..8d2fede82 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -1,12 +1,11 @@ package eu.dnetlib.dhp; -import java.io.IOException; -import java.util.*; +import java.util.List; +import java.util.Optional; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -67,6 +66,12 @@ public class PropagationConstant { public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String cfHbforResultQuery = "select distinct r.id, inst.collectedfrom.key cf, inst.hostedby.key hb " + + + "from result r " + + "lateral view explode(instance) i as inst " + + "where r.datainfo.deletedbyinference=false"; + public static Country getCountry(String classid, String classname) { Country nc = new Country(); nc.setClassid(classid); @@ -130,13 +135,6 @@ public class PropagationConstant { return ret; } - public static void createOutputDirs(String outputPath, FileSystem fs) throws IOException { - if (fs.exists(new Path(outputPath))) { - fs.delete(new Path(outputPath), true); - } - fs.mkdirs(new Path(outputPath)); - } - public static void removeOutputDir(SparkSession spark, String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } @@ -155,50 +153,17 @@ public class PropagationConstant { .orElse(Boolean.FALSE); } - public static void createCfHbforresult(SparkSession spark) { - String query; -// query = "SELECT id, inst.collectedfrom.key cf , inst.hostedby.key hb " -// + "FROM ( SELECT id, instance " -// + "FROM result " -// + " WHERE datainfo.deletedbyinference = false) ds " -// + "LATERAL VIEW EXPLODE(instance) i AS inst"; - query = "select distinct r.id, inst.collectedfrom.key cf, inst.hostedby.key hb " + - "from result r " + - "lateral view explode(instance) i as inst " + - "where r.datainfo.deletedbyinference=false"; - - org.apache.spark.sql.Dataset cfhb = spark.sql(query); + public static void createCfHbforResult(SparkSession spark) { + org.apache.spark.sql.Dataset cfhb = spark.sql(cfHbforResultQuery); cfhb.createOrReplaceTempView("cfhb"); } - public static org.apache.spark.sql.Dataset readPathEntity( - SparkSession spark, String inputEntityPath, Class resultClazz) { - - return spark - .read() - .textFile(inputEntityPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, resultClazz), - Encoders.bean(resultClazz)); - } - - public static org.apache.spark.sql.Dataset readRelations( - SparkSession spark, String inputPath) { + public static Dataset readPath( + SparkSession spark, String inputPath, Class clazz) { return spark .read() .textFile(inputPath) - .map( - (MapFunction) value -> OBJECT_MAPPER.readValue(value, Relation.class), - Encoders.bean(Relation.class)); + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); } - public static org.apache.spark.sql.Dataset readResultCommunityList( - SparkSession spark, String possibleUpdatesPath) { - return spark - .read() - .textFile(possibleUpdatesPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), - Encoders.bean(ResultCommunityList.class)); - } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java deleted file mode 100644 index c29043a2d..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/QueryInformationSystem.java +++ /dev/null @@ -1,20 +0,0 @@ - -package eu.dnetlib.dhp; - -import java.util.List; - -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; - -public class QueryInformationSystem { - private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" - + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" - + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" - + " return $x//CONFIGURATION/context/@id/string()"; - - public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - return isLookUp.quickSearchProfile(XQUERY); - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java index 56185eb72..e91a1e48a 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java @@ -13,6 +13,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,7 +31,6 @@ import eu.dnetlib.dhp.schema.oaf.*; public class PrepareDatasourceCountryAssociation { private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { @@ -80,31 +80,10 @@ public class PrepareDatasourceCountryAssociation { for (String i : whitelist) { whitelisted += " OR id = '" + i + "'"; } - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset datasource = spark - .createDataset( - sc - .textFile(inputPath + "/datasource") - .map(item -> OBJECT_MAPPER.readValue(item, Datasource.class)) - .rdd(), - Encoders.bean(Datasource.class)); - - Dataset relation = spark - .createDataset( - sc - .textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); - - Dataset organization = spark - .createDataset( - sc - .textFile(inputPath + "/organization") - .map(item -> OBJECT_MAPPER.readValue(item, Organization.class)) - .rdd(), - Encoders.bean(Organization.class)); + Dataset datasource = readPath(spark, inputPath + "/datasource", Datasource.class); + Dataset relation = readPath(spark, inputPath + "/relation", Relation.class); + Dataset organization = readPath(spark, inputPath + "/organization", Organization.class); datasource.createOrReplaceTempView("datasource"); relation.createOrReplaceTempView("relation"); @@ -128,14 +107,15 @@ public class PrepareDatasourceCountryAssociation { + "JOIN (SELECT id, country " + " FROM organization " + " WHERE datainfo.deletedbyinference = false " - + " AND length(country.classid)>0) o " + + " AND length(country.classid) > 0) o " + "ON o.id = rel.target"; spark .sql(query) .as(Encoders.bean(DatasourceCountry.class)) - .toJavaRDD() - .map(c -> OBJECT_MAPPER.writeValueAsString(c)) - .saveAsTextFile(outputPath, GzipCodec.class); + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(outputPath); } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java index fec4a08ce..34b376413 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java @@ -4,31 +4,31 @@ package eu.dnetlib.dhp.countrypropagation; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import java.util.Arrays; -import java.util.stream.Collectors; - import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; public class PrepareResultCountrySet { private static final Logger log = LoggerFactory.getLogger(PrepareResultCountrySet.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String RESULT_COUNTRYSET_QUERY = "SELECT id resultId, collect_set(country) countrySet " + + "FROM ( SELECT id, country " + + "FROM datasource_country JOIN cfhb ON cf = dataSourceId " + + "UNION ALL " + + "SELECT id, country FROM datasource_country " + + "JOIN cfhb ON hb = dataSourceId ) tmp " + + "GROUP BY id"; public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( - SparkCountryPropagationJob2.class + PrepareResultCountrySet.class .getResourceAsStream( "/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json")); @@ -42,6 +42,9 @@ public class PrepareResultCountrySet { String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + final String datasourcecountrypath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", datasourcecountrypath); @@ -60,75 +63,36 @@ public class PrepareResultCountrySet { getPotentialResultToUpdate( spark, inputPath, + outputPath, datasourcecountrypath, resultClazz); }); - } private static void getPotentialResultToUpdate( SparkSession spark, String inputPath, + String outputPath, String datasourcecountrypath, Class resultClazz) { - Dataset result = readPathEntity(spark, inputPath, resultClazz); + Dataset result = readPath(spark, inputPath, resultClazz); result.createOrReplaceTempView("result"); // log.info("number of results: {}", result.count()); - createCfHbforresult(spark); - Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); - countryPropagationAssoc(spark, datasourcecountryassoc) - .map((MapFunction) value -> { - R ret = resultClazz.newInstance(); - ret.setId(value.getResultId()); - ret - .setCountry( - value - .getCountrySet() - .stream() - .map(c -> getCountry(c.getClassid(), c.getClassname())) - .collect(Collectors.toList())); - return ret; - }, Encoders.bean(resultClazz)) + createCfHbforResult(spark); + + Dataset datasource_country = readPath(spark, datasourcecountrypath, DatasourceCountry.class); + + datasource_country.createOrReplaceTempView("datasource_country"); + // log.info("datasource_country number : {}", datasource_country.count()); + + spark + .sql(RESULT_COUNTRYSET_QUERY) + .as(Encoders.bean(ResultCountrySet.class)) .write() .option("compression", "gzip") .mode(SaveMode.Append) - .json(inputPath); + .json(outputPath); } - private static Dataset countryPropagationAssoc( - SparkSession spark, - Dataset datasource_country) { - - // Dataset datasource_country = broadcast_datasourcecountryassoc.value(); - datasource_country.createOrReplaceTempView("datasource_country"); - log.info("datasource_country number : {}", datasource_country.count()); - - String query = "SELECT id resultId, collect_set(country) countrySet " - + "FROM ( SELECT id, country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON cf = dataSourceId " - + "UNION ALL " - + "SELECT id , country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON hb = dataSourceId ) tmp " - + "GROUP BY id"; - Dataset potentialUpdates = spark - .sql(query) - .as(Encoders.bean(ResultCountrySet.class)); - // log.info("potential update number : {}", potentialUpdates.count()); - return potentialUpdates; - } - - private static Dataset readAssocDatasourceCountry( - SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, DatasourceCountry.class), - Encoders.bean(DatasourceCountry.class)); - } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java similarity index 59% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java index 8f12fcf3d..9dc17701b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java @@ -5,17 +5,11 @@ import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.util.*; -import java.util.function.Function; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; @@ -26,15 +20,13 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Country; -import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Result; import scala.Tuple2; -public class SparkCountryPropagationJob3 { +public class SparkCountryPropagationJob { - private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob3.class); + private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -42,7 +34,7 @@ public class SparkCountryPropagationJob3 { String jsonConfiguration = IOUtils .toString( - SparkCountryPropagationJob3.class + SparkCountryPropagationJob.class .getResourceAsStream( "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); @@ -53,8 +45,11 @@ public class SparkCountryPropagationJob3 { Boolean isSparkSessionManaged = isSparkSessionManaged(parser); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String sourcePath = parser.get("sourcePath"); + log.info("sourcePath: {}", sourcePath); + + String preparedInfoPath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", preparedInfoPath); final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); @@ -76,7 +71,8 @@ public class SparkCountryPropagationJob3 { isSparkSessionManaged, spark -> execPropagation( spark, - inputPath, + sourcePath, + preparedInfoPath, outputPath, resultClazz, saveGraph)); @@ -84,21 +80,26 @@ public class SparkCountryPropagationJob3 { private static void execPropagation( SparkSession spark, - String inputPath, + String sourcePath, + String preparedInfoPath, String outputPath, Class resultClazz, boolean saveGraph) { if (saveGraph) { // updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); - log.info("Reading Graph table from: {}", inputPath); + log.info("Reading Graph table from: {}", sourcePath); + Dataset res = readPath(spark, sourcePath, resultClazz); - spark + log.info("Reading prepared info: {}", preparedInfoPath); + Dataset prepared = spark .read() - .json(inputPath) - .as(Encoders.bean(resultClazz)) - .groupByKey((MapFunction) r -> r.getId(), Encoders.STRING()) - .mapGroups(getCountryMergeFn(resultClazz), Encoders.bean(resultClazz)) + .json(preparedInfoPath) + .as(Encoders.bean(ResultCountrySet.class)); + + res + .joinWith(prepared, res.col("id").equalTo(prepared.col("resultId")), "left_outer") + .map(getCountryMergeFn(), Encoders.bean(resultClazz)) .write() .option("compression", "gzip") .mode(SaveMode.Overwrite) @@ -106,37 +107,26 @@ public class SparkCountryPropagationJob3 { } } - private static MapGroupsFunction getCountryMergeFn(Class resultClazz) { - return (MapGroupsFunction) (key, values) -> { - R res = resultClazz.newInstance(); - List countries = new ArrayList<>(); - values.forEachRemaining(r -> { - res.mergeFrom(r); - countries.addAll(r.getCountry()); + private static MapFunction, R> getCountryMergeFn() { + return (MapFunction, R>) t -> { + Optional.ofNullable(t._2()).ifPresent(r -> { + t._1().getCountry().addAll(merge(t._1().getCountry(), r.getCountrySet())); }); - res - .setCountry( - countries - .stream() - .collect( - Collectors - .toMap( - Country::getClassid, - Function.identity(), - (c1, c2) -> { - if (Optional - .ofNullable( - c1.getDataInfo().getInferenceprovenance()) - .isPresent()) { - return c2; - } - return c1; - })) - .values() - .stream() - .collect(Collectors.toList())); - return res; + return t._1(); }; } + private static List merge(List c1, List c2) { + HashSet countries = c1 + .stream() + .map(c -> c.getClassid()) + .collect(Collectors.toCollection(HashSet::new)); + + return c2 + .stream() + .filter(c -> !countries.contains(c.getClassid())) + .map(c -> getCountry(c.getClassid(), c.getClassname())) + .collect(Collectors.toList()); + } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java deleted file mode 100644 index 059e388db..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob2.java +++ /dev/null @@ -1,289 +0,0 @@ - -package eu.dnetlib.dhp.countrypropagation; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; -import static jdk.nashorn.internal.objects.NativeDebug.map; - -import java.util.*; -import java.util.stream.Collectors; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.broadcast.Broadcast; -import org.apache.spark.sql.*; -import org.apache.spark.sql.Dataset; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import scala.Tuple2; - -public class SparkCountryPropagationJob2 { - - private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob2.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public static void main(String[] args) throws Exception { - - String jsonConfiguration = IOUtils - .toString( - SparkCountryPropagationJob2.class - .getResourceAsStream( - "/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json")); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - - parser.parseArgument(args); - - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); - - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); - - final String datasourcecountrypath = parser.get("preparedInfoPath"); - log.info("preparedInfoPath: {}", datasourcecountrypath); - - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); - - final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); - log.info("resultType: {}", resultType); - - final String possibleUpdatesPath = datasourcecountrypath - .substring(0, datasourcecountrypath.lastIndexOf("/") + 1) - + "possibleUpdates/" + resultType; - log.info("possibleUpdatesPath: {}", possibleUpdatesPath); - - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); - - Class resultClazz = (Class) Class.forName(resultClassName); - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, possibleUpdatesPath); - execPropagation( - spark, - datasourcecountrypath, - inputPath, - outputPath, - resultClazz, - saveGraph, possibleUpdatesPath); - }); - } - - private static void execPropagation( - SparkSession spark, - String datasourcecountrypath, - String inputPath, - String outputPath, - Class resultClazz, - boolean saveGraph, String possilbeUpdatesPath) { - // final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - // Load file with preprocessed association datasource - country - Dataset datasourcecountryassoc = readAssocDatasourceCountry(spark, datasourcecountrypath); - // broadcasting the result of the preparation step - // Broadcast> broadcast_datasourcecountryassoc = - // sc.broadcast(datasourcecountryassoc); - - Dataset potentialUpdates = getPotentialResultToUpdate( - spark, inputPath, resultClazz, datasourcecountryassoc) - .as(Encoders.bean(ResultCountrySet.class)); - - potentialUpdates.write().option("compression", "gzip").mode(SaveMode.Overwrite).json(possilbeUpdatesPath); - - if (saveGraph) { - // updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); - potentialUpdates = spark - .read() - .textFile(possilbeUpdatesPath) - .map( - (MapFunction) value -> OBJECT_MAPPER - .readValue(value, ResultCountrySet.class), - Encoders.bean(ResultCountrySet.class)); - updateResultTable(spark, potentialUpdates, inputPath, resultClazz, outputPath); - } - } - - private static void updateResultTable( - SparkSession spark, - Dataset potentialUpdates, - String inputPath, - Class resultClazz, - String outputPath) { - - log.info("Reading Graph table from: {}", inputPath); - Dataset result = readPathEntity(spark, inputPath, resultClazz); - - Dataset new_table = result - .joinWith( - potentialUpdates, result - .col("id") - .equalTo(potentialUpdates.col("resultId")), - "left_outer") - .map((MapFunction, R>) value -> { - R r = value._1(); - Optional potentialNewCountries = Optional.ofNullable(value._2()); - if (potentialNewCountries.isPresent()) { - HashSet countries = r - .getCountry() - .stream() - .map(c -> c.getClassid()) - .collect(Collectors.toCollection(HashSet::new)); - - r - .getCountry() - .addAll( - potentialNewCountries - .get() - .getCountrySet() - .stream() - .filter(c -> !countries.contains(c.getClassid())) - .map(c -> getCountry(c.getClassid(), c.getClassname())) - .collect(Collectors.toList())); - -// Result res = new Result(); -// res.setId(r.getId()); -// List countryList = new ArrayList<>(); -// for (CountrySbs country : potentialNewCountries -// .get() -// .getCountrySet()) { -// if (!countries.contains(country.getClassid())) { -// countryList -// .add( -// getCountry( -// country.getClassid(), -// country.getClassname())); -// } -// } -// res.setCountry(countryList); -// r.mergeFrom(res); - } - return r; - }, Encoders.bean(resultClazz)); -// Dataset> result_pair = result -// .map( -// r -> new Tuple2<>(r.getId(), r), -// Encoders.tuple(Encoders.STRING(), Encoders.bean(resultClazz))); -// -// Dataset new_table = result_pair -// .joinWith( -// potentialUpdates, -// result_pair.col("_1").equalTo(potentialUpdates.col("resultId")), -// "left_outer") -// .map( -// (MapFunction, ResultCountrySet>, R>) value -> { -// R r = value._1()._2(); -// Optional potentialNewCountries = Optional.ofNullable(value._2()); -// if (potentialNewCountries.isPresent()) { -// HashSet countries = new HashSet<>(); -// for (Qualifier country : r.getCountry()) { -// countries.add(country.getClassid()); -// } -// Result res = new Result(); -// res.setId(r.getId()); -// List countryList = new ArrayList<>(); -// for (CountrySbs country : potentialNewCountries -// .get() -// .getCountrySet()) { -// if (!countries.contains(country.getClassid())) { -// countryList -// .add( -// getCountry( -// country.getClassid(), -// country.getClassname())); -// } -// } -// res.setCountry(countryList); -// r.mergeFrom(res); -// } -// return r; -// }, -// Encoders.bean(resultClazz)); - - log.info("Saving graph table to path: {}", outputPath); - log.info("number of saved recordsa: {}", new_table.count()); - new_table.write().option("compression", "gzip").mode(SaveMode.Overwrite).json(outputPath); - - } - - private static Dataset getPotentialResultToUpdate( - SparkSession spark, - String inputPath, - Class resultClazz, - Dataset datasourcecountryassoc) { - - Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.createOrReplaceTempView("result"); - // log.info("number of results: {}", result.count()); - createCfHbforresult(spark); - return countryPropagationAssoc(spark, datasourcecountryassoc); - } - - private static Dataset countryPropagationAssoc( - SparkSession spark, - Dataset datasource_country) { - - // Dataset datasource_country = broadcast_datasourcecountryassoc.value(); - datasource_country.createOrReplaceTempView("datasource_country"); - log.info("datasource_country number : {}", datasource_country.count()); - - String query = "SELECT id resultId, collect_set(country) countrySet " - + "FROM ( SELECT id, country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON cf = dataSourceId " - + "UNION ALL " - + "SELECT id , country " - + "FROM datasource_country " - + "JOIN cfhb " - + " ON hb = dataSourceId ) tmp " - + "GROUP BY id"; - - Dataset potentialUpdates = spark - .sql(query) - .as(Encoders.bean(ResultCountrySet.class)) - .map((MapFunction) r -> { - final ArrayList c = r - .getCountrySet() - .stream() - .limit(100) - .collect(Collectors.toCollection(ArrayList::new)); - r.setCountrySet(c); - return r; - }, Encoders.bean(ResultCountrySet.class)); - // log.info("potential update number : {}", potentialUpdates.count()); - return potentialUpdates; - } - - private static Dataset readAssocDatasourceCountry( - SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map( - (MapFunction) value -> OBJECT_MAPPER - .readValue(value, DatasourceCountry.class), - Encoders.bean(DatasourceCountry.class)); - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java index c1644a589..a5fcab360 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java @@ -2,10 +2,11 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel; public class AutoritativeAuthor { - String name; - String surname; - String fullname; - String orcid; + + private String name; + private String surname; + private String fullname; + private String orcid; public String getName() { return name; @@ -38,4 +39,5 @@ public class AutoritativeAuthor { public void setOrcid(String orcid) { this.orcid = orcid; } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java index 1baec07c5..3e16b4b4b 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java @@ -13,6 +13,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,17 +28,14 @@ import eu.dnetlib.dhp.schema.oaf.Result; public class PrepareResultOrcidAssociationStep1 { private static final Logger log = LoggerFactory.getLogger(PrepareResultOrcidAssociationStep1.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils + String jsonConf = IOUtils .toString( - SparkOrcidToResultFromSemRelJob3.class + PrepareResultOrcidAssociationStep1.class .getResourceAsStream( "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConf); parser.parseArgument(args); Boolean isSparkSessionManaged = isSparkSessionManaged(parser); @@ -63,6 +61,15 @@ public class PrepareResultOrcidAssociationStep1 { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + String inputRelationPath = inputPath + "/relation"; + log.info("inputRelationPath: {}", inputRelationPath); + + String inputResultPath = inputPath + "/" + resultType; + log.info("inputResultPath: {}", inputResultPath); + + String outputResultPath = outputPath + "/" + resultType; + log.info("outputResultPath: {}", outputResultPath); + runWithSparkHiveSession( conf, isSparkSessionManaged, @@ -71,39 +78,25 @@ public class PrepareResultOrcidAssociationStep1 { removeOutputDir(spark, outputPath); } prepareInfo( - spark, inputPath, outputPath, resultClazz, resultType, allowedsemrel); + spark, inputRelationPath, inputResultPath, outputResultPath, resultClazz, allowedsemrel); }); } private static void prepareInfo( SparkSession spark, - String inputPath, - String outputPath, + String inputRelationPath, + String inputResultPath, + String outputResultPath, Class resultClazz, - String resultType, List allowedsemrel) { - // read the relation table and the table related to the result it is using - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = spark - .createDataset( - sc - .textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + Dataset relation = readPath(spark, inputRelationPath, Relation.class); relation.createOrReplaceTempView("relation"); - log.info("Reading Graph table from: {}", inputPath + "/" + resultType); - Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); - + log.info("Reading Graph table from: {}", inputResultPath); + Dataset result = readPath(spark, inputResultPath, resultClazz); result.createOrReplaceTempView("result"); - getPossibleResultOrcidAssociation(spark, allowedsemrel, outputPath + "/" + resultType); - } - - private static void getPossibleResultOrcidAssociation( - SparkSession spark, List allowedsemrel, String outputPath) { String query = " select target resultId, author authorList" + " from (select id, collect_set(named_struct('name', name, 'surname', surname, 'fullname', fullname, 'orcid', orcid)) author " + " from ( " @@ -120,18 +113,13 @@ public class PrepareResultOrcidAssociationStep1 { + getConstraintList(" relclass = '", allowedsemrel) + ") rel_rel " + " on source = id"; - spark .sql(query) .as(Encoders.bean(ResultOrcidList.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); - // .toJSON() - // .write() - // .mode(SaveMode.Append) - // .option("compression","gzip") - // .text(outputPath) - // ; + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(outputResultPath); } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java index a8380e8b9..65d8811bc 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java @@ -59,10 +59,10 @@ public class PrepareResultOrcidAssociationStep2 { private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssoc = readAssocResultOrcidList(spark, inputPath + "/publication") - .union(readAssocResultOrcidList(spark, inputPath + "/dataset")) - .union(readAssocResultOrcidList(spark, inputPath + "/otherresearchproduct")) - .union(readAssocResultOrcidList(spark, inputPath + "/software")); + Dataset resultOrcidAssoc = readPath(spark, inputPath + "/publication", ResultOrcidList.class) + .union(readPath(spark, inputPath + "/dataset", ResultOrcidList.class)) + .union(readPath(spark, inputPath + "/otherresearchproduct", ResultOrcidList.class)) + .union(readPath(spark, inputPath + "/software", ResultOrcidList.class)); resultOrcidAssoc .toJavaRDD() @@ -77,7 +77,6 @@ public class PrepareResultOrcidAssociationStep2 { } Set orcid_set = new HashSet<>(); a.getAuthorList().stream().forEach(aa -> orcid_set.add(aa.getOrcid())); - b .getAuthorList() .stream() @@ -95,13 +94,4 @@ public class PrepareResultOrcidAssociationStep2 { .saveAsTextFile(outputPath, GzipCodec.class); } - private static Dataset readAssocResultOrcidList( - SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), - Encoders.bean(ResultOrcidList.class)); - } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java similarity index 81% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java index 6214e18ca..ebb75a5a6 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob3.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java @@ -6,11 +6,11 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import java.util.List; import java.util.Optional; -import java.util.function.Consumer; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; @@ -25,21 +25,19 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import scala.Tuple2; -public class SparkOrcidToResultFromSemRelJob3 { - private static final Logger log = LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob3.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); +public class SparkOrcidToResultFromSemRelJob { + private static final Logger log = LoggerFactory.getLogger(SparkOrcidToResultFromSemRelJob.class); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( - SparkOrcidToResultFromSemRelJob3.class + SparkOrcidToResultFromSemRelJob.class .getResourceAsStream( "/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); Boolean isSparkSessionManaged = isSparkSessionManaged(parser); @@ -88,9 +86,9 @@ public class SparkOrcidToResultFromSemRelJob3 { Class resultClazz) { // read possible updates (resultId and list of possible orcid to add - Dataset possible_updates = readAssocResultOrcidList(spark, possibleUpdatesPath); + Dataset possible_updates = readPath(spark, possibleUpdatesPath, ResultOrcidList.class); // read the result we have been considering - Dataset result = readPathEntity(spark, inputPath, resultClazz); + Dataset result = readPath(spark, inputPath, resultClazz); // make join result left_outer with possible updates result @@ -98,38 +96,29 @@ public class SparkOrcidToResultFromSemRelJob3 { possible_updates, result.col("id").equalTo(possible_updates.col("resultId")), "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rol = Optional.ofNullable(value._2()); - if (rol.isPresent()) { - List toenrich_author = ret.getAuthor(); - List autoritativeAuthors = rol.get().getAuthorList(); - for (Author author : toenrich_author) { - if (!containsAllowedPid(author)) { - enrichAuthor(author, autoritativeAuthors); - } - } - } - - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() + .map(authorEnrichFn(), Encoders.bean(resultClazz)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(outputPath); + .json(outputPath); } - private static Dataset readAssocResultOrcidList( - SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultOrcidList.class), - Encoders.bean(ResultOrcidList.class)); + private static MapFunction, R> authorEnrichFn() { + return (MapFunction, R>) value -> { + R ret = value._1(); + Optional rol = Optional.ofNullable(value._2()); + if (rol.isPresent()) { + List toenrich_author = ret.getAuthor(); + List autoritativeAuthors = rol.get().getAuthorList(); + for (Author author : toenrich_author) { + if (!containsAllowedPid(author)) { + enrichAuthor(author, autoritativeAuthors); + } + } + } + + return ret; + }; } private static void enrichAuthor(Author a, List au) { diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java index b8579156b..05dcdc692 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java @@ -25,7 +25,6 @@ import eu.dnetlib.dhp.schema.oaf.Relation; public class PrepareProjectResultsAssociation { private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { @@ -61,8 +60,6 @@ public class PrepareProjectResultsAssociation { conf, isSparkSessionManaged, spark -> { - // removeOutputDir(spark, potentialUpdatePath); - // removeOutputDir(spark, alreadyLinkedPath); prepareResultProjProjectResults( spark, inputPath, @@ -78,28 +75,21 @@ public class PrepareProjectResultsAssociation { String potentialUpdatePath, String alreadyLinkedPath, List allowedsemrel) { - JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - Dataset relation = spark - .createDataset( - sc - .textFile(inputPath) - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + Dataset relation = readPath(spark, inputPath, Relation.class); relation.createOrReplaceTempView("relation"); - String query = "SELECT source, target " + String resproj_relation_query = "SELECT source, target " + " FROM relation " + " WHERE datainfo.deletedbyinference = false " + " AND relClass = '" + RELATION_RESULT_PROJECT_REL_CLASS + "'"; - Dataset resproj_relation = spark.sql(query); + Dataset resproj_relation = spark.sql(resproj_relation_query); resproj_relation.createOrReplaceTempView("resproj_relation"); - query = "SELECT resultId, collect_set(projectId) projectSet " + String potential_update_query = "SELECT resultId, collect_set(projectId) projectSet " + "FROM ( " + "SELECT r1.target resultId, r2.target projectId " + " FROM (SELECT source, target " @@ -111,46 +101,26 @@ public class PrepareProjectResultsAssociation { + " ON r1.source = r2.source " + " ) tmp " + "GROUP BY resultId "; - // query = - // "SELECT projectId, collect_set(resId) resultSet " - // + "FROM (" - // + " SELECT r1.target resId, r2.target projectId " - // + " FROM (SELECT source, target " - // + " FROM relation " - // + " WHERE datainfo.deletedbyinference = false " - // + getConstraintList(" relClass = '", allowedsemrel) - // + ") r1" - // + " JOIN resproj_relation r2 " - // + " ON r1.source = r2.source " - // + " ) tmp " - // + "GROUP BY projectId "; spark - .sql(query) + .sql(potential_update_query) .as(Encoders.bean(ResultProjectSet.class)) - // .toJSON() - // .write() - // .mode(SaveMode.Overwrite) - // .option("compression", "gzip") - // .text(potentialUpdatePath); - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(potentialUpdatePath, GzipCodec.class); + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(potentialUpdatePath); - query = "SELECT source resultId, collect_set(target) projectSet " + String result_projectset_query = "SELECT source resultId, collect_set(target) projectSet " + "FROM resproj_relation " + "GROUP BY source"; spark - .sql(query) + .sql(result_projectset_query) .as(Encoders.bean(ResultProjectSet.class)) - // .toJSON() - // .write() - // .mode(SaveMode.Overwrite) - // .option("compression", "gzip") - // .text(alreadyLinkedPath); - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(alreadyLinkedPath); } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java new file mode 100644 index 000000000..36694b3dd --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java @@ -0,0 +1,147 @@ + +package eu.dnetlib.dhp.projecttoresult; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; +import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; + +public class SparkResultToProjectThroughSemRelJob { + + private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkResultToProjectThroughSemRelJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + final String potentialUpdatePath = parser.get("potentialUpdatePath"); + log.info("potentialUpdatePath {}: ", potentialUpdatePath); + + final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); + + final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); + log.info("saveGraph: {}", saveGraph); + + SparkConf conf = new SparkConf(); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + execPropagation( + spark, outputPath, alreadyLinkedPath, potentialUpdatePath, saveGraph); + }); + } + + private static void execPropagation( + SparkSession spark, + String outputPath, + String alreadyLinkedPath, + String potentialUpdatePath, + Boolean saveGraph) { + + Dataset toaddrelations = readPath(spark, potentialUpdatePath, ResultProjectSet.class); + Dataset alreadyLinked = readPath(spark, alreadyLinkedPath, ResultProjectSet.class); + + if (saveGraph) { + toaddrelations + .joinWith( + alreadyLinked, + toaddrelations.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") + .flatMap(mapRelationRn(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(outputPath); + } + } + + private static FlatMapFunction, Relation> mapRelationRn() { + return (FlatMapFunction, Relation>) value -> { + List new_relations = new ArrayList<>(); + ResultProjectSet potential_update = value._1(); + Optional already_linked = Optional.ofNullable(value._2()); + if (already_linked.isPresent()) { + already_linked + .get() + .getProjectSet() + .stream() + .forEach( + (p -> { + if (potential_update + .getProjectSet() + .contains(p)) { + potential_update.getProjectSet().remove(p); + } + })); + } + String resId = potential_update.getResultId(); + potential_update + .getProjectSet() + .stream() + .forEach( + projectId -> { + new_relations + .add( + getRelation( + resId, + projectId, + RELATION_RESULT_PROJECT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + new_relations + .add( + getRelation( + projectId, + resId, + RELATION_PROJECT_RESULT_REL_CLASS, + RELATION_RESULTPROJECT_REL_TYPE, + RELATION_RESULTPROJECT_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, + PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); + }); + return new_relations.iterator(); + }; + } + +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java deleted file mode 100644 index 4be072901..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob3.java +++ /dev/null @@ -1,159 +0,0 @@ - -package eu.dnetlib.dhp.projecttoresult; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.sql.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation; -import eu.dnetlib.dhp.schema.oaf.Relation; - -public class SparkResultToProjectThroughSemRelJob3 { - - private static final Logger log = LoggerFactory.getLogger(PrepareDatasourceCountryAssociation.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public static void main(String[] args) throws Exception { - - String jsonConfiguration = IOUtils - .toString( - SparkResultToProjectThroughSemRelJob3.class - .getResourceAsStream( - "/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json")); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - - parser.parseArgument(args); - - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); - - final String potentialUpdatePath = parser.get("potentialUpdatePath"); - log.info("potentialUpdatePath {}: ", potentialUpdatePath); - - final String alreadyLinkedPath = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath {}: ", alreadyLinkedPath); - - final Boolean saveGraph = Boolean.valueOf(parser.get("saveGraph")); - log.info("saveGraph: {}", saveGraph); - - SparkConf conf = new SparkConf(); - - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - execPropagation( - spark, outputPath, alreadyLinkedPath, potentialUpdatePath, saveGraph); - }); - } - - private static void execPropagation( - SparkSession spark, - String outputPath, - String alreadyLinkedPath, - String potentialUpdatePath, - Boolean saveGraph) { - - Dataset toaddrelations = readAssocResultProjects(spark, potentialUpdatePath); - Dataset alreadyLinked = readAssocResultProjects(spark, alreadyLinkedPath); - - if (saveGraph) { - getNewRelations(alreadyLinked, toaddrelations) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression", "gzip") - .text(outputPath); - } - } - - private static Dataset getNewRelations( - Dataset alreadyLinked, Dataset toaddrelations) { - - return toaddrelations - .joinWith( - alreadyLinked, - toaddrelations.col("resultId").equalTo(alreadyLinked.col("resultId")), - "left_outer") - .flatMap( - value -> { - List new_relations = new ArrayList<>(); - ResultProjectSet potential_update = value._1(); - Optional already_linked = Optional.ofNullable(value._2()); - if (already_linked.isPresent()) { - already_linked - .get() - .getProjectSet() - .stream() - .forEach( - (p -> { - if (potential_update - .getProjectSet() - .contains(p)) { - potential_update.getProjectSet().remove(p); - } - })); - } - String resId = potential_update.getResultId(); - potential_update - .getProjectSet() - .stream() - .forEach( - pId -> { - new_relations - .add( - getRelation( - resId, - pId, - RELATION_RESULT_PROJECT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - new_relations - .add( - getRelation( - pId, - resId, - RELATION_PROJECT_RESULT_REL_CLASS, - RELATION_RESULTPROJECT_REL_TYPE, - RELATION_RESULTPROJECT_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_ID, - PROPAGATION_RELATION_RESULT_PROJECT_SEM_REL_CLASS_NAME)); - }); - return new_relations.iterator(); - }, - Encoders.bean(Relation.class)); - } - - private static Dataset readAssocResultProjects( - SparkSession spark, String potentialUpdatePath) { - return spark - .read() - .textFile(potentialUpdatePath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultProjectSet.class), - Encoders.bean(ResultProjectSet.class)); - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index fbe598e89..e2d4d5687 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -8,6 +8,7 @@ import java.util.*; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -22,8 +23,6 @@ public class PrepareResultCommunitySet { private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( @@ -32,7 +31,6 @@ public class PrepareResultCommunitySet { "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json")); final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); Boolean isSparkSessionManaged = isSparkSessionManaged(parser); @@ -69,7 +67,8 @@ public class PrepareResultCommunitySet { String inputPath, String outputPath, OrganizationMap organizationMap) { - Dataset relation = readRelations(spark, inputPath); + + Dataset relation = readPath(spark, inputPath, Relation.class); relation.createOrReplaceTempView("relation"); String query = "SELECT result_organization.source resultId, result_organization.target orgId, org_set merges " @@ -88,46 +87,44 @@ public class PrepareResultCommunitySet { + " GROUP BY source) organization_organization " + "ON result_organization.target = organization_organization.source "; - org.apache.spark.sql.Dataset result_organizationset = spark + Dataset result_organizationset = spark .sql(query) .as(Encoders.bean(ResultOrganizations.class)); result_organizationset - .map( - value -> { - String rId = value.getResultId(); - Optional> orgs = Optional.ofNullable(value.getMerges()); - String oTarget = value.getOrgId(); - Set communitySet = new HashSet<>(); - if (organizationMap.containsKey(oTarget)) { - communitySet.addAll(organizationMap.get(oTarget)); - } - if (orgs.isPresent()) - // try{ - for (String oId : orgs.get()) { - if (organizationMap.containsKey(oId)) { - communitySet.addAll(organizationMap.get(oId)); - } - } - // }catch(Exception e){ - // - // } - if (communitySet.size() > 0) { - ResultCommunityList rcl = new ResultCommunityList(); - rcl.setResultId(rId); - ArrayList communityList = new ArrayList<>(); - communityList.addAll(communitySet); - rcl.setCommunityList(communityList); - return rcl; - } - return null; - }, - Encoders.bean(ResultCommunityList.class)) - .filter(r -> r != null) - .toJSON() + .map(mapResultCommunityFn(organizationMap), Encoders.bean(ResultCommunityList.class)) + .filter(Objects::nonNull) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(outputPath); + .json(outputPath); + } + + private static MapFunction mapResultCommunityFn( + OrganizationMap organizationMap) { + return (MapFunction) value -> { + String rId = value.getResultId(); + Optional> orgs = Optional.ofNullable(value.getMerges()); + String oTarget = value.getOrgId(); + Set communitySet = new HashSet<>(); + if (organizationMap.containsKey(oTarget)) { + communitySet.addAll(organizationMap.get(oTarget)); + } + if (orgs.isPresent()) + for (String oId : orgs.get()) { + if (organizationMap.containsKey(oId)) { + communitySet.addAll(organizationMap.get(oId)); + } + } + if (communitySet.size() > 0) { + ResultCommunityList rcl = new ResultCommunityList(); + rcl.setResultId(rId); + ArrayList communityList = new ArrayList<>(); + communityList.addAll(communitySet); + rcl.setCommunityList(communityList); + return rcl; + } + return null; + }; } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java similarity index 61% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java index 7cc3d6d59..71275cc7f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java @@ -9,6 +9,8 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -19,17 +21,16 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.*; +import scala.Tuple2; -public class SparkResultToCommunityFromOrganizationJob2 { +public class SparkResultToCommunityFromOrganizationJob { - private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob2.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromOrganizationJob.class); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( - SparkResultToCommunityFromOrganizationJob2.class + SparkResultToCommunityFromOrganizationJob.class .getResourceAsStream( "/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json")); @@ -81,54 +82,56 @@ public class SparkResultToCommunityFromOrganizationJob2 { String outputPath, Class resultClazz, String possibleUpdatesPath) { - org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList( - spark, possibleUpdatesPath); - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + + Dataset possibleUpdates = readPath(spark, possibleUpdatesPath, ResultCommunityList.class); + Dataset result = readPath(spark, inputPath, resultClazz); result .joinWith( possibleUpdates, result.col("id").equalTo(possibleUpdates.col("resultId")), "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if (rcl.isPresent()) { - ArrayList communitySet = rcl.get().getCommunityList(); - List contextList = ret - .getContext() - .stream() - .map(con -> con.getId()) - .collect(Collectors.toList()); - Result res = new Result(); - res.setId(ret.getId()); - List propagatedContexts = new ArrayList<>(); - for (String cId : communitySet) { - if (!contextList.contains(cId)) { - Context newContext = new Context(); - newContext.setId(cId); - newContext - .setDataInfo( - Arrays - .asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); - propagatedContexts.add(newContext); - } - } - res.setContext(propagatedContexts); - ret.mergeFrom(res); - } - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() + .map(resultCommunityFn(), Encoders.bean(resultClazz)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(outputPath); + .json(outputPath); } + + private static MapFunction, R> resultCommunityFn() { + return (MapFunction, R>) value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + ArrayList communitySet = rcl.get().getCommunityList(); + List contextList = ret + .getContext() + .stream() + .map(con -> con.getId()) + .collect(Collectors.toList()); + Result res = new Result(); + res.setId(ret.getId()); + List propagatedContexts = new ArrayList<>(); + for (String cId : communitySet) { + if (!contextList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME))); + propagatedContexts.add(newContext); + } + } + res.setContext(propagatedContexts); + ret.mergeFrom(res); + } + return ret; + }; + } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java index 95fad98d7..4f5ac2552 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java @@ -8,29 +8,56 @@ import java.util.Arrays; import java.util.List; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; -import eu.dnetlib.dhp.QueryInformationSystem; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; public class PrepareResultCommunitySetStep1 { private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySetStep1.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String COMMUNITY_LIST_XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType')" + + " where $x//CONFIGURATION/context[./@type='community' or ./@type='ri']" + + " and $x//CONFIGURATION/context/param[./@name='status']/text() != 'hidden'" + + " return $x//CONFIGURATION/context/@id/string()"; + + /** + * associates to each result the set of community contexts they are associated to; associates to each target of a + * relation with allowed semantics the set of community context it could possibly inherit from the source of the + * relation + */ + // TODO + private static final String RESULT_CONTEXT_QUERY_TEMPLATE = "select target resultId, community_context " + + "from (select id, collect_set(co.id) community_context " + + " from result " + + " lateral view explode (context) c as co " + + " where datainfo.deletedbyinference = false %s group by id) p " + + " JOIN " + + " (select source, target from relation " + + " where datainfo.deletedbyinference = false %s ) r ON p.id = r.source"; + + /** + * a dataset for example could be linked to more than one publication. For each publication linked to that dataset + * the previous query will produce a row: targetId set of community context the target could possibly inherit with + * the following query there will be a single row for each result linked to more than one result of the result type + * currently being used + */ + // TODO + private static final String RESULT_COMMUNITY_LIST_QUERY = "select resultId , collect_set(co) communityList " + + "from result_context " + + "lateral view explode (community_context) c as co " + + "where length(co) > 0 " + + "group by resultId"; public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -64,7 +91,7 @@ public class PrepareResultCommunitySetStep1 { final String isLookupUrl = parser.get("isLookUpUrl"); log.info("isLookupUrl: {}", isLookupUrl); - final List communityIdList = QueryInformationSystem.getCommunityList(isLookupUrl); + final List communityIdList = getCommunityList(isLookupUrl); log.info("communityIdList: {}", new Gson().toJson(communityIdList)); final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase(); @@ -98,78 +125,43 @@ public class PrepareResultCommunitySetStep1 { Class resultClazz, String resultType, List communityIdList) { - // read the relation table and the table related to the result it is using - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - org.apache.spark.sql.Dataset relation = spark - .createDataset( - sc - .textFile(inputPath + "/relation") - .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); + + final String inputResultPath = inputPath + "/" + resultType; + log.info("Reading Graph table from: {}", inputResultPath); + + final String inputRelationPath = inputPath + "/relation"; + log.info("Reading relation table from: {}", inputResultPath); + + Dataset relation = readPath(spark, inputRelationPath, Relation.class); relation.createOrReplaceTempView("relation"); - log.info("Reading Graph table from: {}", inputPath + "/" + resultType); - Dataset result = readPathEntity(spark, inputPath + "/" + resultType, resultClazz); - + Dataset result = readPath(spark, inputResultPath, resultClazz); result.createOrReplaceTempView("result"); - getPossibleResultcommunityAssociation( - spark, allowedsemrel, outputPath + "/" + resultType, communityIdList); - } + final String outputResultPath = outputPath + "/" + resultType; + log.info("writing output results to: {}", outputResultPath); - private static void getPossibleResultcommunityAssociation( - SparkSession spark, - List allowedsemrel, - String outputPath, - List communityIdList) { + String resultContextQuery = String + .format( + RESULT_CONTEXT_QUERY_TEMPLATE, + getConstraintList(" co.id = '", communityIdList), + getConstraintList(" relClass = '", allowedsemrel)); - String communitylist = getConstraintList(" co.id = '", communityIdList); - String semrellist = getConstraintList(" relClass = '", allowedsemrel); - - /* - * associates to each result the set of community contexts they are associated to select id, collect_set(co.id) - * community_context " + " from result " + " lateral view explode (context) c as co " + - * " where datainfo.deletedbyinference = false "+ communitylist + " group by id associates to each target - * of a relation with allowed semantics the set of community context it could possibly inherit from the source - * of the relation - */ - String query = "Select target resultId, community_context " - + "from (select id, collect_set(co.id) community_context " - + " from result " - + " lateral view explode (context) c as co " - + " where datainfo.deletedbyinference = false " - + communitylist - + " group by id) p " - + "JOIN " - + "(select source, target " - + "from relation " - + "where datainfo.deletedbyinference = false " - + semrellist - + ") r " - + "ON p.id = r.source"; - - org.apache.spark.sql.Dataset result_context = spark.sql(query); + Dataset result_context = spark.sql(resultContextQuery); result_context.createOrReplaceTempView("result_context"); - // ( target, (mes, dh-ch-, ni)) - /* - * a dataset for example could be linked to more than one publication. For each publication linked to that - * dataset the previous query will produce a row: targetId set of community context the target could possibly - * inherit with the following query there will be a single row for each result linked to more than one result of - * the result type currently being used - */ - query = "select resultId , collect_set(co) communityList " - + "from result_context " - + "lateral view explode (community_context) c as co " - + "where length(co) > 0 " - + "group by resultId"; - spark - .sql(query) + .sql(RESULT_COMMUNITY_LIST_QUERY) .as(Encoders.bean(ResultCommunityList.class)) - .toJavaRDD() - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath, GzipCodec.class); + .write() + .option("compression", "gzip") + .mode(SaveMode.Overwrite) + .json(outputResultPath); } + + public static List getCommunityList(final String isLookupUrl) throws ISLookUpException { + ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); + return isLookUp.quickSearchProfile(COMMUNITY_LIST_XQUERY); + } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java index cbd7e5e50..723aa8960 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java @@ -62,11 +62,11 @@ public class PrepareResultCommunitySetStep2 { private static void mergeInfo(SparkSession spark, String inputPath, String outputPath) { - Dataset resultOrcidAssocCommunityList = readResultCommunityList( - spark, inputPath + "/publication") - .union(readResultCommunityList(spark, inputPath + "/dataset")) - .union(readResultCommunityList(spark, inputPath + "/otherresearchproduct")) - .union(readResultCommunityList(spark, inputPath + "/software")); + Dataset resultOrcidAssocCommunityList = readPath( + spark, inputPath + "/publication", ResultCommunityList.class) + .union(readPath(spark, inputPath + "/dataset", ResultCommunityList.class)) + .union(readPath(spark, inputPath + "/otherresearchproduct", ResultCommunityList.class)) + .union(readPath(spark, inputPath + "/software", ResultCommunityList.class)); resultOrcidAssocCommunityList .toJavaRDD() @@ -80,9 +80,7 @@ public class PrepareResultCommunitySetStep2 { return a; } Set community_set = new HashSet<>(); - a.getCommunityList().stream().forEach(aa -> community_set.add(aa)); - b .getCommunityList() .stream() @@ -100,13 +98,4 @@ public class PrepareResultCommunitySetStep2 { .saveAsTextFile(outputPath, GzipCodec.class); } - private static Dataset readResultCommunityList( - SparkSession spark, String relationPath) { - return spark - .read() - .textFile(relationPath) - .map( - value -> OBJECT_MAPPER.readValue(value, ResultCommunityList.class), - Encoders.bean(ResultCommunityList.class)); - } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java similarity index 61% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java rename to dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java index b513ddd79..0c613d1b4 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob4.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -9,30 +9,28 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.ximpleware.extended.xpath.parser; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.schema.oaf.*; +import scala.Tuple2; -public class SparkResultToCommunityThroughSemRelJob4 { +public class SparkResultToCommunityThroughSemRelJob { - private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob4.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils .toString( - SparkResultToCommunityThroughSemRelJob4.class + SparkResultToCommunityThroughSemRelJob.class .getResourceAsStream( "/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); @@ -87,58 +85,59 @@ public class SparkResultToCommunityThroughSemRelJob4 { String preparedInfoPath, Class resultClazz) { - org.apache.spark.sql.Dataset possibleUpdates = readResultCommunityList( - spark, preparedInfoPath); - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); + Dataset possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class); + Dataset result = readPath(spark, inputPath, resultClazz); result .joinWith( possibleUpdates, result.col("id").equalTo(possibleUpdates.col("resultId")), "left_outer") - .map( - value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if (rcl.isPresent()) { - Set context_set = new HashSet<>(); - ret.getContext().stream().forEach(c -> context_set.add(c.getId())); - List contextList = rcl - .get() - .getCommunityList() - .stream() - .map( - c -> { - if (!context_set.contains(c)) { - Context newContext = new Context(); - newContext.setId(c); - newContext - .setDataInfo( - Arrays - .asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); - return newContext; - } - return null; - }) - .filter(c -> c != null) - .collect(Collectors.toList()); - Result r = new Result(); - r.setId(ret.getId()); - r.setContext(contextList); - ret.mergeFrom(r); - } - - return ret; - }, - Encoders.bean(resultClazz)) - .toJSON() + .map(contextUpdaterFn(), Encoders.bean(resultClazz)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(outputPath); + .json(outputPath); } + + private static MapFunction, R> contextUpdaterFn() { + return (MapFunction, R>) value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + Set context_set = new HashSet<>(); + ret.getContext().stream().forEach(c -> context_set.add(c.getId())); + List contextList = rcl + .get() + .getCommunityList() + .stream() + .map( + c -> { + if (!context_set.contains(c)) { + Context newContext = new Context(); + newContext.setId(c); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME))); + return newContext; + } + return null; + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + Result r = new Result(); + r.setId(ret.getId()); + r.setContext(contextList); + ret.mergeFrom(r); + } + + return ret; + }; + } + } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java index 02faf0086..f8fe1668f 100644 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java @@ -7,7 +7,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -58,8 +58,7 @@ public class PrepareResultInstRepoAssociation { isSparkSessionManaged, spark -> { readNeededResources(spark, inputPath); - prepareDatasourceOrganizationAssociations( - spark, datasourceOrganizationPath, alreadyLinkedPath); + prepareDatasourceOrganization(spark, datasourceOrganizationPath); prepareAlreadyLinkedAssociation(spark, alreadyLinkedPath); }); } @@ -77,45 +76,25 @@ public class PrepareResultInstRepoAssociation { spark .sql(query) .as(Encoders.bean(ResultOrganizationSet.class)) + // TODO retry to stick with datasets .toJavaRDD() .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(alreadyLinkedPath, GzipCodec.class); } private static void readNeededResources(SparkSession spark, String inputPath) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - org.apache.spark.sql.Dataset datasource = spark - .createDataset( - sc - .textFile(inputPath + "/datasource") - .map(item -> new ObjectMapper().readValue(item, Datasource.class)) - .rdd(), - Encoders.bean(Datasource.class)); - - org.apache.spark.sql.Dataset relation = spark - .createDataset( - sc - .textFile(inputPath + "/relation") - .map(item -> new ObjectMapper().readValue(item, Relation.class)) - .rdd(), - Encoders.bean(Relation.class)); - - org.apache.spark.sql.Dataset organization = spark - .createDataset( - sc - .textFile(inputPath + "/organization") - .map(item -> new ObjectMapper().readValue(item, Organization.class)) - .rdd(), - Encoders.bean(Organization.class)); - + Dataset datasource = readPath(spark, inputPath + "/datasource", Datasource.class); datasource.createOrReplaceTempView("datasource"); + + Dataset relation = readPath(spark, inputPath + "/relation", Relation.class); relation.createOrReplaceTempView("relation"); + + Dataset organization = readPath(spark, inputPath + "/organization", Organization.class); organization.createOrReplaceTempView("organization"); } - private static void prepareDatasourceOrganizationAssociations( - SparkSession spark, String datasourceOrganizationPath, String alreadyLinkedPath) { + private static void prepareDatasourceOrganization( + SparkSession spark, String datasourceOrganizationPath) { String query = "SELECT source datasourceId, target organizationId " + "FROM ( SELECT id " @@ -135,10 +114,9 @@ public class PrepareResultInstRepoAssociation { spark .sql(query) .as(Encoders.bean(DatasourceOrganization.class)) - .toJSON() .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .text(datasourceOrganizationPath); + .json(datasourceOrganizationPath); } } diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java new file mode 100644 index 000000000..86634d43f --- /dev/null +++ b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java @@ -0,0 +1,193 @@ + +package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.util.*; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.*; +import scala.Tuple2; + +public class SparkResultToOrganizationFromIstRepoJob { + + private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob.class); + + private static final String RESULT_ORGANIZATIONSET_QUERY = "SELECT id resultId, collect_set(organizationId) organizationSet " + + "FROM ( SELECT id, organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON cf = datasourceId " + + "UNION ALL " + + "SELECT id , organizationId " + + "FROM rels " + + "JOIN cfhb " + + " ON hb = datasourceId ) tmp " + + "GROUP BY id"; + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + SparkResultToOrganizationFromIstRepoJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String datasourceorganization = parser.get("datasourceOrganizationPath"); + log.info("datasourceOrganizationPath: {}", datasourceorganization); + + final String alreadylinked = parser.get("alreadyLinkedPath"); + log.info("alreadyLinkedPath: {}", alreadylinked); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + final Boolean saveGraph = Optional + .ofNullable(parser.get("saveGraph")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("saveGraph: {}", saveGraph); + + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + if (saveGraph) + execPropagation( + spark, + datasourceorganization, + alreadylinked, + inputPath, + outputPath, + resultClazz); + }); + } + + private static void execPropagation( + SparkSession spark, + String datasourceorganization, + String alreadyLinkedPath, + String inputPath, + String outputPath, + Class clazz) { + + Dataset ds_org = readPath(spark, datasourceorganization, DatasourceOrganization.class); + + Dataset potentialUpdates = getPotentialRelations(spark, inputPath, clazz, ds_org); + + Dataset alreadyLinked = readPath(spark, alreadyLinkedPath, ResultOrganizationSet.class); + + potentialUpdates + .joinWith( + alreadyLinked, + potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), + "left_outer") + .flatMap(createRelationFn(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Append) + .option("compression", "gzip") + .json(outputPath); + } + + private static FlatMapFunction, Relation> createRelationFn() { + return (FlatMapFunction, Relation>) value -> { + List new_relations = new ArrayList<>(); + ResultOrganizationSet potential_update = value._1(); + Optional already_linked = Optional.ofNullable(value._2()); + List organization_list = potential_update.getOrganizationSet(); + if (already_linked.isPresent()) { + already_linked + .get() + .getOrganizationSet() + .stream() + .forEach( + rId -> { + if (organization_list.contains(rId)) { + organization_list.remove(rId); + } + }); + } + String resultId = potential_update.getResultId(); + organization_list + .stream() + .forEach( + orgId -> { + new_relations + .add( + getRelation( + orgId, + resultId, + RELATION_ORGANIZATION_RESULT_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + new_relations + .add( + getRelation( + resultId, + orgId, + RELATION_RESULT_ORGANIZATION_REL_CLASS, + RELATION_RESULTORGANIZATION_REL_TYPE, + RELATION_RESULTORGANIZATION_SUBREL_TYPE, + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, + PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); + }); + return new_relations.iterator(); + }; + } + + private static Dataset getPotentialRelations( + SparkSession spark, + String inputPath, + Class resultClazz, + Dataset ds_org) { + + Dataset result = readPath(spark, inputPath, resultClazz); + result.createOrReplaceTempView("result"); + createCfHbforResult(spark); + + ds_org.createOrReplaceTempView("rels"); + + return spark + .sql(RESULT_ORGANIZATIONSET_QUERY) + .as(Encoders.bean(ResultOrganizationSet.class)); + } + +} diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java b/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java deleted file mode 100644 index 72c75e8a6..000000000 --- a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob2.java +++ /dev/null @@ -1,232 +0,0 @@ - -package eu.dnetlib.dhp.resulttoorganizationfrominstrepo; - -import static eu.dnetlib.dhp.PropagationConstant.*; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; - -import java.util.*; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.broadcast.Broadcast; -import org.apache.spark.sql.*; -import org.apache.spark.sql.Dataset; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.*; -import scala.Tuple2; - -public class SparkResultToOrganizationFromIstRepoJob2 { - - private static final Logger log = LoggerFactory.getLogger(SparkResultToOrganizationFromIstRepoJob2.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public static void main(String[] args) throws Exception { - - String jsonConfiguration = IOUtils - .toString( - SparkResultToOrganizationFromIstRepoJob2.class - .getResourceAsStream( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json")); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - - parser.parseArgument(args); - - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); - - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); - - final String datasourceorganization = parser.get("datasourceOrganizationPath"); - log.info("datasourceOrganizationPath: {}", datasourceorganization); - - final String alreadylinked = parser.get("alreadyLinkedPath"); - log.info("alreadyLinkedPath: {}", alreadylinked); - - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); - - final Boolean saveGraph = Optional - .ofNullable(parser.get("saveGraph")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("saveGraph: {}", saveGraph); - - Class resultClazz = (Class) Class.forName(resultClassName); - - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - if (saveGraph) - execPropagation( - spark, - datasourceorganization, - alreadylinked, - inputPath, - outputPath, - resultClazz); - }); - } - - private static void execPropagation( - SparkSession spark, - String datasourceorganization, - String alreadylinked, - String inputPath, - String outputPath, - Class resultClazz) { - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - org.apache.spark.sql.Dataset datasourceorganizationassoc = readAssocDatasourceOrganization( - spark, datasourceorganization); - - // broadcasting the result of the preparation step - Broadcast> broadcast_datasourceorganizationassoc = sc - .broadcast(datasourceorganizationassoc); - - org.apache.spark.sql.Dataset potentialUpdates = getPotentialRelations( - spark, - inputPath, - resultClazz, - broadcast_datasourceorganizationassoc) - .as(Encoders.bean(ResultOrganizationSet.class)); - - getNewRelations( - spark - .read() - .textFile(alreadylinked) - .map( - value -> OBJECT_MAPPER - .readValue( - value, ResultOrganizationSet.class), - Encoders.bean(ResultOrganizationSet.class)), - potentialUpdates) - .toJSON() - .write() - .mode(SaveMode.Append) - .option("compression", "gzip") - .text(outputPath); - - } - - private static Dataset getNewRelations( - Dataset alreadyLinked, - Dataset potentialUpdates) { - - return potentialUpdates - .joinWith( - alreadyLinked, - potentialUpdates.col("resultId").equalTo(alreadyLinked.col("resultId")), - "left_outer") - .flatMap( - (FlatMapFunction, Relation>) value -> { - List new_relations = new ArrayList<>(); - ResultOrganizationSet potential_update = value._1(); - Optional already_linked = Optional.ofNullable(value._2()); - List organization_list = potential_update.getOrganizationSet(); - if (already_linked.isPresent()) { - already_linked - .get() - .getOrganizationSet() - .stream() - .forEach( - rId -> { - if (organization_list.contains(rId)) { - organization_list.remove(rId); - } - }); - } - String resultId = potential_update.getResultId(); - organization_list - .stream() - .forEach( - orgId -> { - new_relations - .add( - getRelation( - orgId, - resultId, - RELATION_ORGANIZATION_RESULT_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - new_relations - .add( - getRelation( - resultId, - orgId, - RELATION_RESULT_ORGANIZATION_REL_CLASS, - RELATION_RESULTORGANIZATION_REL_TYPE, - RELATION_RESULTORGANIZATION_SUBREL_TYPE, - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_ID, - PROPAGATION_RELATION_RESULT_ORGANIZATION_INST_REPO_CLASS_NAME)); - }); - return new_relations.iterator(); - }, - Encoders.bean(Relation.class)); - } - - private static org.apache.spark.sql.Dataset getPotentialRelations( - SparkSession spark, - String inputPath, - Class resultClazz, - Broadcast> broadcast_datasourceorganizationassoc) { - org.apache.spark.sql.Dataset result = readPathEntity(spark, inputPath, resultClazz); - result.createOrReplaceTempView("result"); - createCfHbforresult(spark); - - return organizationPropagationAssoc(spark, broadcast_datasourceorganizationassoc); - } - - private static org.apache.spark.sql.Dataset readAssocDatasourceOrganization( - SparkSession spark, String datasourcecountryorganization) { - return spark - .read() - .textFile(datasourcecountryorganization) - .map( - value -> OBJECT_MAPPER.readValue(value, DatasourceOrganization.class), - Encoders.bean(DatasourceOrganization.class)); - } - - private static org.apache.spark.sql.Dataset organizationPropagationAssoc( - SparkSession spark, - Broadcast> broadcast_datasourceorganizationassoc) { - org.apache.spark.sql.Dataset datasourceorganization = broadcast_datasourceorganizationassoc - .value(); - datasourceorganization.createOrReplaceTempView("rels"); - String query = "SELECT id resultId, collect_set(organizationId) organizationSet " - + "FROM ( SELECT id, organizationId " - + "FROM rels " - + "JOIN cfhb " - + " ON cf = datasourceId " - + "UNION ALL " - + "SELECT id , organizationId " - + "FROM rels " - + "JOIN cfhb " - + " ON hb = datasourceId ) tmp " - + "GROUP BY id"; - return spark.sql(query).as(Encoders.bean(ResultOrganizationSet.class)); - } -} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json index 9956f3474..5efa3dbd6 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json @@ -5,6 +5,12 @@ "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, + { + "paramName":"out", + "paramLongName":"outputPath", + "paramDescription": "the output path", + "paramRequired": true + }, { "paramName":"h", "paramLongName":"hive_metastore_uris", diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index ac0fff2c0..90b004883 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -19,27 +19,22 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - - - - - - - - - - - - - + + + + + + + + @@ -50,11 +45,8 @@ - - - - + ${jobTracker} @@ -98,50 +90,6 @@ - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/publication - ${nameNode}/${workingDir}/publication - - - - - - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/dataset - ${nameNode}/${workingDir}/dataset - - - - - - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/otherresearchproduct - ${nameNode}/${workingDir}/otherresearchproduct - - - - - - - - ${jobTracker} - ${nameNode} - ${nameNode}/${sourcePath}/software - ${nameNode}/${workingDir}/software - - - - - @@ -159,7 +107,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=300 + --conf spark.sql.shuffle.partitions=3840 --sourcePath${sourcePath} --whitelist${whitelist} @@ -198,7 +146,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/publication + --sourcePath${sourcePath}/publication + --outputPath${workingDir}/publication --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --preparedInfoPath${workingDir}/preparedInfo @@ -227,7 +176,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/dataset + --sourcePath${sourcePath}/dataset + --outputPath${workingDir}/dataset --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --preparedInfoPath${workingDir}/preparedInfo @@ -256,7 +206,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/otherresearchproduct + --sourcePath${sourcePath}/otherresearchproduct + --outputPath${workingDir}/otherresearchproduct --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --preparedInfoPath${workingDir}/preparedInfo @@ -285,7 +236,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/software + --sourcePath${sourcePath}/software + --outputPath${workingDir}/software --hive_metastore_uris${hive_metastore_uris} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --preparedInfoPath${workingDir}/preparedInfo @@ -308,7 +260,7 @@ yarn cluster countryPropagationForPublications - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -323,7 +275,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/publication + --sourcePath${sourcePath}/publication + --preparedInfoPath${workingDir}/publication --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication --outputPath${outputPath}/publication @@ -337,7 +290,7 @@ yarn cluster countryPropagationForDataset - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -352,7 +305,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/dataset + --sourcePath${sourcePath}/dataset + --preparedInfoPath${workingDir}/dataset --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset --outputPath${outputPath}/dataset @@ -366,7 +320,7 @@ yarn cluster countryPropagationForORP - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -381,7 +335,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/otherresearchproduct + --sourcePath${sourcePath}/otherresearchproduct + --preparedInfoPath${workingDir}/otherresearchproduct --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct --outputPath${outputPath}/otherresearchproduct @@ -395,7 +350,7 @@ yarn cluster countryPropagationForSoftware - eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob3 + eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -410,7 +365,8 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --sourcePath${workingDir}/software + --sourcePath${sourcePath}/software + --preparedInfoPath${workingDir}/software --saveGraph${saveGraph} --resultTableNameeu.dnetlib.dhp.schema.oaf.Software --outputPath${outputPath}/software diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index 4c4b74b52..243167bd6 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -253,7 +253,7 @@ yarn cluster ORCIDPropagation-Publication - eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -285,7 +285,7 @@ yarn cluster ORCIDPropagation-Dataset - eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -316,7 +316,7 @@ yarn cluster ORCIDPropagation-ORP - eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -347,7 +347,7 @@ yarn cluster ORCIDPropagation-Software - eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob3 + eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 72ced0905..850a2f498 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -166,7 +166,7 @@ yarn cluster ProjectToResultPropagation - eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob3 + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index bf200e242..e041fc39c 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -127,7 +127,7 @@ yarn cluster community2resultfromorganization-Publication - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -155,7 +155,7 @@ yarn cluster community2resultfromorganization-Dataset - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -183,7 +183,7 @@ yarn cluster community2resultfromorganization-ORP - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -211,7 +211,7 @@ yarn cluster community2resultfromorganization-Software - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob2 + eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index 977e4838c..f2d406ad9 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -252,7 +252,7 @@ yarn cluster Result2CommunitySemRelPropagation-Publication - eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -280,7 +280,7 @@ yarn cluster Result2CommunitySemRelPropagation-Dataset - eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -308,7 +308,7 @@ yarn cluster Result2CommunitySemRelPropagation-ORP - eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -336,7 +336,7 @@ yarn cluster Result2CommunitySemRelPropagation-Software - eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob4 + eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 7e124f843..6bf7d0cec 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -166,7 +166,7 @@ yarn cluster resultToOrganizationFromInstRepoPropagationForPublications - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -196,7 +196,7 @@ yarn cluster resultToOrganizationFromInstRepoPropagationForDataset - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -225,7 +225,7 @@ yarn cluster resultToOrganizationFromInstRepoPropagationForORP - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} @@ -255,7 +255,7 @@ yarn cluster resultToOrganizationFromInstRepoPropagationForSoftware - eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob2 + eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob dhp-propagation-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 6c66606e1..10ff48cae 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -66,30 +66,25 @@ public class CountryPropagationJobTest { @Test public void testCountryPropagationSoftware() throws Exception { - SparkCountryPropagationJob2 + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/sample/software") + .getPath(); + final String preparedInfoPath = getClass() + .getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo") + .getPath(); + SparkCountryPropagationJob .main( new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/countrypropagation/sample/software") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", - workingDir.toString() + "/software", - "-preparedInfoPath", - getClass() - .getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo") - .getPath(), + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + "--hive_metastore_uris", "", + "-saveGraph", "true", + "-resultTableName", Software.class.getCanonicalName(), + "-outputPath", workingDir.toString() + "/software", + "-preparedInfoPath", preparedInfoPath }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/software") diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index d18acd550..0b0ec62d1 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -65,33 +65,27 @@ public class OrcidPropagationJobTest { @Test public void noUpdateTest() throws Exception { - SparkOrcidToResultFromSemRelJob3 + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate") + .getPath(); + final String possibleUpdatesPath = getClass() + .getResource( + "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") + .getPath(); + SparkOrcidToResultFromSemRelJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate") - .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-possibleUpdatesPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc") - .getPath() + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-hive_metastore_uris", "", + "-saveGraph", "true", + "-resultTableName", Dataset.class.getCanonicalName(), + "-outputPath", workingDir.toString() + "/dataset", + "-possibleUpdatesPath", possibleUpdatesPath }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") @@ -117,7 +111,7 @@ public class OrcidPropagationJobTest { @Test public void oneUpdateTest() throws Exception { - SparkOrcidToResultFromSemRelJob3 + SparkOrcidToResultFromSemRelJob .main( new String[] { "-isTest", @@ -182,7 +176,7 @@ public class OrcidPropagationJobTest { @Test public void twoUpdatesTest() throws Exception { - SparkOrcidToResultFromSemRelJob3 + SparkOrcidToResultFromSemRelJob .main( new String[] { "-isTest", diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index ac28e9d4b..7ed26b6b2 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -72,7 +72,7 @@ public class ProjectPropagationJobTest { @Test public void NoUpdateTest() throws Exception { - SparkResultToProjectThroughSemRelJob3 + SparkResultToProjectThroughSemRelJob .main( new String[] { "-isTest", @@ -115,7 +115,7 @@ public class ProjectPropagationJobTest { */ @Test public void UpdateTenTest() throws Exception { - SparkResultToProjectThroughSemRelJob3 + SparkResultToProjectThroughSemRelJob .main( new String[] { "-isTest", @@ -194,7 +194,7 @@ public class ProjectPropagationJobTest { */ @Test public void UpdateMixTest() throws Exception { - SparkResultToProjectThroughSemRelJob3 + SparkResultToProjectThroughSemRelJob .main( new String[] { "-isTest", diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index 0dd8c6bd4..ba8fb0831 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -67,8 +67,8 @@ public class ResultToCommunityJobTest { } @Test - public void test1() throws Exception { - SparkResultToCommunityFromOrganizationJob2 + public void testSparkResultToCommunityFromOrganizationJob() throws Exception { + SparkResultToCommunityFromOrganizationJob .main( new String[] { "-isTest", diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index e0ee12be6..13941b4a3 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -78,7 +78,7 @@ public class ResultToCommunityJobTest { @Test public void test1() throws Exception { - SparkResultToCommunityThroughSemRelJob4 + SparkResultToCommunityThroughSemRelJob .main( new String[] { "-isTest", Boolean.TRUE.toString(), diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java index 447cf47b0..e7adb260e 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java @@ -39,11 +39,11 @@ public class Result2OrganizationJobTest { public static void beforeAll() throws IOException { workingDir = Files .createTempDirectory( - SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()); log.info("using work dir {}", workingDir); SparkConf conf = new SparkConf(); - conf.setAppName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()); + conf.setAppName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()); conf.setMaster("local[*]"); conf.set("spark.driver.host", "localhost"); @@ -54,7 +54,7 @@ public class Result2OrganizationJobTest { spark = SparkSession .builder() - .appName(SparkResultToOrganizationFromIstRepoJob2.class.getSimpleName()) + .appName(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()) .config(conf) .getOrCreate(); } @@ -72,7 +72,7 @@ public class Result2OrganizationJobTest { */ @Test public void NoUpdateTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2 + SparkResultToOrganizationFromIstRepoJob .main( new String[] { "-isTest", @@ -123,7 +123,7 @@ public class Result2OrganizationJobTest { */ @Test public void UpdateNoMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2 + SparkResultToOrganizationFromIstRepoJob .main( new String[] { "-isTest", @@ -197,7 +197,7 @@ public class Result2OrganizationJobTest { @Test public void UpdateMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob2 + SparkResultToOrganizationFromIstRepoJob .main( new String[] { "-isTest", From f95d28868139995fd294023a45b90c3aef4a7081 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 May 2020 18:22:32 +0200 Subject: [PATCH 228/259] fixed swithch of parameters --- .../SparkRemoveBlacklistedRelationJob.java | 45 +++++++++++-------- .../dhp/blacklist/oozie_app/workflow.xml | 4 +- 2 files changed, 28 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index d25272263..fe6002710 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -62,8 +62,8 @@ public class SparkRemoveBlacklistedRelationJob { spark -> { removeBlacklistedRelations( spark, - inputPath, blacklistPath, + inputPath, outputPath, mergesPath); }); @@ -76,30 +76,34 @@ public class SparkRemoveBlacklistedRelationJob { Dataset inputRelation = readRelations(spark, inputPath); Dataset mergesRelation = readRelations(spark, mergesPath); + log.info("InputRelationCount: {}", inputRelation.count()); + Dataset dedupSource = blackListed .joinWith(mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), "left_outer") .map(c -> { - Optional merged = Optional.ofNullable(c._2()); - Relation bl = c._1(); - if (merged.isPresent()) { - bl.setSource(merged.get().getSource()); - } - return bl; + Optional + .ofNullable(c._2()) + .ifPresent(mr -> c._1().setSource(mr.getSource())); + return c._1(); }, Encoders.bean(Relation.class)); Dataset dedupBL = dedupSource .joinWith(mergesRelation, dedupSource.col("target").equalTo(mergesRelation.col("target")), "left_outer") .map(c -> { - Optional merged = Optional.ofNullable(c._2()); - Relation bl = c._1(); - if (merged.isPresent()) { - bl.setTarget(merged.get().getSource()); - } - return bl; + Optional + .ofNullable(c._2()) + .ifPresent(mr -> c._1().setTarget(mr.getSource())); + return c._1(); }, Encoders.bean(Relation.class)); - inputRelation - .joinWith(dedupBL, inputRelation.col("source").equalTo(dedupBL.col("source")), "left_outer") + dedupBL + .write() + .json(blacklistPath + "/deduped"); + + Dataset tmp = inputRelation + .joinWith( + dedupBL, inputRelation.col("source").equalTo(dedupBL.col("source")), + "left_outer") .map(c -> { Relation ir = c._1(); Optional obl = Optional.ofNullable(c._2()); @@ -111,12 +115,15 @@ public class SparkRemoveBlacklistedRelationJob { return ir; }, Encoders.bean(Relation.class)) - .filter(r -> !(r == null)) - .toJSON() + .filter(r -> r != null); + + log.info("NumberOfRelationAfterBlacklisting: {} ", tmp.count()); + + tmp .write() .mode(SaveMode.Overwrite) - .option("conpression", "gzip") - .text(outputPath); + .option("compression", "gzip") + .json(outputPath); } diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 78bac5eaf..855cac65e 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -62,7 +62,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePath${sourcePath} + --sourcePath${sourcePath}/relation --outputPath${workingDir}/mergesRelation --hive_metastore_uris${hive_metastore_uris} @@ -86,7 +86,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePath${sourcePath} + --sourcePath${sourcePath}/relation --outputPath${workingDir}/relation --hdfsPath${workingDir}/blacklist --mergesPath${workingDir}/mergesRelation From 53952707b6386bf3b7c5f281d28026a13fa4609f Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 8 May 2020 11:49:19 +0200 Subject: [PATCH 229/259] modified test because of new step of data preparation. It now expects to find ResultCountrySet serialization nstead of DatasourceCountry --- .../CountryPropagationJobTest.java | 1 - .../preparedInfo/preparedInfo.json.gz | Bin 985 -> 351 bytes 2 files changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 10ff48cae..2370d5e6c 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -77,7 +77,6 @@ public class CountryPropagationJobTest { new String[] { "--isSparkSessionManaged", Boolean.FALSE.toString(), "--sourcePath", sourcePath, - "--hive_metastore_uris", "", "-saveGraph", "true", "-resultTableName", Software.class.getCanonicalName(), "-outputPath", workingDir.toString() + "/software", diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz b/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz index 91a89da8c3d58147aebc199c322dd86d977a9d74..d838bee992e2188a81fe78507101f23319132bdb 100644 GIT binary patch literal 351 zcmV-l0igaLiwFpKAhliq18{O>aA9&~WJzvjZ!T(cZ*Bmkk=sfGK@5iPeTp*Iy_m_I z=K+Kk3SwIjM3&4<28wlC_JEe!cem?>$Rdkq?vniE`;(kmLv6qB7MTU6`THSPpCPn^ z5DZ!L5d=aZDoPtGo##pd7O6lfHfQP3*Ijd3Q^&&NnWa6p?QR%$dp~rCw8j@23JunY z*yZVRa~@GM<^EW`mWp!`g5q=qf)Y*?gJf-V#!yVLl!zMS6m>JEZu;h#PCrt=b`{)& z0MRD&77R!p$&oV<2|$&sjKnm#>tZvaE;`&_q4U{y)#_oSTG87s9VWe1Uh3;b406og z8zrnrGG`&Bl|?JC(W6NJ_qJV+RNH#jQJ$?k?5O?4)1t94d7%xLC2>JY=>kzZ=OS7V xZpz~FZUkTA5$ox@2<}5D(&|VB4WLzmJZMf@$5ha{;?evO`T?k=#^DtM0076#rTPE> literal 985 zcmV;~119_*iwFq6kdR&g18{O>aA9&~WJzvjZ!T(cZ*Bm!mr-k5M-YYI`zwN;`>?aS zvoq`0P*Vqz7CV8ymD!nD5OpOZ%cQBB|K3p_1(%8_S0EWl;7@b+oHOUXZcL8yuzxwI z-`VD71L>drlkaRVhu`CIb0c1`bZ#W#t}S(`xg~AW2W6af^W#SMFT3O6%jV{FqmT0Z zygmJ+kN2lpkkL?dNE~nD3EU(V>`UzXPiO4=m-t3@Z*JZEAjj?g!r`G4 z7gnUMWf1pFp40$`RPBuyf57(`9&!lE1Ts60RI4eLnou*2YHYqHyN9rQhB>rm9@q+x zlf%3;v^q1a$+kJiXg;k_i#f;0S#8!1UK!kUO?LLIiIZXNiCU70-BNh%q`kp))oYII zO4(*oQ9-bY?F6r#v=mObCc6Po5uni8$UftssgzbRQ;`96RW^1{Bexj8&+J5#E%aUD zPa+jwjPDP>oUuRb51-}IB*jjS2~(apXJaoOzUlxqr;xa;IWUKlJD0(y1P9}(hO@+` z>e3Kl(OlQ0xEUs|*0oZYDP)$bbB|Yc8wFQQ@x)XUW7xQW`+(Uw4^1wU!3xeb=7pg? z(RdiPmN=AL#%wsqL2|{#^?VIer&C8sQj~^53k`>tjJc3m?2RSn+n>*v51+Tkzy0ti zJG+cLc_K8>*=yuSY^&IfCt=Iz2$9{Zl#xEYdaS}I1a*KA=M2#iENN;*oR*-s=g`}O z?9?wjSB~Alo=0J?!s?{6&KCKRks2>vgB5J^>Yfz}th-c?XKUD%d$^o%w_O;L8DdqAlW=JCF0o9mzvg+QsN#1O6x z>k3g!J1dr~%U-<-%FKvZ2`NpQVVW}MB*o~sP+8Xzl1Kc;nL7BemL_6&C}cd1N;m9_ zKKR|4Tqu8G}qpNlVPTk7vxgqdZ<14HiX)03q%dFeDuF zc8oq1UgbzBS*#P+FjI-x^GTN4oYsn!Di)GW9cIXCQAOXSvVP;d?|(aEzxThky1%fq zC4{&wG0Pd5ti_6{Q6#{lv0H`3fv Date: Fri, 8 May 2020 11:51:45 +0200 Subject: [PATCH 230/259] - --- .../main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index a6662b9fc..90a99926e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -97,10 +97,11 @@ public class SparkBulkTagJob { ResultTagger resultTagger = new ResultTagger(); readPath(spark, inputPath, resultClazz) - .map((MapFunction) value -> resultTagger + .map( + (MapFunction) value -> resultTagger .enrichContextCriteria( - value, communityConfiguration, protoMappingParams), - Encoders.bean(resultClazz)) + value, communityConfiguration, protoMappingParams), + Encoders.bean(resultClazz)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") From b2192fdcdc958bc412527174df25e8147443db12 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 8 May 2020 12:33:31 +0200 Subject: [PATCH 231/259] simplified reset_outputpath nodes across the workflows, applied common xml formatting --- .../dhp/bulktag/oozie_app/workflow.xml | 80 ++++---- .../countrypropagation/oozie_app/workflow.xml | 10 +- .../oozie_app/workflow.xml | 192 +++++++++++------- .../projecttoresult/oozie_app/workflow.xml | 19 +- .../oozie_app/workflow.xml | 22 +- .../oozie_app/workflow.xml | 107 +++++----- .../oozie_app/workflow.xml | 50 +++-- 7 files changed, 252 insertions(+), 228 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 2fea9ff41..524281bc9 100644 --- a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -1,39 +1,33 @@ - - - sourcePath - the source path - - - isLookUpUrl - the isLookup service endpoint - - - pathMap - the json path associated to each selection field - - - outputPath - the output path - - + + + sourcePath + the source path + + + isLookUpUrl + the isLookup service endpoint + + + pathMap + the json path associated to each selection field + + + outputPath + the output path + + - + - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + - + - - - - - - - - + + @@ -67,6 +61,7 @@ + ${jobTracker} @@ -91,14 +86,13 @@ + + + + + + - - - - - - - ${jobTracker} @@ -127,6 +121,7 @@ + ${jobTracker} @@ -155,6 +150,7 @@ + ${jobTracker} @@ -183,6 +179,7 @@ + ${jobTracker} @@ -211,6 +208,9 @@ - - + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index 90b004883..f269c5442 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -27,14 +27,8 @@ - - - - - - - - + + diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index 243167bd6..7b06b6504 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -1,37 +1,29 @@ - - - sourcePath - the source path - - - allowedsemrels - the semantic relationships allowed for propagation - - - outputPath - the output path - - - + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + outputPath + the output path + + + - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - + - - - - - - - - - + + @@ -119,11 +111,16 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels${allowedsemrels} + --sourcePath + ${sourcePath} + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Publication + --outputPath + ${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels + ${allowedsemrels} @@ -147,11 +144,16 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels${allowedsemrels} + --sourcePath + ${sourcePath} + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Dataset + --outputPath + ${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels + ${allowedsemrels} @@ -175,11 +177,16 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels${allowedsemrels} + --sourcePath + ${sourcePath} + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath + ${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels + ${allowedsemrels} @@ -203,11 +210,16 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${sourcePath} - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels${allowedsemrels} + --sourcePath + ${sourcePath} + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Software + --outputPath + ${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels + ${allowedsemrels} @@ -233,11 +245,13 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath${workingDir}/preparedInfo/targetOrcidAssoc - --outputPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath + ${workingDir}/preparedInfo/targetOrcidAssoc + --outputPath + ${workingDir}/preparedInfo/mergedOrcidAssoc - + @@ -270,12 +284,18 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath${sourcePath}/publication - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${outputPath}/publication - --saveGraph${saveGraph} + --possibleUpdatesPath + ${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath + ${sourcePath}/publication + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Publication + --outputPath + ${outputPath}/publication + --saveGraph + ${saveGraph} @@ -301,12 +321,18 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath${sourcePath}/dataset - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${outputPath}/dataset - --saveGraph${saveGraph} + --possibleUpdatesPath + ${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath + ${sourcePath}/dataset + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Dataset + --outputPath + ${outputPath}/dataset + --saveGraph + ${saveGraph} @@ -332,12 +358,18 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath${sourcePath}/otherresearchproduct - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${outputPath}/otherresearchproduct - --saveGraph${saveGraph} + --possibleUpdatesPath + ${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath + ${sourcePath}/otherresearchproduct + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath + ${outputPath}/otherresearchproduct + --saveGraph + ${saveGraph} @@ -363,16 +395,22 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath${sourcePath}/software - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${outputPath}/software - --saveGraph${saveGraph} + --possibleUpdatesPath + ${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath + ${sourcePath}/software + --hive_metastore_uris + ${hive_metastore_uris} + --resultTableName + eu.dnetlib.dhp.schema.oaf.Software + --outputPath + ${outputPath}/software + --saveGraph + ${saveGraph} - + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 850a2f498..dd7f25846 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -14,27 +14,21 @@ - - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - - - - - - - - - + + + @@ -190,4 +184,5 @@ + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index e041fc39c..3be69bde6 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -14,23 +14,16 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - - - - - - - - - + + @@ -64,6 +57,7 @@ + ${jobTracker} @@ -85,6 +79,7 @@ + @@ -111,7 +106,6 @@ --organizationtoresultcommunitymap${organizationtoresultcommunitymap} - @@ -150,6 +144,7 @@ + yarn @@ -178,6 +173,7 @@ + yarn @@ -206,6 +202,7 @@ + yarn @@ -238,4 +235,5 @@ + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index f2d406ad9..b75b2d31e 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -1,43 +1,38 @@ - - - sourcePath - the source path - - - allowedsemrels - the semantic relationships allowed for propagation - - - isLookUpUrl - the isLookup service endpoint - - - outputPath - the output path - - + + + sourcePath + the source path + + + allowedsemrels + the semantic relationships allowed for propagation + + + isLookUpUrl + the isLookup service endpoint + + + outputPath + the output path + + - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + - - - - - - - - - + + + @@ -126,6 +121,7 @@ + yarn @@ -182,6 +178,7 @@ + yarn @@ -214,29 +211,29 @@ - - yarn - cluster - ResultToCommunityEmRelPropagation-PreparePhase2 - eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep2 - dhp-propagation-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --sourcePath${workingDir}/preparedInfo/targetCommunityAssoc - --outputPath${workingDir}/preparedInfo/mergedCommunityAssoc - - - - + + yarn + cluster + ResultToCommunityEmRelPropagation-PreparePhase2 + eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep2 + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${workingDir}/preparedInfo/targetCommunityAssoc + --outputPath${workingDir}/preparedInfo/mergedCommunityAssoc + + + + @@ -275,6 +272,7 @@ + yarn @@ -303,6 +301,7 @@ + yarn @@ -331,6 +330,7 @@ + yarn @@ -362,5 +362,6 @@ - + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 6bf7d0cec..73268fcc7 100644 --- a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -1,32 +1,25 @@ - - - sourcePath - the source path - - - outputPath - sets the outputPath - - + + + sourcePath + the source path + + + outputPath + sets the outputPath + + - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + - - - - - - - - - + + @@ -53,6 +46,7 @@ + ${jobTracker} @@ -85,6 +79,7 @@ + ${jobTracker} @@ -95,6 +90,7 @@ + ${jobTracker} @@ -151,9 +147,9 @@ --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - + @@ -220,6 +216,7 @@ + yarn @@ -281,6 +278,7 @@ + \ No newline at end of file From 28556507e7b0f0ab72c6c992ac1a5d67becbcdd5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 8 May 2020 12:54:52 +0200 Subject: [PATCH 232/259] - --- .../dhp/schema/common/ModelSupport.java | 360 ++++++++++++++---- .../blacklist/PrepareMergedRelationJob.java | 30 +- .../dhp/blacklist/ReadBlacklistFromDB.java | 2 +- .../SparkRemoveBlacklistedRelationJob.java | 2 +- 4 files changed, 312 insertions(+), 82 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java index 1fd2ef2da..7b0b9a1e2 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java @@ -1,4 +1,3 @@ - package eu.dnetlib.dhp.schema.common; import java.util.Map; @@ -13,7 +12,7 @@ import eu.dnetlib.dhp.schema.oaf.*; public class ModelSupport { /** Defines the mapping between the actual entity type and the main entity type */ - private static final Map entityMapping = Maps.newHashMap(); + private static Map entityMapping = Maps.newHashMap(); static { entityMapping.put(EntityType.publication, MainEntityType.result); @@ -53,6 +52,232 @@ public class ModelSupport { oafTypes.put("relation", Relation.class); } + public static final Map entityIdPrefix = Maps.newHashMap(); + + static { + entityIdPrefix.put("datasource", "10"); + entityIdPrefix.put("organization", "20"); + entityIdPrefix.put("project", "40"); + entityIdPrefix.put("result", "50"); + } + + public static final Map relationInverseMap = Maps.newHashMap(); + + static { + relationInverseMap + .put( + "personResult_authorship_isAuthorOf", new RelationInverse() + .setRelation("isAuthorOf") + .setInverse("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap + .put( + "personResult_authorship_hasAuthor", new RelationInverse() + .setInverse("isAuthorOf") + .setRelation("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); + relationInverseMap + .put( + "projectOrganization_participation_isParticipant", new RelationInverse() + .setRelation("isParticipant") + .setInverse("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap + .put( + "projectOrganization_participation_hasParticipant", new RelationInverse() + .setInverse("isParticipant") + .setRelation("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); + relationInverseMap + .put( + "resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() + .setRelation("hasAuthorInstitution") + .setInverse("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap + .put( + "resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() + .setInverse("hasAuthorInstitution") + .setRelation("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); + relationInverseMap + .put( + "organizationOrganization_dedup_merges", new RelationInverse() + .setRelation("merges") + .setInverse("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap + .put( + "organizationOrganization_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); + relationInverseMap + .put( + "organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("organizationOrganization") + .setSubReltype("dedupSimilarity")); + + relationInverseMap + .put( + "resultProject_outcome_isProducedBy", new RelationInverse() + .setRelation("isProducedBy") + .setInverse("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap + .put( + "resultProject_outcome_produces", new RelationInverse() + .setInverse("isProducedBy") + .setRelation("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); + relationInverseMap + .put( + "projectPerson_contactPerson_isContact", new RelationInverse() + .setRelation("isContact") + .setInverse("hasContact") + .setRelType("projectPerson") + .setSubReltype("contactPerson")); + relationInverseMap + .put( + "projectPerson_contactPerson_hasContact", new RelationInverse() + .setInverse("isContact") + .setRelation("hasContact") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap + .put( + "personPerson_coAuthorship_isCoauthorOf", new RelationInverse() + .setInverse("isCoAuthorOf") + .setRelation("isCoAuthorOf") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); + relationInverseMap + .put( + "personPerson_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap + .put( + "personPerson_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("personPerson") + .setSubReltype("dedup")); + relationInverseMap + .put( + "personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("personPerson") + .setSubReltype("dedupSimilarity")); + relationInverseMap + .put( + "datasourceOrganization_provision_isProvidedBy", new RelationInverse() + .setInverse("provides") + .setRelation("isProvidedBy") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap + .put( + "datasourceOrganization_provision_provides", new RelationInverse() + .setInverse("isProvidedBy") + .setRelation("provides") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); + relationInverseMap + .put( + "resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("isAmongTopNSimilarDocuments") + .setRelation("hasAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_relationship_isRelatedTo", new RelationInverse() + .setInverse("isRelatedTo") + .setRelation("isRelatedTo") + .setRelType("resultResult") + .setSubReltype("relationship")); + relationInverseMap + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); + relationInverseMap + .put( + "resultResult_supplement_isSupplementTo", new RelationInverse() + .setInverse("isSupplementedBy") + .setRelation("isSupplementTo") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap + .put( + "resultResult_supplement_isSupplementedBy", new RelationInverse() + .setInverse("isSupplementTo") + .setRelation("isSupplementedBy") + .setRelType("resultResult") + .setSubReltype("supplement")); + relationInverseMap + .put( + "resultResult_part_isPartOf", new RelationInverse() + .setInverse("hasPart") + .setRelation("isPartOf") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap + .put( + "resultResult_part_hasPart", new RelationInverse() + .setInverse("isPartOf") + .setRelation("hasPart") + .setRelType("resultResult") + .setSubReltype("part")); + relationInverseMap + .put( + "resultResult_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap + .put( + "resultResult_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("resultResult") + .setSubReltype("dedup")); + relationInverseMap + .put( + "resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("resultResult") + .setSubReltype("dedupSimilarity")); + + } + private static final String schemeTemplate = "dnet:%s_%s_relations"; private ModelSupport() { @@ -68,7 +293,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - X subClazzObject, Y superClazzObject) { + X subClazzObject, Y superClazzObject) { return isSubClass(subClazzObject.getClass(), superClazzObject.getClass()); } @@ -82,7 +307,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - X subClazzObject, Class superClazz) { + X subClazzObject, Class superClazz) { return isSubClass(subClazzObject.getClass(), superClazz); } @@ -96,7 +321,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - Class subClazz, Class superClazz) { + Class subClazz, Class superClazz) { return superClazz.isAssignableFrom(subClazz); } @@ -108,32 +333,32 @@ public class ModelSupport { */ public static Class[] getOafModelClasses() { return new Class[] { - Author.class, - Context.class, - Country.class, - DataInfo.class, - Dataset.class, - Datasource.class, - ExternalReference.class, - ExtraInfo.class, - Field.class, - GeoLocation.class, - Instance.class, - Journal.class, - KeyValue.class, - Oaf.class, - OafEntity.class, - OAIProvenance.class, - Organization.class, - OriginDescription.class, - OtherResearchProduct.class, - Project.class, - Publication.class, - Qualifier.class, - Relation.class, - Result.class, - Software.class, - StructuredProperty.class + Author.class, + Context.class, + Country.class, + DataInfo.class, + Dataset.class, + Datasource.class, + ExternalReference.class, + ExtraInfo.class, + Field.class, + GeoLocation.class, + Instance.class, + Journal.class, + KeyValue.class, + Oaf.class, + OafEntity.class, + OAIProvenance.class, + Organization.class, + OriginDescription.class, + OtherResearchProduct.class, + Project.class, + Publication.class, + Qualifier.class, + Relation.class, + Result.class, + Software.class, + StructuredProperty.class }; } @@ -147,10 +372,10 @@ public class ModelSupport { public static String getScheme(final String sourceType, final String targetType) { return String - .format( - schemeTemplate, - entityMapping.get(EntityType.valueOf(sourceType)).name(), - entityMapping.get(EntityType.valueOf(targetType)).name()); + .format( + schemeTemplate, + entityMapping.get(EntityType.valueOf(sourceType)).name(), + entityMapping.get(EntityType.valueOf(targetType)).name()); } public static Function idFn() { @@ -165,42 +390,41 @@ public class ModelSupport { private static String idFnForRelation(T t) { Relation r = (Relation) t; return Optional - .ofNullable(r.getSource()) - .map( - source -> Optional - .ofNullable(r.getTarget()) - .map( - target -> Optional - .ofNullable(r.getRelType()) - .map( - relType -> Optional - .ofNullable(r.getSubRelType()) - .map( - subRelType -> Optional - .ofNullable(r.getRelClass()) - .map( - relClass -> String - .join( - source, - target, - relType, - subRelType, - relClass)) - .orElse( - String - .join( - source, - target, - relType, - subRelType))) - .orElse(String.join(source, target, relType))) - .orElse(String.join(source, target))) - .orElse(source)) - .orElse(null); + .ofNullable(r.getSource()) + .map( + source -> Optional + .ofNullable(r.getTarget()) + .map( + target -> Optional + .ofNullable(r.getRelType()) + .map( + relType -> Optional + .ofNullable(r.getSubRelType()) + .map( + subRelType -> Optional + .ofNullable(r.getRelClass()) + .map( + relClass -> String + .join( + source, + target, + relType, + subRelType, + relClass)) + .orElse( + String + .join( + source, + target, + relType, + subRelType))) + .orElse(String.join(source, target, relType))) + .orElse(String.join(source, target))) + .orElse(source)) + .orElse(null); } private static String idFnForOafEntity(T t) { return ((OafEntity) t).getId(); } - } diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java index fbefc1c87..7ac0a3413 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -66,19 +66,25 @@ public class PrepareMergedRelationJob { private static void selectMergesRelations(SparkSession spark, String inputPath, String outputPath) { Dataset relation = readRelations(spark, inputPath); - relation.createOrReplaceTempView("relation"); - spark - .sql( - "Select * from relation " + - "where relclass = 'merges' " + - "and datainfo.deletedbyinference = false") - .as(Encoders.bean(Relation.class)) - .toJSON() - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .text(outputPath); + relation.filter("relclass = 'merges' and datainfo.deletedbyinference=false") + .write() + .mode(SaveMode.Overwrite) + .option("compression","gizp") + .json(outputPath); +// relation.createOrReplaceTempView("relation"); +// +// spark +// .sql( +// "Select * from relation " + +// "where relclass = 'merges' " + +// "and datainfo.deletedbyinference = false") +// .as(Encoders.bean(Relation.class)) +// .toJSON() +// .write() +// .mode(SaveMode.Overwrite) +// .option("compression", "gzip") +// .text(outputPath); } public static org.apache.spark.sql.Dataset readRelations( diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index fc20eabe0..704cab375 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -53,7 +53,7 @@ public class ReadBlacklistFromDB implements Closeable { final String dbUrl = parser.get("postgresUrl"); final String dbUser = parser.get("postgresUser"); final String dbPassword = parser.get("postgresPassword"); - final String hdfsPath = parser.get("hdfsPath"); + final String hdfsPath = parser.get("hdfsPath") + "/blacklist"; final String hdfsNameNode = parser.get("hdfsNameNode"); try (final ReadBlacklistFromDB rbl = new ReadBlacklistFromDB(hdfsPath, hdfsNameNode, dbUrl, dbUser, diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index fe6002710..5bf9f5a3f 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -72,7 +72,7 @@ public class SparkRemoveBlacklistedRelationJob { private static void removeBlacklistedRelations(SparkSession spark, String blacklistPath, String inputPath, String outputPath, String mergesPath) { - Dataset blackListed = readRelations(spark, blacklistPath); + Dataset blackListed = readRelations(spark, blacklistPath + "/blacklist"); Dataset inputRelation = readRelations(spark, inputPath); Dataset mergesRelation = readRelations(spark, mergesPath); From 9a29ab75081aad03fee582d587aaecb02c10aa2b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 8 May 2020 13:08:56 +0200 Subject: [PATCH 233/259] got back to the readPath we have before --- .../java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 90a99926e..e62b4b4fc 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -25,6 +25,7 @@ import eu.dnetlib.dhp.schema.oaf.*; public class SparkBulkTagJob { private static final Logger log = LoggerFactory.getLogger(SparkBulkTagJob.class); + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -108,12 +109,12 @@ public class SparkBulkTagJob { .json(outputPath); } - private static Dataset readPath( - SparkSession spark, String inputEntityPath, Class clazz) { + public static Dataset readPath( + SparkSession spark, String inputPath, Class clazz) { return spark .read() - .json(inputEntityPath) - .as(Encoders.bean(clazz)); + .textFile(inputPath) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); } } From 7e66bc25271c6d8b3b65833654b2e7f6ac68570a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 09:40:58 +0200 Subject: [PATCH 234/259] fix a typo in the compression keyword and added some logging info in the spark job --- .../blacklist/PrepareMergedRelationJob.java | 11 ++-- .../SparkRemoveBlacklistedRelationJob.java | 61 +++++++++++++------ 2 files changed, 50 insertions(+), 22 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java index 7ac0a3413..d5c2b518a 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -67,11 +67,12 @@ public class PrepareMergedRelationJob { Dataset relation = readRelations(spark, inputPath); - relation.filter("relclass = 'merges' and datainfo.deletedbyinference=false") - .write() - .mode(SaveMode.Overwrite) - .option("compression","gizp") - .json(outputPath); + relation + .filter("relclass = 'merges' and datainfo.deletedbyinference=false") + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); // relation.createOrReplaceTempView("relation"); // // spark diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 5bf9f5a3f..1a47a0724 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.blacklist; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import java.util.Objects; import java.util.Optional; import org.apache.commons.io.IOUtils; @@ -18,6 +19,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; public class SparkRemoveBlacklistedRelationJob { private static final Logger log = LoggerFactory.getLogger(SparkRemoveBlacklistedRelationJob.class); @@ -78,8 +80,12 @@ public class SparkRemoveBlacklistedRelationJob { log.info("InputRelationCount: {}", inputRelation.count()); + log.info("NumberOfBlacklistedRelations: {}", blackListed.count()); + Dataset dedupSource = blackListed - .joinWith(mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), "left_outer") + .joinWith( + mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), + "left_outer") .map(c -> { Optional .ofNullable(c._2()) @@ -88,7 +94,9 @@ public class SparkRemoveBlacklistedRelationJob { }, Encoders.bean(Relation.class)); Dataset dedupBL = dedupSource - .joinWith(mergesRelation, dedupSource.col("target").equalTo(mergesRelation.col("target")), "left_outer") + .joinWith( + mergesRelation, dedupSource.col("target").equalTo(mergesRelation.col("target")), + "left_outer") .map(c -> { Optional .ofNullable(c._2()) @@ -98,28 +106,41 @@ public class SparkRemoveBlacklistedRelationJob { dedupBL .write() + .mode(SaveMode.Overwrite) .json(blacklistPath + "/deduped"); - Dataset tmp = inputRelation + log.info("number of dedupedBL: {}", dedupBL.count()); + + Dataset> tmp = inputRelation .joinWith( - dedupBL, inputRelation.col("source").equalTo(dedupBL.col("source")), - "left_outer") - .map(c -> { - Relation ir = c._1(); - Optional obl = Optional.ofNullable(c._2()); - if (obl.isPresent()) { - if (ir.equals(obl.get())) { - return null; - } + dedupBL, (inputRelation + .col("source") + .equalTo(dedupBL.col("source")) + .and( + inputRelation + .col("target") + .equalTo(dedupBL.col("target")) + .and(inputRelation.col("relclass").equalTo(dedupBL.col("relclass"))))), + "left_outer"); + + log.info("numberOfRelationAfterJoin: {}", tmp.count()); + + Dataset tmp1 = tmp.map(c -> { + Relation ir = c._1(); + Optional obl = Optional.ofNullable(c._2()); + if (obl.isPresent()) { + if (areEquals(ir, obl.get())) { + return null; } - return ir; + } + return ir; - }, Encoders.bean(Relation.class)) - .filter(r -> r != null); + }, Encoders.bean(Relation.class)) + .filter(Objects::nonNull); - log.info("NumberOfRelationAfterBlacklisting: {} ", tmp.count()); + log.info("NumberOfRelationAfterBlacklisting: {} ", tmp1.count()); - tmp + tmp1 .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") @@ -127,6 +148,12 @@ public class SparkRemoveBlacklistedRelationJob { } + private static boolean areEquals(Relation ir, Relation bl) { + return ir.getRelClass().equals(bl.getRelClass()) && + ir.getRelType().equals(bl.getRelType()) && + ir.getSubRelType().equals(bl.getSubRelType()); + } + public static org.apache.spark.sql.Dataset readRelations( SparkSession spark, String inputPath) { return spark From 622ba87ec252e7cec3c8ac891309769260170fc5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 10:10:36 +0200 Subject: [PATCH 235/259] changed the version --- dhp-workflows/dhp-bulktag/pom.xml | 2 +- dhp-workflows/dhp-propagation/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-bulktag/pom.xml index 4844ef287..cc2396abf 100644 --- a/dhp-workflows/dhp-bulktag/pom.xml +++ b/dhp-workflows/dhp-bulktag/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.7-SNAPSHOT + 1.1.8-SNAPSHOT 4.0.0 diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml index 7d822b9d2..67fdbd9cf 100644 --- a/dhp-workflows/dhp-propagation/pom.xml +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.7-SNAPSHOT + 1.1.8-SNAPSHOT 4.0.0 From dc8c8fa4803e7fbe6266db5b365b6264a5d60e65 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 10:20:48 +0200 Subject: [PATCH 236/259] changed the version --- dhp-workflows/dhp-blacklist/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 309f62a19..803559221 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.7-SNAPSHOT + 1.1.8-SNAPSHOT 4.0.0 From 5e3548add6f04c34462352ee1a1b0953f405e8b0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 10:33:08 +0200 Subject: [PATCH 237/259] - --- .../dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index b64642e02..5d9533cb0 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -373,7 +373,6 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication qualifier(USER_CLAIM, USER_CLAIM, DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS), "0.9"); - final List collectedFrom = listKeyValues( createOpenaireId(10, "infrastruct_::openaire", true), "OpenAIRE"); From a7e91e23bae38a6534c00569ee8d941e30fdf6de Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 10:34:30 +0200 Subject: [PATCH 238/259] update to versione 1.2.1-SNAPSHOT --- dhp-workflows/dhp-bulktag/pom.xml | 2 +- dhp-workflows/dhp-propagation/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-bulktag/pom.xml index cc2396abf..7c2afa0cc 100644 --- a/dhp-workflows/dhp-bulktag/pom.xml +++ b/dhp-workflows/dhp-bulktag/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.8-SNAPSHOT + 1.2.1-SNAPSHOT 4.0.0 diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml index 67fdbd9cf..9492fa7c5 100644 --- a/dhp-workflows/dhp-propagation/pom.xml +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.8-SNAPSHOT + 1.2.1-SNAPSHOT 4.0.0 From b0f0b24263b7683865553576bbca59cee4214bf0 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 10:37:31 +0200 Subject: [PATCH 239/259] update to version 1.2.1-SNAPSHOT --- dhp-workflows/dhp-propagation/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml index 67fdbd9cf..9492fa7c5 100644 --- a/dhp-workflows/dhp-propagation/pom.xml +++ b/dhp-workflows/dhp-propagation/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.8-SNAPSHOT + 1.2.1-SNAPSHOT 4.0.0 From 9a7ae523c918570c1d5fe8d83a230604b54ccf5e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 13:57:47 +0200 Subject: [PATCH 240/259] update to version 1.2.1-SNAPSHOT --- dhp-workflows/dhp-blacklist/pom.xml | 2 +- .../src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java diff --git a/dhp-workflows/dhp-blacklist/pom.xml b/dhp-workflows/dhp-blacklist/pom.xml index 803559221..37abc22f6 100644 --- a/dhp-workflows/dhp-blacklist/pom.xml +++ b/dhp-workflows/dhp-blacklist/pom.xml @@ -5,7 +5,7 @@ dhp-workflows eu.dnetlib.dhp - 1.1.8-SNAPSHOT + 1.2.1-SNAPSHOT 4.0.0 diff --git a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java new file mode 100644 index 000000000..477c0a98a --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java @@ -0,0 +1,4 @@ +package eu.dnetlib.dhp.blacklist; + +public class BlackListTest { +} From 112b2cb3c3fb8ff186f41558fbf4d9c06a4711be Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 13:58:58 +0200 Subject: [PATCH 241/259] added the test class --- .../dnetlib/dhp/blacklist/BlackListTest.java | 166 ++++++++++++++++++ 1 file changed, 166 insertions(+) diff --git a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java index 477c0a98a..bc8fd0cdc 100644 --- a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java +++ b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java @@ -1,4 +1,170 @@ + package eu.dnetlib.dhp.blacklist; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.oaf.Country; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Software; + public class BlackListTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final ClassLoader cl = eu.dnetlib.dhp.blacklist.BlackListTest.class.getClassLoader(); + + private static SparkSession spark; + + private static Path workingDir; + private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.blacklist.BlackListTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(eu.dnetlib.dhp.blacklist.BlackListTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(eu.dnetlib.dhp.blacklist.BlackListTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(BlackListTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + /* + * String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); final String outputPath = + * parser.get("outputPath"); log.info("outputPath {}: ", outputPath); final String blacklistPath = + * parser.get("hdfsPath"); log.info("blacklistPath {}: ", blacklistPath); final String mergesPath = + * parser.get("mergesPath"); log.info("mergesPath {}: ", mergesPath); + */ + @Test + public void noRemoveTest() throws Exception { + SparkRemoveBlacklistedRelationJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/relationsNoRemoval").getPath(), + "-outputPath", + workingDir.toString() + "/relation", + "-hdfsPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/blacklist").getPath(), + "-mergesPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/mergesRel").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(13, tmp.count()); + + } + + @Test + public void removeNoMergeMatchTest() throws Exception { + SparkRemoveBlacklistedRelationJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/relationsOneRemoval").getPath(), + "-outputPath", + workingDir.toString() + "/relation", + "-hdfsPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/blacklist").getPath(), + "-mergesPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/mergesRel").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(12, tmp.count()); + + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(eu.dnetlib.dhp.schema.oaf.Relation.class)); + + Assertions + .assertEquals( + 0, verificationDataset + .filter( + "source = '40|corda__h2020::5161f53ab205d803c36b4c888fe7deef' and " + + "target = '20|dedup_wf_001::157af406bc653aa4d9749318b644de43'") + .count()); + + Assertions.assertEquals(0, verificationDataset.filter("relClass = 'hasParticipant'").count()); + } + + @Test + public void removeMergeMatchTest() throws Exception { + SparkRemoveBlacklistedRelationJob + .main( + new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/relationOneRemovalWithMatch").getPath(), + "-outputPath", + workingDir.toString() + "/relation", + "-hdfsPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/blacklist").getPath(), + "-mergesPath", + getClass().getResource("/eu/dnetlib/dhp/blacklist/mergesRelOneMerge").getPath(), + }); + + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/relation") + .map(item -> OBJECT_MAPPER.readValue(item, Relation.class)); + + Assertions.assertEquals(12, tmp.count()); + + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(eu.dnetlib.dhp.schema.oaf.Relation.class)); + + Assertions.assertEquals(12, verificationDataset.filter("relClass = 'isProvidedBy'").count()); + + } } From f5d785e0966f949f796cf58056da8d5ea75f35b3 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 13:59:42 +0200 Subject: [PATCH 242/259] used the DbClient moved in dhp-common --- .../dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java | 2 +- .../dhp/oa/graph/raw/MigrateDbEntitiesApplication.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index 503e4c504..739c7a462 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -24,8 +24,8 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 5d9533cb0..e5e348642 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -30,8 +30,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; -import eu.dnetlib.dhp.oa.graph.raw.common.DbClient; import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Dataset; @@ -94,7 +94,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication log.info("Processing orgs..."); smdbe.execute("queryOrganizations.sql", smdbe::processOrganization); - log.info("Processing relations ds <-> orgs ..."); + log.info("Processing relationsNoRemoval ds <-> orgs ..."); smdbe.execute("queryDatasourceOrganization.sql", smdbe::processDatasourceOrganization); log.info("Processing projects <-> orgs ..."); From e563e653351b85b03e0614f6e700fcc3ee905612 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 14:11:44 +0200 Subject: [PATCH 243/259] moved check from join to method --- .../dhp/blacklist/SparkRemoveBlacklistedRelationJob.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 1a47a0724..6abfc70bd 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -119,8 +119,7 @@ public class SparkRemoveBlacklistedRelationJob { .and( inputRelation .col("target") - .equalTo(dedupBL.col("target")) - .and(inputRelation.col("relclass").equalTo(dedupBL.col("relclass"))))), + .equalTo(dedupBL.col("target")))), "left_outer"); log.info("numberOfRelationAfterJoin: {}", tmp.count()); @@ -151,7 +150,8 @@ public class SparkRemoveBlacklistedRelationJob { private static boolean areEquals(Relation ir, Relation bl) { return ir.getRelClass().equals(bl.getRelClass()) && ir.getRelType().equals(bl.getRelType()) && - ir.getSubRelType().equals(bl.getSubRelType()); + ir.getSubRelType().equals(bl.getSubRelType()) && + ir.getRelClass().equals(bl.getRelClass()); } public static org.apache.spark.sql.Dataset readRelations( From b35d57a1ac25ee919a82b96345d06f27acb36c8e Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 14:15:30 +0200 Subject: [PATCH 244/259] added resources for test --- .../dnetlib/dhp/blacklist/blacklist/blacklist | 20 +++++++++++++++++++ .../dhp/blacklist/mergesRel/mergesRel.json | 14 +++++++++++++ .../mergesRelOneMerge/mergesRel.json | 14 +++++++++++++ .../relations.json | 13 ++++++++++++ .../relationsNoRemoval/relations.json | 13 ++++++++++++ .../relationsOneRemove.json | 13 ++++++++++++ 6 files changed, 87 insertions(+) create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/blacklist/blacklist create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRel/mergesRel.json create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRelOneMerge/mergesRel.json create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationOneRemovalWithMatch/relations.json create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsNoRemoval/relations.json create mode 100644 dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsOneRemoval/relationsOneRemove.json diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/blacklist/blacklist b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/blacklist/blacklist new file mode 100644 index 000000000..ea95130af --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/blacklist/blacklist @@ -0,0 +1,20 @@ +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"projectOrganization","subRelType":"participation","relClass":"hasParticipant","source":"40|corda__h2020::5161f53ab205d803c36b4c888fe7deef","target":"20|dedup_wf_001::157af406bc653aa4d9749318b644de43"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"projectOrganization","subRelType":"participation","relClass":"isParticipant","source":"20|dedup_wf_001::157af406bc653aa4d9749318b644de43","target":"40|corda__h2020::5161f53ab205d803c36b4c888fe7deef"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od_______908::a47e1c3ede9a21ee5278a2e5c338d69b","target":"40|corda_______::189ff31d637eaaeaf4d3584dc490b1cf"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::189ff31d637eaaeaf4d3584dc490b1cf","target":"50|od_______908::a47e1c3ede9a21ee5278a2e5c338d69b"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od________18::a727cc288016db7132ef9a799aa83350","target":"40|corda_______::9826e8aba3e8f3a2a46545cf341838a8"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::9826e8aba3e8f3a2a46545cf341838a8","target":"50|od________18::a727cc288016db7132ef9a799aa83350"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od________18::062cf091d5c7a7d730001c34177042e3","target":"40|corda_______::9826e8aba3e8f3a2a46545cf341838a8"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::9826e8aba3e8f3a2a46545cf341838a8","target":"50|od________18::062cf091d5c7a7d730001c34177042e3"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|webcrawl____::68c191d9b972b47a235d311804c7f6f5","target":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64","target":"50|webcrawl____::68c191d9b972b47a235d311804c7f6f5"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od_______908::1b172ab34639e7935e2357119cf20830","target":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64","target":"50|od_______908::1b172ab34639e7935e2357119cf20830"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|doajarticles::cb234c66327d29ba5f13c0db7a4cf423","target":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::c3d0b21615b129cd7395e24f9cf6bb64","target":"50|doajarticles::cb234c66327d29ba5f13c0db7a4cf423"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od______1146::e2fafaba636a14e408f02c6ea26acb0e","target":"40|corda_______::35695c955c51f0bb39482ce5477047c7"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::35695c955c51f0bb39482ce5477047c7","target":"50|od______1146::e2fafaba636a14e408f02c6ea26acb0e"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|od_______908::b8e86ed982ff331764456e1f0759ed9c","target":"40|corda_______::35695c955c51f0bb39482ce5477047c7"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::35695c955c51f0bb39482ce5477047c7","target":"50|od_______908::b8e86ed982ff331764456e1f0759ed9c"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"isProducedBy","source":"50|webcrawl____::c472bf5944ce0495844d505d43d1c021","target":"40|corda_______::35695c955c51f0bb39482ce5477047c7"} +{"collectedfrom":null,"dataInfo":null,"lastupdatetimestamp":null,"relType":"resultProject","subRelType":"outcome","relClass":"produces","source":"40|corda_______::35695c955c51f0bb39482ce5477047c7","target":"50|webcrawl____::c472bf5944ce0495844d505d43d1c021"} \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRel/mergesRel.json b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRel/mergesRel.json new file mode 100644 index 000000000..8f0d296d6 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRel/mergesRel.json @@ -0,0 +1,14 @@ +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|od_______177::67c1385662f2fa0bde310bec15427646"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|doiboost____::8ea1631fa01adcbafc3f384b6a2c5cc3"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|od_______166::67c1385662f2fa0bde310bec15427646"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::d2a45f0f42d8dd66c364219924c37c3f","subRelType":"dedup","target":"50|od_______935::0bf7d9c5d2e1115a31cd558f83ae8ee3"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::d2a45f0f42d8dd66c364219924c37c3f","subRelType":"dedup","target":"50|doajarticles::d695fee344cb367a38ce6622f5fe9430"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|od_______267::14e952745e4b602ff72919aa881b8945"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|doiboost____::43941031067842fac90604d37b2a4149"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|core________::5c62b3ad05a23de613636607a424899d"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|scholexplore::1c467aabe5108ee840a4500d58f19328"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::cd4fc0411683ee762d50bfd30436f95b","subRelType":"dedup","target":"50|doiboost____::0ff61beeb12c49ed8a826b2b1883c8f8"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::cd4fc0411683ee762d50bfd30436f95b","subRelType":"dedup","target":"50|doajarticles::fca1220426b10ccb8b46e4967b353f37"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|doiboost____::dd96d41ee05d4022065c9d3096e1023a"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|erc_________::7d9a29ff323c2fe0ecf037189bf71b8e"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|webcrawl____::fdd999801fec35d4c6190bcabb850c52"} \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRelOneMerge/mergesRel.json b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRelOneMerge/mergesRel.json new file mode 100644 index 000000000..3d74ffa6e --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/mergesRelOneMerge/mergesRel.json @@ -0,0 +1,14 @@ +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|od_______908::a47e1c3ede9a21ee5278a2e5c338d69b"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|doiboost____::8ea1631fa01adcbafc3f384b6a2c5cc3"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"dedup","target":"50|od_______166::67c1385662f2fa0bde310bec15427646"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::d2a45f0f42d8dd66c364219924c37c3f","subRelType":"dedup","target":"50|od_______935::0bf7d9c5d2e1115a31cd558f83ae8ee3"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::d2a45f0f42d8dd66c364219924c37c3f","subRelType":"dedup","target":"50|doajarticles::d695fee344cb367a38ce6622f5fe9430"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|od_______267::14e952745e4b602ff72919aa881b8945"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|doiboost____::43941031067842fac90604d37b2a4149"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|core________::5c62b3ad05a23de613636607a424899d"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::325525b879b17d8059a4e58def2f7225","subRelType":"dedup","target":"50|scholexplore::1c467aabe5108ee840a4500d58f19328"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::cd4fc0411683ee762d50bfd30436f95b","subRelType":"dedup","target":"50|doiboost____::0ff61beeb12c49ed8a826b2b1883c8f8"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::cd4fc0411683ee762d50bfd30436f95b","subRelType":"dedup","target":"50|doajarticles::fca1220426b10ccb8b46e4967b353f37"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|doiboost____::dd96d41ee05d4022065c9d3096e1023a"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|erc_________::7d9a29ff323c2fe0ecf037189bf71b8e"} +{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"decisiontree-dedup-test","inferred":true,"invisible":false,"provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"relClass":"merges","source":"50|dedup_wf_001::a87be24a4fcac13c9298f0cc3acfc6ea","subRelType":"dedup","target":"50|webcrawl____::fdd999801fec35d4c6190bcabb850c52"} \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationOneRemovalWithMatch/relations.json b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationOneRemovalWithMatch/relations.json new file mode 100644 index 000000000..761cba478 --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationOneRemovalWithMatch/relations.json @@ -0,0 +1,13 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProducedBy","relType":"resultProject","source":"50|dedup_wf_001::3668b9bd87532a085dc7a18ce2086715","subRelType":"outcome","target":"40|corda_______::189ff31d637eaaeaf4d3584dc490b1cf"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::05c5c5d2920c01e194d6760f24885a82","subRelType":"provision","target":"20|dedup_wf_001::cd07e6c09886e59266fdbae32a9e319b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::07022f119fc3d1cb66fe84494aa820c9","subRelType":"provision","target":"20|doajarticles::c48e93350cf5287e604ef631f2a67087"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::09ea05970871d7d923caaa8d2416d10e","subRelType":"provision","target":"20|doajarticles::cd84ef51b2de10ff01d679e4e662594e"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0af8c8ecf992b177304eb8f5d978100b","subRelType":"provision","target":"20|doajarticles::4eb6845b141d2b36ed94918d2bf382f0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0b48a767b2b8d323ccdcaf2d40642746","subRelType":"provision","target":"20|doajarticles::46a4942a4707e842611278cfa26789f9"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0dd9573adad4e5cc322612f6e9ecc8ce","subRelType":"provision","target":"20|doajarticles::e34526e7b5efb700ddb4544700234a0b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0e870ab15f231d954306bb658fc747a2","subRelType":"provision","target":"20|doajarticles::ccac83f4f971e3cdc194ddb796850a37"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f19a2d702e31d451e9806f701584c97","subRelType":"provision","target":"20|doajarticles::7a02d64772c121c1f10c17f8e2bf2aec"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f4b6db6c02966acbfb60af527728c85","subRelType":"provision","target":"20|doajarticles::acd96b3bd87b176202b8ea494c318b21"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::11f65dc66da7ef1b1f3a3e59199e4d70","subRelType":"provision","target":"20|dedup_wf_001::6132363e7458cbd7c22aa284c7df1307"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::149fd06e8702d94aa648641fd1602284","subRelType":"provision","target":"20|dedup_wf_001::35ae35032078bc33bc92e2b0f2ecfa17"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::15581a45537ceb854bbddee49b2942b4","subRelType":"provision","target":"20|doajarticles::0b25b0ce56da469cc8ad74c7d83c16a3"} \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsNoRemoval/relations.json b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsNoRemoval/relations.json new file mode 100644 index 000000000..a79d1d8eb --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsNoRemoval/relations.json @@ -0,0 +1,13 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::018cb61ed43c01704decc66183ce5d60","subRelType":"provision","target":"20|dedup_wf_001::b9fff055ce5efacecbe4ef918c127f86"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::05c5c5d2920c01e194d6760f24885a82","subRelType":"provision","target":"20|dedup_wf_001::cd07e6c09886e59266fdbae32a9e319b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::07022f119fc3d1cb66fe84494aa820c9","subRelType":"provision","target":"20|doajarticles::c48e93350cf5287e604ef631f2a67087"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::09ea05970871d7d923caaa8d2416d10e","subRelType":"provision","target":"20|doajarticles::cd84ef51b2de10ff01d679e4e662594e"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0af8c8ecf992b177304eb8f5d978100b","subRelType":"provision","target":"20|doajarticles::4eb6845b141d2b36ed94918d2bf382f0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0b48a767b2b8d323ccdcaf2d40642746","subRelType":"provision","target":"20|doajarticles::46a4942a4707e842611278cfa26789f9"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0dd9573adad4e5cc322612f6e9ecc8ce","subRelType":"provision","target":"20|doajarticles::e34526e7b5efb700ddb4544700234a0b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0e870ab15f231d954306bb658fc747a2","subRelType":"provision","target":"20|doajarticles::ccac83f4f971e3cdc194ddb796850a37"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f19a2d702e31d451e9806f701584c97","subRelType":"provision","target":"20|doajarticles::7a02d64772c121c1f10c17f8e2bf2aec"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f4b6db6c02966acbfb60af527728c85","subRelType":"provision","target":"20|doajarticles::acd96b3bd87b176202b8ea494c318b21"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::11f65dc66da7ef1b1f3a3e59199e4d70","subRelType":"provision","target":"20|dedup_wf_001::6132363e7458cbd7c22aa284c7df1307"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::149fd06e8702d94aa648641fd1602284","subRelType":"provision","target":"20|dedup_wf_001::35ae35032078bc33bc92e2b0f2ecfa17"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::15581a45537ceb854bbddee49b2942b4","subRelType":"provision","target":"20|doajarticles::0b25b0ce56da469cc8ad74c7d83c16a3"} \ No newline at end of file diff --git a/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsOneRemoval/relationsOneRemove.json b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsOneRemoval/relationsOneRemove.json new file mode 100644 index 000000000..f809acfeb --- /dev/null +++ b/dhp-workflows/dhp-blacklist/src/test/resources/eu/dnetlib/dhp/blacklist/relationsOneRemoval/relationsOneRemove.json @@ -0,0 +1,13 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"hasParticipant","relType":"projectOrganization","source":"40|corda__h2020::5161f53ab205d803c36b4c888fe7deef","subRelType":"participation","target":"20|dedup_wf_001::157af406bc653aa4d9749318b644de43"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::05c5c5d2920c01e194d6760f24885a82","subRelType":"provision","target":"20|dedup_wf_001::cd07e6c09886e59266fdbae32a9e319b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::07022f119fc3d1cb66fe84494aa820c9","subRelType":"provision","target":"20|doajarticles::c48e93350cf5287e604ef631f2a67087"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::09ea05970871d7d923caaa8d2416d10e","subRelType":"provision","target":"20|doajarticles::cd84ef51b2de10ff01d679e4e662594e"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0af8c8ecf992b177304eb8f5d978100b","subRelType":"provision","target":"20|doajarticles::4eb6845b141d2b36ed94918d2bf382f0"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0b48a767b2b8d323ccdcaf2d40642746","subRelType":"provision","target":"20|doajarticles::46a4942a4707e842611278cfa26789f9"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0dd9573adad4e5cc322612f6e9ecc8ce","subRelType":"provision","target":"20|doajarticles::e34526e7b5efb700ddb4544700234a0b"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0e870ab15f231d954306bb658fc747a2","subRelType":"provision","target":"20|doajarticles::ccac83f4f971e3cdc194ddb796850a37"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f19a2d702e31d451e9806f701584c97","subRelType":"provision","target":"20|doajarticles::7a02d64772c121c1f10c17f8e2bf2aec"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::0f4b6db6c02966acbfb60af527728c85","subRelType":"provision","target":"20|doajarticles::acd96b3bd87b176202b8ea494c318b21"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::11f65dc66da7ef1b1f3a3e59199e4d70","subRelType":"provision","target":"20|dedup_wf_001::6132363e7458cbd7c22aa284c7df1307"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::149fd06e8702d94aa648641fd1602284","subRelType":"provision","target":"20|dedup_wf_001::35ae35032078bc33bc92e2b0f2ecfa17"} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":true,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"sysimport:crosswalk:entityregistry","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"lastupdatetimestamp":1588608946167,"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::15581a45537ceb854bbddee49b2942b4","subRelType":"provision","target":"20|doajarticles::0b25b0ce56da469cc8ad74c7d83c16a3"} \ No newline at end of file From 757bae53ea4cfdd19a6e3c89e4d3fa169fe2e914 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 14:28:37 +0200 Subject: [PATCH 245/259] removed unusefule serialization points --- .../SparkRemoveBlacklistedRelationJob.java | 26 ++++--------------- 1 file changed, 5 insertions(+), 21 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index 6abfc70bd..c5104058c 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -80,8 +80,6 @@ public class SparkRemoveBlacklistedRelationJob { log.info("InputRelationCount: {}", inputRelation.count()); - log.info("NumberOfBlacklistedRelations: {}", blackListed.count()); - Dataset dedupSource = blackListed .joinWith( mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), @@ -109,9 +107,8 @@ public class SparkRemoveBlacklistedRelationJob { .mode(SaveMode.Overwrite) .json(blacklistPath + "/deduped"); - log.info("number of dedupedBL: {}", dedupBL.count()); - Dataset> tmp = inputRelation + inputRelation .joinWith( dedupBL, (inputRelation .col("source") @@ -120,26 +117,19 @@ public class SparkRemoveBlacklistedRelationJob { inputRelation .col("target") .equalTo(dedupBL.col("target")))), - "left_outer"); - - log.info("numberOfRelationAfterJoin: {}", tmp.count()); - - Dataset tmp1 = tmp.map(c -> { + "left_outer") + .map(c -> { Relation ir = c._1(); Optional obl = Optional.ofNullable(c._2()); if (obl.isPresent()) { - if (areEquals(ir, obl.get())) { + if (ir.equals(obl.get())) { return null; } } return ir; }, Encoders.bean(Relation.class)) - .filter(Objects::nonNull); - - log.info("NumberOfRelationAfterBlacklisting: {} ", tmp1.count()); - - tmp1 + .filter(Objects::nonNull) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") @@ -147,12 +137,6 @@ public class SparkRemoveBlacklistedRelationJob { } - private static boolean areEquals(Relation ir, Relation bl) { - return ir.getRelClass().equals(bl.getRelClass()) && - ir.getRelType().equals(bl.getRelType()) && - ir.getSubRelType().equals(bl.getSubRelType()) && - ir.getRelClass().equals(bl.getRelClass()); - } public static org.apache.spark.sql.Dataset readRelations( SparkSession spark, String inputPath) { From bbc9b4f3298b30051d31f868ccd943dfdbcde667 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 14:28:55 +0200 Subject: [PATCH 246/259] removed unused imports --- .../src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java index bc8fd0cdc..2d6b1061b 100644 --- a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java +++ b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java @@ -9,7 +9,6 @@ import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.AfterAll; @@ -20,10 +19,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.schema.oaf.Country; import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.schema.oaf.Software; public class BlackListTest { From 6d0b11252e665f186223d104e0b14d94432f0f4a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 11 May 2020 17:32:06 +0200 Subject: [PATCH 247/259] bulktagging wfs moved into common dhp-enrichment module --- .../dhp/blacklist/oozie_app/workflow.xml | 61 ++-- .../dhp-bulktag/project-default.properties | 7 - .../{dhp-bulktag => dhp-enrichment}/pom.xml | 23 +- .../eu/dnetlib/dhp/PropagationConstant.java | 0 .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 16 +- .../dhp/bulktag}/community/Community.java | 23 +- .../community/CommunityConfiguration.java | 24 +- .../CommunityConfigurationFactory.java | 42 ++- .../dhp/bulktag}/community/Constraint.java | 8 +- .../dhp/bulktag}/community/Constraints.java | 16 +- .../dnetlib/dhp/bulktag}/community/Pair.java | 6 +- .../dhp/bulktag}/community/ProtoMap.java | 2 +- .../dhp/bulktag/community/Provider.java} | 14 +- .../community/QueryInformationSystem.java | 10 +- .../dhp/bulktag}/community/ResultTagger.java | 23 +- .../community/SelectionConstraints.java | 11 +- .../bulktag/community/TaggingConstants.java} | 10 +- .../bulktag}/community/ZenodoCommunity.java | 9 +- .../dhp/bulktag/criteria}/ContainsVerb.java | 2 +- .../criteria}/ContainsVerbIgnoreCase.java | 2 +- .../dhp/bulktag/criteria}/EqualVerb.java | 2 +- .../criteria}/EqualVerbIgnoreCase.java | 2 +- .../bulktag/criteria}/InterfaceAdapter.java | 6 +- .../bulktag/criteria}/NotContainsVerb.java | 2 +- .../criteria}/NotContainsVerbIgnoreCase.java | 2 +- .../dhp/bulktag/criteria}/NotEqualVerb.java | 2 +- .../criteria}/NotEqualVerbIgnoreCase.java | 2 +- .../dhp/bulktag/criteria}/Selection.java | 2 +- .../dhp/bulktag/criteria}/VerbClass.java | 2 +- .../dhp/bulktag/criteria}/VerbResolver.java | 12 +- .../criteria}/VerbResolverFactory.java | 2 +- .../dhp/countrypropagation/CountrySbs.java | 0 .../countrypropagation/DatasourceCountry.java | 0 .../PrepareDatasourceCountryAssociation.java | 0 .../PrepareResultCountrySet.java | 0 .../countrypropagation/ResultCountrySet.java | 0 .../SparkCountryPropagationJob.java | 0 .../AutoritativeAuthor.java | 0 .../PrepareResultOrcidAssociationStep1.java | 0 .../PrepareResultOrcidAssociationStep2.java | 0 .../ResultOrcidList.java | 0 .../SparkOrcidToResultFromSemRelJob.java | 0 .../PrepareProjectResultsAssociation.java | 0 .../dhp/projecttoresult/ResultProjectSet.java | 0 .../SparkResultToProjectThroughSemRelJob.java | 0 .../OrganizationMap.java | 0 .../PrepareResultCommunitySet.java | 0 .../ResultCommunityList.java | 0 .../ResultOrganizations.java | 0 ...kResultToCommunityFromOrganizationJob.java | 0 .../PrepareResultCommunitySetStep1.java | 0 .../PrepareResultCommunitySetStep2.java | 0 ...parkResultToCommunityThroughSemRelJob.java | 0 .../DatasourceOrganization.java | 0 .../PrepareResultInstRepoAssociation.java | 0 .../ResultOrganizationSet.java | 0 ...arkResultToOrganizationFromIstRepoJob.java | 0 .../dhp/bulktag/input_bulkTag_parameters.json | 0 .../dhp/bulktag/oozie_app/config-default.xml | 0 .../dhp/bulktag/oozie_app/workflow.xml | 0 .../input_countrypropagation_parameters.json | 0 .../input_prepareassoc_parameters.json | 0 ...input_prepareresultcountry_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../countrypropagation/oozie_app/workflow.xml | 0 .../input_orcidtoresult_parameters.json | 0 ...input_prepareorcidtoresult_parameters.json | 0 ...nput_prepareorcidtoresult_parameters2.json | 0 .../oozie_app/config-default.xml | 0 .../oozie_app/workflow.xml | 0 ...put_prepareprojecttoresult_parameters.json | 0 .../input_projecttoresult_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../projecttoresult/oozie_app/workflow.xml | 0 .../input_communitytoresult_parameters.json | 0 ...t_preparecommunitytoresult_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../oozie_app/workflow.xml | 0 .../input_communitytoresult_parameters.json | 0 ..._preparecommunitytoresult2_parameters.json | 0 ...t_preparecommunitytoresult_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../oozie_app/workflow.xml | 0 .../input_prepareresultorg_parameters.json | 0 ...sulaffiliationfrominstrepo_parameters.json | 0 .../oozie_app/config-default.xml | 0 .../oozie_app/workflow.xml | 0 .../dnetlib/dhp/bulktag}/BulkTagJobTest.java | 326 +++++++----------- .../CommunityConfigurationFactoryTest.java | 22 +- .../CountryPropagationJobTest.java | 88 ++--- .../OrcidPropagationJobTest.java | 2 - .../ProjectPropagationJobTest.java | 153 ++++---- .../ResultToCommunityJobTest.java | 44 +-- .../ResultToCommunityJobTest.java | 33 +- .../ResultToOrganizationJobTest.java} | 163 ++++----- .../community_configuration.json | 0 .../community_configuration.xml | 28 +- .../community_configuration_selcrit.json | 0 .../community_configuration_selcrit.xml | 32 +- .../communityconfiguration/tagging_conf.json | 0 .../communityconfiguration/tagging_conf.xml | 62 ++-- .../dataset/no_updates/dataset_10.json.gz | Bin .../dataset_10.json.gz | Bin .../contextnoprovenance/dataset_10.json.gz | Bin .../nocontext/dataset_10.json.gz | Bin .../dataset_10.json.gz | Bin .../dataset_10.json.gz | Bin 0 -> 6968 bytes .../otherresearchproduct_10.json.gz | Bin .../update_datasource/publication_10.json.gz | Bin .../sample/software/software_10.json.gz | Bin .../preparedInfo/preparedInfo.json.gz | Bin .../sample/software/software_10.json.gz | Bin .../mergedOrcidAssoc/mergedOrcid_17.json.gz | Bin .../sample/noupdate/dataset_10.json.gz | Bin .../sample/oneupdate/dataset_10.json.gz | Bin .../sample/twoupdates/dataset_10.json.gz | Bin .../alreadyLinked/alreadyLinked.json.gz | Bin .../potentialUpdates/potentialUpdates.json.gz | Bin .../potentialUpdates/potentialUpdates.json.gz | Bin .../potentialUpdates/potentialUpdates.json.gz | Bin .../preparedInfo/resultCommunityList.json.gz | Bin .../sample/dataset_10.json.gz | Bin .../mergedResultCommunityList.json.gz | Bin .../sample/dataset_10.json.gz | Bin .../alreadyLinked/alreadyLinked_20.json.gz | Bin .../datasourceOrganization_28.json.gz | Bin .../noupdate_updatenomix/software_10.json.gz | Bin .../sample/updatemix/software_10.json.gz | Bin .../alreadyLinked/alreadyLinked_20.json.gz | Bin .../datasourceOrganization_28.json.gz | Bin .../alreadyLinked/alreadyLinked_20.json.gz | Bin .../datasourceOrganization_28.json.gz | Bin dhp-workflows/dhp-propagation/pom.xml | 43 --- .../eu/dnetlib/dhp/wf/profiles/provision.xml | 2 +- dhp-workflows/pom.xml | 3 +- 135 files changed, 550 insertions(+), 796 deletions(-) delete mode 100644 dhp-workflows/dhp-bulktag/project-default.properties rename dhp-workflows/{dhp-bulktag => dhp-enrichment}/pom.xml (81%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/PropagationConstant.java (100%) rename dhp-workflows/{dhp-bulktag => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java (97%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/Community.java (79%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/CommunityConfiguration.java (96%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/CommunityConfigurationFactory.java (86%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/Constraint.java (86%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/Constraints.java (94%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/Pair.java (92%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/ProtoMap.java (80%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java} (86%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/QueryInformationSystem.java (98%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/ResultTagger.java (94%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/SelectionConstraints.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java} (66%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag}/community/ZenodoCommunity.java (95%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/ContainsVerb.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/ContainsVerbIgnoreCase.java (92%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/EqualVerb.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/EqualVerbIgnoreCase.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/InterfaceAdapter.java (96%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/NotContainsVerb.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/NotContainsVerbIgnoreCase.java (92%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/NotEqualVerb.java (91%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/NotEqualVerbIgnoreCase.java (92%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/Selection.java (60%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/VerbClass.java (86%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/VerbResolver.java (97%) rename dhp-workflows/{dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria => dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria}/VerbResolverFactory.java (73%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java (100%) rename dhp-workflows/{dhp-bulktag => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json (100%) rename dhp-workflows/{dhp-bulktag => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-bulktag => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml (100%) rename dhp-workflows/{dhp-bulktag/src/test/java/eu/dnetlib/dhp => dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag}/BulkTagJobTest.java (71%) rename dhp-workflows/{dhp-bulktag/src/test/java/eu/dnetlib/dhp => dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag}/CommunityConfigurationFactoryTest.java (93%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java (81%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java (98%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java (62%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java (88%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java (88%) rename dhp-workflows/{dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java => dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java} (61%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/community_configuration.json (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/community_configuration.xml (95%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/community_configuration_selcrit.json (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/community_configuration_selcrit.xml (95%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/tagging_conf.json (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/communityconfiguration/tagging_conf.xml (98%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/dataset/no_updates/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/dataset/update_subject/nocontext/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject}/update_subject_datasource/dataset_10.json.gz (100%) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/dataset_10.json.gz rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/publication/update_datasource/publication_10.json.gz (100%) rename dhp-workflows/{dhp-bulktag/src/test/resources/eu/dnetlib/dhp => dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag}/sample/software/software_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz (100%) rename dhp-workflows/{dhp-propagation => dhp-enrichment}/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz (100%) delete mode 100644 dhp-workflows/dhp-propagation/pom.xml diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 855cac65e..f71c085b2 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -1,31 +1,38 @@ - - - postgresURL - the url of the postgress server to query - - - postgresUser - the username to access the postgres db - - - postgresPassword - the postgres password - - - sourcePath - the source path - - - + + + postgresURL + the url of the postgress server to query + + + postgresUser + the username to access the postgres db + + + postgresPassword + the postgres password + + + sourcePath + the source path + + + outputPath + the graph output path + + - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + - + + + @@ -87,12 +94,14 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --sourcePath${sourcePath}/relation - --outputPath${workingDir}/relation + --outputPath${outputPath}/relation --hdfsPath${workingDir}/blacklist --mergesPath${workingDir}/mergesRelation - + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/project-default.properties b/dhp-workflows/dhp-bulktag/project-default.properties deleted file mode 100644 index 84a56f19f..000000000 --- a/dhp-workflows/dhp-bulktag/project-default.properties +++ /dev/null @@ -1,7 +0,0 @@ -#sandboxName when not provided explicitly will be generated -sandboxName=${sandboxName} -sandboxDir=/user/${dhp.hadoop.frontend.user.name}/${sandboxName} -workingDir=${sandboxDir}/working_dir -oozie.wf.application.path = ${nameNode}${sandboxDir}/${oozieAppDir} -oozieTopWfApplicationPath = ${oozie.wf.application.path} - diff --git a/dhp-workflows/dhp-bulktag/pom.xml b/dhp-workflows/dhp-enrichment/pom.xml similarity index 81% rename from dhp-workflows/dhp-bulktag/pom.xml rename to dhp-workflows/dhp-enrichment/pom.xml index 7c2afa0cc..fe9833e3e 100644 --- a/dhp-workflows/dhp-bulktag/pom.xml +++ b/dhp-workflows/dhp-enrichment/pom.xml @@ -9,7 +9,7 @@ 4.0.0 - dhp-bulktag + dhp-enrichment @@ -31,6 +31,12 @@ dhp-schemas ${project.version} + + org.apache.spark + spark-hive_2.11 + test + + dom4j dom4j @@ -43,23 +49,16 @@ com.jayway.jsonpath json-path - - org.reflections - reflections - 0.9.11 - compile - - - com.google.guava - guava - 23.3-jre - + io.github.classgraph classgraph 4.8.71 + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/PropagationConstant.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index e62b4b4fc..e3d74ef3e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -1,10 +1,11 @@ package eu.dnetlib.dhp.bulktag; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.util.Optional; - +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.bulktag.community.*; +import eu.dnetlib.dhp.schema.oaf.Result; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -15,12 +16,9 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; +import java.util.Optional; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.community.*; -import eu.dnetlib.dhp.schema.oaf.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; public class SparkBulkTagJob { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java similarity index 79% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java index a73ff4d3e..d492b848e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Community.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java @@ -1,15 +1,14 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; + +import com.google.gson.Gson; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import com.google.gson.Gson; - /** Created by miriam on 01/08/2018. */ public class Community implements Serializable { @@ -17,7 +16,7 @@ public class Community implements Serializable { private String id; private List subjects = new ArrayList<>(); - private List datasources = new ArrayList<>(); + private List providers = new ArrayList<>(); private List zenodoCommunities = new ArrayList<>(); public String toJson() { @@ -27,7 +26,7 @@ public class Community implements Serializable { public boolean isValid() { return !getSubjects().isEmpty() - || !getDatasources().isEmpty() + || !getProviders().isEmpty() || !getZenodoCommunities().isEmpty(); } @@ -47,12 +46,12 @@ public class Community implements Serializable { this.subjects = subjects; } - public List getDatasources() { - return datasources; + public List getProviders() { + return providers; } - public void setDatasources(List datasources) { - this.datasources = datasources; + public void setProviders(List providers) { + this.providers = providers; } public List getZenodoCommunities() { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java similarity index 96% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java index c5bbb66eb..4e5b9fc9f 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfiguration.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java @@ -1,5 +1,14 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; +import eu.dnetlib.dhp.bulktag.criteria.Selection; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import java.io.Serializable; import java.util.ArrayList; @@ -8,17 +17,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; - -import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; -import eu.dnetlib.dhp.selectioncriteria.Selection; - /** Created by miriam on 02/08/2018. */ public class CommunityConfiguration implements Serializable { @@ -84,7 +82,7 @@ public class CommunityConfiguration implements Serializable { add(sbj.toLowerCase().trim(), p, subjectMap); } // get datasources - for (Datasource d : c.getDatasources()) { + for (Provider d : c.getProviders()) { add(d.getOpenaireId(), new Pair<>(id, d.getSelectionConstraints()), datasourceMap); } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java similarity index 86% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java index 508f0663d..dc83497c3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java @@ -1,11 +1,14 @@ -package eu.dnetlib.dhp.community; - -import java.io.StringReader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +package eu.dnetlib.dhp.bulktag.community; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; +import eu.dnetlib.dhp.bulktag.criteria.Selection; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -14,15 +17,10 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; - -import eu.dnetlib.dhp.selectioncriteria.InterfaceAdapter; -import eu.dnetlib.dhp.selectioncriteria.Selection; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; -import eu.dnetlib.dhp.selectioncriteria.VerbResolverFactory; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactory { @@ -77,7 +75,7 @@ public class CommunityConfigurationFactory { log.info(String.format("community id: %s", c.getId())); c.setSubjects(parseSubjects(node)); - c.setDatasources(parseDatasources(node)); + c.setProviders(parseDatasources(node)); c.setZenodoCommunities(parseZenodoCommunities(node)); return c; } @@ -96,17 +94,17 @@ public class CommunityConfigurationFactory { return subjects; } - private static List parseDatasources(final Node node) { + private static List parseDatasources(final Node node) { final List list = node.selectNodes("./datasources/datasource"); - final List datasourceList = new ArrayList<>(); + final List providerList = new ArrayList<>(); for (Node n : list) { - Datasource d = new Datasource(); + Provider d = new Provider(); d.setOpenaireId(n.selectSingleNode("./openaireId").getText()); d.setSelCriteria(n.selectSingleNode("./selcriteria"), resolver); - datasourceList.add(d); + providerList.add(d); } - log.info("size of the datasource list " + datasourceList.size()); - return datasourceList; + log.info("size of the datasource list " + providerList.size()); + return providerList; } private static List parseZenodoCommunities(final Node node) { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java similarity index 86% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java index 54f381d4a..8e28a7a5f 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraint.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java @@ -1,12 +1,12 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; + +import eu.dnetlib.dhp.bulktag.criteria.Selection; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; -import eu.dnetlib.dhp.selectioncriteria.Selection; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; - public class Constraint implements Serializable { private String verb; private String field; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java similarity index 94% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java index af095c513..eace3bc35 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Constraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java @@ -1,5 +1,11 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; @@ -8,14 +14,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; - -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; - /** Created by miriam on 02/08/2018. */ public class Constraints implements Serializable { private static final Log log = LogFactory.getLog(Constraints.class); diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java similarity index 92% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java index 01cd3ce22..1130a0770 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Pair.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java @@ -1,10 +1,10 @@ -package eu.dnetlib.dhp.community; - -import java.io.Serializable; +package eu.dnetlib.dhp.bulktag.community; import com.google.gson.Gson; +import java.io.Serializable; + /** Created by miriam on 03/08/2018. */ public class Pair implements Serializable { private A fst; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ProtoMap.java similarity index 80% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ProtoMap.java index d48dce2c6..fd7481719 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ProtoMap.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ProtoMap.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; import java.io.Serializable; import java.util.HashMap; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java similarity index 86% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java index a3d343087..c4362610e 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/Datasource.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java @@ -1,19 +1,17 @@ -package eu.dnetlib.dhp.community; - -import java.io.Serializable; +package eu.dnetlib.dhp.bulktag.community; +import com.google.gson.Gson; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.dom4j.Node; -import com.google.gson.Gson; - -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import java.io.Serializable; /** Created by miriam on 01/08/2018. */ -public class Datasource implements Serializable { - private static final Log log = LogFactory.getLog(Datasource.class); +public class Provider implements Serializable { + private static final Log log = LogFactory.getLog(Provider.class); private String openaireId; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java similarity index 98% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java index 2c18392c7..43eb40940 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/QueryInformationSystem.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java @@ -1,15 +1,13 @@ -package eu.dnetlib.dhp.community; - -import java.util.List; - -import org.dom4j.DocumentException; +package eu.dnetlib.dhp.bulktag.community; import com.google.common.base.Joiner; - import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.dom4j.DocumentException; + +import java.util.List; public class QueryInformationSystem { private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java similarity index 94% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index eb531c6b1..fd4f5497a 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -1,20 +1,19 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; -import static eu.dnetlib.dhp.community.TagginConstants.*; +import com.google.gson.Gson; +import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.JsonPath; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.lang3.StringUtils; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.commons.lang3.StringUtils; - -import com.google.gson.Gson; -import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.JsonPath; - -import eu.dnetlib.dhp.schema.oaf.*; +import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.*; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; /** Created by miriam on 02/08/2018. */ public class ResultTagger implements Serializable { @@ -51,7 +50,7 @@ public class ResultTagger implements Serializable { } public R enrichContextCriteria( - final R result, final CommunityConfiguration conf, final Map criteria) { + final R result, final CommunityConfiguration conf, final Map criteria) { // } // public Result enrichContextCriteria(final Result result, final CommunityConfiguration @@ -239,8 +238,8 @@ public class ResultTagger implements Serializable { Qualifier pa = new Qualifier(); pa.setClassid(inference_class_id); pa.setClassname(inference_class_name); - pa.setSchemeid(DNET_SCHEMA_ID); - pa.setSchemename(DNET_SCHEMA_NAME); + pa.setSchemeid(DNET_PROVENANCE_ACTIONS); + pa.setSchemename(DNET_PROVENANCE_ACTIONS); return pa; } } diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java index 802e2f5d6..28674d9ef 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java @@ -1,5 +1,9 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; import java.io.Serializable; import java.lang.reflect.Type; @@ -7,11 +11,6 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; - -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; - public class SelectionConstraints implements Serializable { private List criteria; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java similarity index 66% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java index 92d37d089..3cdc7c941 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/TagginConstants.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/TaggingConstants.java @@ -1,20 +1,14 @@ -package eu.dnetlib.dhp.community; +package eu.dnetlib.dhp.bulktag.community; -public class TagginConstants { +public class TaggingConstants { public static final String BULKTAG_DATA_INFO_TYPE = "bulktagging"; - public static final String DNET_SCHEMA_NAME = "dnet:provenanceActions"; - public static final String DNET_SCHEMA_ID = "dnet:provenanceActions"; - public static final String CLASS_ID_SUBJECT = "community:subject"; public static final String CLASS_ID_DATASOURCE = "community:datasource"; public static final String CLASS_ID_CZENODO = "community:zenodocommunity"; - public static final String SCHEMA_ID = "dnet:provenanceActions"; - public static final String COUNTER_GROUP = "Bulk Tagging"; - public static final String ZENODO_COMMUNITY_INDICATOR = "zenodo.org/communities/"; public static final String CLASS_NAME_BULKTAG_SUBJECT = "Bulktagging for Community - Subject"; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java similarity index 95% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java index e1492f6a5..eb0577ffc 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/community/ZenodoCommunity.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java @@ -1,11 +1,10 @@ -package eu.dnetlib.dhp.community; - -import java.io.Serializable; - -import org.dom4j.Node; +package eu.dnetlib.dhp.bulktag.community; import com.google.gson.Gson; +import org.dom4j.Node; + +import java.io.Serializable; /** Created by miriam on 01/08/2018. */ public class ZenodoCommunity implements Serializable { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerb.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerb.java index a6ef2d908..496630fa3 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerb.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerb.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerbIgnoreCase.java similarity index 92% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerbIgnoreCase.java index b8b0262e9..a4a6f5663 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/ContainsVerbIgnoreCase.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/ContainsVerbIgnoreCase.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerb.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerb.java index 3f17a6bb3..b9088d012 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerb.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerb.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerbIgnoreCase.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerbIgnoreCase.java index 934406859..c5f0ce070 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/EqualVerbIgnoreCase.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/EqualVerbIgnoreCase.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java similarity index 96% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java index 9ef3bd60c..2bc1ab3cf 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/InterfaceAdapter.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java @@ -1,10 +1,10 @@ -package eu.dnetlib.dhp.selectioncriteria; - -import java.lang.reflect.Type; +package eu.dnetlib.dhp.bulktag.criteria; import com.google.gson.*; +import java.lang.reflect.Type; + public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { private static final String CLASSNAME = "CLASSNAME"; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerb.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerb.java index eb83b256e..03ec9804b 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerb.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerb.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerbIgnoreCase.java similarity index 92% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerbIgnoreCase.java index fab3efef3..b21be83f0 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotContainsVerbIgnoreCase.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotContainsVerbIgnoreCase.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerb.java similarity index 91% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerb.java index 2311c2987..86bf00012 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerb.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerb.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerbIgnoreCase.java similarity index 92% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerbIgnoreCase.java index de2f682a5..c6958a641 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/NotEqualVerbIgnoreCase.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/NotEqualVerbIgnoreCase.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.io.Serializable; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/Selection.java similarity index 60% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/Selection.java index b488bda01..ec9fb716d 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/Selection.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/Selection.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; public interface Selection { diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbClass.java similarity index 86% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbClass.java index d467f934f..5b35919bd 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbClass.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbClass.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java similarity index 97% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java index 6a8ceebc3..19d6c4615 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolver.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java @@ -1,16 +1,16 @@ -package eu.dnetlib.dhp.selectioncriteria; - -import java.io.Serializable; -import java.lang.reflect.InvocationTargetException; -import java.util.Map; -import java.util.stream.Collectors; +package eu.dnetlib.dhp.bulktag.criteria; import io.github.classgraph.ClassGraph; import io.github.classgraph.ClassInfo; import io.github.classgraph.ClassInfoList; import io.github.classgraph.ScanResult; +import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.util.Map; +import java.util.stream.Collectors; + public class VerbResolver implements Serializable { private Map> map = null; // = new HashMap<>(); private final ClassGraph classgraph = new ClassGraph(); diff --git a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolverFactory.java similarity index 73% rename from dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolverFactory.java index 58bf60d42..0bb801999 100644 --- a/dhp-workflows/dhp-bulktag/src/main/java/eu/dnetlib/dhp/selectioncriteria/VerbResolverFactory.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolverFactory.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.selectioncriteria; +package eu.dnetlib.dhp.bulktag.criteria; public class VerbResolverFactory { diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/CountrySbs.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/DatasourceCountry.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareDatasourceCountryAssociation.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/ResultCountrySet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/SparkCountryPropagationJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/AutoritativeAuthor.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep1.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/PrepareResultOrcidAssociationStep2.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/ResultOrcidList.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/SparkOrcidToResultFromSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/PrepareProjectResultsAssociation.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/ResultProjectSet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/projecttoresult/SparkResultToProjectThroughSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultCommunityList.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultOrganizations.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep1.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/PrepareResultCommunitySetStep2.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/DatasourceOrganization.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/PrepareResultInstRepoAssociation.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultOrganizationSet.java diff --git a/dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/SparkResultToOrganizationFromIstRepoJob.java diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-bulktag/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_countrypropagation_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareassoc_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/input_prepareresultcountry_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_orcidtoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/input_prepareorcidtoresult_parameters2.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_prepareprojecttoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/input_projecttoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_communitytoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult2_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/input_preparecommunitytoresult_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_prepareresultorg_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/input_propagationresulaffiliationfrominstrepo_parameters.json diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml similarity index 100% rename from dhp-workflows/dhp-propagation/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml rename to dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java similarity index 71% rename from dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 75ecb0298..f20678f7b 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -1,13 +1,13 @@ -package eu.dnetlib.dhp; - -import static eu.dnetlib.dhp.community.TagginConstants.ZENODO_COMMUNITY_INDICATOR; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; +package eu.dnetlib.dhp.bulktag; +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Software; import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -18,37 +18,44 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.mortbay.util.IO; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; -import eu.dnetlib.dhp.bulktag.SparkBulkTagJob; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Software; +import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY_INDICATOR; public class BulkTagJobTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = eu.dnetlib.dhp.BulkTagJobTest.class.getClassLoader(); + public static final String MOCK_IS_LOOK_UP_URL = "BASEURL:8280/is/services/isLookUp"; + + public static final String pathMap = + "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}"; private static SparkSession spark; private static Path workingDir; - private static final Logger log = LoggerFactory.getLogger(eu.dnetlib.dhp.BulkTagJobTest.class); + + private static final Logger log = LoggerFactory.getLogger(BulkTagJobTest.class); private static String taggingConf = ""; static { try { - taggingConf = IO + taggingConf = IOUtils .toString( BulkTagJobTest.class .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); + "/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml")); } catch (IOException e) { e.printStackTrace(); } @@ -56,11 +63,11 @@ public class BulkTagJobTest { @BeforeAll public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + workingDir = Files.createTempDirectory(BulkTagJobTest.class.getSimpleName()); log.info("using work dir {}", workingDir); SparkConf conf = new SparkConf(); - conf.setAppName(eu.dnetlib.dhp.BulkTagJobTest.class.getSimpleName()); + conf.setAppName(BulkTagJobTest.class.getSimpleName()); conf.setMaster("local[*]"); conf.set("spark.driver.host", "localhost"); @@ -84,34 +91,21 @@ public class BulkTagJobTest { @Test public void noUpdatesTest() throws Exception { + final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/sample/dataset/no_updates").getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" - // "-preparedInfoPath", - // getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo").getPath() + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") @@ -134,34 +128,24 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectNoPreviousContextTest() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext") + .getPath(); + final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") @@ -240,32 +224,22 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { + final String sourcePath = getClass() + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance") + .getPath(); + final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); @@ -332,34 +306,23 @@ public class BulkTagJobTest { @Test public void bulktagByDatasourceTest() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource") + .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/publication/update_datasource") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Publication", - "-outputPath", - workingDir.toString() + "/publication", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", workingDir.toString() + "/publication", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/publication") @@ -415,35 +378,24 @@ public class BulkTagJobTest { @Test public void bulktagByZenodoCommunityTest() throws Exception { + final String sourcePath = getClass() + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity") + .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", - workingDir.toString() + "/orp", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", + "-outputPath", workingDir.toString() + "/orp", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/orp") @@ -548,34 +500,23 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectDatasourceTest() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource") + .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource("/eu/dnetlib/dhp/sample/dataset/update_subject_datasource") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") @@ -691,29 +632,17 @@ public class BulkTagJobTest { SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/sample/software/").getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", - workingDir.toString() + "/software", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/software/").getPath(), + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-outputPath", workingDir.toString() + "/software", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/software") @@ -796,35 +725,24 @@ public class BulkTagJobTest { @Test public void bulktagDatasourcewithConstraintsTest() throws Exception { + final String sourcePath = getClass() + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints") + .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints") - .getPath(), - "-taggingConf", - taggingConf, - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-isLookUpUrl", - "http://beta.services.openaire.eu:8280/is/services/isLookUp", - "-pathMap", - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}" + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") diff --git a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java similarity index 93% rename from dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index 3aae9ebee..056c3345c 100644 --- a/dhp-workflows/dhp-bulktag/src/test/java/eu/dnetlib/dhp/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -1,23 +1,21 @@ -package eu.dnetlib.dhp; - -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.util.*; +package eu.dnetlib.dhp.bulktag; +import com.google.gson.Gson; +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; +import eu.dnetlib.dhp.bulktag.community.CommunityConfigurationFactory; +import eu.dnetlib.dhp.bulktag.community.Constraint; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.dom4j.DocumentException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import com.google.gson.Gson; - -import eu.dnetlib.dhp.community.CommunityConfiguration; -import eu.dnetlib.dhp.community.CommunityConfigurationFactory; -import eu.dnetlib.dhp.community.Constraint; -import eu.dnetlib.dhp.community.SelectionConstraints; -import eu.dnetlib.dhp.selectioncriteria.VerbResolver; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.util.*; /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactoryTest { diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java similarity index 81% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index 2370d5e6c..b62238089 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -5,12 +5,15 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; 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.MapFunction; import org.apache.spark.sql.*; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; @@ -26,12 +29,11 @@ import eu.dnetlib.dhp.schema.oaf.Software; import scala.Tuple2; public class CountryPropagationJobTest { + private static final Logger log = LoggerFactory.getLogger(CountryPropagationJobTest.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = CountryPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; private static Path workingDir; @@ -101,8 +103,8 @@ public class CountryPropagationJobTest { Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); Dataset countryExploded = verificationDs - .flatMap(row -> row.getCountry().iterator(), Encoders.bean(Country.class)) - .map(c -> c.getClassid(), Encoders.STRING()); + .flatMap((FlatMapFunction) row -> row.getCountry().iterator(), Encoders.bean(Country.class)) + .map((MapFunction) c -> c.getClassid(), Encoders.STRING()); Assertions.assertEquals(9, countryExploded.count()); @@ -115,20 +117,18 @@ public class CountryPropagationJobTest { Assertions.assertEquals(2, countryExploded.filter("value = 'JP'").count()); Dataset> countryExplodedWithCountryclassid = verificationDs - .flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list - .stream() - .forEach( + .flatMap((FlatMapFunction>) row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( c -> prova .add( new Tuple2<>( row.getId(), c.getClassid()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + return prova.iterator(); + }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); Assertions.assertEquals(9, countryExplodedWithCountryclassid.count()); @@ -178,20 +178,20 @@ public class CountryPropagationJobTest { Dataset> countryExplodedWithCountryclassname = verificationDs .flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list - .stream() - .forEach( - c -> prova - .add( - new Tuple2<>( - row.getId(), - c.getClassname()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + (FlatMapFunction>) row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c.getClassname()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); countryExplodedWithCountryclassname.show(false); Assertions @@ -239,22 +239,22 @@ public class CountryPropagationJobTest { Dataset> countryExplodedWithCountryProvenance = verificationDs .flatMap( - row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list - .stream() - .forEach( - c -> prova - .add( - new Tuple2<>( - row.getId(), - c - .getDataInfo() - .getInferenceprovenance()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + (FlatMapFunction>) row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c + .getDataInfo() + .getInferenceprovenance()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); Assertions .assertEquals( diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java similarity index 98% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java index 0b0ec62d1..edd2e7ba7 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/orcidtoresultfromsemrel/OrcidPropagationJobTest.java @@ -29,8 +29,6 @@ public class OrcidPropagationJobTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = OrcidPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; private static Path workingDir; diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java similarity index 62% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index 7ed26b6b2..ff63753b8 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -9,6 +9,7 @@ import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; @@ -29,8 +30,6 @@ public class ProjectPropagationJobTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = ProjectPropagationJobTest.class.getClassLoader(); - private static SparkSession spark; private static Path workingDir; @@ -72,34 +71,26 @@ public class ProjectPropagationJobTest { @Test public void NoUpdateTest() throws Exception { - SparkResultToProjectThroughSemRelJob - .main( + final String potentialUpdateDate = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-hive_metastore_uris", "", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", potentialUpdateDate, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") @@ -115,34 +106,26 @@ public class ProjectPropagationJobTest { */ @Test public void UpdateTenTest() throws Exception { - SparkResultToProjectThroughSemRelJob - .main( + final String potentialUpdatePath = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-hive_metastore_uris", "", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", potentialUpdatePath, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") @@ -160,18 +143,18 @@ public class ProjectPropagationJobTest { .assertEquals( 5, verificationDs - .filter( - r -> r.getSource().substring(0, 2).equals("50") - && r.getTarget().substring(0, 2).equals("40") + .filter((FilterFunction) r -> + r.getSource().startsWith("50") + && r.getTarget().startsWith("40") && r.getRelClass().equals("isProducedBy")) .count()); Assertions .assertEquals( 5, verificationDs - .filter( - r -> r.getSource().substring(0, 2).equals("40") - && r.getTarget().substring(0, 2).equals("50") + .filter((FilterFunction) r -> + r.getSource().startsWith("40") + && r.getTarget().startsWith("50") && r.getRelClass().equals("produces")) .count()); @@ -194,34 +177,26 @@ public class ProjectPropagationJobTest { */ @Test public void UpdateMixTest() throws Exception { - SparkResultToProjectThroughSemRelJob - .main( + final String potentialUpdatepath = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - // "-sourcePath", - // getClass().getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/relation").getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-potentialUpdatePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-hive_metastore_uris", "", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-potentialUpdatePath", potentialUpdatepath, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") @@ -242,18 +217,18 @@ public class ProjectPropagationJobTest { .assertEquals( 4, verificationDs - .filter( - r -> r.getSource().substring(0, 2).equals("50") - && r.getTarget().substring(0, 2).equals("40") + .filter((FilterFunction) r -> + r.getSource().startsWith("50") + && r.getTarget().startsWith("40") && r.getRelClass().equals("isProducedBy")) .count()); Assertions .assertEquals( 4, verificationDs - .filter( - r -> r.getSource().substring(0, 2).equals("40") - && r.getTarget().substring(0, 2).equals("50") + .filter((FilterFunction) r -> + r.getSource().startsWith("40") + && r.getTarget().startsWith("50") && r.getRelClass().equals("produces")) .count()); diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java similarity index 88% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index ba8fb0831..20b20d4ed 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -32,8 +32,6 @@ public class ResultToCommunityJobTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = ResultToCommunityJobTest.class.getClassLoader(); - private static SparkSession spark; private static Path workingDir; @@ -68,33 +66,24 @@ public class ResultToCommunityJobTest { @Test public void testSparkResultToCommunityFromOrganizationJob() throws Exception { - SparkResultToCommunityFromOrganizationJob - .main( + final String preparedInfoPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") + .getPath(); + SparkResultToCommunityFromOrganizationJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass() .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") .getPath(), - "-hive_metastore_uris", - "", - "-saveGraph", - "true", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", - workingDir.toString() + "/dataset", - "-preparedInfoPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") - .getPath() + "-hive_metastore_uris", "", + "-saveGraph", "true", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-preparedInfoPath", preparedInfoPath }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") @@ -217,13 +206,6 @@ public class ResultToCommunityJobTest { .get(0) .getString(0)); - /* - * {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8"} - * "context" ["id": euromarine] updates = 1 - * {"communityList":["euromarine","mes"],"resultId":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6"} context - * = [ni, euromarine] updates = 1 - */ - query = "select id, MyT.id community " + "from dataset " + "lateral view explode(context) c as MyT " diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java similarity index 88% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java index 13941b4a3..a8e1ab841 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/ResultToCommunityJobTest.java @@ -29,32 +29,21 @@ import eu.dnetlib.dhp.schema.oaf.Dataset; public class ResultToCommunityJobTest { - private static final Logger log = LoggerFactory - .getLogger( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class); + private static final Logger log = LoggerFactory.getLogger(ResultToCommunityJobTest.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getClassLoader(); - private static SparkSession spark; private static Path workingDir; @BeforeAll public static void beforeAll() throws IOException { - workingDir = Files - .createTempDirectory( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getSimpleName()); + workingDir = Files.createTempDirectory(ResultToCommunityJobTest.class.getSimpleName()); log.info("using work dir {}", workingDir); SparkConf conf = new SparkConf(); - conf - .setAppName( - eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest.class - .getSimpleName()); + conf.setAppName(ResultToCommunityJobTest.class.getSimpleName()); conf.setMaster("local[*]"); conf.set("spark.driver.host", "localhost"); @@ -65,7 +54,7 @@ public class ResultToCommunityJobTest { spark = SparkSession .builder() - .appName(OrcidPropagationJobTest.class.getSimpleName()) + .appName(ResultToCommunityJobTest.class.getSimpleName()) .config(conf) .getOrCreate(); } @@ -83,22 +72,18 @@ public class ResultToCommunityJobTest { new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample") + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample") .getPath(), "-hive_metastore_uris", "", "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", - "-preparedInfoPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo") + "-preparedInfoPath", getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo") .getPath() }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/dataset") diff --git a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java similarity index 61% rename from dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java index e7adb260e..30be118d1 100644 --- a/dhp-workflows/dhp-propagation/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/Result2OrganizationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java @@ -23,23 +23,19 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.schema.oaf.Relation; -public class Result2OrganizationJobTest { +public class ResultToOrganizationJobTest { - private static final Logger log = LoggerFactory.getLogger(Result2OrganizationJobTest.class); + private static final Logger log = LoggerFactory.getLogger(ResultToOrganizationJobTest.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final ClassLoader cl = Result2OrganizationJobTest.class.getClassLoader(); - private static SparkSession spark; private static Path workingDir; @BeforeAll public static void beforeAll() throws IOException { - workingDir = Files - .createTempDirectory( - SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()); + workingDir = Files.createTempDirectory(SparkResultToOrganizationFromIstRepoJob.class.getSimpleName()); log.info("using work dir {}", workingDir); SparkConf conf = new SparkConf(); @@ -72,40 +68,29 @@ public class Result2OrganizationJobTest { */ @Test public void NoUpdateTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob - .main( + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(); + final String datasourceOrganizationPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", datasourceOrganizationPath, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") @@ -123,40 +108,29 @@ public class Result2OrganizationJobTest { */ @Test public void UpdateNoMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob - .main( + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(); + final String datasourceOrganizationPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", datasourceOrganizationPath, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") @@ -197,40 +171,29 @@ public class Result2OrganizationJobTest { @Test public void UpdateMixTest() throws Exception { - SparkResultToOrganizationFromIstRepoJob - .main( + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") + .getPath(); + final String datasourceOrganizationPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") + .getPath(); + final String alreadyLinkedPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob.main( new String[] { - "-isTest", - Boolean.TRUE.toString(), - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-sourcePath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") - .getPath(), - "-hive_metastore_uris", - "", - "-resultTableName", - "eu.dnetlib.dhp.schema.oaf.Software", - - "-saveGraph", - "true", - "-outputPath", - workingDir.toString() + "/relation", - "-datasourceOrganizationPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") - .getPath(), - "-alreadyLinkedPath", - getClass() - .getResource( - "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") - .getPath(), + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", sourcePath, + "-hive_metastore_uris", "", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", + "-saveGraph", "true", + "-outputPath", workingDir.toString() + "/relation", + "-datasourceOrganizationPath", datasourceOrganizationPath, + "-alreadyLinkedPath", alreadyLinkedPath, }); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc .textFile(workingDir.toString() + "/relation") diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.json similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.json rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.json diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.xml similarity index 95% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.xml index 8fec18593..e2cc41063 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.xml @@ -2,17 +2,17 @@ - + - + - + @@ -35,7 +35,7 @@ SDG9 - Industry innovation and infrastructure SDG16 - Peace justice and strong institutions - + 123 @@ -45,12 +45,12 @@ - + - + @@ -74,7 +74,7 @@ brain magnetic resonance imaging brain abnormalities - + re3data_____::5b9bf9171d92df854cf3c520692e9122 @@ -95,7 +95,7 @@ doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a - + @@ -106,12 +106,12 @@ aqua sea - + re3data_____::9633d1e8c4309c833c2c442abeb0cfeb - + @@ -134,7 +134,7 @@ food distribution forestry - + opendoar____::1a551829d50f1400b0dab21fdd969c04 @@ -159,18 +159,18 @@ opendoar____::87ae6fb631f7c8a627e8e28785d9992d - + oac_clarin - + re3data_____::a507cdacc5bbcc08761c92185dee5cab - + \ No newline at end of file diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.json similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.json diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml similarity index 95% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml index ad31e1763..89ace0e5f 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml @@ -2,17 +2,17 @@ - + - + - + @@ -35,7 +35,7 @@ SDG9 - Industry innovation and infrastructure SDG16 - Peace justice and strong institutions - + 123 @@ -45,12 +45,12 @@ - + - + @@ -74,7 +74,7 @@ brain magnetic resonance imaging brain abnormalities - + re3data_____::5b9bf9171d92df854cf3c520692e9122 @@ -95,7 +95,7 @@ doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a - + @@ -106,12 +106,12 @@ aqua sea - + re3data_____::9633d1e8c4309c833c2c442abeb0cfeb - + @@ -134,7 +134,7 @@ food distribution forestry - + opendoar____::1a551829d50f1400b0dab21fdd969c04 @@ -159,30 +159,30 @@ opendoar____::87ae6fb631f7c8a627e8e28785d9992d - + oac_clarin - + re3data_____::a507cdacc5bbcc08761c92185dee5cab - + oaa_dariah - + openaire____::1cfdb2e14977f31a98e0118283401f32 {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - + dimpo diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.json similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.json diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml similarity index 98% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml index 4f0d25f34..ec52d4799 100644 --- a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml @@ -1,7 +1,7 @@ - + zenodo @@ -11,27 +11,27 @@ Result: 2 - + Result: 3 - + Result: 4 - + re3data_____::a507cdacc5bbcc08761c92185dee5cab - + Result: 5 - + rda @@ -59,7 +59,7 @@ SDG9 - Industry innovation and infrastructure SDG16 - Peace justice and strong institutions - + Result: 7 @@ -132,7 +132,7 @@ architectural vessels - + re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 @@ -169,7 +169,7 @@ opendoar____::97275a23ca44226c9964043c8462be96 - + storm @@ -275,7 +275,7 @@ Fishing fleet Aquaculture - + doajarticles::8cec81178926caaca531afbd8eb5d64c @@ -328,7 +328,7 @@ doajarticles::dd70e44479f0ade25aa106aef3e87a0a - + discardless @@ -393,7 +393,7 @@ brain magnetic resonance imaging brain abnormalities - + re3data_____::5b9bf9171d92df854cf3c520692e9122 @@ -418,7 +418,7 @@ rest________::fb1a3d4523c95e63496e3bc7ba36244b - + neuroinformatics @@ -494,7 +494,7 @@ aqua sea - + adriplan @@ -688,7 +688,7 @@ Result: 11 - + instruct @@ -702,7 +702,7 @@ Result: 12 - + Result: 13 @@ -726,7 +726,7 @@ food distribution forestry - + opendoar____::1a551829d50f1400b0dab21fdd969c04 @@ -751,7 +751,7 @@ opendoar____::87ae6fb631f7c8a627e8e28785d9992d - + edenis @@ -829,7 +829,7 @@ Result: 14 - + opendoar____::7e7757b1e12abcb736ab9a754ffb617a {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} @@ -838,7 +838,7 @@ opendoar____::96da2f590cd7246bbde0051047b0d6f7 {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - + dimpo @@ -848,12 +848,12 @@ Result: 15 - + Result: 16 - + Result: 17 @@ -1009,7 +1009,7 @@ Innovative freight delivery systems urban freight delivery infrastructures - + doajarticles::1c5bdf8fca58937894ad1441cca99b76 @@ -1094,7 +1094,7 @@ doajarticles::fba6191177ede7c51ea1cdf58eae7f8b - + jsdtl @@ -1156,17 +1156,17 @@ Result: 18 - + Result: 19 - + Result: 20 - + Result: 21 @@ -1179,7 +1179,7 @@ science policy Policy and Law - + doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d @@ -1264,7 +1264,7 @@ doajarticles::7ffc35ac5133da01d421ccf8af5b70bc - + risis @@ -1287,7 +1287,7 @@ mesh:COVID-19 COVID2019 - + opendoar____::358aee4cc897452c00244351e4d91f69 {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, @@ -1366,7 +1366,7 @@ re3data_____::978378def740bbf2bfb420de868c460b {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCov"}]}]} - + chicago-covid-19 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/no_updates/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/no_updates/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject/nocontext/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject_datasource/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/update_subject_datasource/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/dataset/update_subject_datasource/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/update_subject_datasource/dataset_10.json.gz diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/dataset_10.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..fdc76a04c8b66fa7d14e03e25f67e7715b7e5843 GIT binary patch literal 6968 zcmV-88^`1yiwFoNN1$E+17u-zVRL14UokK)YIARH0PS7fj^j3#zMrQc3=A*>xLlHK z`R}e$UCCC^oiw_82FOki0!pN9RwU7(q)IB81~$*IPtccr(HDch%dKzo6#FE54oO+E z>%XTeQWEYqGG&`Thvz%|Js%H$>@ix<(`+Go^B;R6I~Tc#65g9HXp-|je6!$6akl+} z@q`zgMHl)Pp2crL?`tK`c}8{09{f;$rg0%;h84#N&2s^@^zw^b zq-&{)d8}lfe@ByV=8CUnE()nGAiGYQ(UjjMaUQRD%5}jkpCadjVr13*sOM z{Fxtl(JW%4#c&u;`D`2ps4c0)Qe-qag#!N3yJz-)f>vab@ECwuD49YLKlZ-i7a)sV z^NdNVu(bIc%Y(|!X;NZipYZkO1L}%Af!Yh25PtBto(6U%brTf|f`kqDiGnyn+F?n!IQyQ;ipL`AlaH!KJs6OVU zx+V+!2#9_IgO=^h*tMa)#X=;)Vyt_6f2ZbGsyq`(BFU#*Wt+Dd<3C{K`Q{yYLDS8L ztSLjzfB6Z&mcRUz?Fh3|3WZ;xn5{s;y#&%NB}WNZH>lw#qqP9$LtSe#>9)4vEwzFD zDxQ}~fdkFnAO_gxyx1Sew%_h|T2l^=STd1Iam}E1IhCGnvSmq^djjx0IRwMn6X4~V zW+Go*AwX5i)fLyjzQHM^%nBHKbO+AM=nO`Roe}@|%fFvKIq+xKkMVnA?aH$+fg8c^ zK7#ja6fVSyGLfg!!O$iQ=R$$*8D$QJR$Xcq;yN@^K`aOMrkDZl_*{LOosljkzoye~vda|cYD zz>J5!H}Hq#m0LcwdPH4_0#(yEa5_nd$e4h^3(){zNG#z$JTDY2_)^@k-$<#*h$(Tm zQ9D2EHvrK(76t!$H$hYSy(A9ekLhx!O;ibtg)}1|2jmhtXT;)r!WB zX;v`p(7|J5)^zwZdiF~WkOQ(mP}`*w@>O-NN#MVQ5>a`Bu$&$P(BR+VY{VfycT;+HmM~-PK5@}r&$2`Gd!l8IiS0Rm|O-1Ftt8lB_UY_rC zdiQ~^w=N?X=~g(38K)CZBdV5?XUy~!^KC9m72BSJ{;q<5U)WFD3v2$4Wv07FVV#5( zHqST{^eK(V0lDPFcL3cwbQ$Fvm!E+nhNMb+;85)m`hKBP4JWAj-V7kRN#vRtlmp2C>Dr+Mo8Lyu_X~G3PyerL-M3*!-GN8G{9-tL%@tT6lZ9s_g`J_3z|B` zx1o3#k_1B+JbnYi4st1y7ksO49H>N=+j$#6Nkk0mr=2<6@=;>NLF{vmErU8W9mS^3 z52K5NEI!EC;b`C;x|!r2Q~k~q(Ukei=O=~~n1(~2E+RJNQOL%lxc4y(#eT0;334Z`d9PS9Ska6nd;sDeUZQj1 zLf7syunmcfE0$aD5}WdY$&_x1{TNGZC9yZaa2>B;Y0YVhbf#*Rw*PBfDs28m^}EtT zC}%YTEC(TB?blqMGkz}Oda*8k*#51zo)v%0Foi27pkWMGzf|Aa3|W~*^`>BnXTy~l z{^Or-{`8~6|Dnj);pkd&cF3mjB=kq)U@?oLDGMTRJPXHc5yfFViF&t?vv14(E0VMC z!SkKW?#c5FhO@`y`5JlJ=J^J}q<4wuJ2~+E13!G_kLST`9)^$0^L>G5ThBM|6tZle z!KB$>w^ko(eyd5%fB8u@*tECNdmX>^YJO`hmCWe5fRPfU8~}Bi`v#axAkY?tRG@B|n!r(R~CRQwjXf2^51L4Qk^2 zrrzV+nO%hZ?(WQcn?j$&Pi0Fj#=}P&=xPD_9X%(kMI&$l;GwnbHiA z0io-bSIMA(q5#!E39yLdM1rdFfjgmjK*w@;;EfLg&*`)qj^&4OiaIw%Hx0+RDXwr+ z?4HN@Gz|S|Fk_*&@JF5(d5lJZzhG=Op2gni(cKhy8r5Ytg&VM)o5Hy%9?(rO^JbH4 z-4u)IY&0IlBN~rl_-etuc;b(S!^oe6RL7^d(M@q%_HMc<#`kqogrmpwQj9y{rI>nG zMk7Hl#R#L3488g2k)x5^xLfU$W7rhNg6fSVY^SgXoa?lo!oCfrW4)bBNPVDXi{0jX zx|2!#@{`W>)c~?Iv(=7}s|CVfB$fc|TqFO9Yhi9GV&VEtR~%S%r;!=g zr||b%xRK-IqfejI_f4F4>o;T%clU060&7Kp{)J4rlKpzDWm0K_kTGmwIGW5o&F2oD zufBEsL{Pr8<>n;Wyw5njV&tzpK`V;xrkpATTTJU@E>M_t(53iT61NkN;hwzdo4f%F=Ry75mv{m0kNuZ z@H(@)1{ouENf^Bm)Zy{A-=4#f*o|_{=_{M}q_pkD)std_XVR z7!M@^#|`g@m#oI-p^royyuaq82XX4pX3;EUJ`2a=Y3S3%gbm{`yu}fDYxWL0BE!Kr9NyCrIT=5m zCvq!K`<_VuMo*+an-8bX6X`sW5BLbdeF1ex%`LiUkn8PSGOYvgUxF#nnnR*>B;2^v z84zGM>q76g3&aVox|K|XyIb0cF2SwuDu=$Imst)J$E0xZ+{&bQy6c@aTa#5|(S>f( zSf*=U2urqTbh18ytmN}j`>YdkEb|+Hb!3Vzn#A476gx2Qk||*83gs_&4u2X4|7X<& z;d?se@!Sin3TzDT>5`l83L|A=MeU_qUEj?_38;TvZ|v}UXen!#vR-HT>dPdWtZraF0@NH5lmj4|ZS%bJRTxyy`_lx7_AX|6@im!-mM)6eh| z7#yC~X?f?Qaqd)|Y8|2z;m{i%ZdEfF+!j>YxpW;1;y0wzUx@|rpNv=zO-Fig*gDcf zS4a=OBR%}iI+i0n{DJYmmGm&0UVHmG;KRjm6!V!E2Ht}5Vd&8b_x;h7O<1h&<=sSj zxHWtCNDsr&VCDz+B0Y=-fj@mL(nBpz`=kdO^5RO;!*D(t&I7+y(!<|j1cELfI1lt{$njSs}MVnB3!ZW;YE8GC(C=3psJ5U%-cb&oq#Ln+T%b=q) zUyzLBD|zSIE1QrqudbfQc12cSv%h(j>_`kW^E7@ptECdfI4AlSTZ#$d?q7U`V) z1&^=(N^%*gUDLYI;m5hP%DB`e=2ft8S{dCy`}gy!9sJY7gdRjue2{DZ{_9M$F`jJM z7)+Ss5N+O1X(ScZCqdZGYez8K2W~l&$#kc&p)BIf2lI~o<}Jwup6}@Z50^Dr>knh9 z?{FcIWwE#V(~JvDn3t&*t5TkcAK&`|553f-iV&_Wr7FgTWrZ@)eYkM8J zLf0sxGp+-(qsRr!yr~XC(TK-x!cti5bSfP-esAd5qkABOMP#v9(@3M}sQAMi%wgjZ$r?XNo8VLVh zvr!Bcg@VM%J{LvvQz$75f#zpZg18EMTLDc5#Wb= zP*hb8pr_>p0Ru?Q$U7&6GK`s+LgS$y7{G2ejIESSjnm9>Z?tvUr7If51=Ja@pr~_> z@k+oxY<|#vW>G*H8~Q@A*Cl5Le`U0>oeEH}0X|A#Ohg}?%7%UD!6CQZ0+EHTZn$)P zYg4xH6-BUg%M?lZnN)q7%luB!#6b6UqN2ub>|jxFl?o^vD`>X&bx*7RIT{9vw%BLEy8#2C|U9kcZ40S)6fvCr&85z9mmw^joWIrd_79!idu4Q?5HhAFJzas4m_B`!-%`cAaOctr7Klhfm)lUrXlao9XQHe=r(4g4Y5E~*?H-C&IA z-5s$zx`U7FK#mC3MXc{m1Z&zre;d7q-iPURK~C~D$ECSuA3)A}`&6AIT@bs5e`(k7 z#^J3wOUQZP&8c)ikd1JX87_x}6u2X{hZ9>_Gfna_EF7MYTr4wz5A+yUAXoygcMQmU zgybt4;WIm$nP%yanyq7E+8`05%u9+Pa5Nv{=1HXrLz8IEKumefRm03r@2ov!0cb(D zUjJj6@Fi;Nxf16TA0$HGCgXT|C(JuW|Mx%td%StC;?mF=jzRfY%?T@G&6^>A(X##( z6B4PGG^^M{NcqT+b;ZQCLapef!UVR#%ruySJ~!`DeAw4s<~`msKmXlr{yi!ICR^rMFx z=eB!5HqaDZp~2AU)Ukp7kZ3wK(C-&p#|G*I8wj1gU1kHhDbTTj92@B2*gz96oCepi zfhJ+FU|tkN5siIL{WuOij^B;OJQ|O&7v97Mx;1+z*+AjgzZV-Q98CQ2BeH?4JngfA zroEfkK;dK_j9p+VM=-i0!RR@@Np9{ZIiE|rQC?!Xl8!~xcIYoWymYUkv<*ZRd?3Gk z@6rL&fniGIsf+5;DiANY^=+LCg?=uFMyumhy^KPK7%O9kNq1N*m*O6sqMPl!^8Ugm z4#RBl%*2^#bgDW;4XalM+964ATdq0QOBZdUOW}{uabEB^tHuqgi7H@-V0XwT^%^7j#XpgtU%*HxU+JpX*0B*@+<1Zem}vu zFVDojo-jxeM%Xcq@JdTc^AL*7yI3fC8dU(g{? z){x%v8br1dktMIxFHKP$&L)=`Z3HOFuG21(2{Kcx4aS3RM~N-S%W9O@4WJ;>_oAW? ztSA;_;|Vo{$G^@^Oqsn9PPRfDW2Is>?P{}6K;s-4DVlq5doU_%o=8#<7+)zv1A%%^ zGbpqu3wt!PgV-lRrki)n;F6I4lKEPn>-hmD{t_lN91bt`Yk!Wfe>DJoMwt?ZmD)!IiSwevM_K`NV`8nJ6rO#-j1c?2j zj0DJf#uu6_0!syGIxSQc66R_xW6U6JxlH#Mz%iGaJkmD?tu0CSRuCuFgb_@ZmuI}t zErSmj2S}4((DX9R$} zf4%PM2u+u}AZnUnkn>U+fi>L;1w%&PijtJac)*obd%U{C)(fb9-FcC}XWE7wAFSyI zLqdy%vhB0|=PPU~Ko>zLTy5Ulm+_zvOU+1HP-u14qJ$LqexqeQX)@MAniylYQf!gd z;p^T`#hStcmRt%w9AGr}*F8e9yz|XFL-0~^si>u=Y0^|iw}HZ%w6ZM;5$rwZ%EpH8 zgQzJUV`3mXsQ&X3hB_VNN+K-{N3+dafGfW=hV~H*8bPPZOqYO$r7-L1VaOn@I~9C^ z@!u|VS^c$Qcc0I$*BPEALn^?GR z(HSUEUDXv&heW$9JJ_r^FuDQ8b$5y#D(wzdc%B9SIIdqE7G0S*&}}Lt+q};UnAIIF zZQ}AN{f3i_f9ICCt-=t_d~e_dRrDSmo2D6&MDsyhAi~a&AwQRD$;oFFsaQ;26V&T^ zupFR@4Kl04u1jE*)f&k+V|qGU+EkVt2v*G2%_Gw)_RFT--#<=Yqv)rw}M zJ}EXYHKr6FL9@Cr1fzT#7bulgz%aX$fiH|iA0MNq(;>kMu41hY7ZS{c1al$5Tu87# z*aYk1q5PJJeU4}Ldo~=$GwTG;Y}{#{nH!NE&&=`69*}1ic%FY9&x}oG{t*5gPa+8Ay?zxi0-ltI+4n} zFmWX?;)IQfz^@1hND>qP-EtV7A(Gk=(~N$@D`&}_T&q;O#&YeqfkkE-sjagG6S#l~ z59>Fij0oZ{qwWe*nJ;9*=d~Z{kK|=QnYFlLz#h1lPXo!Db#CQhzZXQ5K9x zQOG0CrgRua!PFa$ZhP6|*6f}1o6P+C-Ory4CZowC`c14nZTn3ovn%~3ctL;s$`9wG z;oSF}-^BS%+y(vL;4INRjs(NUV~dkgL0@$zlI||D1H;DLmWk$=yFSq_4Em0%d*Y*Q z9F9c^Ya@#H6S$r$q`O4f@???9SYdc~2SlqtoWK^z>_C{gvgqJww_j)wa8@0W7c@m!o|Ymff2S(myj3wz@M6C9SU0wcgyQu@efP1bpN6^P2F9@`x@Zn} zXHOiWK$ zM2U!=Ni3v*JsVyqf8F~SgT=R2>O1x;eEoN}aQhPvN;rl6@cle@D{frKirae`x0!s_&lb19t;g@HS zaPvMp_=Kv6E9vMW_v#LJxzy$RPN$A8((rmXx`?BT{9frI=oRcXUBr#ZjxO@%PyY{f KBST|kQ~?0ZJb}sp literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/update_datasource/publication_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/publication/update_datasource/publication_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication_10.json.gz diff --git a/dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/software/software_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz similarity index 100% rename from dhp-workflows/dhp-bulktag/src/test/resources/eu/dnetlib/dhp/sample/software/software_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/countrypropagation/preparedInfo/preparedInfo.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/countrypropagation/sample/software/software_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/preparedInfo/mergedOrcidAssoc/mergedOrcid_17.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/noupdate/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/oneupdate/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/sample/twoupdates/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked/alreadyLinked.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates/potentialUpdates.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates/potentialUpdates.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates/potentialUpdates.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/preparedInfo/mergedResultCommunityList.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/sample/dataset_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix/software_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix/software_10.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked/alreadyLinked_20.json.gz diff --git a/dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz similarity index 100% rename from dhp-workflows/dhp-propagation/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization/datasourceOrganization_28.json.gz diff --git a/dhp-workflows/dhp-propagation/pom.xml b/dhp-workflows/dhp-propagation/pom.xml deleted file mode 100644 index 9492fa7c5..000000000 --- a/dhp-workflows/dhp-propagation/pom.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - dhp-workflows - eu.dnetlib.dhp - 1.2.1-SNAPSHOT - - 4.0.0 - - dhp-propagation - - - - 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} - - - org.apache.spark - spark-hive_2.11 - test - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml index 0467e618f..f99ea7aed 100644 --- a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml @@ -482,7 +482,7 @@ - marks as belonging to communities the result collected from datasources related to the organizations specified in the organizationCommunityMap + marks as belonging to communities the result collected from providers related to the organizations specified in the organizationCommunityMap executeOozieJob IIS diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index badd8ca8a..271c66939 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -22,8 +22,7 @@ dhp-actionmanager dhp-graph-mapper dhp-dedup-openaire - dhp-bulktag - dhp-propagation + dhp-enrichment dhp-graph-provision dhp-dedup-scholexplorer dhp-graph-provision-scholexplorer From c6b028f2af0c07671c5fe8c799896b87c36aa474 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 11 May 2020 17:38:08 +0200 Subject: [PATCH 248/259] code formatting --- .../dhp/schema/common/ModelSupport.java | 491 +++++++++--------- .../blacklist/PrepareMergedRelationJob.java | 13 - .../dhp/blacklist/ReadBlacklistFromDB.java | 3 +- .../SparkRemoveBlacklistedRelationJob.java | 26 +- .../dnetlib/dhp/blacklist/BlackListTest.java | 1 + .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 16 +- .../dhp/bulktag/community/Community.java | 9 +- .../community/CommunityConfiguration.java | 20 +- .../CommunityConfigurationFactory.java | 26 +- .../dhp/bulktag/community/Constraint.java | 6 +- .../dhp/bulktag/community/Constraints.java | 14 +- .../dnetlib/dhp/bulktag/community/Pair.java | 4 +- .../dhp/bulktag/community/Provider.java | 8 +- .../community/QueryInformationSystem.java | 8 +- .../dhp/bulktag/community/ResultTagger.java | 18 +- .../community/SelectionConstraints.java | 9 +- .../bulktag/community/ZenodoCommunity.java | 5 +- .../bulktag/criteria/InterfaceAdapter.java | 4 +- .../dhp/bulktag/criteria/VerbResolver.java | 10 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 69 +-- .../CommunityConfigurationFactoryTest.java | 20 +- .../CountryPropagationJobTest.java | 71 +-- .../ProjectPropagationJobTest.java | 61 +-- .../ResultToCommunityJobTest.java | 7 +- .../ResultToOrganizationJobTest.java | 48 +- 25 files changed, 491 insertions(+), 476 deletions(-) diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java index 7b0b9a1e2..fc85b1ac1 100644 --- a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/common/ModelSupport.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.schema.common; import java.util.Map; @@ -65,216 +66,216 @@ public class ModelSupport { static { relationInverseMap - .put( - "personResult_authorship_isAuthorOf", new RelationInverse() - .setRelation("isAuthorOf") - .setInverse("hasAuthor") - .setRelType("personResult") - .setSubReltype("authorship")); + .put( + "personResult_authorship_isAuthorOf", new RelationInverse() + .setRelation("isAuthorOf") + .setInverse("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); relationInverseMap - .put( - "personResult_authorship_hasAuthor", new RelationInverse() - .setInverse("isAuthorOf") - .setRelation("hasAuthor") - .setRelType("personResult") - .setSubReltype("authorship")); + .put( + "personResult_authorship_hasAuthor", new RelationInverse() + .setInverse("isAuthorOf") + .setRelation("hasAuthor") + .setRelType("personResult") + .setSubReltype("authorship")); relationInverseMap - .put( - "projectOrganization_participation_isParticipant", new RelationInverse() - .setRelation("isParticipant") - .setInverse("hasParticipant") - .setRelType("projectOrganization") - .setSubReltype("participation")); + .put( + "projectOrganization_participation_isParticipant", new RelationInverse() + .setRelation("isParticipant") + .setInverse("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); relationInverseMap - .put( - "projectOrganization_participation_hasParticipant", new RelationInverse() - .setInverse("isParticipant") - .setRelation("hasParticipant") - .setRelType("projectOrganization") - .setSubReltype("participation")); + .put( + "projectOrganization_participation_hasParticipant", new RelationInverse() + .setInverse("isParticipant") + .setRelation("hasParticipant") + .setRelType("projectOrganization") + .setSubReltype("participation")); relationInverseMap - .put( - "resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() - .setRelation("hasAuthorInstitution") - .setInverse("isAuthorInstitutionOf") - .setRelType("resultOrganization") - .setSubReltype("affiliation")); + .put( + "resultOrganization_affiliation_hasAuthorInstitution", new RelationInverse() + .setRelation("hasAuthorInstitution") + .setInverse("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); relationInverseMap - .put( - "resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() - .setInverse("hasAuthorInstitution") - .setRelation("isAuthorInstitutionOf") - .setRelType("resultOrganization") - .setSubReltype("affiliation")); + .put( + "resultOrganization_affiliation_isAuthorInstitutionOf", new RelationInverse() + .setInverse("hasAuthorInstitution") + .setRelation("isAuthorInstitutionOf") + .setRelType("resultOrganization") + .setSubReltype("affiliation")); relationInverseMap - .put( - "organizationOrganization_dedup_merges", new RelationInverse() - .setRelation("merges") - .setInverse("isMergedIn") - .setRelType("organizationOrganization") - .setSubReltype("dedup")); + .put( + "organizationOrganization_dedup_merges", new RelationInverse() + .setRelation("merges") + .setInverse("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); relationInverseMap - .put( - "organizationOrganization_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("organizationOrganization") - .setSubReltype("dedup")); + .put( + "organizationOrganization_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("organizationOrganization") + .setSubReltype("dedup")); relationInverseMap - .put( - "organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("organizationOrganization") - .setSubReltype("dedupSimilarity")); + .put( + "organizationOrganization_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("organizationOrganization") + .setSubReltype("dedupSimilarity")); relationInverseMap - .put( - "resultProject_outcome_isProducedBy", new RelationInverse() - .setRelation("isProducedBy") - .setInverse("produces") - .setRelType("resultProject") - .setSubReltype("outcome")); + .put( + "resultProject_outcome_isProducedBy", new RelationInverse() + .setRelation("isProducedBy") + .setInverse("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); relationInverseMap - .put( - "resultProject_outcome_produces", new RelationInverse() - .setInverse("isProducedBy") - .setRelation("produces") - .setRelType("resultProject") - .setSubReltype("outcome")); + .put( + "resultProject_outcome_produces", new RelationInverse() + .setInverse("isProducedBy") + .setRelation("produces") + .setRelType("resultProject") + .setSubReltype("outcome")); relationInverseMap - .put( - "projectPerson_contactPerson_isContact", new RelationInverse() - .setRelation("isContact") - .setInverse("hasContact") - .setRelType("projectPerson") - .setSubReltype("contactPerson")); + .put( + "projectPerson_contactPerson_isContact", new RelationInverse() + .setRelation("isContact") + .setInverse("hasContact") + .setRelType("projectPerson") + .setSubReltype("contactPerson")); relationInverseMap - .put( - "projectPerson_contactPerson_hasContact", new RelationInverse() - .setInverse("isContact") - .setRelation("hasContact") - .setRelType("personPerson") - .setSubReltype("coAuthorship")); + .put( + "projectPerson_contactPerson_hasContact", new RelationInverse() + .setInverse("isContact") + .setRelation("hasContact") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); relationInverseMap - .put( - "personPerson_coAuthorship_isCoauthorOf", new RelationInverse() - .setInverse("isCoAuthorOf") - .setRelation("isCoAuthorOf") - .setRelType("personPerson") - .setSubReltype("coAuthorship")); + .put( + "personPerson_coAuthorship_isCoauthorOf", new RelationInverse() + .setInverse("isCoAuthorOf") + .setRelation("isCoAuthorOf") + .setRelType("personPerson") + .setSubReltype("coAuthorship")); relationInverseMap - .put( - "personPerson_dedup_merges", new RelationInverse() - .setInverse("isMergedIn") - .setRelation("merges") - .setRelType("personPerson") - .setSubReltype("dedup")); + .put( + "personPerson_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("personPerson") + .setSubReltype("dedup")); relationInverseMap - .put( - "personPerson_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("personPerson") - .setSubReltype("dedup")); + .put( + "personPerson_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("personPerson") + .setSubReltype("dedup")); relationInverseMap - .put( - "personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("personPerson") - .setSubReltype("dedupSimilarity")); + .put( + "personPerson_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("personPerson") + .setSubReltype("dedupSimilarity")); relationInverseMap - .put( - "datasourceOrganization_provision_isProvidedBy", new RelationInverse() - .setInverse("provides") - .setRelation("isProvidedBy") - .setRelType("datasourceOrganization") - .setSubReltype("provision")); + .put( + "datasourceOrganization_provision_isProvidedBy", new RelationInverse() + .setInverse("provides") + .setRelation("isProvidedBy") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); relationInverseMap - .put( - "datasourceOrganization_provision_provides", new RelationInverse() - .setInverse("isProvidedBy") - .setRelation("provides") - .setRelType("datasourceOrganization") - .setSubReltype("provision")); + .put( + "datasourceOrganization_provision_provides", new RelationInverse() + .setInverse("isProvidedBy") + .setRelation("provides") + .setRelType("datasourceOrganization") + .setSubReltype("provision")); relationInverseMap - .put( - "resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("isAmongTopNSimilarDocuments") - .setRelation("hasAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); + .put( + "resultResult_similarity_hasAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("isAmongTopNSimilarDocuments") + .setRelation("hasAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); relationInverseMap - .put( - "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("hasAmongTopNSimilarDocuments") - .setRelation("isAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); relationInverseMap - .put( - "resultResult_relationship_isRelatedTo", new RelationInverse() - .setInverse("isRelatedTo") - .setRelation("isRelatedTo") - .setRelType("resultResult") - .setSubReltype("relationship")); + .put( + "resultResult_relationship_isRelatedTo", new RelationInverse() + .setInverse("isRelatedTo") + .setRelation("isRelatedTo") + .setRelType("resultResult") + .setSubReltype("relationship")); relationInverseMap - .put( - "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() - .setInverse("hasAmongTopNSimilarDocuments") - .setRelation("isAmongTopNSimilarDocuments") - .setRelType("resultResult") - .setSubReltype("similarity")); + .put( + "resultResult_similarity_isAmongTopNSimilarDocuments", new RelationInverse() + .setInverse("hasAmongTopNSimilarDocuments") + .setRelation("isAmongTopNSimilarDocuments") + .setRelType("resultResult") + .setSubReltype("similarity")); relationInverseMap - .put( - "resultResult_supplement_isSupplementTo", new RelationInverse() - .setInverse("isSupplementedBy") - .setRelation("isSupplementTo") - .setRelType("resultResult") - .setSubReltype("supplement")); + .put( + "resultResult_supplement_isSupplementTo", new RelationInverse() + .setInverse("isSupplementedBy") + .setRelation("isSupplementTo") + .setRelType("resultResult") + .setSubReltype("supplement")); relationInverseMap - .put( - "resultResult_supplement_isSupplementedBy", new RelationInverse() - .setInverse("isSupplementTo") - .setRelation("isSupplementedBy") - .setRelType("resultResult") - .setSubReltype("supplement")); + .put( + "resultResult_supplement_isSupplementedBy", new RelationInverse() + .setInverse("isSupplementTo") + .setRelation("isSupplementedBy") + .setRelType("resultResult") + .setSubReltype("supplement")); relationInverseMap - .put( - "resultResult_part_isPartOf", new RelationInverse() - .setInverse("hasPart") - .setRelation("isPartOf") - .setRelType("resultResult") - .setSubReltype("part")); + .put( + "resultResult_part_isPartOf", new RelationInverse() + .setInverse("hasPart") + .setRelation("isPartOf") + .setRelType("resultResult") + .setSubReltype("part")); relationInverseMap - .put( - "resultResult_part_hasPart", new RelationInverse() - .setInverse("isPartOf") - .setRelation("hasPart") - .setRelType("resultResult") - .setSubReltype("part")); + .put( + "resultResult_part_hasPart", new RelationInverse() + .setInverse("isPartOf") + .setRelation("hasPart") + .setRelType("resultResult") + .setSubReltype("part")); relationInverseMap - .put( - "resultResult_dedup_merges", new RelationInverse() - .setInverse("isMergedIn") - .setRelation("merges") - .setRelType("resultResult") - .setSubReltype("dedup")); + .put( + "resultResult_dedup_merges", new RelationInverse() + .setInverse("isMergedIn") + .setRelation("merges") + .setRelType("resultResult") + .setSubReltype("dedup")); relationInverseMap - .put( - "resultResult_dedup_isMergedIn", new RelationInverse() - .setInverse("merges") - .setRelation("isMergedIn") - .setRelType("resultResult") - .setSubReltype("dedup")); + .put( + "resultResult_dedup_isMergedIn", new RelationInverse() + .setInverse("merges") + .setRelation("isMergedIn") + .setRelType("resultResult") + .setSubReltype("dedup")); relationInverseMap - .put( - "resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() - .setInverse("isSimilarTo") - .setRelation("isSimilarTo") - .setRelType("resultResult") - .setSubReltype("dedupSimilarity")); + .put( + "resultResult_dedupSimilarity_isSimilarTo", new RelationInverse() + .setInverse("isSimilarTo") + .setRelation("isSimilarTo") + .setRelType("resultResult") + .setSubReltype("dedupSimilarity")); } @@ -293,7 +294,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - X subClazzObject, Y superClazzObject) { + X subClazzObject, Y superClazzObject) { return isSubClass(subClazzObject.getClass(), superClazzObject.getClass()); } @@ -307,7 +308,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - X subClazzObject, Class superClazz) { + X subClazzObject, Class superClazz) { return isSubClass(subClazzObject.getClass(), superClazz); } @@ -321,7 +322,7 @@ public class ModelSupport { * @return True if X is a subclass of Y */ public static Boolean isSubClass( - Class subClazz, Class superClazz) { + Class subClazz, Class superClazz) { return superClazz.isAssignableFrom(subClazz); } @@ -333,32 +334,32 @@ public class ModelSupport { */ public static Class[] getOafModelClasses() { return new Class[] { - Author.class, - Context.class, - Country.class, - DataInfo.class, - Dataset.class, - Datasource.class, - ExternalReference.class, - ExtraInfo.class, - Field.class, - GeoLocation.class, - Instance.class, - Journal.class, - KeyValue.class, - Oaf.class, - OafEntity.class, - OAIProvenance.class, - Organization.class, - OriginDescription.class, - OtherResearchProduct.class, - Project.class, - Publication.class, - Qualifier.class, - Relation.class, - Result.class, - Software.class, - StructuredProperty.class + Author.class, + Context.class, + Country.class, + DataInfo.class, + Dataset.class, + Datasource.class, + ExternalReference.class, + ExtraInfo.class, + Field.class, + GeoLocation.class, + Instance.class, + Journal.class, + KeyValue.class, + Oaf.class, + OafEntity.class, + OAIProvenance.class, + Organization.class, + OriginDescription.class, + OtherResearchProduct.class, + Project.class, + Publication.class, + Qualifier.class, + Relation.class, + Result.class, + Software.class, + StructuredProperty.class }; } @@ -372,10 +373,10 @@ public class ModelSupport { public static String getScheme(final String sourceType, final String targetType) { return String - .format( - schemeTemplate, - entityMapping.get(EntityType.valueOf(sourceType)).name(), - entityMapping.get(EntityType.valueOf(targetType)).name()); + .format( + schemeTemplate, + entityMapping.get(EntityType.valueOf(sourceType)).name(), + entityMapping.get(EntityType.valueOf(targetType)).name()); } public static Function idFn() { @@ -390,38 +391,38 @@ public class ModelSupport { private static String idFnForRelation(T t) { Relation r = (Relation) t; return Optional - .ofNullable(r.getSource()) - .map( - source -> Optional - .ofNullable(r.getTarget()) - .map( - target -> Optional - .ofNullable(r.getRelType()) - .map( - relType -> Optional - .ofNullable(r.getSubRelType()) - .map( - subRelType -> Optional - .ofNullable(r.getRelClass()) - .map( - relClass -> String - .join( - source, - target, - relType, - subRelType, - relClass)) - .orElse( - String - .join( - source, - target, - relType, - subRelType))) - .orElse(String.join(source, target, relType))) - .orElse(String.join(source, target))) - .orElse(source)) - .orElse(null); + .ofNullable(r.getSource()) + .map( + source -> Optional + .ofNullable(r.getTarget()) + .map( + target -> Optional + .ofNullable(r.getRelType()) + .map( + relType -> Optional + .ofNullable(r.getSubRelType()) + .map( + subRelType -> Optional + .ofNullable(r.getRelClass()) + .map( + relClass -> String + .join( + source, + target, + relType, + subRelType, + relClass)) + .orElse( + String + .join( + source, + target, + relType, + subRelType))) + .orElse(String.join(source, target, relType))) + .orElse(String.join(source, target))) + .orElse(source)) + .orElse(null); } private static String idFnForOafEntity(T t) { diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java index d5c2b518a..0ef59e8c2 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/PrepareMergedRelationJob.java @@ -73,19 +73,6 @@ public class PrepareMergedRelationJob { .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath); -// relation.createOrReplaceTempView("relation"); -// -// spark -// .sql( -// "Select * from relation " + -// "where relclass = 'merges' " + -// "and datainfo.deletedbyinference = false") -// .as(Encoders.bean(Relation.class)) -// .toJSON() -// .write() -// .mode(SaveMode.Overwrite) -// .option("compression", "gzip") -// .text(outputPath); } public static org.apache.spark.sql.Dataset readRelations( diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java index 704cab375..2caa66db4 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/ReadBlacklistFromDB.java @@ -65,8 +65,7 @@ public class ReadBlacklistFromDB implements Closeable { } } - public void execute(final String sql, final Function> producer) - throws Exception { + public void execute(final String sql, final Function> producer) throws Exception { final Consumer consumer = rs -> producer.apply(rs).forEach(r -> writeRelation(r)); diff --git a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java index c5104058c..86587bfc9 100644 --- a/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java +++ b/dhp-workflows/dhp-blacklist/src/main/java/eu/dnetlib/dhp/blacklist/SparkRemoveBlacklistedRelationJob.java @@ -84,7 +84,7 @@ public class SparkRemoveBlacklistedRelationJob { .joinWith( mergesRelation, blackListed.col("source").equalTo(mergesRelation.col("target")), "left_outer") - .map(c -> { + .map((MapFunction, Relation>) c -> { Optional .ofNullable(c._2()) .ifPresent(mr -> c._1().setSource(mr.getSource())); @@ -95,7 +95,7 @@ public class SparkRemoveBlacklistedRelationJob { .joinWith( mergesRelation, dedupSource.col("target").equalTo(mergesRelation.col("target")), "left_outer") - .map(c -> { + .map((MapFunction, Relation>) c -> { Optional .ofNullable(c._2()) .ifPresent(mr -> c._1().setTarget(mr.getSource())); @@ -107,7 +107,6 @@ public class SparkRemoveBlacklistedRelationJob { .mode(SaveMode.Overwrite) .json(blacklistPath + "/deduped"); - inputRelation .joinWith( dedupBL, (inputRelation @@ -118,26 +117,23 @@ public class SparkRemoveBlacklistedRelationJob { .col("target") .equalTo(dedupBL.col("target")))), "left_outer") - .map(c -> { - Relation ir = c._1(); - Optional obl = Optional.ofNullable(c._2()); - if (obl.isPresent()) { - if (ir.equals(obl.get())) { - return null; + .map((MapFunction, Relation>) c -> { + Relation ir = c._1(); + Optional obl = Optional.ofNullable(c._2()); + if (obl.isPresent()) { + if (ir.equals(obl.get())) { + return null; + } } - } - return ir; - - }, Encoders.bean(Relation.class)) + return ir; + }, Encoders.bean(Relation.class)) .filter(Objects::nonNull) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath); - } - public static org.apache.spark.sql.Dataset readRelations( SparkSession spark, String inputPath) { return spark diff --git a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java index 2d6b1061b..bbfd15674 100644 --- a/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java +++ b/dhp-workflows/dhp-blacklist/src/test/java/eu/dnetlib/dhp/blacklist/BlackListTest.java @@ -19,6 +19,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; + import eu.dnetlib.dhp.schema.oaf.Relation; public class BlackListTest { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index e3d74ef3e..75d85e2ba 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -1,11 +1,10 @@ package eu.dnetlib.dhp.bulktag; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.bulktag.community.*; -import eu.dnetlib.dhp.schema.oaf.Result; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -16,9 +15,12 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Optional; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.bulktag.community.*; +import eu.dnetlib.dhp.schema.oaf.Result; public class SparkBulkTagJob { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java index d492b848e..0f45d3beb 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java @@ -1,14 +1,15 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import com.google.gson.Gson; + /** Created by miriam on 01/08/2018. */ public class Community implements Serializable { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java index 4e5b9fc9f..29ddde15f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java @@ -1,15 +1,6 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; -import eu.dnetlib.dhp.bulktag.criteria.Selection; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; @@ -17,6 +8,17 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; +import eu.dnetlib.dhp.bulktag.criteria.Selection; + /** Created by miriam on 02/08/2018. */ public class CommunityConfiguration implements Serializable { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java index dc83497c3..607315f3f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java @@ -1,14 +1,11 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; -import eu.dnetlib.dhp.bulktag.criteria.Selection; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -17,10 +14,15 @@ import org.dom4j.DocumentException; import org.dom4j.Node; import org.dom4j.io.SAXReader; -import java.io.StringReader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +import eu.dnetlib.dhp.bulktag.criteria.InterfaceAdapter; +import eu.dnetlib.dhp.bulktag.criteria.Selection; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactory { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java index 8e28a7a5f..e0856ae8f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java @@ -1,12 +1,12 @@ package eu.dnetlib.dhp.bulktag.community; -import eu.dnetlib.dhp.bulktag.criteria.Selection; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; +import eu.dnetlib.dhp.bulktag.criteria.Selection; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; + public class Constraint implements Serializable { private String verb; private String field; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java index eace3bc35..b56dfaaa3 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraints.java @@ -1,12 +1,6 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Type; @@ -14,6 +8,14 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; + +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; + /** Created by miriam on 02/08/2018. */ public class Constraints implements Serializable { private static final Log log = LogFactory.getLog(Constraints.class); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java index 1130a0770..50e1836fa 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Pair.java @@ -1,10 +1,10 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; - import java.io.Serializable; +import com.google.gson.Gson; + /** Created by miriam on 03/08/2018. */ public class Pair implements Serializable { private A fst; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java index c4362610e..b9c37f4dc 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Provider.java @@ -1,13 +1,15 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import java.io.Serializable; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.dom4j.Node; -import java.io.Serializable; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; /** Created by miriam on 01/08/2018. */ public class Provider implements Serializable { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java index 43eb40940..7ec2f916f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java @@ -1,13 +1,15 @@ package eu.dnetlib.dhp.bulktag.community; +import java.util.List; + +import org.dom4j.DocumentException; + import com.google.common.base.Joiner; + import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import org.dom4j.DocumentException; - -import java.util.List; public class QueryInformationSystem { private static final String XQUERY = "for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') " diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index fd4f5497a..f5a985d15 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -1,19 +1,21 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; -import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.JsonPath; -import eu.dnetlib.dhp.schema.oaf.*; -import org.apache.commons.lang3.StringUtils; +import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.*; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; import java.io.Serializable; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.*; -import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import org.apache.commons.lang3.StringUtils; + +import com.google.gson.Gson; +import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.JsonPath; + +import eu.dnetlib.dhp.schema.oaf.*; /** Created by miriam on 02/08/2018. */ public class ResultTagger implements Serializable { @@ -50,7 +52,7 @@ public class ResultTagger implements Serializable { } public R enrichContextCriteria( - final R result, final CommunityConfiguration conf, final Map criteria) { + final R result, final CommunityConfiguration conf, final Map criteria) { // } // public Result enrichContextCriteria(final Result result, final CommunityConfiguration diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java index 28674d9ef..71ff61d1b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java @@ -1,16 +1,17 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; -import com.google.gson.reflect.TypeToken; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; - import java.io.Serializable; import java.lang.reflect.Type; import java.util.Collection; import java.util.List; import java.util.Map; +import com.google.gson.Gson; +import com.google.gson.reflect.TypeToken; + +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; + public class SelectionConstraints implements Serializable { private List criteria; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java index eb0577ffc..bc6b75fba 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ZenodoCommunity.java @@ -1,10 +1,11 @@ package eu.dnetlib.dhp.bulktag.community; -import com.google.gson.Gson; +import java.io.Serializable; + import org.dom4j.Node; -import java.io.Serializable; +import com.google.gson.Gson; /** Created by miriam on 01/08/2018. */ public class ZenodoCommunity implements Serializable { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java index 2bc1ab3cf..e9b948b2b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/InterfaceAdapter.java @@ -1,10 +1,10 @@ package eu.dnetlib.dhp.bulktag.criteria; -import com.google.gson.*; - import java.lang.reflect.Type; +import com.google.gson.*; + public class InterfaceAdapter implements JsonSerializer, JsonDeserializer { private static final String CLASSNAME = "CLASSNAME"; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java index 19d6c4615..fe46c6936 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java @@ -1,16 +1,16 @@ package eu.dnetlib.dhp.bulktag.criteria; -import io.github.classgraph.ClassGraph; -import io.github.classgraph.ClassInfo; -import io.github.classgraph.ClassInfoList; -import io.github.classgraph.ScanResult; - import java.io.Serializable; import java.lang.reflect.InvocationTargetException; import java.util.Map; import java.util.stream.Collectors; +import io.github.classgraph.ClassGraph; +import io.github.classgraph.ClassInfo; +import io.github.classgraph.ClassInfoList; +import io.github.classgraph.ScanResult; + public class VerbResolver implements Serializable { private Map> map = null; // = new HashMap<>(); private final ClassGraph classgraph = new ClassGraph(); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index f20678f7b..72e0a63fa 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -1,11 +1,12 @@ package eu.dnetlib.dhp.bulktag; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Software; +import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY_INDICATOR; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -18,15 +19,15 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; +import com.fasterxml.jackson.databind.ObjectMapper; -import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY_INDICATOR; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Software; public class BulkTagJobTest { @@ -34,12 +35,11 @@ public class BulkTagJobTest { public static final String MOCK_IS_LOOK_UP_URL = "BASEURL:8280/is/services/isLookUp"; - public static final String pathMap = - "{ \"author\" : \"$['author'][*]['fullname']\"," - + " \"title\" : \"$['title'][*]['value']\"," - + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," - + " \"contributor\" : \"$['contributor'][*]['value']\"," - + " \"description\" : \"$['description'][*]['value']\"}"; + public static final String pathMap = "{ \"author\" : \"$['author'][*]['fullname']\"," + + " \"title\" : \"$['title'][*]['value']\"," + + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," + + " \"contributor\" : \"$['contributor'][*]['value']\"," + + " \"description\" : \"$['description'][*]['value']\"}"; private static SparkSession spark; @@ -97,7 +97,8 @@ public class BulkTagJobTest { new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), "-taggingConf", taggingConf, "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", @@ -129,8 +130,8 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectNoPreviousContextTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext") - .getPath(); + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext") + .getPath(); final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( @@ -139,7 +140,7 @@ public class BulkTagJobTest { "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, "-pathMap", pathMap @@ -225,9 +226,9 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { final String sourcePath = getClass() - .getResource( - "/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance") + .getPath(); final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( @@ -236,7 +237,7 @@ public class BulkTagJobTest { "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, "-pathMap", pathMap @@ -307,8 +308,8 @@ public class BulkTagJobTest { @Test public void bulktagByDatasourceTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource") - .getPath(); + .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource") + .getPath(); SparkBulkTagJob .main( new String[] { @@ -379,9 +380,9 @@ public class BulkTagJobTest { @Test public void bulktagByZenodoCommunityTest() throws Exception { final String sourcePath = getClass() - .getResource( - "/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity") + .getPath(); SparkBulkTagJob .main( new String[] { @@ -501,8 +502,8 @@ public class BulkTagJobTest { @Test public void bulktagBySubjectDatasourceTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource") - .getPath(); + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource") + .getPath(); SparkBulkTagJob .main( new String[] { @@ -726,9 +727,9 @@ public class BulkTagJobTest { public void bulktagDatasourcewithConstraintsTest() throws Exception { final String sourcePath = getClass() - .getResource( - "/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints") + .getPath(); SparkBulkTagJob .main( new String[] { diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index 056c3345c..aaf670fd7 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -1,21 +1,23 @@ package eu.dnetlib.dhp.bulktag; -import com.google.gson.Gson; -import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; -import eu.dnetlib.dhp.bulktag.community.CommunityConfigurationFactory; -import eu.dnetlib.dhp.bulktag.community.Constraint; -import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.dom4j.DocumentException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.util.*; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; +import eu.dnetlib.dhp.bulktag.community.CommunityConfigurationFactory; +import eu.dnetlib.dhp.bulktag.community.Constraint; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; /** Created by miriam on 03/08/2018. */ public class CommunityConfigurationFactoryTest { diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java index b62238089..88ad43b6b 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/countrypropagation/CountryPropagationJobTest.java @@ -103,7 +103,8 @@ public class CountryPropagationJobTest { Assertions.assertEquals(0, verificationDs.filter("size(country) > 2").count()); Dataset countryExploded = verificationDs - .flatMap((FlatMapFunction) row -> row.getCountry().iterator(), Encoders.bean(Country.class)) + .flatMap( + (FlatMapFunction) row -> row.getCountry().iterator(), Encoders.bean(Country.class)) .map((MapFunction) c -> c.getClassid(), Encoders.STRING()); Assertions.assertEquals(9, countryExploded.count()); @@ -123,10 +124,10 @@ public class CountryPropagationJobTest { country_list .stream() .forEach( - c -> prova - .add( - new Tuple2<>( - row.getId(), c.getClassid()))); + c -> prova + .add( + new Tuple2<>( + row.getId(), c.getClassid()))); return prova.iterator(); }, Encoders.tuple(Encoders.STRING(), Encoders.STRING())); @@ -178,20 +179,20 @@ public class CountryPropagationJobTest { Dataset> countryExplodedWithCountryclassname = verificationDs .flatMap( - (FlatMapFunction>) row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list - .stream() - .forEach( - c -> prova - .add( - new Tuple2<>( - row.getId(), - c.getClassname()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + (FlatMapFunction>) row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c.getClassname()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); countryExplodedWithCountryclassname.show(false); Assertions @@ -239,22 +240,22 @@ public class CountryPropagationJobTest { Dataset> countryExplodedWithCountryProvenance = verificationDs .flatMap( - (FlatMapFunction>) row -> { - List> prova = new ArrayList(); - List country_list = row.getCountry(); - country_list - .stream() - .forEach( - c -> prova - .add( - new Tuple2<>( - row.getId(), - c - .getDataInfo() - .getInferenceprovenance()))); - return prova.iterator(); - }, - Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + (FlatMapFunction>) row -> { + List> prova = new ArrayList(); + List country_list = row.getCountry(); + country_list + .stream() + .forEach( + c -> prova + .add( + new Tuple2<>( + row.getId(), + c + .getDataInfo() + .getInferenceprovenance()))); + return prova.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); Assertions .assertEquals( diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java index ff63753b8..abed028e1 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/projecttoresult/ProjectPropagationJobTest.java @@ -72,14 +72,15 @@ public class ProjectPropagationJobTest { public void NoUpdateTest() throws Exception { final String potentialUpdateDate = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/noupdates/potentialUpdates") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToProjectThroughSemRelJob.main( + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), @@ -107,14 +108,15 @@ public class ProjectPropagationJobTest { @Test public void UpdateTenTest() throws Exception { final String potentialUpdatePath = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/tenupdates/potentialUpdates") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToProjectThroughSemRelJob.main( + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), @@ -143,8 +145,8 @@ public class ProjectPropagationJobTest { .assertEquals( 5, verificationDs - .filter((FilterFunction) r -> - r.getSource().startsWith("50") + .filter( + (FilterFunction) r -> r.getSource().startsWith("50") && r.getTarget().startsWith("40") && r.getRelClass().equals("isProducedBy")) .count()); @@ -152,8 +154,8 @@ public class ProjectPropagationJobTest { .assertEquals( 5, verificationDs - .filter((FilterFunction) r -> - r.getSource().startsWith("40") + .filter( + (FilterFunction) r -> r.getSource().startsWith("40") && r.getTarget().startsWith("50") && r.getRelClass().equals("produces")) .count()); @@ -178,14 +180,15 @@ public class ProjectPropagationJobTest { @Test public void UpdateMixTest() throws Exception { final String potentialUpdatepath = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/updatesmixed/potentialUpdates") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource( - "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToProjectThroughSemRelJob.main( + .getResource( + "/eu/dnetlib/dhp/projecttoresult/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToProjectThroughSemRelJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), @@ -217,8 +220,8 @@ public class ProjectPropagationJobTest { .assertEquals( 4, verificationDs - .filter((FilterFunction) r -> - r.getSource().startsWith("50") + .filter( + (FilterFunction) r -> r.getSource().startsWith("50") && r.getTarget().startsWith("40") && r.getRelClass().equals("isProducedBy")) .count()); @@ -226,8 +229,8 @@ public class ProjectPropagationJobTest { .assertEquals( 4, verificationDs - .filter((FilterFunction) r -> - r.getSource().startsWith("40") + .filter( + (FilterFunction) r -> r.getSource().startsWith("40") && r.getTarget().startsWith("50") && r.getRelClass().equals("produces")) .count()); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index 20b20d4ed..d739516fc 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -67,9 +67,10 @@ public class ResultToCommunityJobTest { @Test public void testSparkResultToCommunityFromOrganizationJob() throws Exception { final String preparedInfoPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") - .getPath(); - SparkResultToCommunityFromOrganizationJob.main( + .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo") + .getPath(); + SparkResultToCommunityFromOrganizationJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java index 30be118d1..435b76605 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/ResultToOrganizationJobTest.java @@ -69,15 +69,17 @@ public class ResultToOrganizationJobTest { @Test public void NoUpdateTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(); + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(); final String datasourceOrganizationPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/datasourceOrganization") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToOrganizationFromIstRepoJob.main( + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/noupdate/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), @@ -109,15 +111,17 @@ public class ResultToOrganizationJobTest { @Test public void UpdateNoMixTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") - .getPath(); + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/noupdate_updatenomix") + .getPath(); final String datasourceOrganizationPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/datasourceOrganization") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToOrganizationFromIstRepoJob.main( + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatenomix/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), @@ -172,15 +176,17 @@ public class ResultToOrganizationJobTest { @Test public void UpdateMixTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") - .getPath(); + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/sample/updatemix") + .getPath(); final String datasourceOrganizationPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") - .getPath(); + .getResource( + "/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/datasourceOrganization") + .getPath(); final String alreadyLinkedPath = getClass() - .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") - .getPath(); - SparkResultToOrganizationFromIstRepoJob.main( + .getResource("/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/updatemix/preparedInfo/alreadyLinked") + .getPath(); + SparkResultToOrganizationFromIstRepoJob + .main( new String[] { "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), From ad63effb4e078fb99b2471d98c964f2bbba3c1f8 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 11 May 2020 17:48:22 +0200 Subject: [PATCH 249/259] removed deletion of working dir --- .../resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index f71c085b2..7cbf166a1 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -30,7 +30,6 @@ - From f9a62ba63b21e1687623a0f58abeb15f74db95a1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 11 May 2020 18:16:39 +0200 Subject: [PATCH 250/259] added wf nodes to copy entities to the output path --- .../dhp/blacklist/oozie_app/workflow.xml | 91 ++++++++++++++++++- 1 file changed, 90 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index f71c085b2..397a48f09 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -34,10 +34,99 @@ - + + + + + + + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/publication + ${nameNode}/${outputPath}/publication + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/dataset + ${nameNode}/${outputPath}/dataset + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/otherresearchproduct + ${nameNode}/${outputPath}/otherresearchproduct + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/software + ${nameNode}/${outputPath}/software + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${jobTracker} + ${nameNode} + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + ${jobTracker} From 527e8169a80addd181e8cabbe1866e2891a3fbf9 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 11 May 2020 18:17:05 +0200 Subject: [PATCH 251/259] adjusted paths pointing to test configurations, cleanup --- .../CommunityConfigurationFactoryTest.java | 88 +------------------ 1 file changed, 2 insertions(+), 86 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index aaf670fd7..c4beac4e6 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -11,12 +11,9 @@ import org.dom4j.DocumentException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import com.google.gson.Gson; - import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; import eu.dnetlib.dhp.bulktag.community.CommunityConfigurationFactory; import eu.dnetlib.dhp.bulktag.community.Constraint; -import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; /** Created by miriam on 03/08/2018. */ @@ -30,7 +27,7 @@ public class CommunityConfigurationFactoryTest { .toString( getClass() .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml")); + "/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.xml")); final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); Assertions.assertEquals(5, cc.size()); cc @@ -57,7 +54,7 @@ public class CommunityConfigurationFactoryTest { .toString( getClass() .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml")); + "/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml")); final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); Map> param = new HashMap<>(); param.put("author", new ArrayList<>(Collections.singletonList("Pippo Pippi"))); @@ -82,85 +79,4 @@ public class CommunityConfigurationFactoryTest { Assertions.assertEquals("dariah", comm.get(0)); } - @Test - public void test4() throws DocumentException, IOException { - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json"))); - cc.toString(); - } - - @Test - public void test5() throws IOException, DocumentException { - - // final CommunityConfiguration cc = - // CommunityConfigurationFactory.newInstance(IOUtils.toString(getClass().getResourceAsStream("test.xml"))); - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.json"))); - - System.out.println(cc.toJson()); - } - - @Test - public void test6() { - String json = "{\"criteria\":[{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}]}"; - - String step1 = "{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}"; - - Constraint c = new Gson().fromJson(step1, Constraint.class); - // - // String step2 = - // "{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}"; - // - // ConstraintEncapsulator ce = new - // Gson().fromJson(step2,ConstraintEncapsulator.class); - // - // - // String step3 = - // "{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}"; - // - // Constraints cons = new Gson().fromJson(step3,Constraints.class); - // - // String step4 = - // "{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}"; - // - // ConstraintsList cl = new Gson().fromJson(step4,ConstraintsList.class); - // - // String step5 = - // "{\"cl\":{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}}"; - SelectionConstraints sl = new Gson().fromJson(json, SelectionConstraints.class); - } - - @Test - public void test7() throws IOException { - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json"))); - - System.out.println(cc.toJson()); - } - - @Test - public void temporaneo() throws Exception { - String xml = IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); - final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); - System.out.println(cc.toJson()); - } } From 8ffc050b8ac8ee79201e4f6b2fea78efbd19ba66 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 10:01:09 +0200 Subject: [PATCH 252/259] fixed problem in communityconfigurationfactory test --- .../CommunityConfigurationFactoryTest.java | 87 +------------------ .../community_configuration_selcrit.xml | 24 ++--- 2 files changed, 15 insertions(+), 96 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index aaf670fd7..3d08b5915 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -30,7 +30,7 @@ public class CommunityConfigurationFactoryTest { .toString( getClass() .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.xml")); + "/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration.xml")); final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); Assertions.assertEquals(5, cc.size()); cc @@ -57,7 +57,7 @@ public class CommunityConfigurationFactoryTest { .toString( getClass() .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.xml")); + "/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml")); final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); Map> param = new HashMap<>(); param.put("author", new ArrayList<>(Collections.singletonList("Pippo Pippi"))); @@ -81,86 +81,5 @@ public class CommunityConfigurationFactoryTest { Assertions.assertEquals(1, comm.size()); Assertions.assertEquals("dariah", comm.get(0)); } - - @Test - public void test4() throws DocumentException, IOException { - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration_selcrit.json"))); - cc.toString(); - } - - @Test - public void test5() throws IOException, DocumentException { - - // final CommunityConfiguration cc = - // CommunityConfigurationFactory.newInstance(IOUtils.toString(getClass().getResourceAsStream("test.xml"))); - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/community_configuration.json"))); - - System.out.println(cc.toJson()); - } - - @Test - public void test6() { - String json = "{\"criteria\":[{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}]}"; - - String step1 = "{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}"; - - Constraint c = new Gson().fromJson(step1, Constraint.class); - // - // String step2 = - // "{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}"; - // - // ConstraintEncapsulator ce = new - // Gson().fromJson(step2,ConstraintEncapsulator.class); - // - // - // String step3 = - // "{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}"; - // - // Constraints cons = new Gson().fromJson(step3,Constraints.class); - // - // String step4 = - // "{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}"; - // - // ConstraintsList cl = new Gson().fromJson(step4,ConstraintsList.class); - // - // String step5 = - // "{\"cl\":{\"criteria\":[{\"ce\":{\"constraint\":[{\"verb\":\"contains\",\"field\":\"contributor\",\"value\":\"DARIAH\"}]}}]}}"; - SelectionConstraints sl = new Gson().fromJson(json, SelectionConstraints.class); - } - - @Test - public void test7() throws IOException { - final CommunityConfiguration cc = CommunityConfigurationFactory - .fromJson( - IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.json"))); - - System.out.println(cc.toJson()); - } - - @Test - public void temporaneo() throws Exception { - String xml = IOUtils - .toString( - getClass() - .getResourceAsStream( - "/eu/dnetlib/dhp/communityconfiguration/tagging_conf.xml")); - final CommunityConfiguration cc = CommunityConfigurationFactory.newInstance(xml); - System.out.println(cc.toJson()); - } + } diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml index 89ace0e5f..cd5ea38d0 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/community_configuration_selcrit.xml @@ -35,7 +35,7 @@ SDG9 - Industry innovation and infrastructure SDG16 - Peace justice and strong institutions - + 123 @@ -50,7 +50,7 @@ - + @@ -74,7 +74,7 @@ brain magnetic resonance imaging brain abnormalities - + re3data_____::5b9bf9171d92df854cf3c520692e9122 @@ -95,7 +95,7 @@ doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a - + @@ -106,12 +106,12 @@ aqua sea - + re3data_____::9633d1e8c4309c833c2c442abeb0cfeb - + @@ -134,7 +134,7 @@ food distribution forestry - + opendoar____::1a551829d50f1400b0dab21fdd969c04 @@ -159,30 +159,30 @@ opendoar____::87ae6fb631f7c8a627e8e28785d9992d - + oac_clarin - + re3data_____::a507cdacc5bbcc08761c92185dee5cab - + oaa_dariah - + openaire____::1cfdb2e14977f31a98e0118283401f32 {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - + dimpo From 7387f3449a16111ec7e492a7c3fcf43c35924898 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 11:27:38 +0200 Subject: [PATCH 253/259] changed the route to find the verb resolver classes --- .../java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java index fe46c6936..3d0db2063 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/criteria/VerbResolver.java @@ -22,12 +22,12 @@ public class VerbResolver implements Serializable { .verbose() // If you want to enable logging to stderr .enableAllInfo() // Scan classes, methods, fields, annotations .whitelistPackages( - "eu.dnetlib.dhp.selectioncriteria") // Scan com.xyz and subpackages + "eu.dnetlib.dhp.bulktag.criteria") // Scan com.xyz and subpackages .scan()) { // Perform the scan and return a ScanResult ClassInfoList routeClassInfoList = scanResult .getClassesWithAnnotation( - "eu.dnetlib.dhp.selectioncriteria.VerbClass"); + "eu.dnetlib.dhp.bulktag.criteria.VerbClass"); this.map = routeClassInfoList .stream() From f8aef6161a3001c14207e57fae15330ad4e25afc Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 11:28:07 +0200 Subject: [PATCH 254/259] minor modification --- .../dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index 71252dd82..ca737b79f 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -81,5 +81,5 @@ public class CommunityConfigurationFactoryTest { Assertions.assertEquals(1, comm.size()); Assertions.assertEquals("dariah", comm.get(0)); } - + } From 14979f299ebe2641f0b8906678eb617af66c5767 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 11:28:38 +0200 Subject: [PATCH 255/259] changed the configuration factory --- .../communityconfiguration/tagging_conf.xml | 127 ++++++++++-------- 1 file changed, 74 insertions(+), 53 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml index ec52d4799..a44372e4d 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf.xml @@ -1,7 +1,7 @@ - + zenodo @@ -9,29 +9,33 @@ - Result: 2 + + - + - Result: 3 + + - + - Result: 4 + + - + re3data_____::a507cdacc5bbcc08761c92185dee5cab - + - Result: 5 + + - + rda @@ -39,7 +43,8 @@ - Result: 6 + + SDG13 - Climate action SDG8 - Decent work and economic growth @@ -59,10 +64,11 @@ SDG9 - Industry innovation and infrastructure SDG16 - Peace justice and strong institutions - + - Result: 7 + + modern art monuments @@ -132,7 +138,7 @@ architectural vessels - + re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 @@ -169,7 +175,7 @@ opendoar____::97275a23ca44226c9964043c8462be96 - + storm @@ -253,7 +259,8 @@ - Result: 8 + + Stock Assessment pelagic @@ -275,7 +282,7 @@ Fishing fleet Aquaculture - + doajarticles::8cec81178926caaca531afbd8eb5d64c @@ -328,7 +335,7 @@ doajarticles::dd70e44479f0ade25aa106aef3e87a0a - + discardless @@ -372,7 +379,8 @@ - Result: 9 + + brain mapping brain imaging @@ -393,7 +401,7 @@ brain magnetic resonance imaging brain abnormalities - + re3data_____::5b9bf9171d92df854cf3c520692e9122 @@ -418,7 +426,7 @@ rest________::fb1a3d4523c95e63496e3bc7ba36244b - + neuroinformatics @@ -486,7 +494,8 @@ - Result: 10 + + marine ocean @@ -494,7 +503,7 @@ aqua sea - + adriplan @@ -686,9 +695,10 @@ - Result: 11 + + - + instruct @@ -700,12 +710,14 @@ - Result: 12 + + - + - Result: 13 + + animal production and health fisheries and aquaculture @@ -726,7 +738,7 @@ food distribution forestry - + opendoar____::1a551829d50f1400b0dab21fdd969c04 @@ -751,7 +763,7 @@ opendoar____::87ae6fb631f7c8a627e8e28785d9992d - + edenis @@ -827,9 +839,10 @@ - Result: 14 + + - + opendoar____::7e7757b1e12abcb736ab9a754ffb617a {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} @@ -838,7 +851,7 @@ opendoar____::96da2f590cd7246bbde0051047b0d6f7 {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - + dimpo @@ -846,17 +859,20 @@ - Result: 15 + + - + - Result: 16 + + - + - Result: 17 + + Green Transport City mobility systems @@ -1009,7 +1025,7 @@ Innovative freight delivery systems urban freight delivery infrastructures - + doajarticles::1c5bdf8fca58937894ad1441cca99b76 @@ -1094,7 +1110,7 @@ doajarticles::fba6191177ede7c51ea1cdf58eae7f8b - + jsdtl @@ -1154,22 +1170,26 @@ - Result: 18 + + - + - Result: 19 + + - + - Result: 20 + + - + - Result: 21 + + Sustainability-oriented science policy STI policies @@ -1179,7 +1199,7 @@ science policy Policy and Law - + doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d @@ -1264,7 +1284,7 @@ doajarticles::7ffc35ac5133da01d421ccf8af5b70bc - + risis @@ -1272,7 +1292,8 @@ - Result: 22 + + COVID-19 Severe acute respiratory syndrome coronavirus 2 @@ -1287,7 +1308,7 @@ mesh:COVID-19 COVID2019 - + opendoar____::358aee4cc897452c00244351e4d91f69 {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]}, @@ -1366,7 +1387,7 @@ re3data_____::978378def740bbf2bfb420de868c460b {"criteria":[{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_ignorecase","field":"title","value":"2019-nCov"}]}]} - + chicago-covid-19 @@ -1387,4 +1408,4 @@ - + \ No newline at end of file From 1547ca7e15a7353476b794ec51e3d53791f2d66c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 12:17:27 +0200 Subject: [PATCH 256/259] added blacklist step to the end of the provision wf --- .../eu/dnetlib/dhp/wf/profiles/provision.xml | 130 +++++++----------- 1 file changed, 52 insertions(+), 78 deletions(-) diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml index f99ea7aed..7c918a0d7 100644 --- a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/provision.xml @@ -4,7 +4,7 @@ - + Data Provision [OCEAN] @@ -131,6 +131,16 @@ + + Set the target path to store the blacklisted graph + + blacklistedGraphPath + /tmp/beta_provision/graph/12_graph_blacklisted + + + + + Set the lookup address @@ -155,64 +165,8 @@ Set the map of associations organization, community list for the propagation of community to result through organization propagationOrganizationCommunityMap - - { - "20|corda__h2020::3fb05a9524c3f790391261347852f638":["mes","euromarine"], - "20|corda__h2020::e8dbe14cca9bf6fce09d468872f813f8":["mes","euromarine"], - "20|snsf________::9b253f265e3bef5cae6d881fdf61aceb":["mes","euromarine"], - "20|rcuk________::e054eea0a47665af8c3656b5785ccf76":["mes","euromarine"], - "20|corda__h2020::edc18d67c9b11fb616ca9f6e1db1b151":["mes","euromarine"], - "20|rcuk________::d5736d9da90521ddcdc7828a05a85e9a":["mes","euromarine"], - "20|corda__h2020::f5d418d3aa1cf817ddefcc3fdc039f27":["mes","euromarine"], - "20|snsf________::8fa091f8f25a846779acb4ea97b50aef":["mes","euromarine"], - "20|corda__h2020::81e020977211c2c40fae2e1a50bffd71":["mes","euromarine"], - "20|corda_______::81e020977211c2c40fae2e1a50bffd71":["mes","euromarine"], - "20|snsf________::31d0a100e54e3cdb3c6f52d91e638c78":["mes","euromarine"], - "20|corda__h2020::ea379ef91b8cc86f9ac5edc4169292db":["mes","euromarine"], - "20|corda__h2020::f75ee2ee48e5cb0ec8c8d30aaa8fef70":["mes","euromarine"], - "20|rcuk________::e16010089551a1a9182a94604fc0ea59":["mes","euromarine"], - "20|corda__h2020::38531a2cce7c5c347ffc439b07c1f43b":["mes","euromarine"], - "20|corda_______::38531a2cce7c5c347ffc439b07c1f43b":["mes","euromarine"], - "20|grid________::b2cbbf5eadbbf87d534b022bad3191d7":["mes","euromarine"], - "20|snsf________::74730ef1439d7f7636a8be58a6b471b8":["mes","euromarine"], - "20|nsf_________::ad72e19043a5a467e35f9b444d11563e":["mes","euromarine"], - "20|rcuk________::0fc3e92500290902a2d38ec2445e74c3":["mes","euromarine"], - "20|grid________::ad2c29905da0eb3c06b3fa80cacd89ea":["mes","euromarine"], - "20|corda__h2020::30b53e4d63d3724f00acb9cbaca40860":["mes","euromarine"], - "20|corda__h2020::f60f84bee14ad93f0db0e49af1d5c317":["mes","euromarine"], - "20|corda__h2020::7bf251ac3765b5e89d82270a1763d09f":["mes","euromarine"], - "20|corda__h2020::65531bd11be9935948c7f2f4db1c1832":["mes","euromarine"], - "20|corda__h2020::e0e98f86bbc76638bbb72a8fe2302946":["mes","euromarine"], - "20|snsf________::3eb43582ac27601459a8d8b3e195724b":["mes","euromarine"], - "20|corda__h2020::af2481dab65d06c8ea0ae02b5517b9b6":["mes","euromarine"], - "20|corda__h2020::c19d05cfde69a50d3ebc89bd0ee49929":["mes","euromarine"], - "20|corda__h2020::af0bfd9fc09f80d9488f56d71a9832f0":["mes","euromarine"], - "20|rcuk________::f33c02afb0dc66c49d0ed97ca5dd5cb0":["beopen"], - "20|grid________::a867f78acdc5041b34acfe4f9a349157":["beopen"], - "20|grid________::7bb116a1a9f95ab812bf9d2dea2be1ff":["beopen"], - "20|corda__h2020::6ab0e0739dbe625b99a2ae45842164ad":["beopen"], - "20|corda__h2020::8ba50792bc5f4d51d79fca47d860c602":["beopen"], - "20|corda_______::8ba50792bc5f4d51d79fca47d860c602":["beopen"], - "20|corda__h2020::e70e9114979e963eef24666657b807c3":["beopen"], - "20|corda_______::e70e9114979e963eef24666657b807c3":["beopen"], - "20|corda_______::15911e01e9744d57205825d77c218737":["beopen"], - "20|opendoar____::056a41e24e2a9a67215e87bbee6a80ab":["beopen"], - "20|opendoar____::7f67f2e6c6fbb0628f8160fcd3d92ae3":["beopen"], - "20|grid________::a8ecfd7c084e561168bcbe6bf0daf3e3":["beopen"], - "20|corda_______::7bbe6cc5d8ec1864739a04b0d020c9e9":["beopen"], - "20|corda_______::3ff558e30c2e434d688539548300b050":["beopen"], - "20|corda__h2020::5ffee5b3b83b33a8cf0e046877bd3a39":["beopen"], - "20|corda__h2020::5187217e2e806a6df3579c46f82401bc":["beopen"], - "20|grid________::5fa7e2709bcd945e26bfa18689adeec1":["beopen"], - "20|corda_______::d8696683c53027438031a96ad27c3c07":["beopen"], - "20|corda__h2020::d8696683c53027438031a96ad27c3c07":["beopen"], - "20|rcuk________::23a79ebdfa59790864e4a485881568c1":["beopen"], - "20|corda__h2020::b76cf8fe49590a966953c37e18608af9":["beopen"], - "20|grid________::d2f0204126ee709244a488a4cd3b91c2":["beopen"], - "20|corda__h2020::05aba9d2ed17533d15221e5655ac11e6":["beopen"], - "20|grid________::802401579481dc32062bdee69f5e6a34":["beopen"], - "20|corda__h2020::3f6d9d54cac975a517ba6b252c81582d":["beopen"] - } + {"20|corda__h2020::3fb05a9524c3f790391261347852f638":["mes","euromarine"], "20|corda__h2020::e8dbe14cca9bf6fce09d468872f813f8":["mes","euromarine"], "20|snsf________::9b253f265e3bef5cae6d881fdf61aceb":["mes","euromarine"],"20|rcuk________::e054eea0a47665af8c3656b5785ccf76":["mes","euromarine"],"20|corda__h2020::edc18d67c9b11fb616ca9f6e1db1b151":["mes","euromarine"],"20|rcuk________::d5736d9da90521ddcdc7828a05a85e9a":["mes","euromarine"],"20|corda__h2020::f5d418d3aa1cf817ddefcc3fdc039f27":["mes","euromarine"],"20|snsf________::8fa091f8f25a846779acb4ea97b50aef":["mes","euromarine"],"20|corda__h2020::81e020977211c2c40fae2e1a50bffd71":["mes","euromarine"],"20|corda_______::81e020977211c2c40fae2e1a50bffd71":["mes","euromarine"],"20|snsf________::31d0a100e54e3cdb3c6f52d91e638c78":["mes","euromarine"],"20|corda__h2020::ea379ef91b8cc86f9ac5edc4169292db":["mes","euromarine"],"20|corda__h2020::f75ee2ee48e5cb0ec8c8d30aaa8fef70":["mes","euromarine"],"20|rcuk________::e16010089551a1a9182a94604fc0ea59":["mes","euromarine"],"20|corda__h2020::38531a2cce7c5c347ffc439b07c1f43b":["mes","euromarine"],"20|corda_______::38531a2cce7c5c347ffc439b07c1f43b":["mes","euromarine"],"20|grid________::b2cbbf5eadbbf87d534b022bad3191d7":["mes","euromarine"],"20|snsf________::74730ef1439d7f7636a8be58a6b471b8":["mes","euromarine"],"20|nsf_________::ad72e19043a5a467e35f9b444d11563e":["mes","euromarine"],"20|rcuk________::0fc3e92500290902a2d38ec2445e74c3":["mes","euromarine"],"20|grid________::ad2c29905da0eb3c06b3fa80cacd89ea":["mes","euromarine"],"20|corda__h2020::30b53e4d63d3724f00acb9cbaca40860":["mes","euromarine"],"20|corda__h2020::f60f84bee14ad93f0db0e49af1d5c317":["mes","euromarine"], "20|corda__h2020::7bf251ac3765b5e89d82270a1763d09f":["mes","euromarine"], "20|corda__h2020::65531bd11be9935948c7f2f4db1c1832":["mes","euromarine"], "20|corda__h2020::e0e98f86bbc76638bbb72a8fe2302946":["mes","euromarine"], "20|snsf________::3eb43582ac27601459a8d8b3e195724b":["mes","euromarine"], "20|corda__h2020::af2481dab65d06c8ea0ae02b5517b9b6":["mes","euromarine"], "20|corda__h2020::c19d05cfde69a50d3ebc89bd0ee49929":["mes","euromarine"], "20|corda__h2020::af0bfd9fc09f80d9488f56d71a9832f0":["mes","euromarine"], "20|rcuk________::f33c02afb0dc66c49d0ed97ca5dd5cb0":["beopen"], + "20|grid________::a867f78acdc5041b34acfe4f9a349157":["beopen"], "20|grid________::7bb116a1a9f95ab812bf9d2dea2be1ff":["beopen"], "20|corda__h2020::6ab0e0739dbe625b99a2ae45842164ad":["beopen"], "20|corda__h2020::8ba50792bc5f4d51d79fca47d860c602":["beopen"], "20|corda_______::8ba50792bc5f4d51d79fca47d860c602":["beopen"], "20|corda__h2020::e70e9114979e963eef24666657b807c3":["beopen"], "20|corda_______::e70e9114979e963eef24666657b807c3":["beopen"], "20|corda_______::15911e01e9744d57205825d77c218737":["beopen"], "20|opendoar____::056a41e24e2a9a67215e87bbee6a80ab":["beopen"], "20|opendoar____::7f67f2e6c6fbb0628f8160fcd3d92ae3":["beopen"], "20|grid________::a8ecfd7c084e561168bcbe6bf0daf3e3":["beopen"], "20|corda_______::7bbe6cc5d8ec1864739a04b0d020c9e9":["beopen"], "20|corda_______::3ff558e30c2e434d688539548300b050":["beopen"], "20|corda__h2020::5ffee5b3b83b33a8cf0e046877bd3a39":["beopen"], "20|corda__h2020::5187217e2e806a6df3579c46f82401bc":["beopen"], "20|grid________::5fa7e2709bcd945e26bfa18689adeec1":["beopen"], "20|corda_______::d8696683c53027438031a96ad27c3c07":["beopen"], "20|corda__h2020::d8696683c53027438031a96ad27c3c07":["beopen"], "20|rcuk________::23a79ebdfa59790864e4a485881568c1":["beopen"], "20|corda__h2020::b76cf8fe49590a966953c37e18608af9":["beopen"], "20|grid________::d2f0204126ee709244a488a4cd3b91c2":["beopen"], "20|corda__h2020::05aba9d2ed17533d15221e5655ac11e6":["beopen"], "20|grid________::802401579481dc32062bdee69f5e6a34":["beopen"], "20|corda__h2020::3f6d9d54cac975a517ba6b252c81582d":["beopen"]} @@ -273,8 +227,8 @@ 'mongoDb' : 'mdstore', 'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus', 'postgresUser' : 'dnet', - 'postgresPassword' : '*****', - 'reuseContent' : 'false', + 'postgresPassword' : 'dnetPwd', + 'reuseContent' : 'true', 'contentPath' : '/tmp/beta_provision/aggregator', 'workingDir' : '/tmp/beta_provision/working_dir/aggregator' } @@ -403,7 +357,6 @@ - propagates ORCID among results linked by allowedsemrels semantic relationships @@ -429,7 +382,6 @@ - mark results respecting some rules as belonging to communities @@ -440,7 +392,7 @@ 'sourcePath' : 'orcidGraphPath', 'outputPath': 'bulkTaggingGraphPath', 'isLookUpUrl' : 'isLookUpUrl', - 'pathMap' : 'bulkTaggingPathMap', + 'pathMap' : 'bulkTaggingPathMap' } @@ -455,7 +407,6 @@ - creates relashionships between results and organizations when the organizations are associated to institutional repositories @@ -464,14 +415,14 @@ { 'sourcePath' : 'bulkTaggingGraphPath', - 'outputPath': 'affiliationGraphPath', - 'saveGraph' : 'true' + 'outputPath': 'affiliationGraphPath' } { 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/affiliation/oozie_app', - 'workingDir' : '/tmp/beta_provision/working_dir/affiliation' + 'workingDir' : '/tmp/beta_provision/working_dir/affiliation', + 'saveGraph' : 'true' } build-report @@ -480,9 +431,8 @@ - - marks as belonging to communities the result collected from providers related to the organizations specified in the organizationCommunityMap + marks as belonging to communities the result collected from datasources related to the organizations specified in the organizationCommunityMap executeOozieJob IIS @@ -506,7 +456,6 @@ - created relation between projects and results linked to other results trough allowedsemrel semantic relations linked to projects @@ -532,7 +481,6 @@ - tag as belonging to communitites result in in allowedsemrels relation with other result already linked to communities @@ -542,14 +490,15 @@ { 'sourcePath' : 'fundingGraphPath', 'outputPath': 'communitySemRelGraphPath', - 'isLookupUrl' : 'isLookUpUrl' + 'isLookUpUrl' : 'isLookUpUrl' } { 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/community_semrel/oozie_app', 'workingDir' : '/tmp/beta_provision/working_dir/community_semrel', - 'allowedsemrels' : 'isSupplementedBy;isSupplementTo' + 'allowedsemrels' : 'isSupplementedBy;isSupplementTo', + 'saveGraph' : 'true' } build-report @@ -558,7 +507,6 @@ - associated to results colleced from allowedtypes and those in the whithelist the country of the organization(s) handling the datasource it is collected from @@ -581,16 +529,42 @@ build-report + + + + + + removes blacklisted relations + + executeOozieJob + IIS + + { + 'sourcePath' : 'countryGraphPath', + 'outputPath': 'blacklistedGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/blacklist/oozie_app', + 'workingDir' : '/tmp/beta_provision/working_dir/blacklist', + 'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus', + 'postgresUser' : 'dnet', + 'postgresPassword' : 'dnetPwd' + } + + build-report + - wf_20200428_155848_495 - 2020-04-28T16:53:23+00:00 + wf_20200509_100941_857 + 2020-05-09T13:26:09+00:00 FAILURE - + eu.dnetlib.data.hadoop.rmi.HadoopServiceException: hadoop job: 0002933-200403132837156-oozie-oozi-W failed with status: KILLED, oozie log: 2020-05-09 13:23:31,194 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[] No results found 2020-05-09 13:23:31,216 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@:start:] Start action [0002933-200403132837156-oozie-oozi-W@:start:] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:31,216 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@:start:] [***0002933-200403132837156-oozie-oozi-W@:start:***]Action status=DONE 2020-05-09 13:23:31,216 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@:start:] [***0002933-200403132837156-oozie-oozi-W@:start:***]Action updated in DB! 2020-05-09 13:23:31,257 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@:start:] No results found 2020-05-09 13:23:31,275 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@:start:] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@:start: 2020-05-09 13:23:31,275 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W 2020-05-09 13:23:31,314 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@reset-outputpath] Start action [0002933-200403132837156-oozie-oozi-W@reset-outputpath] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:33,897 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@reset-outputpath] [***0002933-200403132837156-oozie-oozi-W@reset-outputpath***]Action status=DONE 2020-05-09 13:23:33,897 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@reset-outputpath] [***0002933-200403132837156-oozie-oozi-W@reset-outputpath***]Action updated in DB! 2020-05-09 13:23:33,947 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@reset-outputpath] No results found 2020-05-09 13:23:33,966 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] Start action [0002933-200403132837156-oozie-oozi-W@copy_entities] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:33,966 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] [***0002933-200403132837156-oozie-oozi-W@copy_entities***]Action status=DONE 2020-05-09 13:23:33,966 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] [***0002933-200403132837156-oozie-oozi-W@copy_entities***]Action updated in DB! 2020-05-09 13:23:34,012 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] No results found 2020-05-09 13:23:34,018 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] No results found 2020-05-09 13:23:34,023 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] No results found 2020-05-09 13:23:34,029 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] No results found 2020-05-09 13:23:34,124 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] Start action [0002933-200403132837156-oozie-oozi-W@copy_relation] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:34,130 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] Start action [0002933-200403132837156-oozie-oozi-W@copy_projects] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:34,130 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] Start action [0002933-200403132837156-oozie-oozi-W@copy_datasources] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:34,140 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] Start action [0002933-200403132837156-oozie-oozi-W@copy_organization] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:23:35,010 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] checking action, hadoop job ID [job_1585920557248_14569] status [RUNNING] 2020-05-09 13:23:35,018 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] [***0002933-200403132837156-oozie-oozi-W@copy_projects***]Action status=RUNNING 2020-05-09 13:23:35,018 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] [***0002933-200403132837156-oozie-oozi-W@copy_projects***]Action updated in DB! 2020-05-09 13:23:35,022 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] checking action, hadoop job ID [job_1585920557248_14568] status [RUNNING] 2020-05-09 13:23:35,027 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_projects 2020-05-09 13:23:35,028 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] [***0002933-200403132837156-oozie-oozi-W@copy_relation***]Action status=RUNNING 2020-05-09 13:23:35,028 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] [***0002933-200403132837156-oozie-oozi-W@copy_relation***]Action updated in DB! 2020-05-09 13:23:35,031 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] checking action, hadoop job ID [job_1585920557248_14570] status [RUNNING] 2020-05-09 13:23:35,035 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] [***0002933-200403132837156-oozie-oozi-W@copy_datasources***]Action status=RUNNING 2020-05-09 13:23:35,035 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] [***0002933-200403132837156-oozie-oozi-W@copy_datasources***]Action updated in DB! 2020-05-09 13:23:35,037 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_relation 2020-05-09 13:23:35,048 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_datasources 2020-05-09 13:23:35,072 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] checking action, hadoop job ID [job_1585920557248_14571] status [RUNNING] 2020-05-09 13:23:35,076 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] [***0002933-200403132837156-oozie-oozi-W@copy_organization***]Action status=RUNNING 2020-05-09 13:23:35,076 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] [***0002933-200403132837156-oozie-oozi-W@copy_organization***]Action updated in DB! 2020-05-09 13:23:35,084 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_organization 2020-05-09 13:23:35,090 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_entities] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_entities 2020-05-09 13:23:35,090 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@reset-outputpath] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@reset-outputpath 2020-05-09 13:23:58,926 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] callback for action [0002933-200403132837156-oozie-oozi-W@copy_datasources] 2020-05-09 13:23:59,085 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] checking action, hadoop job ID [job_1585920557248_14570] status [RUNNING] 2020-05-09 13:23:59,242 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] callback for action [0002933-200403132837156-oozie-oozi-W@copy_projects] 2020-05-09 13:23:59,386 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] checking action, hadoop job ID [job_1585920557248_14569] status [RUNNING] 2020-05-09 13:24:01,343 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] callback for action [0002933-200403132837156-oozie-oozi-W@copy_datasources] 2020-05-09 13:24:01,418 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] Hadoop Jobs launched : [job_1585920557248_14573] 2020-05-09 13:24:01,418 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] action completed, external ID [job_1585920557248_14570] 2020-05-09 13:24:01,493 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_datasources] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_datasources 2020-05-09 13:24:01,935 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] callback for action [0002933-200403132837156-oozie-oozi-W@copy_projects] 2020-05-09 13:24:02,012 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] Hadoop Jobs launched : [job_1585920557248_14572] 2020-05-09 13:24:02,012 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] action completed, external ID [job_1585920557248_14569] 2020-05-09 13:24:02,076 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_projects] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_projects 2020-05-09 13:25:03,172 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] callback for action [0002933-200403132837156-oozie-oozi-W@copy_organization] 2020-05-09 13:25:03,336 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] checking action, hadoop job ID [job_1585920557248_14571] status [RUNNING] 2020-05-09 13:25:05,598 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] callback for action [0002933-200403132837156-oozie-oozi-W@copy_organization] 2020-05-09 13:25:05,688 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] Hadoop Jobs launched : [job_1585920557248_14574] 2020-05-09 13:25:05,691 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] action completed, external ID [job_1585920557248_14571] 2020-05-09 13:25:05,748 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_organization] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_organization 2020-05-09 13:25:23,274 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] callback for action [0002933-200403132837156-oozie-oozi-W@copy_relation] 2020-05-09 13:25:23,409 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] checking action, hadoop job ID [job_1585920557248_14568] status [RUNNING] 2020-05-09 13:25:25,419 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] callback for action [0002933-200403132837156-oozie-oozi-W@copy_relation] 2020-05-09 13:25:25,510 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] Hadoop Jobs launched : [job_1585920557248_14575] 2020-05-09 13:25:25,511 INFO org.apache.oozie.action.hadoop.DistcpActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] action completed, external ID [job_1585920557248_14568] 2020-05-09 13:25:25,565 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] No results found 2020-05-09 13:25:25,585 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_wait] Start action [0002933-200403132837156-oozie-oozi-W@copy_wait] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:25,585 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_wait] [***0002933-200403132837156-oozie-oozi-W@copy_wait***]Action status=DONE 2020-05-09 13:25:25,585 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_wait] [***0002933-200403132837156-oozie-oozi-W@copy_wait***]Action updated in DB! 2020-05-09 13:25:25,627 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_wait] No results found 2020-05-09 13:25:25,648 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] Start action [0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:25,648 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] [***0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1***]Action status=DONE 2020-05-09 13:25:25,648 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] [***0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1***]Action updated in DB! 2020-05-09 13:25:25,694 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] No results found 2020-05-09 13:25:25,700 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] No results found 2020-05-09 13:25:25,706 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] No results found 2020-05-09 13:25:25,711 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] No results found 2020-05-09 13:25:25,801 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] Start action [0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:25,825 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] Start action [0002933-200403132837156-oozie-oozi-W@join_prepare_software] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:25,825 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] Start action [0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:25,828 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] Start action [0002933-200403132837156-oozie-oozi-W@join_prepare_publication] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:27,165 INFO org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] checking action, hadoop job ID [job_1585920557248_14578] status [RUNNING] 2020-05-09 13:25:27,170 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] [***0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct***]Action status=RUNNING 2020-05-09 13:25:27,170 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] [***0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct***]Action updated in DB! 2020-05-09 13:25:27,179 INFO org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] checking action, hadoop job ID [job_1585920557248_14577] status [RUNNING] 2020-05-09 13:25:27,181 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct 2020-05-09 13:25:27,183 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] [***0002933-200403132837156-oozie-oozi-W@join_prepare_software***]Action status=RUNNING 2020-05-09 13:25:27,183 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] [***0002933-200403132837156-oozie-oozi-W@join_prepare_software***]Action updated in DB! 2020-05-09 13:25:27,188 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_software 2020-05-09 13:25:27,617 INFO org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] checking action, hadoop job ID [job_1585920557248_14576] status [RUNNING] 2020-05-09 13:25:27,622 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] [***0002933-200403132837156-oozie-oozi-W@join_prepare_publication***]Action status=RUNNING 2020-05-09 13:25:27,622 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] [***0002933-200403132837156-oozie-oozi-W@join_prepare_publication***]Action updated in DB! 2020-05-09 13:25:27,625 INFO org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] checking action, hadoop job ID [job_1585920557248_14579] status [RUNNING] 2020-05-09 13:25:27,628 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_publication 2020-05-09 13:25:27,629 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] [***0002933-200403132837156-oozie-oozi-W@join_prepare_dataset***]Action status=RUNNING 2020-05-09 13:25:27,629 INFO org.apache.oozie.command.wf.ForkedActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] [***0002933-200403132837156-oozie-oozi-W@join_prepare_dataset***]Action updated in DB! 2020-05-09 13:25:27,634 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_dataset 2020-05-09 13:25:27,639 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@fork_prepare_assoc_step1 2020-05-09 13:25:27,639 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_wait] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_wait 2020-05-09 13:25:27,640 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@copy_relation] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@copy_relation 2020-05-09 13:25:41,416 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] callback for action [0002933-200403132837156-oozie-oozi-W@join_prepare_software] 2020-05-09 13:25:41,490 INFO org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] action completed, external ID [job_1585920557248_14577] 2020-05-09 13:25:41,495 WARN org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] Launcher ERROR, reason: Main class [org.apache.oozie.action.hadoop.SparkMain], main() threw exception, File file:/data/3/yarn/nm/usercache/dnet.beta/appcache/application_1585920557248_14577/container_e68_1585920557248_14577_01_000002/dhp-propagation-1.1.8-SNAPSHOT.jar does not exist 2020-05-09 13:25:41,495 WARN org.apache.oozie.action.hadoop.SparkActionExecutor: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] Launcher exception: File file:/data/3/yarn/nm/usercache/dnet.beta/appcache/application_1585920557248_14577/container_e68_1585920557248_14577_01_000002/dhp-propagation-1.1.8-SNAPSHOT.jar does not exist java.io.FileNotFoundException: File file:/data/3/yarn/nm/usercache/dnet.beta/appcache/application_1585920557248_14577/container_e68_1585920557248_14577_01_000002/dhp-propagation-1.1.8-SNAPSHOT.jar does not exist at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:598) at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:811) at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:588) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:432) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:340) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:292) at org.apache.spark.deploy.yarn.Client.copyFileToRemote(Client.scala:404) at org.apache.spark.deploy.yarn.Client.org$apache$spark$deploy$yarn$Client$$distribute$1(Client.scala:496) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$9.apply(Client.scala:595) at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$9.apply(Client.scala:594) at scala.Option.foreach(Option.scala:257) at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:594) at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:886) at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:180) at org.apache.spark.deploy.yarn.Client.run(Client.scala:1156) at org.apache.spark.deploy.yarn.YarnClusterApplication.start(Client.scala:1608) at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:849) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:195) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:924) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:933) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) at org.apache.oozie.action.hadoop.SparkMain.runSpark(SparkMain.java:178) at org.apache.oozie.action.hadoop.SparkMain.run(SparkMain.java:90) at org.apache.oozie.action.hadoop.LauncherMain.run(LauncherMain.java:81) at org.apache.oozie.action.hadoop.SparkMain.main(SparkMain.java:57) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.oozie.action.hadoop.LauncherMapper.map(LauncherMapper.java:235) at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:54) at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:459) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:343) at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1924) at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158) 2020-05-09 13:25:41,514 INFO org.apache.oozie.command.wf.ActionEndXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] ERROR is considered as FAILED for SLA 2020-05-09 13:25:41,541 INFO org.apache.oozie.service.JPAService: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] No results found 2020-05-09 13:25:41,580 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@Kill] Start action [0002933-200403132837156-oozie-oozi-W@Kill] with user-retry state : userRetryCount [0], userRetryMax [0], userRetryInterval [10] 2020-05-09 13:25:41,580 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@Kill] [***0002933-200403132837156-oozie-oozi-W@Kill***]Action status=DONE 2020-05-09 13:25:41,580 INFO org.apache.oozie.command.wf.ActionStartXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@Kill] [***0002933-200403132837156-oozie-oozi-W@Kill***]Action updated in DB! 2020-05-09 13:25:41,692 WARN org.apache.oozie.workflow.lite.LiteWorkflowInstance: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@Kill] Workflow completed [KILLED], killing [3] running nodes 2020-05-09 13:25:41,760 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@Kill] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@Kill 2020-05-09 13:25:41,766 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_software] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_software 2020-05-09 13:25:41,852 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct 2020-05-09 13:25:41,914 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] callback for action [0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] 2020-05-09 13:25:41,920 ERROR org.apache.oozie.command.wf.CompletedActionXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] XException, org.apache.oozie.command.CommandException: E0800: Action it is not running its in [KILLED] state, action [0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] at org.apache.oozie.command.wf.CompletedActionXCommand.eagerVerifyPrecondition(CompletedActionXCommand.java:92) at org.apache.oozie.command.XCommand.call(XCommand.java:257) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at org.apache.oozie.service.CallableQueueService$CallableWrapper.run(CallableQueueService.java:179) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 2020-05-09 13:25:41,938 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_publication 2020-05-09 13:25:42,005 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] callback for action [0002933-200403132837156-oozie-oozi-W@join_prepare_publication] 2020-05-09 13:25:42,010 ERROR org.apache.oozie.command.wf.CompletedActionXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_publication] XException, org.apache.oozie.command.CommandException: E0800: Action it is not running its in [KILLED] state, action [0002933-200403132837156-oozie-oozi-W@join_prepare_publication] at org.apache.oozie.command.wf.CompletedActionXCommand.eagerVerifyPrecondition(CompletedActionXCommand.java:92) at org.apache.oozie.command.XCommand.call(XCommand.java:257) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at org.apache.oozie.service.CallableQueueService$CallableWrapper.run(CallableQueueService.java:179) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 2020-05-09 13:25:42,028 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[dnet.beta] GROUP[-] TOKEN[] APP[orcid_to_result_from_semrel_propagation] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W 2020-05-09 13:25:42,028 INFO org.apache.oozie.command.wf.WorkflowNotificationXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_dataset] No Notification URL is defined. Therefore nothing to notify for job 0002933-200403132837156-oozie-oozi-W@join_prepare_dataset 2020-05-09 13:25:42,113 INFO org.apache.oozie.servlet.CallbackServlet: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[-] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] callback for action [0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] 2020-05-09 13:25:42,116 ERROR org.apache.oozie.command.wf.CompletedActionXCommand: SERVER[iis-cdh5-test-m3.ocean.icm.edu.pl] USER[-] GROUP[-] TOKEN[] APP[-] JOB[0002933-200403132837156-oozie-oozi-W] ACTION[0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] XException, org.apache.oozie.command.CommandException: E0800: Action it is not running its in [KILLED] state, action [0002933-200403132837156-oozie-oozi-W@join_prepare_otherresearchproduct] at org.apache.oozie.command.wf.CompletedActionXCommand.eagerVerifyPrecondition(CompletedActionXCommand.java:92) at org.apache.oozie.command.XCommand.call(XCommand.java:257) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at org.apache.oozie.service.CallableQueueService$CallableWrapper.run(CallableQueueService.java:179) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) \ No newline at end of file From ec0782e582961bea41f8fa30ac43a3d4a3a366fa Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 12 May 2020 15:49:28 +0200 Subject: [PATCH 257/259] renamed jar containing the bulktagging and propagation workflows from dhp-[bulktagging|propagation] to dhp-enrichment; adjusted xml formatting --- .../dhp/bulktag/oozie_app/workflow.xml | 8 +- .../countrypropagation/oozie_app/workflow.xml | 18 +- .../oozie_app/workflow.xml | 160 +++++++----------- .../projecttoresult/oozie_app/workflow.xml | 107 ++++++------ .../oozie_app/workflow.xml | 10 +- .../oozie_app/workflow.xml | 21 ++- .../oozie_app/workflow.xml | 10 +- 7 files changed, 144 insertions(+), 190 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 524281bc9..754aba4f2 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -101,7 +101,7 @@ cluster bulkTagging-publication eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-bulktag-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --num-executors=${sparkExecutorNumber} --executor-memory=${sparkExecutorMemory} @@ -130,7 +130,7 @@ cluster bulkTagging-dataset eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-bulktag-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --num-executors=${sparkExecutorNumber} --executor-memory=${sparkExecutorMemory} @@ -159,7 +159,7 @@ cluster bulkTagging-orp eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-bulktag-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --num-executors=${sparkExecutorNumber} --executor-memory=${sparkExecutorMemory} @@ -188,7 +188,7 @@ cluster bulkTagging-software eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-bulktag-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --num-executors=${sparkExecutorNumber} --executor-memory=${sparkExecutorMemory} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml index f269c5442..fc877071d 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/countrypropagation/oozie_app/workflow.xml @@ -92,7 +92,7 @@ cluster PrepareDatasourceCountryAssociation eu.dnetlib.dhp.countrypropagation.PrepareDatasourceCountryAssociation - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -126,7 +126,7 @@ cluster prepareResultCountry-Publication eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -156,7 +156,7 @@ cluster prepareResultCountry-Dataset eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -186,7 +186,7 @@ cluster prepareResultCountry-ORP eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -216,7 +216,7 @@ cluster prepareResultCountry-Software eu.dnetlib.dhp.countrypropagation.PrepareResultCountrySet - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -255,7 +255,7 @@ cluster countryPropagationForPublications eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -285,7 +285,7 @@ cluster countryPropagationForDataset eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -315,7 +315,7 @@ cluster countryPropagationForORP eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -345,7 +345,7 @@ cluster countryPropagationForSoftware eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml index 7b06b6504..e4429b710 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/orcidtoresultfromsemrel/oozie_app/workflow.xml @@ -95,7 +95,7 @@ cluster ORCIDPropagation-PreparePhase1-Publications eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -111,16 +111,11 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --sourcePath - ${sourcePath} - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Publication - --outputPath - ${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels - ${allowedsemrels} + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} @@ -132,7 +127,7 @@ cluster ORCIDPropagation-PreparePhase1-Dataset eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -144,16 +139,11 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath - ${sourcePath} - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Dataset - --outputPath - ${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels - ${allowedsemrels} + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} @@ -165,7 +155,7 @@ cluster ORCIDPropagation-PreparePhase1-ORP eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -177,16 +167,11 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath - ${sourcePath} - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath - ${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels - ${allowedsemrels} + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} @@ -198,7 +183,7 @@ cluster ORCIDPropagation-PreparePhase1-Software eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -210,16 +195,11 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath - ${sourcePath} - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Software - --outputPath - ${workingDir}/preparedInfo/targetOrcidAssoc - --allowedsemrels - ${allowedsemrels} + --sourcePath${sourcePath} + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/preparedInfo/targetOrcidAssoc + --allowedsemrels${allowedsemrels} @@ -233,7 +213,7 @@ cluster ORCIDPropagation-PreparePhase2 eu.dnetlib.dhp.orcidtoresultfromsemrel.PrepareResultOrcidAssociationStep2 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -245,13 +225,10 @@ --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - --sourcePath - ${workingDir}/preparedInfo/targetOrcidAssoc - --outputPath - ${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${workingDir}/preparedInfo/targetOrcidAssoc + --outputPath${workingDir}/preparedInfo/mergedOrcidAssoc - @@ -268,7 +245,7 @@ cluster ORCIDPropagation-Publication eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -284,18 +261,12 @@ --conf spark.hadoop.mapreduce.reduce.speculative=false --conf spark.sql.shuffle.partitions=3840 - --possibleUpdatesPath - ${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath - ${sourcePath}/publication - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Publication - --outputPath - ${outputPath}/publication - --saveGraph - ${saveGraph} + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/publication + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${outputPath}/publication + --saveGraph${saveGraph} @@ -306,7 +277,7 @@ cluster ORCIDPropagation-Dataset eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -321,18 +292,12 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath - ${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath - ${sourcePath}/dataset - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Dataset - --outputPath - ${outputPath}/dataset - --saveGraph - ${saveGraph} + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/dataset + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${outputPath}/dataset + --saveGraph${saveGraph} @@ -343,7 +308,7 @@ cluster ORCIDPropagation-ORP eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -358,18 +323,12 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath - ${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath - ${sourcePath}/otherresearchproduct - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath - ${outputPath}/otherresearchproduct - --saveGraph - ${saveGraph} + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/otherresearchproduct + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${outputPath}/otherresearchproduct + --saveGraph${saveGraph} @@ -380,7 +339,7 @@ cluster ORCIDPropagation-Software eu.dnetlib.dhp.orcidtoresultfromsemrel.SparkOrcidToResultFromSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -395,22 +354,19 @@ --conf spark.hadoop.mapreduce.map.speculative=false --conf spark.hadoop.mapreduce.reduce.speculative=false - --possibleUpdatesPath - ${workingDir}/preparedInfo/mergedOrcidAssoc - --sourcePath - ${sourcePath}/software - --hive_metastore_uris - ${hive_metastore_uris} - --resultTableName - eu.dnetlib.dhp.schema.oaf.Software - --outputPath - ${outputPath}/software - --saveGraph - ${saveGraph} + --possibleUpdatesPath${workingDir}/preparedInfo/mergedOrcidAssoc + --sourcePath${sourcePath}/software + --hive_metastore_uris${hive_metastore_uris} + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${outputPath}/software + --saveGraph${saveGraph} + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index dd7f25846..687d66869 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -127,61 +127,60 @@ + + + + yarn + cluster + PrepareProjectResultsAssociation + eu.dnetlib.dhp.projecttoresult.PrepareProjectResultsAssociation + dhp-propagation-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath}/relation + --allowedsemrels${allowedsemrels} + --hive_metastore_uris${hive_metastore_uris} + --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked + + + + - - - - yarn - cluster - PrepareProjectResultsAssociation - eu.dnetlib.dhp.projecttoresult.PrepareProjectResultsAssociation - dhp-propagation-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/relation - --allowedsemrels${allowedsemrels} - --hive_metastore_uris${hive_metastore_uris} - --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates - --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - - - - - - - - yarn - cluster - ProjectToResultPropagation - eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob - dhp-propagation-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --saveGraph${saveGraph} - --hive_metastore_uris${hive_metastore_uris} - --outputPath${outputPath}/relation - --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates - --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked - - - - + + + yarn + cluster + ProjectToResultPropagation + eu.dnetlib.dhp.projecttoresult.SparkResultToProjectThroughSemRelJob + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --saveGraph${saveGraph} + --hive_metastore_uris${hive_metastore_uris} + --outputPath${outputPath}/relation + --potentialUpdatePath${workingDir}/preparedInfo/potentialUpdates + --alreadyLinkedPath${workingDir}/preparedInfo/alreadyLinked + + + + diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index 3be69bde6..d481cad05 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -88,7 +88,7 @@ cluster Prepare-Community-Result-Organization eu.dnetlib.dhp.resulttocommunityfromorganization.PrepareResultCommunitySet - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -122,7 +122,7 @@ cluster community2resultfromorganization-Publication eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -151,7 +151,7 @@ cluster community2resultfromorganization-Dataset eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -180,7 +180,7 @@ cluster community2resultfromorganization-ORP eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -209,7 +209,7 @@ cluster community2resultfromorganization-Software eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml index b75b2d31e..81b51443c 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromsemrel/oozie_app/workflow.xml @@ -99,7 +99,7 @@ cluster ResultToCommunitySemRel-PreparePhase1-Publications eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -128,7 +128,7 @@ cluster ResultToCommunitySemRel-PreparePhase1-Dataset eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -150,13 +150,14 @@ + yarn cluster ResultToCommunitySemRel-PreparePhase1-ORP eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -185,7 +186,7 @@ cluster ResultToCommunitySemRel-PreparePhase1-Software eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep1 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -216,7 +217,7 @@ cluster ResultToCommunityEmRelPropagation-PreparePhase2 eu.dnetlib.dhp.resulttocommunityfromsemrel.PrepareResultCommunitySetStep2 - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -232,9 +233,7 @@ --outputPath${workingDir}/preparedInfo/mergedCommunityAssoc - - @@ -250,7 +249,7 @@ cluster Result2CommunitySemRelPropagation-Publication eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -279,7 +278,7 @@ cluster Result2CommunitySemRelPropagation-Dataset eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -308,7 +307,7 @@ cluster Result2CommunitySemRelPropagation-ORP eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -337,7 +336,7 @@ cluster Result2CommunitySemRelPropagation-Software eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml index 73268fcc7..a1b7f4ad7 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttoorganizationfrominstrepo/oozie_app/workflow.xml @@ -131,7 +131,7 @@ cluster PrepareResultOrganizationAssociation eu.dnetlib.dhp.resulttoorganizationfrominstrepo.PrepareResultInstRepoAssociation - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -163,7 +163,7 @@ cluster resultToOrganizationFromInstRepoPropagationForPublications eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -193,7 +193,7 @@ cluster resultToOrganizationFromInstRepoPropagationForDataset eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -223,7 +223,7 @@ cluster resultToOrganizationFromInstRepoPropagationForORP eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} @@ -253,7 +253,7 @@ cluster resultToOrganizationFromInstRepoPropagationForSoftware eu.dnetlib.dhp.resulttoorganizationfrominstrepo.SparkResultToOrganizationFromIstRepoJob - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} From 43f127448ddf38d2a2328585ff4d2b1562bd0ff5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 12 May 2020 18:24:26 +0200 Subject: [PATCH 258/259] changed the package name from dhp-propagation to dhp-enrichment for the preparation phase of funding propagation --- .../eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml index 687d66869..24e1d3b7f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/projecttoresult/oozie_app/workflow.xml @@ -134,7 +134,7 @@ cluster PrepareProjectResultsAssociation eu.dnetlib.dhp.projecttoresult.PrepareProjectResultsAssociation - dhp-propagation-${projectVersion}.jar + dhp-enrichment-${projectVersion}.jar --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} From 85f3c55992b9d9b526020aa096e9867774b92270 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 13 May 2020 09:04:33 +0200 Subject: [PATCH 259/259] fixed node names in blacklist workflow --- .../eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml index 59fd30fea..1538318c1 100644 --- a/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-blacklist/src/main/resources/eu/dnetlib/dhp/blacklist/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + postgresURL @@ -102,7 +102,7 @@ - + ${jobTracker} ${nameNode} @@ -113,7 +113,7 @@ - + ${jobTracker} ${nameNode}